From 0d3014c10cc10ddd2826e848fe30ef8687ee0d62 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Tue, 18 Feb 2025 23:15:34 +0530 Subject: [PATCH 01/24] HBASE-28996: Implement Custom ReplicationEndpoint to Enable WAL Backup to External Storage (#6633) * HBASE-28996: Implement Custom ReplicationEndpoint to Enable WAL Backup to External Storage * fix spotless error --- .../replication/BackupFileSystemManager.java | 71 +++ .../backup/replication/BulkLoadProcessor.java | 96 ++++ .../ContinuousBackupReplicationEndpoint.java | 440 +++++++++++++++ .../ObjectStoreProtobufWalWriter.java | 73 +++ .../hbase/backup/replication/Utils.java | 30 + ...stContinuousBackupReplicationEndpoint.java | 513 ++++++++++++++++++ .../replication/ReplicationEndpoint.java | 18 +- .../hbase/replication/ReplicationResult.java | 33 ++ .../VerifyWALEntriesReplicationEndpoint.java | 4 +- .../HBaseInterClusterReplicationEndpoint.java | 13 +- .../regionserver/ReplicationSource.java | 35 +- .../ReplicationSourceInterface.java | 9 +- .../ReplicationSourceShipper.java | 23 +- .../VisibilityReplicationEndpoint.java | 3 +- .../replication/DummyReplicationEndpoint.java | 4 +- .../SerialReplicationTestBase.java | 4 +- .../TestHBaseReplicationEndpoint.java | 8 +- .../TestNonHBaseReplicationEndpoint.java | 4 +- .../replication/TestReplicationBase.java | 2 +- .../replication/TestReplicationEndpoint.java | 22 +- .../TestVerifyCellsReplicationEndpoint.java | 2 +- ...ClusterReplicationEndpointFilterEdits.java | 4 +- ...TestRaceWhenCreatingReplicationSource.java | 5 +- .../TestReplicationSourceManager.java | 5 +- .../regionserver/TestReplicator.java | 3 +- .../TestVisibilityLabelsReplication.java | 5 +- 26 files changed, 1369 insertions(+), 60 deletions(-) create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ObjectStoreProtobufWalWriter.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/Utils.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationResult.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java new file mode 100644 index 000000000000..225d32172766 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java @@ -0,0 +1,71 @@ +/* + * 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; + +/** + * Initializes and organizes backup directories for continuous Write-Ahead Logs (WALs) and + * bulk-loaded files within the specified backup root directory. + */ +@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 final Path walsDir; + private final 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); + this.walsDir = createDirectory(WALS_DIR); + this.bulkLoadFilesDir = createDirectory(BULKLOAD_FILES_DIR); + } + + private Path createDirectory(String dirName) throws IOException { + Path dirPath = new Path(backupRootDir, dirName); + backupFs.mkdirs(dirPath); + LOG.info("{} Initialized directory: {}", Utils.logPeerId(peerId), dirPath); + return dirPath; + } + + public Path getWalsDir() { + return walsDir; + } + + public Path getBulkLoadFilesDir() { + return bulkLoadFilesDir; + } + + public FileSystem getBackupFs() { + return backupFs; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java new file mode 100644 index 000000000000..6e1271313bcd --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java @@ -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. + *

+ * 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. + *

+ * The class is designed for scenarios where replicable bulk load operations need to be parsed and + * their file paths need to be determined programmatically. + *

+ */ +@InterfaceAudience.Private +public final class BulkLoadProcessor { + private BulkLoadProcessor() { + } + + public static List processBulkLoadFiles(List walEntries) throws IOException { + List bulkLoadFilePaths = new ArrayList<>(); + + for (WAL.Entry entry : walEntries) { + WALEdit edit = entry.getEdit(); + for (Cell cell : edit.getCells()) { + if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) { + TableName tableName = entry.getKey().getTableName(); + String namespace = tableName.getNamespaceAsString(); + String table = tableName.getQualifierAsString(); + bulkLoadFilePaths.addAll(processBulkLoadDescriptor(cell, namespace, table)); + } + } + } + return bulkLoadFilePaths; + } + + private static List processBulkLoadDescriptor(Cell cell, String namespace, String table) + throws IOException { + List 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 processStoreDescriptor(WALProtos.StoreDescriptor storeDescriptor, + String namespace, String table, String regionName) { + List 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; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java new file mode 100644 index 000000000000..c973af8102e7 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -0,0 +1,440 @@ +/* + * 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.io.UncheckedIOException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; +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.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; +import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; +import org.apache.hadoop.hbase.replication.ReplicationResult; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.FSHLogProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ContinuousBackupReplicationEndpoint is responsible for replicating WAL entries to a backup + * storage. It organizes WAL entries by day and periodically flushes the data, ensuring that WAL + * files do not exceed the configured size. The class includes mechanisms for handling the WAL + * files, performing bulk load backups, and ensuring that the replication process is safe. + */ +@InterfaceAudience.Private +public class ContinuousBackupReplicationEndpoint extends BaseReplicationEndpoint { + private static final Logger LOG = + LoggerFactory.getLogger(ContinuousBackupReplicationEndpoint.class); + public static final String CONF_PEER_UUID = "hbase.backup.wal.replication.peerUUID"; + 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; + + public static final String CONF_STAGED_WAL_FLUSH_INITIAL_DELAY = + "hbase.backup.staged.wal.flush.initial.delay.seconds"; + public static final int DEFAULT_STAGED_WAL_FLUSH_INITIAL_DELAY_SECONDS = 5 * 60; // 5 minutes + public static final String CONF_STAGED_WAL_FLUSH_INTERVAL = + "hbase.backup.staged.wal.flush.interval.seconds"; + public static final int DEFAULT_STAGED_WAL_FLUSH_INTERVAL_SECONDS = 5 * 60; // 5 minutes + public static final int EXECUTOR_TERMINATION_TIMEOUT_SECONDS = 60; // TODO: configurable?? + + private final Map walWriters = new ConcurrentHashMap<>(); + private final ReentrantLock lock = new ReentrantLock(); + + private ReplicationSourceInterface replicationSource; + private Configuration conf; + private BackupFileSystemManager backupFileSystemManager; + private UUID peerUUID; + private String peerId; + private ScheduledExecutorService flushExecutor; + + public static final long ONE_DAY_IN_MILLISECONDS = TimeUnit.DAYS.toMillis(1); + public static final String WAL_FILE_PREFIX = "wal_file."; + public static final String DATE_FORMAT = "yyyy-MM-dd"; + + @Override + public void init(Context context) throws IOException { + super.init(context); + this.replicationSource = context.getReplicationSource(); + this.peerId = context.getPeerId(); + this.conf = HBaseConfiguration.create(context.getConfiguration()); + + initializePeerUUID(); + initializeBackupFileSystemManager(); + startWalFlushExecutor(); + LOG.info("{} Initialization complete", Utils.logPeerId(peerId)); + } + + private void initializePeerUUID() throws IOException { + String peerUUIDStr = conf.get(CONF_PEER_UUID); + if (peerUUIDStr == null || peerUUIDStr.isEmpty()) { + throw new IOException("Peer UUID is not specified. Please configure " + CONF_PEER_UUID); + } + try { + this.peerUUID = UUID.fromString(peerUUIDStr); + LOG.info("{} Peer UUID initialized to {}", Utils.logPeerId(peerId), peerUUID); + } catch (IllegalArgumentException e) { + throw new IOException("Invalid Peer UUID format: " + peerUUIDStr, e); + } + } + + private void initializeBackupFileSystemManager() throws IOException { + String backupRootDir = conf.get(CONF_BACKUP_ROOT_DIR); + if (backupRootDir == null || backupRootDir.isEmpty()) { + throw new IOException( + "Backup root directory is not specified. Configure " + CONF_BACKUP_ROOT_DIR); + } + + try { + this.backupFileSystemManager = new BackupFileSystemManager(peerId, conf, backupRootDir); + LOG.info("{} BackupFileSystemManager initialized successfully for {}", + Utils.logPeerId(peerId), backupRootDir); + } catch (IOException e) { + throw new IOException("Failed to initialize BackupFileSystemManager", e); + } + } + + private void startWalFlushExecutor() { + int initialDelay = conf.getInt(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, + DEFAULT_STAGED_WAL_FLUSH_INITIAL_DELAY_SECONDS); + int flushInterval = + conf.getInt(CONF_STAGED_WAL_FLUSH_INTERVAL, DEFAULT_STAGED_WAL_FLUSH_INTERVAL_SECONDS); + + flushExecutor = Executors.newSingleThreadScheduledExecutor(); + flushExecutor.scheduleAtFixedRate(this::flushAndBackupSafely, initialDelay, flushInterval, + TimeUnit.SECONDS); + LOG.info("{} Scheduled WAL flush executor started with initial delay {}s and interval {}s", + Utils.logPeerId(peerId), initialDelay, flushInterval); + } + + private void flushAndBackupSafely() { + lock.lock(); + try { + LOG.info("{} Periodic WAL flush triggered", Utils.logPeerId(peerId)); + flushWriters(); + replicationSource.persistOffsets(); + LOG.info("{} Periodic WAL flush and offset persistence completed successfully", + Utils.logPeerId(peerId)); + } catch (IOException e) { + LOG.error("{} Error during WAL flush: {}", Utils.logPeerId(peerId), e.getMessage(), e); + } finally { + lock.unlock(); + } + } + + private void flushWriters() throws IOException { + LOG.info("{} Flushing {} WAL writers", Utils.logPeerId(peerId), walWriters.size()); + for (Map.Entry entry : walWriters.entrySet()) { + FSHLogProvider.Writer writer = entry.getValue(); + if (writer != null) { + LOG.debug("{} Closing WAL writer for day: {}", Utils.logPeerId(peerId), entry.getKey()); + try { + writer.close(); + LOG.debug("{} Successfully closed WAL writer for day: {}", Utils.logPeerId(peerId), + entry.getKey()); + } catch (IOException e) { + LOG.error("{} Failed to close WAL writer for day: {}. Error: {}", Utils.logPeerId(peerId), + entry.getKey(), e.getMessage(), e); + throw e; + } + } + } + walWriters.clear(); + LOG.info("{} WAL writers flushed and cleared", Utils.logPeerId(peerId)); + } + + @Override + public UUID getPeerUUID() { + return peerUUID; + } + + @Override + public void start() { + LOG.info("{} Starting ContinuousBackupReplicationEndpoint", Utils.logPeerId(peerId)); + startAsync(); + } + + @Override + protected void doStart() { + LOG.info("{} ContinuousBackupReplicationEndpoint started successfully.", + Utils.logPeerId(peerId)); + notifyStarted(); + } + + @Override + public ReplicationResult replicate(ReplicateContext replicateContext) { + final List entries = replicateContext.getEntries(); + if (entries.isEmpty()) { + LOG.debug("{} No WAL entries to replicate", Utils.logPeerId(peerId)); + return ReplicationResult.SUBMITTED; + } + + LOG.debug("{} Received {} WAL entries for replication", Utils.logPeerId(peerId), + entries.size()); + + Map> groupedEntries = groupEntriesByDay(entries); + LOG.debug("{} Grouped WAL entries by day: {}", Utils.logPeerId(peerId), + groupedEntries.keySet()); + + lock.lock(); + try { + for (Map.Entry> entry : groupedEntries.entrySet()) { + LOG.debug("{} Backing up {} WAL entries for day {}", Utils.logPeerId(peerId), + entry.getValue().size(), entry.getKey()); + backupWalEntries(entry.getKey(), entry.getValue()); + } + + if (isAnyWriterFull()) { + LOG.debug("{} Some WAL writers reached max size, triggering flush", + Utils.logPeerId(peerId)); + flushWriters(); + LOG.debug("{} Replication committed after WAL flush", Utils.logPeerId(peerId)); + return ReplicationResult.COMMITTED; + } + + LOG.debug("{} Replication submitted successfully", Utils.logPeerId(peerId)); + return ReplicationResult.SUBMITTED; + } catch (IOException e) { + LOG.error("{} Replication failed. Error details: {}", Utils.logPeerId(peerId), e.getMessage(), + e); + return ReplicationResult.FAILED; + } finally { + lock.unlock(); + } + } + + private Map> groupEntriesByDay(List entries) { + return entries.stream().collect( + Collectors.groupingBy(entry -> (entry.getKey().getWriteTime() / ONE_DAY_IN_MILLISECONDS) + * ONE_DAY_IN_MILLISECONDS)); + } + + private boolean isAnyWriterFull() { + return walWriters.values().stream().anyMatch(this::isWriterFull); + } + + private boolean isWriterFull(FSHLogProvider.Writer writer) { + long maxWalSize = conf.getLong(CONF_BACKUP_MAX_WAL_SIZE, DEFAULT_MAX_WAL_SIZE); + return writer.getLength() >= maxWalSize; + } + + private void backupWalEntries(long day, List walEntries) throws IOException { + LOG.debug("{} Starting backup of {} WAL entries for day {}", Utils.logPeerId(peerId), + walEntries.size(), day); + + try { + FSHLogProvider.Writer walWriter = walWriters.computeIfAbsent(day, this::createWalWriter); + List bulkLoadFiles = BulkLoadProcessor.processBulkLoadFiles(walEntries); + + if (LOG.isTraceEnabled()) { + LOG.trace("{} Processed {} bulk load files for WAL entries", Utils.logPeerId(peerId), + bulkLoadFiles.size()); + LOG.trace("{} Bulk load files: {}", Utils.logPeerId(peerId), + bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); + } + + for (WAL.Entry entry : walEntries) { + walWriter.append(entry); + } + walWriter.sync(true); + uploadBulkLoadFiles(bulkLoadFiles); + } catch (UncheckedIOException e) { + String errorMsg = Utils.logPeerId(peerId) + " Failed to get or create WAL Writer for " + day; + LOG.error("{} Backup failed for day {}. Error: {}", Utils.logPeerId(peerId), day, + e.getMessage(), e); + throw new IOException(errorMsg, e); + } + } + + private FSHLogProvider.Writer createWalWriter(long dayInMillis) { + // Convert dayInMillis to "yyyy-MM-dd" format + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + String dayDirectoryName = dateFormat.format(new Date(dayInMillis)); + + FileSystem fs = backupFileSystemManager.getBackupFs(); + Path walsDir = backupFileSystemManager.getWalsDir(); + + try { + // Create a directory for the day + Path dayDir = new Path(walsDir, dayDirectoryName); + fs.mkdirs(dayDir); + + // Generate a unique WAL file name + long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); + String walFileName = WAL_FILE_PREFIX + currentTime + "." + UUID.randomUUID(); + Path walFilePath = new Path(dayDir, walFileName); + + // Initialize the WAL writer + FSHLogProvider.Writer writer = + ObjectStoreProtobufWalWriter.class.getDeclaredConstructor().newInstance(); + writer.init(fs, walFilePath, conf, true, WALUtil.getWALBlockSize(conf, fs, walFilePath), + StreamSlowMonitor.create(conf, walFileName)); + + LOG.info("{} WAL writer created: {}", Utils.logPeerId(peerId), walFilePath); + return writer; + } catch (Exception e) { + throw new UncheckedIOException( + Utils.logPeerId(peerId) + " Failed to initialize WAL Writer for day: " + dayDirectoryName, + new IOException(e)); + } + } + + @Override + public void stop() { + LOG.info("{} Stopping ContinuousBackupReplicationEndpoint...", Utils.logPeerId(peerId)); + stopAsync(); + } + + @Override + protected void doStop() { + close(); + LOG.info("{} ContinuousBackupReplicationEndpoint stopped successfully.", + Utils.logPeerId(peerId)); + notifyStopped(); + } + + private void close() { + LOG.info("{} Closing WAL replication component...", Utils.logPeerId(peerId)); + shutdownFlushExecutor(); + lock.lock(); + try { + flushWriters(); + replicationSource.persistOffsets(); + } catch (IOException e) { + LOG.error("{} Failed to Flush Open Wal Writers: {}", Utils.logPeerId(peerId), e.getMessage(), + e); + } finally { + lock.unlock(); + LOG.info("{} WAL replication component closed.", Utils.logPeerId(peerId)); + } + } + + private void uploadBulkLoadFiles(List bulkLoadFiles) throws IOException { + LOG.debug("{} Starting upload of {} bulk load files", Utils.logPeerId(peerId), + bulkLoadFiles.size()); + + if (LOG.isTraceEnabled()) { + LOG.trace("{} Bulk load files to upload: {}", Utils.logPeerId(peerId), + bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); + } + for (Path file : bulkLoadFiles) { + Path sourcePath = getBulkLoadFileStagingPath(file); + Path destPath = new Path(backupFileSystemManager.getBulkLoadFilesDir(), file); + + try { + LOG.debug("{} Copying bulk load file from {} to {}", Utils.logPeerId(peerId), sourcePath, + destPath); + + FileUtil.copy(CommonFSUtils.getRootDirFileSystem(conf), 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.getMessage(), e); + throw e; + } + } + + LOG.debug("{} Completed upload of bulk load files", Utils.logPeerId(peerId)); + } + + private Path getBulkLoadFileStagingPath(Path relativePathFromNamespace) throws IOException { + FileSystem rootFs = CommonFSUtils.getRootDirFileSystem(conf); + Path rootDir = CommonFSUtils.getRootDir(conf); + Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR); + Path baseNamespaceDir = new Path(rootDir, baseNSDir); + Path hFileArchiveDir = + new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir)); + + LOG.debug("{} Searching for bulk load file: {} in paths: {}, {}", Utils.logPeerId(peerId), + relativePathFromNamespace, baseNamespaceDir, hFileArchiveDir); + + Path result = + findExistingPath(rootFs, baseNamespaceDir, hFileArchiveDir, relativePathFromNamespace); + + if (result == null) { + LOG.error("{} No bulk loaded file found in relative path: {}", Utils.logPeerId(peerId), + relativePathFromNamespace); + throw new IOException( + "No Bulk loaded file found in relative path: " + relativePathFromNamespace); + } + + LOG.debug("{} Bulk load file found at {}", Utils.logPeerId(peerId), result); + return result; + } + + private static Path findExistingPath(FileSystem rootFs, Path baseNamespaceDir, + Path hFileArchiveDir, Path filePath) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Checking for bulk load file at: {} and {}", new Path(baseNamespaceDir, filePath), + new Path(hFileArchiveDir, filePath)); + } + + for (Path candidate : new Path[] { new Path(baseNamespaceDir, filePath), + new Path(hFileArchiveDir, filePath) }) { + if (rootFs.exists(candidate)) { + LOG.debug("Found bulk load file at: {}", candidate); + return candidate; + } + } + return null; + } + + private void shutdownFlushExecutor() { + if (flushExecutor != null) { + LOG.info("{} Initiating WAL flush executor shutdown.", Utils.logPeerId(peerId)); + + flushExecutor.shutdown(); + try { + if ( + !flushExecutor.awaitTermination(EXECUTOR_TERMINATION_TIMEOUT_SECONDS, TimeUnit.SECONDS) + ) { + LOG.warn("{} Flush executor did not terminate within timeout, forcing shutdown.", + Utils.logPeerId(peerId)); + flushExecutor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + flushExecutor.shutdownNow(); + LOG.warn("{} Flush executor shutdown was interrupted.", Utils.logPeerId(peerId), e); + } + LOG.info("{} WAL flush thread stopped.", Utils.logPeerId(peerId)); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ObjectStoreProtobufWalWriter.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ObjectStoreProtobufWalWriter.java new file mode 100644 index 000000000000..27f4fbdc027e --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ObjectStoreProtobufWalWriter.java @@ -0,0 +1,73 @@ +/* + * 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.concurrent.atomic.AtomicLong; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; +import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter; +import org.apache.hadoop.hbase.util.AtomicUtils; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A custom implementation of {@link ProtobufLogWriter} that provides support for writing + * protobuf-based WAL (Write-Ahead Log) entries to object store-backed files. + *

+ * This class overrides the {@link ProtobufLogWriter#sync(boolean)} and + * {@link ProtobufLogWriter#initOutput(FileSystem, Path, boolean, int, short, long, StreamSlowMonitor, boolean)} + * methods to ensure compatibility with object stores, while ignoring specific capability checks + * such as HFLUSH and HSYNC. These checks are often not supported by some object stores, and + * bypassing them ensures smooth operation in such environments. + *

+ */ +@InterfaceAudience.Private +public class ObjectStoreProtobufWalWriter extends ProtobufLogWriter { + private final AtomicLong syncedLength = new AtomicLong(0); + + @Override + public void sync(boolean forceSync) throws IOException { + FSDataOutputStream fsDataOutputstream = this.output; + if (fsDataOutputstream == null) { + return; // Presume closed + } + // Special case for Hadoop S3: Unlike traditional file systems, where flush() ensures data is + // durably written, in Hadoop S3, flush() only writes data to the internal buffer and does not + // immediately persist it to S3. The actual upload to S3 happens asynchronously, typically when + // a block is full or when close() is called, which finalizes the upload process. + fsDataOutputstream.flush(); + AtomicUtils.updateMax(this.syncedLength, fsDataOutputstream.getPos()); + } + + @Override + protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize, + short replication, long blockSize, StreamSlowMonitor monitor, boolean noLocalWrite) + throws IOException { + try { + super.initOutput(fs, path, overwritable, bufferSize, replication, blockSize, monitor, + noLocalWrite); + } catch (CommonFSUtils.StreamLacksCapabilityException e) { + // Ignore capability check for HFLUSH and HSYNC capabilities + // Some object stores may not support these capabilities, so we bypass the exception handling + // to ensure compatibility with such stores. + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/Utils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/Utils.java new file mode 100644 index 000000000000..69365674acca --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/Utils.java @@ -0,0 +1,30 @@ +/* + * 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 final class Utils { + private Utils() { + } + + public static String logPeerId(String peerId) { + return "[Source for peer " + peerId + "]:"; + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java new file mode 100644 index 000000000000..cd1f758f7607 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java @@ -0,0 +1,513 @@ +/* + * 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 static org.apache.hadoop.hbase.HConstants.REPLICATION_BULKLOAD_ENABLE_KEY; +import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_MAX_WAL_SIZE; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_ROOT_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_PEER_UUID; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INITIAL_DELAY; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INTERVAL; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.WAL_FILE_PREFIX; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.tool.BulkLoadHFiles; +import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; +import org.apache.hadoop.hbase.util.HFileTestUtil; +import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.apache.hadoop.util.ToolRunner; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestContinuousBackupReplicationEndpoint { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestContinuousBackupReplicationEndpoint.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestContinuousBackupReplicationEndpoint.class); + + private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final Configuration conf = TEST_UTIL.getConfiguration(); + private static Admin admin; + + private final String replicationEndpoint = ContinuousBackupReplicationEndpoint.class.getName(); + private static final String CF_NAME = "cf"; + private static final byte[] QUALIFIER = Bytes.toBytes("my-qualifier"); + static FileSystem fs = null; + static Path root; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + // Set the configuration properties as required + conf.setBoolean(REPLICATION_BULKLOAD_ENABLE_KEY, true); + conf.set(REPLICATION_CLUSTER_ID, "clusterId1"); + + TEST_UTIL.startMiniZKCluster(); + TEST_UTIL.startMiniCluster(3); + fs = FileSystem.get(conf); + root = TEST_UTIL.getDataTestDirOnTestFS(); + admin = TEST_UTIL.getAdmin(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + if (fs != null) { + fs.close(); + } + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testWALAndBulkLoadFileBackup() throws IOException { + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + String peerId = "peerId"; + + createTable(tableName); + + Path backupRootDir = new Path(root, methodName); + fs.mkdirs(backupRootDir); + + Map> tableMap = new HashMap<>(); + tableMap.put(tableName, new ArrayList<>()); + + addReplicationPeer(peerId, backupRootDir, tableMap); + + loadRandomData(tableName, 100); + assertEquals(100, getRowCount(tableName)); + + Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoadByFamily"); + generateHFiles(dir); + bulkLoadHFiles(tableName, dir); + assertEquals(1100, getRowCount(tableName)); + + waitForReplication(15000); + deleteReplicationPeer(peerId); + + verifyBackup(backupRootDir.toString(), true, Map.of(tableName, 1100)); + + deleteTable(tableName); + } + + @Test + public void testMultiTableWALBackup() throws IOException { + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName table1 = TableName.valueOf("table_" + methodName + "1"); + TableName table2 = TableName.valueOf("table_" + methodName + "2"); + TableName table3 = TableName.valueOf("table_" + methodName + "3"); + String peerId = "peerMulti"; + + for (TableName table : List.of(table1, table2, table3)) { + createTable(table); + } + + Path backupRootDir = new Path(root, methodName); + fs.mkdirs(backupRootDir); + + Map> initialTableMap = new HashMap<>(); + initialTableMap.put(table1, new ArrayList<>()); + initialTableMap.put(table2, new ArrayList<>()); + + addReplicationPeer(peerId, backupRootDir, initialTableMap); + + for (TableName table : List.of(table1, table2, table3)) { + loadRandomData(table, 50); + assertEquals(50, getRowCount(table)); + } + + waitForReplication(15000); + + // Update the Replication Peer to Include table3 + admin.updateReplicationPeerConfig(peerId, + ReplicationPeerConfig.newBuilder(admin.getReplicationPeerConfig(peerId)) + .setTableCFsMap( + Map.of(table1, new ArrayList<>(), table2, new ArrayList<>(), table3, new ArrayList<>())) + .build()); + + for (TableName table : List.of(table1, table2, table3)) { + loadRandomData(table, 50); + assertEquals(100, getRowCount(table)); + } + + waitForReplication(15000); + deleteReplicationPeer(peerId); + + verifyBackup(backupRootDir.toString(), false, Map.of(table1, 100, table2, 100, table3, 50)); + + for (TableName table : List.of(table1, table2, table3)) { + deleteTable(table); + } + } + + @Test + public void testWALBackupWithPeerRestart() throws IOException, InterruptedException { + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + String peerId = "peerId"; + + createTable(tableName); + + Path backupRootDir = new Path(root, methodName); + fs.mkdirs(backupRootDir); + + Map> tableMap = new HashMap<>(); + tableMap.put(tableName, new ArrayList<>()); + + addReplicationPeer(peerId, backupRootDir, tableMap); + + AtomicBoolean stopLoading = new AtomicBoolean(false); + + // Start a separate thread to load data continuously + Thread dataLoaderThread = new Thread(() -> { + try { + while (!stopLoading.get()) { + loadRandomData(tableName, 10); + Thread.sleep(1000); // Simulate delay + } + } catch (Exception e) { + LOG.error("Data loading thread encountered an error", e); + } + }); + + dataLoaderThread.start(); + + // Main thread enables and disables replication peer + try { + for (int i = 0; i < 5; i++) { + LOG.info("Disabling replication peer..."); + admin.disableReplicationPeer(peerId); + Thread.sleep(2000); + + LOG.info("Enabling replication peer..."); + admin.enableReplicationPeer(peerId); + Thread.sleep(2000); + } + } finally { + stopLoading.set(true); // Stop the data loader thread + dataLoaderThread.join(); + } + + waitForReplication(20000); + deleteReplicationPeer(peerId); + + verifyBackup(backupRootDir.toString(), false, Map.of(tableName, getRowCount(tableName))); + + deleteTable(tableName); + } + + @Test + public void testDayWiseWALBackup() throws IOException { + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + String peerId = "peerId"; + + createTable(tableName); + + Path backupRootDir = new Path(root, methodName); + fs.mkdirs(backupRootDir); + + Map> tableMap = new HashMap<>(); + tableMap.put(tableName, new ArrayList<>()); + + addReplicationPeer(peerId, backupRootDir, tableMap); + + // Mock system time using ManualEnvironmentEdge + ManualEnvironmentEdge manualEdge = new ManualEnvironmentEdge(); + EnvironmentEdgeManagerTestHelper.injectEdge(manualEdge); + + long currentTime = System.currentTimeMillis(); + long oneDayBackTime = currentTime - ONE_DAY_IN_MILLISECONDS; + + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + String expectedPrevDayDir = dateFormat.format(new Date(oneDayBackTime)); + String expectedCurrentDayDir = dateFormat.format(new Date(currentTime)); + + manualEdge.setValue(oneDayBackTime); + loadRandomData(tableName, 100); + assertEquals(100, getRowCount(tableName)); + + manualEdge.setValue(currentTime); + loadRandomData(tableName, 100); + assertEquals(200, getRowCount(tableName)); + + // Reset time mocking + EnvironmentEdgeManagerTestHelper.reset(); + + waitForReplication(15000); + deleteReplicationPeer(peerId); + + verifyBackup(backupRootDir.toString(), false, Map.of(tableName, 200)); + + // Verify that WALs are stored in two directories, one for each day + Path walDir = new Path(backupRootDir, WALS_DIR); + Set walDirectories = new HashSet<>(); + + FileStatus[] fileStatuses = fs.listStatus(walDir); + for (FileStatus fileStatus : fileStatuses) { + if (fileStatus.isDirectory()) { + String dirName = fileStatus.getPath().getName(); + walDirectories.add(dirName); + } + } + + assertEquals("WALs should be stored in exactly two directories", 2, walDirectories.size()); + assertTrue("Expected previous day's WAL directory missing", + walDirectories.contains(expectedPrevDayDir)); + assertTrue("Expected current day's WAL directory missing", + walDirectories.contains(expectedCurrentDayDir)); + + deleteTable(tableName); + } + + private void createTable(TableName tableName) throws IOException { + ColumnFamilyDescriptor columnFamilyDescriptor = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CF_NAME)).setScope(1).build(); + TableDescriptor tableDescriptor = + TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(columnFamilyDescriptor).build(); + + if (!admin.tableExists(tableName)) { + admin.createTable(tableDescriptor); + } + } + + private void deleteTable(TableName tableName) throws IOException { + admin.disableTable(tableName); + admin.truncateTable(tableName, false); + admin.disableTable(tableName); + admin.deleteTable(tableName); + } + + private void addReplicationPeer(String peerId, Path backupRootDir, + Map> tableMap) throws IOException { + Map additionalArgs = new HashMap<>(); + additionalArgs.put(CONF_PEER_UUID, UUID.randomUUID().toString()); + additionalArgs.put(CONF_BACKUP_ROOT_DIR, backupRootDir.toString()); + additionalArgs.put(CONF_BACKUP_MAX_WAL_SIZE, "10240"); + additionalArgs.put(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, "10"); + additionalArgs.put(CONF_STAGED_WAL_FLUSH_INTERVAL, "10"); + + ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() + .setReplicationEndpointImpl(replicationEndpoint).setReplicateAllUserTables(false) + .setTableCFsMap(tableMap).putAllConfiguration(additionalArgs).build(); + + admin.addReplicationPeer(peerId, peerConfig); + } + + private void deleteReplicationPeer(String peerId) throws IOException { + admin.disableReplicationPeer(peerId); + admin.removeReplicationPeer(peerId); + } + + private void loadRandomData(TableName tableName, int totalRows) throws IOException { + int rowSize = 32; + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + TEST_UTIL.loadRandomRows(table, Bytes.toBytes(CF_NAME), rowSize, totalRows); + } + } + + private void bulkLoadHFiles(TableName tableName, Path inputDir) throws IOException { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true); + + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + BulkLoadHFiles loader = new BulkLoadHFilesTool(TEST_UTIL.getConfiguration()); + loader.bulkLoad(table.getName(), inputDir); + } finally { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false); + } + } + + private void bulkLoadHFiles(TableName tableName, Map> family2Files) + throws IOException { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true); + + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + BulkLoadHFiles loader = new BulkLoadHFilesTool(TEST_UTIL.getConfiguration()); + loader.bulkLoad(table.getName(), family2Files); + } finally { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false); + } + } + + private void generateHFiles(Path outputDir) throws IOException { + String hFileName = "MyHFile"; + int numRows = 1000; + outputDir = outputDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()); + + byte[] from = Bytes.toBytes(CF_NAME + "begin"); + byte[] to = Bytes.toBytes(CF_NAME + "end"); + + Path familyDir = new Path(outputDir, CF_NAME); + HFileTestUtil.createHFile(TEST_UTIL.getConfiguration(), fs, new Path(familyDir, hFileName), + Bytes.toBytes(CF_NAME), QUALIFIER, from, to, numRows); + } + + private void waitForReplication(int durationInMillis) { + LOG.info("Waiting for replication to complete for {} ms", durationInMillis); + try { + Thread.sleep(durationInMillis); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Thread was interrupted while waiting", e); + } + } + + /** + * Verifies the backup process by: 1. Checking whether any WAL (Write-Ahead Log) files were + * generated in the backup directory. 2. Checking whether any bulk-loaded files were generated in + * the backup directory. 3. Replaying the WAL and bulk-loaded files (if present) to restore data + * and check consistency by verifying that the restored data matches the expected row count for + * each table. + */ + private void verifyBackup(String backupRootDir, boolean hasBulkLoadFiles, + Map tablesWithExpectedRows) throws IOException { + verifyWALBackup(backupRootDir); + if (hasBulkLoadFiles) { + verifyBulkLoadBackup(backupRootDir); + } + + for (Map.Entry entry : tablesWithExpectedRows.entrySet()) { + TableName tableName = entry.getKey(); + int expectedRows = entry.getValue(); + + admin.disableTable(tableName); + admin.truncateTable(tableName, false); + assertEquals(0, getRowCount(tableName)); + + replayWALs(new Path(backupRootDir, WALS_DIR).toString(), tableName); + replayBulkLoadHFilesIfPresent(new Path(backupRootDir, BULKLOAD_FILES_DIR).toString(), + tableName); + assertEquals(expectedRows, getRowCount(tableName)); + } + } + + private void verifyWALBackup(String backupRootDir) throws IOException { + Path walDir = new Path(backupRootDir, WALS_DIR); + assertTrue("WAL directory does not exist!", fs.exists(walDir)); + + RemoteIterator fileStatusIterator = fs.listFiles(walDir, true); + List walFiles = new ArrayList<>(); + + while (fileStatusIterator.hasNext()) { + LocatedFileStatus fileStatus = fileStatusIterator.next(); + Path filePath = fileStatus.getPath(); + + // Check if the file starts with the expected WAL prefix + if (!fileStatus.isDirectory() && filePath.getName().startsWith(WAL_FILE_PREFIX)) { + walFiles.add(filePath); + } + } + + assertNotNull("No WAL files found!", walFiles); + assertFalse("Expected some WAL files but found none!", walFiles.isEmpty()); + } + + private void verifyBulkLoadBackup(String backupRootDir) throws IOException { + Path bulkLoadFilesDir = new Path(backupRootDir, BULKLOAD_FILES_DIR); + assertTrue("BulkLoad Files directory does not exist!", fs.exists(bulkLoadFilesDir)); + + FileStatus[] bulkLoadFiles = fs.listStatus(bulkLoadFilesDir); + assertNotNull("No Bulk load files found!", bulkLoadFiles); + assertTrue("Expected some Bulk load files but found none!", bulkLoadFiles.length > 0); + } + + private void replayWALs(String walDir, TableName tableName) { + WALPlayer player = new WALPlayer(); + try { + assertEquals(0, ToolRunner.run(TEST_UTIL.getConfiguration(), player, + new String[] { walDir, tableName.getQualifierAsString() })); + } catch (Exception e) { + fail("Failed to replay WALs properly: " + e.getMessage()); + } + } + + private void replayBulkLoadHFilesIfPresent(String bulkLoadDir, TableName tableName) { + try { + Path tableBulkLoadDir = new Path(bulkLoadDir + "/default/" + tableName); + if (fs.exists(tableBulkLoadDir)) { + RemoteIterator fileStatusIterator = fs.listFiles(tableBulkLoadDir, true); + List bulkLoadFiles = new ArrayList<>(); + + while (fileStatusIterator.hasNext()) { + LocatedFileStatus fileStatus = fileStatusIterator.next(); + Path filePath = fileStatus.getPath(); + + if (!fileStatus.isDirectory()) { + bulkLoadFiles.add(filePath); + } + } + bulkLoadHFiles(tableName, Map.of(Bytes.toBytes(CF_NAME), bulkLoadFiles)); + } + } catch (Exception e) { + fail("Failed to replay BulkLoad HFiles properly: " + e.getMessage()); + } + } + + private int getRowCount(TableName tableName) throws IOException { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + return HBaseTestingUtil.countRows(table); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java index 5edd5b3e8c92..fc5c2bf62659 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.replication.regionserver.MetricsSource; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.yetus.audience.InterfaceAudience; @@ -51,6 +52,7 @@ public interface ReplicationEndpoint extends ReplicationPeerConfigListener { @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION) class Context { + private final ReplicationSourceInterface replicationSource; private final Server server; private final Configuration localConf; private final Configuration conf; @@ -63,10 +65,12 @@ class Context { private final Abortable abortable; @InterfaceAudience.Private - public Context(final Server server, final Configuration localConf, final Configuration conf, - final FileSystem fs, final String peerId, final UUID clusterId, - final ReplicationPeer replicationPeer, final MetricsSource metrics, - final TableDescriptors tableDescriptors, final Abortable abortable) { + public Context(final ReplicationSourceInterface replicationSource, final Server server, + final Configuration localConf, final Configuration conf, final FileSystem fs, + final String peerId, final UUID clusterId, final ReplicationPeer replicationPeer, + final MetricsSource metrics, final TableDescriptors tableDescriptors, + final Abortable abortable) { + this.replicationSource = replicationSource; this.server = server; this.localConf = localConf; this.conf = conf; @@ -79,6 +83,10 @@ public Context(final Server server, final Configuration localConf, final Configu this.abortable = abortable; } + public ReplicationSourceInterface getReplicationSource() { + return replicationSource; + } + public Server getServer() { return server; } @@ -208,7 +216,7 @@ public int getTimeout() { * the context are assumed to be persisted in the target cluster. * @param replicateContext a context where WAL entries and other parameters can be obtained. */ - boolean replicate(ReplicateContext replicateContext); + ReplicationResult replicate(ReplicateContext replicateContext); // The below methods are inspired by Guava Service. See // https://github.com/google/guava/wiki/ServiceExplained for overview of Guava Service. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationResult.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationResult.java new file mode 100644 index 000000000000..03ed0ce6799f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationResult.java @@ -0,0 +1,33 @@ +/* + * 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.replication; + +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION) +public enum ReplicationResult { + /* Batch has been replicated and persisted successfully. */ + COMMITTED, + + /* Batch has been submitted for replication, but not persisted yet. */ + SUBMITTED, + + /* Batch replicaton failed, should be re-tried */ + FAILED +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/VerifyWALEntriesReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/VerifyWALEntriesReplicationEndpoint.java index 229cec57e976..a9674407bd2a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/VerifyWALEntriesReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/VerifyWALEntriesReplicationEndpoint.java @@ -59,10 +59,10 @@ private void checkCell(Cell cell) { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { replicateContext.entries.stream().map(WAL.Entry::getEdit).flatMap(e -> e.getCells().stream()) .forEach(this::checkCell); - return true; + return ReplicationResult.COMMITTED; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java index 6bdc97732644..4f9a4909d784 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; @@ -424,7 +425,7 @@ private long parallelReplicate(ReplicateContext replicateContext, List> batches = createBatches(replicateContext.getEntries()); @@ -458,7 +459,7 @@ public boolean replicate(ReplicateContext replicateContext) { try { // replicate the batches to sink side. parallelReplicate(replicateContext, batches); - return true; + return ReplicationResult.COMMITTED; } catch (IOException ioe) { if (ioe instanceof RemoteException) { if (dropOnDeletedTables && isTableNotFoundException(ioe)) { @@ -467,14 +468,14 @@ public boolean replicate(ReplicateContext replicateContext) { batches = filterNotExistTableEdits(batches); if (batches.isEmpty()) { LOG.warn("After filter not exist table's edits, 0 edits to replicate, just return"); - return true; + return ReplicationResult.COMMITTED; } } else if (dropOnDeletedColumnFamilies && isNoSuchColumnFamilyException(ioe)) { batches = filterNotExistColumnFamilyEdits(batches); if (batches.isEmpty()) { LOG.warn("After filter not exist column family's edits, 0 edits to replicate, " + "just return"); - return true; + return ReplicationResult.COMMITTED; } } else { LOG.warn("{} Peer encountered RemoteException, rechecking all sinks: ", logPeerId(), @@ -506,7 +507,7 @@ public boolean replicate(ReplicateContext replicateContext) { } } } - return false; // in case we exited before replicating + return ReplicationResult.FAILED; // in case we exited before replicating } protected boolean isPeerEnabled() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index dc17ed12ff0a..73421dced454 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueData; import org.apache.hadoop.hbase.replication.ReplicationQueueId; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter; import org.apache.hadoop.hbase.replication.WALEntryFilter; import org.apache.hadoop.hbase.util.Bytes; @@ -166,6 +167,8 @@ public class ReplicationSource implements ReplicationSourceInterface { */ private final List baseFilterOutWALEntries; + private final Map lastEntryBatch = new ConcurrentHashMap<>(); + ReplicationSource() { // Default, filters *in* all WALs but meta WALs & filters *out* all WALEntries of System Tables. this(p -> !AbstractFSWALProvider.isMetaFile(p), @@ -318,8 +321,8 @@ private void initAndStartReplicationEndpoint(ReplicationEndpoint replicationEndp if (server instanceof HRegionServer) { tableDescriptors = ((HRegionServer) server).getTableDescriptors(); } - replicationEndpoint - .init(new ReplicationEndpoint.Context(server, conf, replicationPeer.getConfiguration(), fs, + replicationEndpoint.init( + new ReplicationEndpoint.Context(this, server, conf, replicationPeer.getConfiguration(), fs, replicationPeer.getId(), clusterId, replicationPeer, metrics, tableDescriptors, server)); replicationEndpoint.start(); replicationEndpoint.awaitRunning(waitOnEndpointSeconds, TimeUnit.SECONDS); @@ -862,4 +865,32 @@ public String logPeerId() { public long getTotalReplicatedEdits() { return totalReplicatedEdits.get(); } + + @Override + public void logPositionAndCleanOldLogs(WALEntryBatch entryBatch, ReplicationResult replicated) { + String walName = entryBatch.getLastWalPath().getName(); + String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(walName); + + synchronized (lastEntryBatch) { // Synchronize addition and processing + lastEntryBatch.put(walPrefix, entryBatch); + + if (replicated == ReplicationResult.COMMITTED) { + processAndClearEntries(); + } + } + } + + public void persistOffsets() { + synchronized (lastEntryBatch) { + processAndClearEntries(); + } + } + + private void processAndClearEntries() { + // Process all entries + lastEntryBatch + .forEach((prefix, batch) -> getSourceManager().logPositionAndCleanOldLogs(this, batch)); + // Clear all processed entries + lastEntryBatch.clear(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java index 69ad2887064a..f482cc73e717 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueData; import org.apache.hadoop.hbase.replication.ReplicationQueueId; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.yetus.audience.InterfaceAudience; @@ -207,7 +208,11 @@ default boolean isRecovered() { * @param entryBatch the wal entry batch we just shipped * @return The instance of queueStorage used by this ReplicationSource. */ - default void logPositionAndCleanOldLogs(WALEntryBatch entryBatch) { - getSourceManager().logPositionAndCleanOldLogs(this, entryBatch); + default void logPositionAndCleanOldLogs(WALEntryBatch entryBatch, ReplicationResult replicated) { + + } + + default public void persistOffsets() { + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java index 6d0730d76b6e..ee819faa77b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WAL.Entry; @@ -155,7 +156,7 @@ private void shipEdits(WALEntryBatch entryBatch) { List entries = entryBatch.getWalEntries(); int sleepMultiplier = 0; if (entries.isEmpty()) { - updateLogPosition(entryBatch); + updateLogPosition(entryBatch, ReplicationResult.COMMITTED); return; } int currentSize = (int) entryBatch.getHeapSize(); @@ -182,21 +183,23 @@ private void shipEdits(WALEntryBatch entryBatch) { long startTimeNs = System.nanoTime(); // send the edits to the endpoint. Will block until the edits are shipped and acknowledged - boolean replicated = source.getReplicationEndpoint().replicate(replicateContext); + ReplicationResult replicated = source.getReplicationEndpoint().replicate(replicateContext); long endTimeNs = System.nanoTime(); - if (!replicated) { + if (replicated == ReplicationResult.FAILED) { continue; } else { sleepMultiplier = Math.max(sleepMultiplier - 1, 0); } - // Clean up hfile references - for (Entry entry : entries) { - cleanUpHFileRefs(entry.getEdit()); - LOG.trace("shipped entry {}: ", entry); + if (replicated == ReplicationResult.COMMITTED) { + // Clean up hfile references + for (Entry entry : entries) { + cleanUpHFileRefs(entry.getEdit()); + LOG.trace("shipped entry {}: ", entry); + } } // Log and clean up WAL logs - updateLogPosition(entryBatch); + updateLogPosition(entryBatch, replicated); // offsets totalBufferUsed by deducting shipped batchSize (excludes bulk load size) // this sizeExcludeBulkLoad has to use same calculation that when calling @@ -253,7 +256,7 @@ private void cleanUpHFileRefs(WALEdit edit) throws IOException { } } - private boolean updateLogPosition(WALEntryBatch batch) { + private boolean updateLogPosition(WALEntryBatch batch, ReplicationResult replicated) { boolean updated = false; // if end of file is true, then the logPositionAndCleanOldLogs method will remove the file // record on zk, so let's call it. The last wal position maybe zero if end of file is true and @@ -263,7 +266,7 @@ private boolean updateLogPosition(WALEntryBatch batch) { batch.isEndOfFile() || !batch.getLastWalPath().equals(currentPath) || batch.getLastWalPosition() != currentPosition ) { - source.logPositionAndCleanOldLogs(batch); + source.logPositionAndCleanOldLogs(batch, replicated); updated = true; } // if end of file is true, then we can just skip to the next file in queue. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java index b97a08c01c38..a32ce78b0c78 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.WALEntryFilter; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALEdit; @@ -63,7 +64,7 @@ public void peerConfigUpdated(ReplicationPeerConfig rpc) { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { if (!delegator.canReplicateToSameCluster()) { // Only when the replication is inter cluster replication we need to // convert the visibility tags to diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DummyReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DummyReplicationEndpoint.java index e6a39e7fede1..f0e627316cd4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DummyReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DummyReplicationEndpoint.java @@ -42,8 +42,8 @@ public WALEntryFilter getWALEntryfilter() { } @Override - public boolean replicate(ReplicateContext replicateContext) { - return true; + public ReplicationResult replicate(ReplicateContext replicateContext) { + return ReplicationResult.COMMITTED; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java index f54c39316997..a8c76033d02d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java @@ -81,7 +81,7 @@ public UUID getPeerUUID() { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { synchronized (WRITER) { try { for (Entry entry : replicateContext.getEntries()) { @@ -92,7 +92,7 @@ public boolean replicate(ReplicateContext replicateContext) { throw new UncheckedIOException(e); } } - return true; + return ReplicationResult.COMMITTED; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestHBaseReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestHBaseReplicationEndpoint.java index 058564dc0ecf..d9a75b8ca8a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestHBaseReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestHBaseReplicationEndpoint.java @@ -59,8 +59,8 @@ public void setUp() throws Exception { when(replicationPeer.getPeerConfig()).thenReturn(peerConfig); when(peerConfig.getClusterKey()).thenReturn("hbase+zk://server1:2181/hbase"); ReplicationEndpoint.Context context = - new ReplicationEndpoint.Context(null, UTIL.getConfiguration(), UTIL.getConfiguration(), null, - null, null, replicationPeer, null, null, null); + new ReplicationEndpoint.Context(null, null, UTIL.getConfiguration(), UTIL.getConfiguration(), + null, null, null, replicationPeer, null, null, null); endpoint = new DummyHBaseReplicationEndpoint(); endpoint.init(context); } @@ -199,8 +199,8 @@ protected Collection fetchPeerAddresses() { } @Override - public boolean replicate(ReplicateContext replicateContext) { - return false; + public ReplicationResult replicate(ReplicateContext replicateContext) { + return ReplicationResult.FAILED; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java index 70cae18b4561..c98b46c8e4be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNonHBaseReplicationEndpoint.java @@ -127,9 +127,9 @@ public WALEntryFilter getWALEntryfilter() { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { REPLICATED.set(true); - return true; + return ReplicationResult.COMMITTED; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java index b03b89d6d69a..a8bea44750fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java @@ -430,7 +430,7 @@ public ReplicationEndpointTest() { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { replicateCount.incrementAndGet(); replicatedEntries.addAll(replicateContext.getEntries()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java index 057a9f3567f5..77cd5da8de0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java @@ -463,10 +463,10 @@ public UUID getPeerUUID() { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { replicateCount.incrementAndGet(); lastEntries = new ArrayList<>(replicateContext.entries); - return true; + return ReplicationResult.COMMITTED; } @Override @@ -526,12 +526,12 @@ public void init(Context context) throws IOException { } @Override - public boolean replicate(ReplicateContext context) { + public ReplicationResult replicate(ReplicateContext context) { try { Thread.sleep(duration); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - return false; + return ReplicationResult.FAILED; } return super.replicate(context); } @@ -548,9 +548,9 @@ public InterClusterReplicationEndpointForTest() { } @Override - public boolean replicate(ReplicateContext replicateContext) { - boolean success = super.replicate(replicateContext); - if (success) { + public ReplicationResult replicate(ReplicateContext replicateContext) { + ReplicationResult success = super.replicate(replicateContext); + if (success == ReplicationResult.COMMITTED) { replicateCount.addAndGet(replicateContext.entries.size()); } return success; @@ -577,7 +577,7 @@ public static class ReplicationEndpointReturningFalse extends ReplicationEndpoin static AtomicBoolean replicated = new AtomicBoolean(false); @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { try { // check row doAssert(row); @@ -589,7 +589,7 @@ public boolean replicate(ReplicateContext replicateContext) { LOG.info("Replicated " + Bytes.toString(row) + ", count=" + replicateCount.get()); replicated.set(replicateCount.get() > COUNT); // first 10 times, we return false - return replicated.get(); + return replicated.get() ? ReplicationResult.COMMITTED : ReplicationResult.FAILED; } } @@ -598,14 +598,14 @@ public static class ReplicationEndpointWithWALEntryFilter extends ReplicationEnd static AtomicReference ex = new AtomicReference<>(null); @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { try { super.replicate(replicateContext); doAssert(row); } catch (Exception e) { ex.set(e); } - return true; + return ReplicationResult.COMMITTED; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java index b990916ae75f..50b0911970a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyCellsReplicationEndpoint.java @@ -71,7 +71,7 @@ public class TestVerifyCellsReplicationEndpoint { public static final class EndpointForTest extends VerifyWALEntriesReplicationEndpoint { @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { LOG.info(replicateContext.getEntries().toString()); replicateContext.entries.stream().map(WAL.Entry::getEdit).map(WALEdit::getCells) .forEachOrdered(CELLS::addAll); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestHBaseInterClusterReplicationEndpointFilterEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestHBaseInterClusterReplicationEndpointFilterEdits.java index 7b108f5ca148..92e7c8290f0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestHBaseInterClusterReplicationEndpointFilterEdits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestHBaseInterClusterReplicationEndpointFilterEdits.java @@ -83,8 +83,8 @@ public static void setUpBeforeClass() throws Exception { when(rpc.isSerial()).thenReturn(false); when(replicationPeer.getPeerConfig()).thenReturn(rpc); when(rpc.getClusterKey()).thenReturn("hbase+zk://localhost:2181"); - Context context = new Context(null, UTIL.getConfiguration(), UTIL.getConfiguration(), null, - null, null, replicationPeer, null, null, null); + Context context = new Context(null, null, UTIL.getConfiguration(), UTIL.getConfiguration(), + null, null, null, replicationPeer, null, null, null); endpoint = new HBaseInterClusterReplicationEndpoint(); endpoint.init(context); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRaceWhenCreatingReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRaceWhenCreatingReplicationSource.java index 66f04dca36d5..d7b5bcdcccbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRaceWhenCreatingReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRaceWhenCreatingReplicationSource.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; @@ -94,7 +95,7 @@ public UUID getPeerUUID() { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { synchronized (WRITER) { try { for (Entry entry : replicateContext.getEntries()) { @@ -105,7 +106,7 @@ public boolean replicate(ReplicateContext replicateContext) { throw new UncheckedIOException(e); } } - return true; + return ReplicationResult.COMMITTED; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index 663b444dc4e4..c99f25380de4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueId; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.replication.SyncReplicationState; @@ -94,13 +95,13 @@ public static final class ReplicationEndpointForTest extends DummyReplicationEnd private String clusterKey; @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { // if you want to block the replication, for example, do not want the recovered source to be // removed if (clusterKey.endsWith("error")) { throw new RuntimeException("Inject error"); } - return true; + return ReplicationResult.COMMITTED; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java index 979db712ef34..cdbd1c73a2a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.TestReplicationBase; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -218,7 +219,7 @@ public static void setEntriesCount(int i) { } @Override - public boolean replicate(ReplicateContext replicateContext) { + public ReplicationResult replicate(ReplicateContext replicateContext) { try { await(); } catch (InterruptedException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java index 7d5a5627d2c0..ffbc0d2cee5a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java @@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; @@ -473,8 +474,8 @@ public VisibilityReplicationEndPointForTest(ReplicationEndpoint endpoint, } @Override - public boolean replicate(ReplicateContext replicateContext) { - boolean ret = super.replicate(replicateContext); + public ReplicationResult replicate(ReplicateContext replicateContext) { + ReplicationResult ret = super.replicate(replicateContext); lastEntries = replicateContext.getEntries(); replicateCount.incrementAndGet(); return ret; From 0bff7ebf7eb1fd087da4dd6416f99371b26ea74b Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Tue, 4 Mar 2025 22:27:44 +0530 Subject: [PATCH 02/24] HBASE-29025: Enhance the full backup command to support Continuous Backup (#6710) * HBASE-29025: Enhance the full backup command to support continuous backup * add new check for full backup command regards to continuous backup flag * minor fixes --- .../hadoop/hbase/backup/BackupDriver.java | 6 +- .../hadoop/hbase/backup/BackupInfo.java | 12 + .../hadoop/hbase/backup/BackupRequest.java | 14 + .../hbase/backup/BackupRestoreConstants.java | 12 + .../hbase/backup/impl/BackupAdminImpl.java | 3 +- .../hbase/backup/impl/BackupCommands.java | 63 +++- .../hbase/backup/impl/BackupManager.java | 18 +- .../hbase/backup/impl/BackupSystemTable.java | 94 ++++++ .../backup/impl/FullTableBackupClient.java | 245 ++++++++++---- .../hbase/backup/impl/TableBackupClient.java | 2 +- .../hbase/backup/TestContinuousBackup.java | 302 ++++++++++++++++++ .../src/main/protobuf/Backup.proto | 1 + 12 files changed, 707 insertions(+), 65 deletions(-) create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java index d55a280b4aa4..e096bbee161c 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java @@ -17,11 +17,14 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.LONG_OPTION_ENABLE_CONTINUOUS_BACKUP; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BACKUP_LIST_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP; @@ -159,7 +162,8 @@ protected void addOptions() { addOptWithArg(OPTION_PATH, OPTION_PATH_DESC); addOptWithArg(OPTION_KEEP, OPTION_KEEP_DESC); addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_DESC); - + addOptNoArg(OPTION_ENABLE_CONTINUOUS_BACKUP, LONG_OPTION_ENABLE_CONTINUOUS_BACKUP, + OPTION_ENABLE_CONTINUOUS_BACKUP_DESC); } @Override diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java index f0dc10b83619..862a9cbad107 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java @@ -71,6 +71,7 @@ public enum BackupState { */ public enum BackupPhase { REQUEST, + SETUP_WAL_REPLICATION, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, @@ -170,6 +171,8 @@ public enum BackupPhase { */ private boolean noChecksumVerify; + private boolean continuousBackupEnabled; + public BackupInfo() { backupTableInfoMap = new HashMap<>(); } @@ -185,6 +188,7 @@ public BackupInfo(String backupId, BackupType type, TableName[] tables, String t } this.startTs = 0; this.completeTs = 0; + this.continuousBackupEnabled = false; } public int getWorkers() { @@ -592,4 +596,12 @@ public int compareTo(BackupInfo o) { Long otherTS = Long.valueOf(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1)); return thisTS.compareTo(otherTS); } + + public void setContinuousBackupEnabled(boolean continuousBackupEnabled) { + this.continuousBackupEnabled = continuousBackupEnabled; + } + + public boolean isContinuousBackupEnabled() { + return this.continuousBackupEnabled; + } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java index aa2d5b44259f..822c84c57c06 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java @@ -75,6 +75,11 @@ public Builder withYarnPoolName(String name) { return this; } + public Builder withContinuousBackupEnabled(boolean continuousBackupEnabled) { + request.setContinuousBackupEnabled(continuousBackupEnabled); + return this; + } + public BackupRequest build() { return request; } @@ -89,6 +94,7 @@ public BackupRequest build() { private boolean noChecksumVerify = false; private String backupSetName; private String yarnPoolName; + private boolean continuousBackupEnabled; private BackupRequest() { } @@ -163,4 +169,12 @@ public String getYarnPoolName() { public void setYarnPoolName(String yarnPoolName) { this.yarnPoolName = yarnPoolName; } + + private void setContinuousBackupEnabled(boolean continuousBackupEnabled) { + this.continuousBackupEnabled = continuousBackupEnabled; + } + + public boolean isContinuousBackupEnabled() { + return this.continuousBackupEnabled; + } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java index 57454d402173..f5c49adb6960 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java @@ -99,6 +99,11 @@ public interface BackupRestoreConstants { String OPTION_YARN_QUEUE_NAME_DESC = "Yarn queue name to run backup create command on"; String OPTION_YARN_QUEUE_NAME_RESTORE_DESC = "Yarn queue name to run backup restore command on"; + String OPTION_ENABLE_CONTINUOUS_BACKUP = "cb"; + String LONG_OPTION_ENABLE_CONTINUOUS_BACKUP = "continuous-backup-enabled"; + String OPTION_ENABLE_CONTINUOUS_BACKUP_DESC = + "Flag indicating that the full backup is part of a continuous backup process."; + String JOB_NAME_CONF_KEY = "mapreduce.job.name"; String BACKUP_CONFIG_STRING = BackupRestoreConstants.BACKUP_ENABLE_KEY + "=true\n" @@ -126,6 +131,13 @@ public interface BackupRestoreConstants { String BACKUPID_PREFIX = "backup_"; + String CONTINUOUS_BACKUP_REPLICATION_PEER = "continuous_backup_replication_peer"; + + String DEFAULT_CONTINUOUS_BACKUP_REPLICATION_ENDPOINT = + "org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint"; + + String CONF_CONTINUOUS_BACKUP_WAL_DIR = "hbase.backup.continuous.wal.dir"; + enum BackupCommand { CREATE, CANCEL, diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java index c36b398e5e86..1e745c69cdad 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -581,7 +581,8 @@ public String backupTables(BackupRequest request) throws IOException { request = builder.withBackupType(request.getBackupType()).withTableList(tableList) .withTargetRootDir(request.getTargetRootDir()).withBackupSetName(request.getBackupSetName()) .withTotalTasks(request.getTotalTasks()).withBandwidthPerTasks((int) request.getBandwidth()) - .withNoChecksumVerify(request.getNoChecksumVerify()).build(); + .withNoChecksumVerify(request.getNoChecksumVerify()) + .withContinuousBackupEnabled(request.isContinuousBackupEnabled()).build(); TableBackupClient client; try { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 66694f4384f4..ab9ca1c4ed27 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -22,6 +22,8 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP; @@ -45,6 +47,7 @@ import java.io.IOException; import java.net.URI; import java.util.List; +import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -339,14 +342,64 @@ public void execute() throws IOException { boolean ignoreChecksum = cmdline.hasOption(OPTION_IGNORECHECKSUM); + BackupType backupType = BackupType.valueOf(args[1].toUpperCase()); + List tableNameList = null; + if (tables != null) { + tableNameList = Lists.newArrayList(BackupUtils.parseTableNames(tables)); + } + boolean continuousBackup = cmdline.hasOption(OPTION_ENABLE_CONTINUOUS_BACKUP); + if (continuousBackup && !BackupType.FULL.equals(backupType)) { + System.out.println("ERROR: Continuous backup can Only be specified for Full Backup"); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + /* + * The `continuousBackup` flag is specified only during the first full backup to initiate + * continuous WAL replication. After that, it is redundant because the tables are already set + * up for continuous backup. If the `continuousBackup` flag is not explicitly enabled, we need + * to determine the backup mode based on the current state of the specified tables: - If all + * the specified tables are already part of continuous backup, we treat the request as a + * continuous backup request and proceed accordingly (since these tables are already + * continuously backed up, no additional setup is needed). - If none of the specified tables + * are part of continuous backup, we treat the request as a normal full backup without + * continuous backup. - If the request includes a mix of tables—some with continuous backup + * enabled and others without—we cannot determine a clear backup strategy. In this case, we + * throw an error. If all tables are already in continuous backup mode, we explicitly set the + * `continuousBackup` flag to `true` so that the request is processed using the continuous + * backup approach rather than the normal full backup flow. + */ + if (!continuousBackup && tableNameList != null && !tableNameList.isEmpty()) { + try (BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { + Set continuousBackupTableSet = + backupSystemTable.getContinuousBackupTableSet().keySet(); + + boolean allTablesInContinuousBackup = continuousBackupTableSet.containsAll(tableNameList); + boolean noTablesInContinuousBackup = + tableNameList.stream().noneMatch(continuousBackupTableSet::contains); + + // Ensure that all tables are either fully in continuous backup or not at all + if (!allTablesInContinuousBackup && !noTablesInContinuousBackup) { + System.err + .println("ERROR: Some tables are already in continuous backup, while others are not. " + + "Cannot mix both in a single request."); + printUsage(); + throw new IOException(INCORRECT_USAGE); + } + + // If all tables are already in continuous backup, enable the flag + if (allTablesInContinuousBackup) { + continuousBackup = true; + } + } + } + try (BackupAdminImpl admin = new BackupAdminImpl(conn)) { BackupRequest.Builder builder = new BackupRequest.Builder(); - BackupRequest request = builder.withBackupType(BackupType.valueOf(args[1].toUpperCase())) - .withTableList( - tables != null ? Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null) + BackupRequest request = builder.withBackupType(backupType).withTableList(tableNameList) .withTargetRootDir(targetBackupDir).withTotalTasks(workers) .withBandwidthPerTasks(bandwidth).withNoChecksumVerify(ignoreChecksum) - .withBackupSetName(setName).build(); + .withBackupSetName(setName).withContinuousBackupEnabled(continuousBackup).build(); String backupId = admin.backupTables(request); System.out.println("Backup session " + backupId + " finished. Status: SUCCESS"); } catch (IOException e) { @@ -400,6 +453,8 @@ protected void printUsage() { options.addOption(OPTION_YARN_QUEUE_NAME, true, OPTION_YARN_QUEUE_NAME_DESC); options.addOption(OPTION_DEBUG, false, OPTION_DEBUG_DESC); options.addOption(OPTION_IGNORECHECKSUM, false, OPTION_IGNORECHECKSUM_DESC); + options.addOption(OPTION_ENABLE_CONTINUOUS_BACKUP, false, + OPTION_ENABLE_CONTINUOUS_BACKUP_DESC); HelpFormatter helpFormatter = new HelpFormatter(); helpFormatter.setLeftPadding(2); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java index 810af8f032ce..8b17e93868be 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java @@ -199,8 +199,8 @@ public void close() { * @throws BackupException exception */ public BackupInfo createBackupInfo(String backupId, BackupType type, List tableList, - String targetRootDir, int workers, long bandwidth, boolean noChecksumVerify) - throws BackupException { + String targetRootDir, int workers, long bandwidth, boolean noChecksumVerify, + boolean continuousBackupEnabled) throws BackupException { if (targetRootDir == null) { throw new BackupException("Wrong backup request parameter: target backup root directory"); } @@ -238,6 +238,7 @@ public BackupInfo createBackupInfo(String backupId, BackupType type, List tables) throws IOExcepti public Connection getConnection() { return conn; } + + /** + * Adds a set of tables to the global continuous backup set. Only tables that do not already have + * continuous backup enabled will be updated. + * @param tables set of tables to add to continuous backup + * @param startTimestamp timestamp indicating when continuous backup started for newly added + * tables + * @throws IOException if an error occurs while updating the backup system table + */ + public void addContinuousBackupTableSet(Set tables, long startTimestamp) + throws IOException { + systemTable.addContinuousBackupTableSet(tables, startTimestamp); + } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index f2ddcf5e7573..752f448a3016 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -169,6 +169,7 @@ public String toString() { private final static byte[] ACTIVE_SESSION_NO = Bytes.toBytes("no"); private final static String INCR_BACKUP_SET = "incrbackupset:"; + private final static String CONTINUOUS_BACKUP_SET = "continuousbackupset"; private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm:"; private final static String RS_LOG_TS_PREFIX = "rslogts:"; @@ -892,6 +893,37 @@ public Set getIncrementalBackupTableSet(String backupRoot) throws IOE } } + /** + * Retrieves the current set of tables covered by continuous backup along with the timestamp + * indicating when continuous backup started for each table. + * @return a map where the key is the table name and the value is the timestamp representing the + * start time of continuous backup for that table. + * @throws IOException if an I/O error occurs while accessing the backup system table. + */ + public Map getContinuousBackupTableSet() throws IOException { + LOG.trace("Retrieving continuous backup table set from the backup system table."); + Map tableMap = new TreeMap<>(); + + try (Table systemTable = connection.getTable(tableName)) { + Get getOperation = createGetForContinuousBackupTableSet(); + Result result = systemTable.get(getOperation); + + if (result.isEmpty()) { + return tableMap; + } + + // Extract table names and timestamps from the result cells + List cells = result.listCells(); + for (Cell cell : cells) { + TableName tableName = TableName.valueOf(CellUtil.cloneQualifier(cell)); + long timestamp = Bytes.toLong(CellUtil.cloneValue(cell)); + tableMap.put(tableName, timestamp); + } + } + + return tableMap; + } + /** * Add tables to global incremental backup set * @param tables set of tables @@ -913,6 +945,34 @@ public void addIncrementalBackupTableSet(Set tables, String backupRoo } } + /** + * Add tables to the global continuous backup set. Only updates tables that are not already in the + * continuous backup set. + * @param tables set of tables to add + * @param startTimestamp timestamp indicating when continuous backup started + * @throws IOException if an error occurs while updating the backup system table + */ + public void addContinuousBackupTableSet(Set tables, long startTimestamp) + throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Add continuous backup table set to backup system table. tables [" + + StringUtils.join(tables, " ") + "]"); + } + if (LOG.isDebugEnabled()) { + tables.forEach(table -> LOG.debug(Objects.toString(table))); + } + + // Get existing continuous backup tables + Map existingTables = getContinuousBackupTableSet(); + + try (Table table = connection.getTable(tableName)) { + Put put = createPutForContinuousBackupTableSet(tables, existingTables, startTimestamp); + if (!put.isEmpty()) { + table.put(put); + } + } + } + /** * Deletes incremental backup set for a backup destination * @param backupRoot backup root @@ -1241,6 +1301,18 @@ private Get createGetForIncrBackupTableSet(String backupRoot) throws IOException return get; } + /** + * Creates a Get operation to retrieve the continuous backup table set from the backup system + * table. + * @return a Get operation for retrieving the table set + */ + private Get createGetForContinuousBackupTableSet() throws IOException { + Get get = new Get(rowkey(CONTINUOUS_BACKUP_SET)); + get.addFamily(BackupSystemTable.META_FAMILY); + get.readVersions(1); + return get; + } + /** * Creates Put to store incremental backup table set * @param tables tables @@ -1255,6 +1327,28 @@ private Put createPutForIncrBackupTableSet(Set tables, String backupR return put; } + /** + * Creates a Put operation to store the continuous backup table set. Only includes tables that are + * not already in the set. + * @param tables tables to add + * @param existingTables tables that already have continuous backup enabled + * @param startTimestamp timestamp indicating when continuous backup started + * @return put operation + */ + private Put createPutForContinuousBackupTableSet(Set tables, + Map existingTables, long startTimestamp) { + Put put = new Put(rowkey(CONTINUOUS_BACKUP_SET)); + + for (TableName table : tables) { + if (!existingTables.containsKey(table)) { + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes(table.getNameAsString()), + Bytes.toBytes(startTimestamp)); + } + } + + return put; + } + /** * Creates Delete for incremental backup table set * @param backupRoot backup root diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java index 2293fd4f8149..de47e8f33915 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -17,16 +17,24 @@ */ package org.apache.hadoop.hbase.backup.impl; +import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_GLOBAL; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_ATTEMPTS_PAUSE_MS_KEY; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_MAX_ATTEMPTS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_REPLICATION_ENDPOINT; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_ROOT_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_PEER_UUID; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupCopyJob; import org.apache.hadoop.hbase.backup.BackupInfo; @@ -37,7 +45,13 @@ import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -61,9 +75,9 @@ public FullTableBackupClient(final Connection conn, final String backupId, Backu /** * Do snapshot copy. * @param backupInfo backup info - * @throws Exception exception + * @throws IOException exception */ - protected void snapshotCopy(BackupInfo backupInfo) throws Exception { + protected void snapshotCopy(BackupInfo backupInfo) throws IOException { LOG.info("Snapshot copy is starting."); // set overall backup phase: snapshot_copy @@ -131,72 +145,22 @@ protected void snapshotCopy(BackupInfo backupInfo) throws Exception { @Override public void execute() throws IOException { try (Admin admin = conn.getAdmin()) { - // Begin BACKUP beginBackup(backupManager, backupInfo); - String savedStartCode; - boolean firstBackup; - // do snapshot for full table backup - - savedStartCode = backupManager.readBackupStartCode(); - firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L; - if (firstBackup) { - // This is our first backup. Let's put some marker to system table so that we can hold the - // logs while we do the backup. - backupManager.writeBackupStartCode(0L); - } - // We roll log here before we do the snapshot. It is possible there is duplicate data - // in the log that is already in the snapshot. But if we do it after the snapshot, we - // could have data loss. - // A better approach is to do the roll log on each RS in the same global procedure as - // the snapshot. - LOG.info("Execute roll log procedure for full backup ..."); // Gather the bulk loads being tracked by the system, which can be deleted (since their data // will be part of the snapshot being taken). We gather this list before taking the actual // snapshots for the same reason as the log rolls. List bulkLoadsToDelete = backupManager.readBulkloadRows(tableList); - BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf); - - newTimestamps = backupManager.readRegionServerLastLogRollResult(); - - // SNAPSHOT_TABLES: - backupInfo.setPhase(BackupPhase.SNAPSHOT); - for (TableName tableName : tableList) { - String snapshotName = "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) - + "_" + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString(); - - snapshotTable(admin, tableName, snapshotName); - backupInfo.setSnapshotName(tableName, snapshotName); + if (backupInfo.isContinuousBackupEnabled()) { + handleContinuousBackup(admin); + } else { + handleNonContinuousBackup(admin); } - // SNAPSHOT_COPY: - // do snapshot copy - LOG.debug("snapshot copy for " + backupId); - snapshotCopy(backupInfo); - // Updates incremental backup table set - backupManager.addIncrementalBackupTableSet(backupInfo.getTables()); - - // BACKUP_COMPLETE: - // set overall backup status: complete. Here we make sure to complete the backup. - // After this checkpoint, even if entering cancel process, will let the backup finished - backupInfo.setState(BackupState.COMPLETE); - // The table list in backupInfo is good for both full backup and incremental backup. - // For incremental backup, it contains the incremental backup table set. - backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); - - Map> newTableSetTimestampMap = - backupManager.readLogTimestampMap(); - - backupInfo.setTableSetTimestampMap(newTableSetTimestampMap); - Long newStartCode = - BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); - backupManager.writeBackupStartCode(newStartCode); - backupManager .deleteBulkLoadedRows(bulkLoadsToDelete.stream().map(BulkLoad::getRowKey).toList()); - // backup complete completeBackup(conn, backupInfo, BackupType.FULL, conf); } catch (Exception e) { failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", @@ -205,6 +169,175 @@ public void execute() throws IOException { } } + private void handleContinuousBackup(Admin admin) throws IOException { + backupInfo.setPhase(BackupInfo.BackupPhase.SETUP_WAL_REPLICATION); + long startTimestamp = startContinuousWALBackup(admin); + + performBackupSnapshots(admin); + + backupManager.addContinuousBackupTableSet(backupInfo.getTables(), startTimestamp); + + // set overall backup status: complete. Here we make sure to complete the backup. + // After this checkpoint, even if entering cancel process, will let the backup finished + backupInfo.setState(BackupState.COMPLETE); + + if (!conf.getBoolean("hbase.replication.bulkload.enabled", false)) { + System.out.println("NOTE: Bulkload replication is not enabled. " + + "Bulk loaded files will not be backed up as part of continuous backup. " + + "To ensure bulk loaded files are included in the backup, please enable bulkload replication " + + "(hbase.replication.bulkload.enabled=true) and configure other necessary settings " + + "to properly enable bulkload replication."); + } + } + + private void handleNonContinuousBackup(Admin admin) throws IOException { + initializeBackupStartCode(backupManager); + performLogRoll(); + performBackupSnapshots(admin); + backupManager.addIncrementalBackupTableSet(backupInfo.getTables()); + + // set overall backup status: complete. Here we make sure to complete the backup. + // After this checkpoint, even if entering cancel process, will let the backup finished + backupInfo.setState(BackupState.COMPLETE); + + updateBackupMetadata(); + } + + private void initializeBackupStartCode(BackupManager backupManager) throws IOException { + String savedStartCode; + boolean firstBackup; + // do snapshot for full table backup + savedStartCode = backupManager.readBackupStartCode(); + firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L; + if (firstBackup) { + // This is our first backup. Let's put some marker to system table so that we can hold the + // logs while we do the backup. + backupManager.writeBackupStartCode(0L); + } + } + + private void performLogRoll() throws IOException { + // We roll log here before we do the snapshot. It is possible there is duplicate data + // in the log that is already in the snapshot. But if we do it after the snapshot, we + // could have data loss. + // A better approach is to do the roll log on each RS in the same global procedure as + // the snapshot. + LOG.info("Execute roll log procedure for full backup ..."); + BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf); + newTimestamps = backupManager.readRegionServerLastLogRollResult(); + } + + private void performBackupSnapshots(Admin admin) throws IOException { + backupInfo.setPhase(BackupPhase.SNAPSHOT); + performSnapshots(admin); + LOG.debug("Performing snapshot copy for backup ID: {}", backupInfo.getBackupId()); + snapshotCopy(backupInfo); + } + + private void performSnapshots(Admin admin) throws IOException { + backupInfo.setPhase(BackupPhase.SNAPSHOT); + + for (TableName tableName : tableList) { + String snapshotName = String.format("snapshot_%d_%s_%s", EnvironmentEdgeManager.currentTime(), + tableName.getNamespaceAsString(), tableName.getQualifierAsString()); + snapshotTable(admin, tableName, snapshotName); + backupInfo.setSnapshotName(tableName, snapshotName); + } + } + + private void updateBackupMetadata() throws IOException { + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + Map> timestampMap = backupManager.readLogTimestampMap(); + backupInfo.setTableSetTimestampMap(timestampMap); + Long newStartCode = BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(timestampMap)); + backupManager.writeBackupStartCode(newStartCode); + } + + private long startContinuousWALBackup(Admin admin) throws IOException { + enableTableReplication(admin); + if (continuousBackupReplicationPeerExists(admin)) { + updateContinuousBackupReplicationPeer(admin); + } else { + addContinuousBackupReplicationPeer(admin); + } + LOG.info("Continuous WAL Backup setup completed."); + return EnvironmentEdgeManager.getDelegate().currentTime(); + } + + private void enableTableReplication(Admin admin) throws IOException { + for (TableName table : tableList) { + TableDescriptor tableDescriptor = admin.getDescriptor(table); + TableDescriptorBuilder tableDescriptorBuilder = + TableDescriptorBuilder.newBuilder(tableDescriptor); + + for (ColumnFamilyDescriptor cfDescriptor : tableDescriptor.getColumnFamilies()) { + if (cfDescriptor.getScope() != REPLICATION_SCOPE_GLOBAL) { + ColumnFamilyDescriptor newCfDescriptor = ColumnFamilyDescriptorBuilder + .newBuilder(cfDescriptor).setScope(REPLICATION_SCOPE_GLOBAL).build(); + + tableDescriptorBuilder.modifyColumnFamily(newCfDescriptor); + } + } + + admin.modifyTable(tableDescriptorBuilder.build()); + LOG.info("Enabled Global replication scope for table: {}", table); + } + } + + private void updateContinuousBackupReplicationPeer(Admin admin) throws IOException { + Map> tableMap = tableList.stream() + .collect(Collectors.toMap(tableName -> tableName, tableName -> new ArrayList<>())); + + try { + admin.appendReplicationPeerTableCFs(CONTINUOUS_BACKUP_REPLICATION_PEER, tableMap); + LOG.info("Updated replication peer {} with table and column family map.", + CONTINUOUS_BACKUP_REPLICATION_PEER); + } catch (ReplicationException e) { + LOG.error("Error while updating the replication peer: {}. Error: {}", + CONTINUOUS_BACKUP_REPLICATION_PEER, e.getMessage(), e); + throw new IOException("Error while updating the continuous backup replication peer.", e); + } + } + + private void addContinuousBackupReplicationPeer(Admin admin) throws IOException { + String backupWalDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + + if (backupWalDir == null || backupWalDir.isEmpty()) { + String errorMsg = "WAL Directory is not specified for continuous backup."; + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + + Map additionalArgs = new HashMap<>(); + additionalArgs.put(CONF_PEER_UUID, UUID.randomUUID().toString()); + additionalArgs.put(CONF_BACKUP_ROOT_DIR, backupWalDir); + + Map> tableMap = tableList.stream() + .collect(Collectors.toMap(tableName -> tableName, tableName -> new ArrayList<>())); + + ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() + .setReplicationEndpointImpl(DEFAULT_CONTINUOUS_BACKUP_REPLICATION_ENDPOINT) + .setReplicateAllUserTables(false).setTableCFsMap(tableMap).putAllConfiguration(additionalArgs) + .build(); + + try { + admin.addReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER, peerConfig, true); + LOG.info("Successfully added replication peer with ID: {}", + CONTINUOUS_BACKUP_REPLICATION_PEER); + } catch (IOException e) { + LOG.error("Failed to add replication peer with ID: {}. Error: {}", + CONTINUOUS_BACKUP_REPLICATION_PEER, e.getMessage(), e); + throw e; + } + } + + private boolean continuousBackupReplicationPeerExists(Admin admin) throws IOException { + return admin.listReplicationPeers().stream() + .anyMatch(peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER)); + } + protected void snapshotTable(Admin admin, TableName tableName, String snapshotName) throws IOException { int maxAttempts = conf.getInt(BACKUP_MAX_ATTEMPTS_KEY, DEFAULT_BACKUP_MAX_ATTEMPTS); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java index 30c27f01faaf..9e31ca409ada 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java @@ -92,7 +92,7 @@ public void init(final Connection conn, final String backupId, BackupRequest req this.fs = CommonFSUtils.getCurrentFileSystem(conf); backupInfo = backupManager.createBackupInfo(backupId, request.getBackupType(), tableList, request.getTargetRootDir(), request.getTotalTasks(), request.getBandwidth(), - request.getNoChecksumVerify()); + request.getNoChecksumVerify(), request.isContinuousBackupEnabled()); if (tableList == null || tableList.isEmpty()) { this.tableList = new ArrayList<>(backupInfo.getTables()); } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java new file mode 100644 index 000000000000..fe44ebf420df --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java @@ -0,0 +1,302 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.util.ToolRunner; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + +@Category(LargeTests.class) +public class TestContinuousBackup extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestContinuousBackup.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestContinuousBackup.class); + + String backupWalDirName = "TestContinuousBackupWalDir"; + + @Before + public void beforeTest() throws IOException { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, backupWalDirName); + FileSystem fs = FileSystem.get(conf1); + fs.mkdirs(backupWalDir); + conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); + } + + @After + public void afterTest() throws IOException { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, backupWalDirName); + FileSystem fs = FileSystem.get(conf1); + + if (fs.exists(backupWalDir)) { + fs.delete(backupWalDir, true); + } + + conf1.unset(CONF_CONTINUOUS_BACKUP_WAL_DIR); + deleteContinuousBackupReplicationPeerIfExists(TEST_UTIL.getAdmin()); + } + + @Test + public void testContinuousBackupWithFullBackup() throws Exception { + LOG.info("Testing successful continuous backup with full backup"); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + TEST_UTIL.createTable(tableName, "cf"); + + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + + // Run backup + String[] args = buildBackupArgs("full", new TableName[] { tableName }, true); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + // Verify backup history increased and all the backups are succeeded + List backups = table.getBackupHistory(); + assertEquals("Backup history should increase", before + 1, backups.size()); + for (BackupInfo data : List.of(backups.get(0))) { + String backupId = data.getBackupId(); + assertTrue(checkSucceeded(backupId)); + } + + // Verify backup manifest contains the correct tables + BackupManifest manifest = getLatestBackupManifest(backups); + assertEquals("Backup should contain the expected tables", Sets.newHashSet(tableName), + new HashSet<>(manifest.getTableList())); + } + + // Verify replication peer subscription + verifyReplicationPeerSubscription(tableName); + + // Verify table is registered in Backup System Table + verifyTableInBackupSystemTable(tableName); + } + + @Test + public void testContinuousBackupForMultipleTables() throws Exception { + LOG.info("Test continuous backup for multiple tables"); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName1 = TableName.valueOf("table_" + methodName); + TEST_UTIL.createTable(tableName1, "cf"); + TableName tableName2 = TableName.valueOf("table_" + methodName + "2"); + TEST_UTIL.createTable(tableName2, "cf"); + + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + + // Create full backup for table1 + String[] args = buildBackupArgs("full", new TableName[] { tableName1 }, true); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + // Create full backup for table2 + args = buildBackupArgs("full", new TableName[] { tableName2 }, true); + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + // Verify backup history increased and all the backups are succeeded + List backups = table.getBackupHistory(); + assertEquals("Backup history should increase", before + 2, backups.size()); + for (BackupInfo data : List.of(backups.get(0), backups.get(1))) { + String backupId = data.getBackupId(); + assertTrue(checkSucceeded(backupId)); + } + + // Verify backup manifest contains the correct tables + BackupManifest manifest = getLatestBackupManifest(backups); + assertEquals("Backup should contain the expected tables", Sets.newHashSet(tableName2), + new HashSet<>(manifest.getTableList())); + } + + // Verify replication peer subscription for each table + verifyReplicationPeerSubscription(tableName1); + verifyReplicationPeerSubscription(tableName2); + + // Verify tables are registered in Backup System Table + verifyTableInBackupSystemTable(tableName1); + verifyTableInBackupSystemTable(tableName2); + } + + @Test + public void testInvalidBackupScenarioWithContinuousEnabled() throws Exception { + LOG.info("Testing invalid backup scenario with continuous backup enabled"); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName1 = TableName.valueOf("table_" + methodName); + TEST_UTIL.createTable(tableName1, "cf"); + TableName tableName2 = TableName.valueOf("table_" + methodName + "2"); + TEST_UTIL.createTable(tableName2, "cf"); + + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + + // Create full backup for table1 with continuous backup enabled + String[] args = buildBackupArgs("full", new TableName[] { tableName1 }, true); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + // Create full backup for table2 without continuous backup enabled + args = buildBackupArgs("full", new TableName[] { tableName2 }, false); + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + // Attempt full backup for both tables without continuous backup enabled (should fail) + args = buildBackupArgs("full", new TableName[] { tableName1, tableName2 }, false); + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue("Backup should fail due to mismatch in continuous backup settings", ret != 0); + + // Verify backup history size is unchanged after the failed backup + int after = table.getBackupHistory().size(); + assertEquals("Backup history should remain unchanged on failure", before + 2, after); + } + } + + @Test + public void testContinuousBackupWithWALDirNotSpecified() throws Exception { + LOG.info("Testing that continuous backup fails when WAL directory is not specified"); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + TEST_UTIL.createTable(tableName, "cf"); + + conf1.unset(CONF_CONTINUOUS_BACKUP_WAL_DIR); + LOG.info("CONF_CONTINUOUS_BACKUP_WAL_DIR: {}", conf1.get(CONF_CONTINUOUS_BACKUP_WAL_DIR)); + + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + + // Run full backup without specifying WAL directory (invalid scenario) + String[] args = buildBackupArgs("full", new TableName[] { tableName }, true); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + + assertTrue("Backup should fail when WAL directory is not specified", ret != 0); + + List backups = table.getBackupHistory(); + int after = backups.size(); + assertEquals("Backup history should increase", before + 1, after); + + // last backup should be a failure + assertFalse(checkSucceeded(backups.get(0).getBackupId())); + } + } + + @Test + public void testContinuousBackupWithIncrementalBackup() throws Exception { + LOG.info("Testing that continuous backup cannot be enabled with incremental backup"); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + TEST_UTIL.createTable(tableName, "cf"); + + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + + // Run incremental backup with continuous backup flag (invalid scenario) + String[] args = buildBackupArgs("incremental", new TableName[] { tableName }, true); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + + assertTrue("Backup should fail when using continuous backup with incremental mode", ret != 0); + + // Backup history should remain unchanged + int after = table.getBackupHistory().size(); + assertEquals("Backup history should remain unchanged on failure", before, after); + } + } + + private void verifyReplicationPeerSubscription(TableName table) throws IOException { + try (Admin admin = TEST_UTIL.getAdmin()) { + ReplicationPeerDescription peerDesc = admin.listReplicationPeers().stream() + .filter(peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER)).findFirst() + .orElseThrow(() -> new AssertionError("Replication peer not found")); + + assertTrue("Table should be subscribed to the replication peer", + peerDesc.getPeerConfig().getTableCFsMap().containsKey(table)); + } + } + + private String[] buildBackupArgs(String backupType, TableName[] tables, + boolean continuousEnabled) { + String tableNames = + Arrays.stream(tables).map(TableName::getNameAsString).collect(Collectors.joining(",")); + + if (continuousEnabled) { + return new String[] { "create", backupType, BACKUP_ROOT_DIR, "-t", tableNames, + "-" + OPTION_ENABLE_CONTINUOUS_BACKUP }; + } else { + return new String[] { "create", backupType, BACKUP_ROOT_DIR, "-t", tableNames }; + } + } + + private BackupManifest getLatestBackupManifest(List backups) throws IOException { + BackupInfo newestBackup = backups.get(0); + return HBackupFileSystem.getManifest(conf1, new Path(BACKUP_ROOT_DIR), + newestBackup.getBackupId()); + } + + private void verifyTableInBackupSystemTable(TableName table) throws IOException { + try (BackupSystemTable backupTable = new BackupSystemTable(TEST_UTIL.getConnection())) { + Map tableBackupMap = backupTable.getContinuousBackupTableSet(); + + assertTrue("Table is missing in the continuous backup table set", + tableBackupMap.containsKey(table)); + + assertTrue("Timestamp for table should be greater than 0", tableBackupMap.get(table) > 0); + } + } + + private void deleteContinuousBackupReplicationPeerIfExists(Admin admin) throws IOException { + if ( + admin.listReplicationPeers().stream() + .anyMatch(peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER)) + ) { + admin.disableReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); + admin.removeReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); + } + } + +} diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto index a114001ba504..95a298673251 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Backup.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto @@ -118,5 +118,6 @@ message BackupInfo { SNAPSHOTCOPY = 3; INCREMENTAL_COPY = 4; STORE_MANIFEST = 5; + SETUP_WAL_REPLICATION = 6; } } From 912ef675765095172932facb90502c4c10730020 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Thu, 10 Apr 2025 16:35:02 +0530 Subject: [PATCH 03/24] HBASE-29210: Introduce Validation for PITR-Critical Backup Deletion (#6848) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Andor Molnár Signed-off-by: Wellington Chevreuil --- .../hadoop/hbase/backup/BackupDriver.java | 4 + .../hbase/backup/BackupRestoreConstants.java | 8 + .../hbase/backup/impl/BackupCommands.java | 173 +++++++++++++++- .../hadoop/hbase/backup/TestBackupDelete.java | 6 +- ...ckupDeleteWithContinuousBackupAndPITR.java | 186 ++++++++++++++++++ 5 files changed, 369 insertions(+), 8 deletions(-) create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java index e096bbee161c..eb27e9a60e0d 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.backup; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.LONG_OPTION_ENABLE_CONTINUOUS_BACKUP; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.LONG_OPTION_FORCE_DELETE; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BACKUP_LIST_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC; @@ -25,6 +26,8 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_FORCE_DELETE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_FORCE_DELETE_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP; @@ -164,6 +167,7 @@ protected void addOptions() { addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_DESC); addOptNoArg(OPTION_ENABLE_CONTINUOUS_BACKUP, LONG_OPTION_ENABLE_CONTINUOUS_BACKUP, OPTION_ENABLE_CONTINUOUS_BACKUP_DESC); + addOptNoArg(OPTION_FORCE_DELETE, LONG_OPTION_FORCE_DELETE, OPTION_FORCE_DELETE_DESC); } @Override diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java index f5c49adb6960..3df67ac1aeff 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java @@ -104,6 +104,11 @@ public interface BackupRestoreConstants { String OPTION_ENABLE_CONTINUOUS_BACKUP_DESC = "Flag indicating that the full backup is part of a continuous backup process."; + String OPTION_FORCE_DELETE = "fd"; + String LONG_OPTION_FORCE_DELETE = "force-delete"; + String OPTION_FORCE_DELETE_DESC = + "Flag to forcefully delete the backup, even if it may be required for Point-in-Time Restore"; + String JOB_NAME_CONF_KEY = "mapreduce.job.name"; String BACKUP_CONFIG_STRING = BackupRestoreConstants.BACKUP_ENABLE_KEY + "=true\n" @@ -138,6 +143,9 @@ public interface BackupRestoreConstants { String CONF_CONTINUOUS_BACKUP_WAL_DIR = "hbase.backup.continuous.wal.dir"; + String CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS = "hbase.backup.continuous.pitr.window.days"; + long DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS = 30; + enum BackupCommand { CREATE, CANCEL, diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index ab9ca1c4ed27..e9d14d1426de 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.backup.impl; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BACKUP_LIST_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC; @@ -24,6 +26,8 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_FORCE_DELETE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_FORCE_DELETE_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP; @@ -46,8 +50,12 @@ import java.io.IOException; import java.net.URI; +import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; +import org.agrona.collections.MutableLong; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -632,15 +640,18 @@ public void execute() throws IOException { printUsage(); throw new IOException(INCORRECT_USAGE); } + + boolean isForceDelete = cmdline.hasOption(OPTION_FORCE_DELETE); super.execute(); if (cmdline.hasOption(OPTION_KEEP)) { - executeDeleteOlderThan(cmdline); + executeDeleteOlderThan(cmdline, isForceDelete); } else if (cmdline.hasOption(OPTION_LIST)) { - executeDeleteListOfBackups(cmdline); + executeDeleteListOfBackups(cmdline, isForceDelete); } } - private void executeDeleteOlderThan(CommandLine cmdline) throws IOException { + private void executeDeleteOlderThan(CommandLine cmdline, boolean isForceDelete) + throws IOException { String value = cmdline.getOptionValue(OPTION_KEEP); int days = 0; try { @@ -662,6 +673,7 @@ public boolean apply(BackupInfo info) { BackupAdminImpl admin = new BackupAdminImpl(conn)) { history = sysTable.getBackupHistory(-1, dateFilter); String[] backupIds = convertToBackupIds(history); + validatePITRBackupDeletion(backupIds, isForceDelete); int deleted = admin.deleteBackups(backupIds); System.out.println("Deleted " + deleted + " backups. Total older than " + days + " days: " + backupIds.length); @@ -680,10 +692,11 @@ private String[] convertToBackupIds(List history) { return ids; } - private void executeDeleteListOfBackups(CommandLine cmdline) throws IOException { + private void executeDeleteListOfBackups(CommandLine cmdline, boolean isForceDelete) + throws IOException { String value = cmdline.getOptionValue(OPTION_LIST); String[] backupIds = value.split(","); - + validatePITRBackupDeletion(backupIds, isForceDelete); try (BackupAdminImpl admin = new BackupAdminImpl(conn)) { int deleted = admin.deleteBackups(backupIds); System.out.println("Deleted " + deleted + " backups. Total requested: " + backupIds.length); @@ -695,12 +708,162 @@ private void executeDeleteListOfBackups(CommandLine cmdline) throws IOException } + /** + * Validates whether the specified backups can be deleted while preserving Point-In-Time + * Recovery (PITR) capabilities. If a backup is the only remaining full backup enabling PITR for + * certain tables, deletion is prevented unless forced. + * @param backupIds Array of backup IDs to validate. + * @param isForceDelete Flag indicating whether deletion should proceed regardless of PITR + * constraints. + * @throws IOException If a backup is essential for PITR and force deletion is not enabled. + */ + private void validatePITRBackupDeletion(String[] backupIds, boolean isForceDelete) + throws IOException { + if (!isForceDelete) { + for (String backupId : backupIds) { + List affectedTables = getTablesDependentOnBackupForPITR(backupId); + if (!affectedTables.isEmpty()) { + String errMsg = String.format( + "Backup %s is the only FULL backup remaining that enables PITR for tables: %s. " + + "Use the force option to delete it anyway.", + backupId, affectedTables); + System.err.println(errMsg); + throw new IOException(errMsg); + } + } + } + } + + /** + * Identifies tables that rely on the specified backup for PITR. If a table has no other valid + * FULL backups that can facilitate recovery to all points within the PITR retention window, it + * is added to the dependent list. + * @param backupId The backup ID being evaluated. + * @return List of tables dependent on the specified backup for PITR. + * @throws IOException If backup metadata cannot be retrieved. + */ + private List getTablesDependentOnBackupForPITR(String backupId) throws IOException { + List dependentTables = new ArrayList<>(); + + try (final BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { + BackupInfo targetBackup = backupSystemTable.readBackupInfo(backupId); + + if (targetBackup == null) { + throw new IOException("Backup info not found for backupId: " + backupId); + } + + // Only full backups are mandatory for PITR + if (!BackupType.FULL.equals(targetBackup.getType())) { + return List.of(); + } + + // Retrieve the tables with continuous backup enabled and their start times + Map continuousBackupStartTimes = + backupSystemTable.getContinuousBackupTableSet(); + + // Determine the PITR time window + long pitrWindowDays = getConf().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, + DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS); + long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); + final MutableLong pitrMaxStartTime = + new MutableLong(currentTime - TimeUnit.DAYS.toMillis(pitrWindowDays)); + + // For all tables, determine the earliest (minimum) continuous backup start time. + // This represents the actual earliest point-in-time recovery (PITR) timestamp + // that can be used, ensuring we do not go beyond the available backup data. + long minContinuousBackupStartTime = currentTime; + for (TableName table : targetBackup.getTableNames()) { + minContinuousBackupStartTime = Math.min(minContinuousBackupStartTime, + continuousBackupStartTimes.getOrDefault(table, currentTime)); + } + + // The PITR max start time should be the maximum of the calculated minimum continuous backup + // start time and the default PITR max start time (based on the configured window). + // This ensures that PITR does not extend beyond what is practically possible. + pitrMaxStartTime.set(Math.max(minContinuousBackupStartTime, pitrMaxStartTime.longValue())); + + for (TableName table : targetBackup.getTableNames()) { + // This backup is not necessary for this table since it doesn't have PITR enabled + if (!continuousBackupStartTimes.containsKey(table)) { + continue; + } + if ( + !isValidPITRBackup(targetBackup, table, continuousBackupStartTimes, + pitrMaxStartTime.longValue()) + ) { + continue; // This backup is not crucial for PITR of this table + } + + // Check if another valid full backup exists for this table + List backupHistory = backupSystemTable.getBackupInfos(BackupState.COMPLETE); + boolean hasAnotherValidBackup = backupHistory.stream() + .anyMatch(backup -> !backup.getBackupId().equals(backupId) && isValidPITRBackup(backup, + table, continuousBackupStartTimes, pitrMaxStartTime.longValue())); + + if (!hasAnotherValidBackup) { + dependentTables.add(table); + } + } + } + return dependentTables; + } + + /** + * Determines if a given backup is a valid candidate for Point-In-Time Recovery (PITR) for a + * specific table. A valid backup ensures that recovery is possible to any point within the PITR + * retention window. A backup qualifies if: + *
    + *
  • It is a FULL backup.
  • + *
  • It contains the specified table.
  • + *
  • Its completion timestamp is before the PITR retention window start time.
  • + *
  • Its completion timestamp is on or after the table’s continuous backup start time.
  • + *
+ * @param backupInfo The backup information being evaluated. + * @param tableName The table for which PITR validity is being checked. + * @param continuousBackupTables A map of tables to their continuous backup start time. + * @param pitrMaxStartTime The maximum allowed start timestamp for PITR eligibility. + * @return {@code true} if the backup enables recovery to all valid points in time for the + * table; {@code false} otherwise. + */ + private boolean isValidPITRBackup(BackupInfo backupInfo, TableName tableName, + Map continuousBackupTables, long pitrMaxStartTime) { + // Only FULL backups are mandatory for PITR + if (!BackupType.FULL.equals(backupInfo.getType())) { + return false; + } + + // The backup must include the table to be relevant for PITR + if (!backupInfo.getTableNames().contains(tableName)) { + return false; + } + + // The backup must have been completed before the PITR retention window starts, + // otherwise, it won't be helpful in cases where the recovery point is between + // pitrMaxStartTime and the backup completion time. + if (backupInfo.getCompleteTs() > pitrMaxStartTime) { + return false; + } + + // Retrieve the table's continuous backup start time + long continuousBackupStartTime = continuousBackupTables.getOrDefault(tableName, 0L); + + // The backup must have been started on or after the table’s continuous backup start time, + // otherwise, it won't be helpful in few cases because we wouldn't have the WAL entries + // between the backup start time and the continuous backup start time. + if (backupInfo.getStartTs() < continuousBackupStartTime) { + return false; + } + + return true; + } + @Override protected void printUsage() { System.out.println(DELETE_CMD_USAGE); Options options = new Options(); options.addOption(OPTION_KEEP, true, OPTION_KEEP_DESC); options.addOption(OPTION_LIST, true, OPTION_BACKUP_LIST_DESC); + options.addOption(OPTION_FORCE_DELETE, false, OPTION_FORCE_DELETE_DESC); HelpFormatter helpFormatter = new HelpFormatter(); helpFormatter.setLeftPadding(2); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java index 785859c52805..31eaaff50512 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.ByteArrayOutputStream; import java.io.PrintStream; @@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.ToolRunner; -import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -138,7 +138,7 @@ public long currentTime() { assertTrue(ret == 0); } catch (Exception e) { LOG.error("failed", e); - Assert.fail(e.getMessage()); + fail(e.getMessage()); } String output = baos.toString(); LOG.info(baos.toString()); @@ -154,7 +154,7 @@ public long currentTime() { assertTrue(ret == 0); } catch (Exception e) { LOG.error("failed", e); - Assert.fail(e.getMessage()); + fail(e.getMessage()); } output = baos.toString(); LOG.info(baos.toString()); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java new file mode 100644 index 000000000000..919d3e79f720 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java @@ -0,0 +1,186 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.ToolRunner; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDeleteWithContinuousBackupAndPITR extends TestBackupBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupDeleteWithContinuousBackupAndPITR.class); + + private BackupSystemTable backupSystemTable; + private String backupId1; + private String backupId2; + private String backupId3; + private String backupId4; + private String backupId5; + + /** + * Sets up the backup environment before each test. + *

+ * This includes: + *

    + *
  • Setting a 30-day PITR (Point-In-Time Recovery) window
  • + *
  • Registering table2 as a continuous backup table starting 40 days ago
  • + *
  • Creating a mix of full and incremental backups at specific time offsets (using + * EnvironmentEdge injection) to simulate scenarios like: - backups outside PITR window - valid + * PITR backups - incomplete PITR chains
  • + *
  • Resetting the system clock after time manipulation
  • + *
+ * This setup enables tests to evaluate deletion behavior of backups based on age, table type, and + * PITR chain requirements. + */ + @Before + public void setup() throws Exception { + conf1.setLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, 30); + backupSystemTable = new BackupSystemTable(TEST_UTIL.getConnection()); + + long currentTime = System.currentTimeMillis(); + long backupStartTime = currentTime - 40 * ONE_DAY_IN_MILLISECONDS; + backupSystemTable.addContinuousBackupTableSet(Set.of(table2), backupStartTime); + + backupId1 = fullTableBackup(Lists.newArrayList(table1)); + assertTrue(checkSucceeded(backupId1)); + + // 31 days back + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - 31 * ONE_DAY_IN_MILLISECONDS); + backupId2 = fullTableBackup(Lists.newArrayList(table2)); + assertTrue(checkSucceeded(backupId2)); + + // 32 days back + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - 32 * ONE_DAY_IN_MILLISECONDS); + backupId3 = fullTableBackup(Lists.newArrayList(table2)); + assertTrue(checkSucceeded(backupId3)); + + // 15 days back + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - 15 * ONE_DAY_IN_MILLISECONDS); + backupId4 = fullTableBackup(Lists.newArrayList(table2)); + assertTrue(checkSucceeded(backupId4)); + + // Reset clock + EnvironmentEdgeManager.reset(); + + backupId5 = incrementalTableBackup(Lists.newArrayList(table1)); + assertTrue(checkSucceeded(backupId5)); + } + + @After + public void teardown() throws Exception { + EnvironmentEdgeManager.reset(); + // Try to delete all backups forcefully if they exist + for (String id : List.of(backupId1, backupId2, backupId3, backupId4, backupId5)) { + try { + deleteBackup(id, true); + } catch (Exception ignored) { + } + } + } + + @Test + public void testDeleteIncrementalBackup() throws Exception { + assertDeletionSucceeds(backupSystemTable, backupId5, false); + } + + @Test + public void testDeleteFullBackupNonContinuousTable() throws Exception { + assertDeletionSucceeds(backupSystemTable, backupId1, false); + } + + @Test + public void testDeletePITRIncompleteBackup() throws Exception { + assertDeletionSucceeds(backupSystemTable, backupId4, false); + } + + @Test + public void testDeleteValidPITRBackupWithAnotherPresent() throws Exception { + assertDeletionSucceeds(backupSystemTable, backupId2, false); + } + + @Test + public void testDeleteOnlyValidPITRBackupFails() throws Exception { + // Delete backupId2 (31 days ago) — this should succeed + assertDeletionSucceeds(backupSystemTable, backupId2, false); + + // Now backupId3 (32 days ago) is the only remaining PITR backup — deletion should fail + assertDeletionFails(backupSystemTable, backupId3, false); + } + + @Test + public void testForceDeleteOnlyValidPITRBackup() throws Exception { + // Delete backupId2 (31 days ago) + assertDeletionSucceeds(backupSystemTable, backupId2, false); + + // Force delete backupId3 — should succeed despite PITR constraints + assertDeletionSucceeds(backupSystemTable, backupId3, true); + } + + private void assertDeletionSucceeds(BackupSystemTable table, String backupId, + boolean isForceDelete) throws Exception { + int ret = deleteBackup(backupId, isForceDelete); + assertEquals(0, ret); + assertFalse("Backup should be deleted but still exists!", backupExists(table, backupId)); + } + + private void assertDeletionFails(BackupSystemTable table, String backupId, boolean isForceDelete) + throws Exception { + int ret = deleteBackup(backupId, isForceDelete); + assertNotEquals(0, ret); + assertTrue("Backup should still exist after failed deletion!", backupExists(table, backupId)); + } + + private boolean backupExists(BackupSystemTable table, String backupId) throws Exception { + return table.getBackupHistory().stream() + .anyMatch(backup -> backup.getBackupId().equals(backupId)); + } + + private int deleteBackup(String backupId, boolean isForceDelete) throws Exception { + String[] args = buildBackupDeleteArgs(backupId, isForceDelete); + return ToolRunner.run(conf1, new BackupDriver(), args); + } + + private String[] buildBackupDeleteArgs(String backupId, boolean isForceDelete) { + return isForceDelete + ? new String[] { "delete", "-l", backupId, "-fd" } + : new String[] { "delete", "-l", backupId }; + } +} From 0e3b5e483828e86cb67ee410277fc09ac30d8fb2 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Tue, 20 May 2025 21:39:27 +0530 Subject: [PATCH 04/24] HBASE-29261: Investigate flaw in backup deletion validation of PITR-critical backups and propose correct approach (#6922) * improve the logic of backup deletion validation of PITR-critical backups * add new tests --- .../hbase/backup/impl/BackupCommands.java | 169 ++++++------ .../hbase/backup/impl/BackupSystemTable.java | 43 +++ ...ckupDeleteWithContinuousBackupAndPITR.java | 248 ++++++++++++------ 3 files changed, 305 insertions(+), 155 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index e9d14d1426de..804dc7141a19 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -53,9 +53,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; -import org.agrona.collections.MutableLong; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.base.Splitter; @@ -735,19 +736,20 @@ private void validatePITRBackupDeletion(String[] backupIds, boolean isForceDelet } /** - * Identifies tables that rely on the specified backup for PITR. If a table has no other valid - * FULL backups that can facilitate recovery to all points within the PITR retention window, it - * is added to the dependent list. - * @param backupId The backup ID being evaluated. - * @return List of tables dependent on the specified backup for PITR. - * @throws IOException If backup metadata cannot be retrieved. + * Identifies tables that rely on the specified backup for PITR (Point-In-Time Recovery). A + * table is considered dependent on the backup if it does not have any other valid full backups + * that can cover the PITR window enabled by the specified backup. + * @param backupId The ID of the backup being evaluated for PITR coverage. + * @return A list of tables that are dependent on the specified backup for PITR recovery. + * @throws IOException If there is an error retrieving the backup metadata or backup system + * table. */ private List getTablesDependentOnBackupForPITR(String backupId) throws IOException { List dependentTables = new ArrayList<>(); try (final BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { + // Fetch the target backup's info using the backup ID BackupInfo targetBackup = backupSystemTable.readBackupInfo(backupId); - if (targetBackup == null) { throw new IOException("Backup info not found for backupId: " + backupId); } @@ -757,104 +759,121 @@ private List getTablesDependentOnBackupForPITR(String backupId) throw return List.of(); } - // Retrieve the tables with continuous backup enabled and their start times + // Retrieve the tables with continuous backup enabled along with their start times Map continuousBackupStartTimes = backupSystemTable.getContinuousBackupTableSet(); - // Determine the PITR time window + // Calculate the PITR window by fetching configuration and current time long pitrWindowDays = getConf().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS); long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); - final MutableLong pitrMaxStartTime = - new MutableLong(currentTime - TimeUnit.DAYS.toMillis(pitrWindowDays)); - - // For all tables, determine the earliest (minimum) continuous backup start time. - // This represents the actual earliest point-in-time recovery (PITR) timestamp - // that can be used, ensuring we do not go beyond the available backup data. - long minContinuousBackupStartTime = currentTime; - for (TableName table : targetBackup.getTableNames()) { - minContinuousBackupStartTime = Math.min(minContinuousBackupStartTime, - continuousBackupStartTimes.getOrDefault(table, currentTime)); - } - - // The PITR max start time should be the maximum of the calculated minimum continuous backup - // start time and the default PITR max start time (based on the configured window). - // This ensures that PITR does not extend beyond what is practically possible. - pitrMaxStartTime.set(Math.max(minContinuousBackupStartTime, pitrMaxStartTime.longValue())); + final long maxAllowedPITRTime = currentTime - TimeUnit.DAYS.toMillis(pitrWindowDays); + // Check each table associated with the target backup for (TableName table : targetBackup.getTableNames()) { - // This backup is not necessary for this table since it doesn't have PITR enabled + // Skip tables without continuous backup enabled if (!continuousBackupStartTimes.containsKey(table)) { continue; } - if ( - !isValidPITRBackup(targetBackup, table, continuousBackupStartTimes, - pitrMaxStartTime.longValue()) - ) { - continue; // This backup is not crucial for PITR of this table + + // Calculate the PITR window this backup covers for the table + Optional> coveredPitrWindow = getCoveredPitrWindowForTable(targetBackup, + continuousBackupStartTimes.get(table), maxAllowedPITRTime, currentTime); + + // If this backup does not cover a valid PITR window for the table, skip + if (coveredPitrWindow.isEmpty()) { + continue; } - // Check if another valid full backup exists for this table - List backupHistory = backupSystemTable.getBackupInfos(BackupState.COMPLETE); - boolean hasAnotherValidBackup = backupHistory.stream() - .anyMatch(backup -> !backup.getBackupId().equals(backupId) && isValidPITRBackup(backup, - table, continuousBackupStartTimes, pitrMaxStartTime.longValue())); + // Check if there is any other valid backup that can cover the PITR window + List allBackups = backupSystemTable.getBackupInfos(BackupState.COMPLETE); + boolean hasAnotherValidBackup = + canAnyOtherBackupCover(allBackups, targetBackup, table, coveredPitrWindow.get(), + continuousBackupStartTimes.get(table), maxAllowedPITRTime, currentTime); + // If no other valid backup exists, add the table to the dependent list if (!hasAnotherValidBackup) { dependentTables.add(table); } } } + return dependentTables; } /** - * Determines if a given backup is a valid candidate for Point-In-Time Recovery (PITR) for a - * specific table. A valid backup ensures that recovery is possible to any point within the PITR - * retention window. A backup qualifies if: - *
    - *
  • It is a FULL backup.
  • - *
  • It contains the specified table.
  • - *
  • Its completion timestamp is before the PITR retention window start time.
  • - *
  • Its completion timestamp is on or after the table’s continuous backup start time.
  • - *
- * @param backupInfo The backup information being evaluated. - * @param tableName The table for which PITR validity is being checked. - * @param continuousBackupTables A map of tables to their continuous backup start time. - * @param pitrMaxStartTime The maximum allowed start timestamp for PITR eligibility. - * @return {@code true} if the backup enables recovery to all valid points in time for the - * table; {@code false} otherwise. + * Calculates the PITR (Point-In-Time Recovery) window that the given backup enables for a + * table. + * @param backupInfo Metadata of the backup being evaluated. + * @param continuousBackupStartTime When continuous backups started for the table. + * @param maxAllowedPITRTime The earliest timestamp from which PITR is supported in the + * cluster. + * @param currentTime Current time. + * @return Optional PITR window as a pair (start, end), or empty if backup is not useful for + * PITR. */ - private boolean isValidPITRBackup(BackupInfo backupInfo, TableName tableName, - Map continuousBackupTables, long pitrMaxStartTime) { - // Only FULL backups are mandatory for PITR - if (!BackupType.FULL.equals(backupInfo.getType())) { - return false; - } + private Optional> getCoveredPitrWindowForTable(BackupInfo backupInfo, + long continuousBackupStartTime, long maxAllowedPITRTime, long currentTime) { - // The backup must include the table to be relevant for PITR - if (!backupInfo.getTableNames().contains(tableName)) { - return false; - } + long backupStartTs = backupInfo.getStartTs(); + long backupEndTs = backupInfo.getCompleteTs(); + long effectiveStart = Math.max(continuousBackupStartTime, maxAllowedPITRTime); - // The backup must have been completed before the PITR retention window starts, - // otherwise, it won't be helpful in cases where the recovery point is between - // pitrMaxStartTime and the backup completion time. - if (backupInfo.getCompleteTs() > pitrMaxStartTime) { - return false; + if (backupStartTs < continuousBackupStartTime) { + return Optional.empty(); } - // Retrieve the table's continuous backup start time - long continuousBackupStartTime = continuousBackupTables.getOrDefault(tableName, 0L); + return Optional.of(Pair.newPair(Math.max(backupEndTs, effectiveStart), currentTime)); + } - // The backup must have been started on or after the table’s continuous backup start time, - // otherwise, it won't be helpful in few cases because we wouldn't have the WAL entries - // between the backup start time and the continuous backup start time. - if (backupInfo.getStartTs() < continuousBackupStartTime) { - return false; + /** + * Checks if any backup (excluding the current backup) can cover the specified PITR window for + * the given table. A backup can cover the PITR window if it fully encompasses the target time + * range specified. + * @param allBackups List of all backups available. + * @param currentBackup The current backup that should not be considered for + * coverage. + * @param table The table for which we need to check backup coverage. + * @param targetWindow A pair representing the target PITR window (start and end + * times). + * @param continuousBackupStartTime When continuous backups started for the table. + * @param maxAllowedPITRTime The earliest timestamp from which PITR is supported in the + * cluster. + * @param currentTime Current time. + * @return {@code true} if any backup (excluding the current one) fully covers the target PITR + * window; {@code false} otherwise. + */ + private boolean canAnyOtherBackupCover(List allBackups, BackupInfo currentBackup, + TableName table, Pair targetWindow, long continuousBackupStartTime, + long maxAllowedPITRTime, long currentTime) { + + long targetStart = targetWindow.getFirst(); + long targetEnd = targetWindow.getSecond(); + + // Iterate through all backups (including the current one) + for (BackupInfo backup : allBackups) { + // Skip if the backup is not full or doesn't contain the table + if (!BackupType.FULL.equals(backup.getType())) continue; + if (!backup.getTableNames().contains(table)) continue; + + // Skip the current backup itself + if (backup.equals(currentBackup)) continue; + + // Get the covered PITR window for this backup + Optional> coveredWindow = getCoveredPitrWindowForTable(backup, + continuousBackupStartTime, maxAllowedPITRTime, currentTime); + + if (coveredWindow.isPresent()) { + Pair covered = coveredWindow.get(); + + // The backup must fully cover the target window + if (covered.getFirst() <= targetStart && covered.getSecond() >= targetEnd) { + return true; + } + } } - return true; + return false; } @Override diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 752f448a3016..7ea78cedfcbc 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -973,6 +973,36 @@ public void addContinuousBackupTableSet(Set tables, long startTimesta } } + /** + * Removes tables from the global continuous backup set. Only removes entries that currently exist + * in the backup system table. + * @param tables set of tables to remove + * @throws IOException if an error occurs while updating the backup system table + */ + public void removeContinuousBackupTableSet(Set tables) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Remove continuous backup table set from backup system table. tables [" + + StringUtils.join(tables, " ") + "]"); + } + if (LOG.isDebugEnabled()) { + tables.forEach(table -> LOG.debug("Removing: " + table)); + } + + Map existingTables = getContinuousBackupTableSet(); + Set toRemove = + tables.stream().filter(existingTables::containsKey).collect(Collectors.toSet()); + + if (toRemove.isEmpty()) { + LOG.debug("No matching tables found to remove from continuous backup set."); + return; + } + + try (Table table = connection.getTable(tableName)) { + Delete delete = createDeleteForContinuousBackupTableSet(toRemove); + table.delete(delete); + } + } + /** * Deletes incremental backup set for a backup destination * @param backupRoot backup root @@ -1360,6 +1390,19 @@ private Delete createDeleteForIncrBackupTableSet(String backupRoot) { return delete; } + /** + * Creates Delete for continuous backup table set + * @param tables tables to remove + * @return delete operation + */ + private Delete createDeleteForContinuousBackupTableSet(Set tables) { + Delete delete = new Delete(rowkey(CONTINUOUS_BACKUP_SET)); + for (TableName tableName : tables) { + delete.addColumns(META_FAMILY, Bytes.toBytes(tableName.getNameAsString())); + } + return delete; + } + /** * Creates Scan operation to load backup history * @return scan operation diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java index 919d3e79f720..248e8e7b757c 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithContinuousBackupAndPITR.java @@ -24,21 +24,33 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; -import java.util.List; +import java.io.IOException; import java.util.Set; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.ToolRunner; -import org.junit.After; -import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +/** + * Tests the deletion of HBase backups under continuous backup and PITR settings. + *

+ * Terminology: + *

    + *
  • ct (current time): Current timestamp
  • + *
  • maxAllowedPITRTime (mapt): Maximum allowed time range for PITR, typically a + * cluster-level config (e.g., 30 days ago)
  • + *
  • cst (continuousBackupStartTime): Earliest time from which continuous backup is + * available
  • + *
  • fs: Full backup start time (not reliably usable)
  • + *
  • fm: Time when snapshot (logical freeze) was taken (we don't have this)
  • + *
  • fe: Full backup end time (used as conservative proxy for fm)
  • + *
+ */ @Category(LargeTests.class) public class TestBackupDeleteWithContinuousBackupAndPITR extends TestBackupBase { @ClassRule @@ -46,112 +58,183 @@ public class TestBackupDeleteWithContinuousBackupAndPITR extends TestBackupBase HBaseClassTestRule.forClass(TestBackupDeleteWithContinuousBackupAndPITR.class); private BackupSystemTable backupSystemTable; - private String backupId1; - private String backupId2; - private String backupId3; - private String backupId4; - private String backupId5; /** - * Sets up the backup environment before each test. - *

- * This includes: - *

    - *
  • Setting a 30-day PITR (Point-In-Time Recovery) window
  • - *
  • Registering table2 as a continuous backup table starting 40 days ago
  • - *
  • Creating a mix of full and incremental backups at specific time offsets (using - * EnvironmentEdge injection) to simulate scenarios like: - backups outside PITR window - valid - * PITR backups - incomplete PITR chains
  • - *
  • Resetting the system clock after time manipulation
  • - *
- * This setup enables tests to evaluate deletion behavior of backups based on age, table type, and - * PITR chain requirements. + * Configures continuous backup with the specified CST (continuous backup start time). */ - @Before - public void setup() throws Exception { + private void configureContinuousBackup(long cstTimestamp) throws IOException { conf1.setLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, 30); backupSystemTable = new BackupSystemTable(TEST_UTIL.getConnection()); - long currentTime = System.currentTimeMillis(); - long backupStartTime = currentTime - 40 * ONE_DAY_IN_MILLISECONDS; - backupSystemTable.addContinuousBackupTableSet(Set.of(table2), backupStartTime); + backupSystemTable.addContinuousBackupTableSet(Set.of(table1), cstTimestamp); + } - backupId1 = fullTableBackup(Lists.newArrayList(table1)); - assertTrue(checkSucceeded(backupId1)); + private void cleanupContinuousBackup() throws IOException { + backupSystemTable.removeContinuousBackupTableSet(Set.of(table1)); + } - // 31 days back - EnvironmentEdgeManager - .injectEdge(() -> System.currentTimeMillis() - 31 * ONE_DAY_IN_MILLISECONDS); - backupId2 = fullTableBackup(Lists.newArrayList(table2)); - assertTrue(checkSucceeded(backupId2)); + /** + * Main Case: continuousBackupStartTime < maxAllowedPITRTime + *

+ * Sub Case: fe < cst + */ + @Test + public void testDeletionWhenBackupCompletesBeforeCST() throws Exception { + long now = System.currentTimeMillis(); + long cst = now - 40 * ONE_DAY_IN_MILLISECONDS; // CST = 40 days ago + configureContinuousBackup(cst); - // 32 days back - EnvironmentEdgeManager - .injectEdge(() -> System.currentTimeMillis() - 32 * ONE_DAY_IN_MILLISECONDS); - backupId3 = fullTableBackup(Lists.newArrayList(table2)); - assertTrue(checkSucceeded(backupId3)); + String backupId = + createAndUpdateBackup(cst - ONE_DAY_IN_MILLISECONDS, cst - ONE_DAY_IN_MILLISECONDS + 1000); + assertDeletionSucceeds(backupSystemTable, backupId, false); - // 15 days back - EnvironmentEdgeManager - .injectEdge(() -> System.currentTimeMillis() - 15 * ONE_DAY_IN_MILLISECONDS); - backupId4 = fullTableBackup(Lists.newArrayList(table2)); - assertTrue(checkSucceeded(backupId4)); + cleanupContinuousBackup(); + } - // Reset clock - EnvironmentEdgeManager.reset(); + /** + * Main Case: continuousBackupStartTime < maxAllowedPITRTime + *

+ * Sub Case: fs < cst < fe + */ + @Test + public void testDeletionWhenBackupStraddlesCST() throws Exception { + long now = System.currentTimeMillis(); + long cst = now - 40 * ONE_DAY_IN_MILLISECONDS; // CST = 40 days ago + configureContinuousBackup(cst); - backupId5 = incrementalTableBackup(Lists.newArrayList(table1)); - assertTrue(checkSucceeded(backupId5)); - } + String backupId = createAndUpdateBackup(cst - 1000, cst + 1000); + assertDeletionSucceeds(backupSystemTable, backupId, false); - @After - public void teardown() throws Exception { - EnvironmentEdgeManager.reset(); - // Try to delete all backups forcefully if they exist - for (String id : List.of(backupId1, backupId2, backupId3, backupId4, backupId5)) { - try { - deleteBackup(id, true); - } catch (Exception ignored) { - } - } + cleanupContinuousBackup(); } + /** + * Main Case: continuousBackupStartTime < maxAllowedPITRTime + *

+ * Sub Case: fs >= cst && fe < mapt + */ @Test - public void testDeleteIncrementalBackup() throws Exception { - assertDeletionSucceeds(backupSystemTable, backupId5, false); + public void testDeletionWhenBackupWithinCSTToMAPTRangeAndUncovered() throws Exception { + long now = System.currentTimeMillis(); + long cst = now - 40 * ONE_DAY_IN_MILLISECONDS; + long mapt = now - 30 * ONE_DAY_IN_MILLISECONDS; + configureContinuousBackup(cst); + + String backupId = createAndUpdateBackup(cst, mapt - 1000); + assertDeletionFails(backupSystemTable, backupId); + + // Cover the backup with another backup + String coverId = createAndUpdateBackup(cst, mapt - 1000); + + // Now, deletion should succeed because the backup is covered by the new one + assertDeletionSucceeds(backupSystemTable, backupId, false); + assertDeletionSucceeds(backupSystemTable, coverId, true); + + cleanupContinuousBackup(); } + /** + * Main Case: continuousBackupStartTime < maxAllowedPITRTime + *

+ * Sub Case: fs >= cst && fe >= mapt + */ @Test - public void testDeleteFullBackupNonContinuousTable() throws Exception { - assertDeletionSucceeds(backupSystemTable, backupId1, false); + public void testDeletionWhenBackupExtendsBeyondMAPTAndUncovered() throws Exception { + long now = System.currentTimeMillis(); + long cst = now - 40 * ONE_DAY_IN_MILLISECONDS; + long mapt = now - 30 * ONE_DAY_IN_MILLISECONDS; + configureContinuousBackup(cst); + + String backupId = createAndUpdateBackup(cst + 1000, mapt + 1000); + assertDeletionFails(backupSystemTable, backupId); + + // Cover the backup with another backup + String coverId = createAndUpdateBackup(cst + 1000, mapt + 1000); + + // Now, deletion should succeed because the backup is covered by the new one + assertDeletionSucceeds(backupSystemTable, backupId, false); + assertDeletionSucceeds(backupSystemTable, coverId, true); + + cleanupContinuousBackup(); } + /** + * Main Case: continuousBackupStartTime >= maxAllowedPITRTime + *

+ * Sub Case: fs < cst + */ @Test - public void testDeletePITRIncompleteBackup() throws Exception { - assertDeletionSucceeds(backupSystemTable, backupId4, false); + public void testDeletionWhenBackupBeforeCST_ShouldSucceed() throws Exception { + long now = System.currentTimeMillis(); + long cst = now - 20 * ONE_DAY_IN_MILLISECONDS; + configureContinuousBackup(cst); + + String backupId = createAndUpdateBackup(cst - 1000, cst + 1000); + assertDeletionSucceeds(backupSystemTable, backupId, false); + + cleanupContinuousBackup(); } + /** + * Main Case: continuousBackupStartTime >= maxAllowedPITRTime + *

+ * Sub Case: fs >= cst + */ @Test - public void testDeleteValidPITRBackupWithAnotherPresent() throws Exception { - assertDeletionSucceeds(backupSystemTable, backupId2, false); + public void testDeletionWhenBackupAfterCST_ShouldFailUnlessCovered() throws Exception { + long now = System.currentTimeMillis(); + long cst = now - 20 * ONE_DAY_IN_MILLISECONDS; + configureContinuousBackup(cst); + + String backupId = createAndUpdateBackup(cst + 1000, cst + 2000); + assertDeletionFails(backupSystemTable, backupId); + + // Cover the backup with another backup + String coverId = createAndUpdateBackup(cst + 1000, cst + 2000); + + assertDeletionSucceeds(backupSystemTable, backupId, false); + assertDeletionSucceeds(backupSystemTable, coverId, true); + + cleanupContinuousBackup(); } @Test - public void testDeleteOnlyValidPITRBackupFails() throws Exception { - // Delete backupId2 (31 days ago) — this should succeed - assertDeletionSucceeds(backupSystemTable, backupId2, false); + public void testDeleteIncrementalBackup() throws Exception { + long now = System.currentTimeMillis(); + long cst = now - 20 * ONE_DAY_IN_MILLISECONDS; + configureContinuousBackup(cst); + + String fullBackupId = fullTableBackup(Lists.newArrayList(table1)); + String incrementalTableBackupId = incrementalTableBackup(Lists.newArrayList(table1)); + assertDeletionSucceeds(backupSystemTable, incrementalTableBackupId, false); - // Now backupId3 (32 days ago) is the only remaining PITR backup — deletion should fail - assertDeletionFails(backupSystemTable, backupId3, false); + assertDeletionSucceeds(backupSystemTable, fullBackupId, true); } @Test - public void testForceDeleteOnlyValidPITRBackup() throws Exception { - // Delete backupId2 (31 days ago) - assertDeletionSucceeds(backupSystemTable, backupId2, false); + public void testDeleteFullBackupNonContinuousTable() throws Exception { + conf1.setLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, 30); + backupSystemTable = new BackupSystemTable(TEST_UTIL.getConnection()); + + long now = System.currentTimeMillis(); + String backupId = + createAndUpdateBackup(now - ONE_DAY_IN_MILLISECONDS, now - ONE_DAY_IN_MILLISECONDS + 1000); + assertDeletionSucceeds(backupSystemTable, backupId, false); + } - // Force delete backupId3 — should succeed despite PITR constraints - assertDeletionSucceeds(backupSystemTable, backupId3, true); + /** + * Creates a full backup and updates its timestamps. + */ + private String createAndUpdateBackup(long startTs, long completeTs) throws Exception { + String backupId = fullTableBackup(Lists.newArrayList(table1)); + assertTrue(checkSucceeded(backupId)); + + BackupInfo backupInfo = getBackupInfoById(backupId); + backupInfo.setStartTs(startTs); + backupInfo.setCompleteTs(completeTs); + backupSystemTable.updateBackupInfo(backupInfo); + + return backupId; } private void assertDeletionSucceeds(BackupSystemTable table, String backupId, @@ -161,9 +244,8 @@ private void assertDeletionSucceeds(BackupSystemTable table, String backupId, assertFalse("Backup should be deleted but still exists!", backupExists(table, backupId)); } - private void assertDeletionFails(BackupSystemTable table, String backupId, boolean isForceDelete) - throws Exception { - int ret = deleteBackup(backupId, isForceDelete); + private void assertDeletionFails(BackupSystemTable table, String backupId) throws Exception { + int ret = deleteBackup(backupId, false); assertNotEquals(0, ret); assertTrue("Backup should still exist after failed deletion!", backupExists(table, backupId)); } @@ -183,4 +265,10 @@ private String[] buildBackupDeleteArgs(String backupId, boolean isForceDelete) { ? new String[] { "delete", "-l", backupId, "-fd" } : new String[] { "delete", "-l", backupId }; } + + private BackupInfo getBackupInfoById(String backupId) throws IOException { + return backupSystemTable.getBackupInfos(BackupInfo.BackupState.COMPLETE).stream() + .filter(b -> b.getBackupId().equals(backupId)).findFirst() + .orElseThrow(() -> new IllegalStateException("Backup should exist: " + backupId)); + } } From c4bef9eba9917175d9a8c23118299670deb9d23a Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Sat, 31 May 2025 00:07:44 +0530 Subject: [PATCH 05/24] HBASE-29133: Implement "pitr" Command for Point-in-Time Restore (#6717) Signed-off-by: Andor Molnar Signed-off-by: Tak Lon (Stephen) Wu --- bin/hbase | 17 + .../hbase/backup/AbstractRestoreDriver.java | 232 +++++++++++ .../hadoop/hbase/backup/BackupAdmin.java | 7 + .../hbase/backup/BackupRestoreConstants.java | 10 + .../hbase/backup/HBackupFileSystem.java | 40 ++ .../backup/PointInTimeRestoreDriver.java | 137 +++++++ .../backup/PointInTimeRestoreRequest.java | 111 ++++++ .../hadoop/hbase/backup/RestoreDriver.java | 218 +---------- .../hbase/backup/impl/BackupAdminImpl.java | 360 +++++++++++++++++- .../hbase/backup/impl/BackupSystemTable.java | 86 +++++ .../backup/impl/FullTableBackupClient.java | 12 +- .../ContinuousBackupReplicationEndpoint.java | 33 ++ .../hadoop/hbase/backup/util/BackupUtils.java | 162 ++++++++ .../hadoop/hbase/backup/TestBackupBase.java | 8 + .../hbase/backup/TestPointInTimeRestore.java | 277 ++++++++++++++ 15 files changed, 1496 insertions(+), 214 deletions(-) create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/AbstractRestoreDriver.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java diff --git a/bin/hbase b/bin/hbase index d8d9b6ec5b25..d0343dc85a3d 100755 --- a/bin/hbase +++ b/bin/hbase @@ -102,6 +102,7 @@ show_usage() { echo " version Print the version" echo " backup Backup tables for recovery" echo " restore Restore tables from existing backup image" + echo " pitr Restore tables to a specific point in time using backup and WAL replay" echo " completebulkload Run BulkLoadHFiles tool" echo " regionsplitter Run RegionSplitter tool" echo " rowcounter Run RowCounter tool" @@ -639,6 +640,22 @@ elif [ "$COMMAND" = "restore" ] ; then fi done fi +elif [ "$COMMAND" = "pitr" ] ; then + CLASS='org.apache.hadoop.hbase.backup.PointInTimeRestoreDriver' + if [ -n "${shaded_jar}" ] ; then + for f in "${HBASE_HOME}"/lib/hbase-backup*.jar; do + if [ -f "${f}" ]; then + CLASSPATH="${CLASSPATH}:${f}" + break + fi + done + for f in "${HBASE_HOME}"/lib/commons-lang3*.jar; do + if [ -f "${f}" ]; then + CLASSPATH="${CLASSPATH}:${f}" + break + fi + done + fi elif [ "$COMMAND" = "upgrade" ] ; then echo "This command was used to upgrade to HBase 0.96, it was removed in HBase 2.0.0." echo "Please follow the documentation at http://hbase.apache.org/book.html#upgrading." diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/AbstractRestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/AbstractRestoreDriver.java new file mode 100644 index 000000000000..8e053a73a6bb --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/AbstractRestoreDriver.java @@ -0,0 +1,232 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_RESTORE_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_RESTORE_DESC; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.logging.Log4jUtils; +import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; + +@InterfaceAudience.Private +public abstract class AbstractRestoreDriver extends AbstractHBaseTool { + protected static final Logger LOG = LoggerFactory.getLogger(AbstractRestoreDriver.class); + protected CommandLine cmd; + + protected static final String USAGE_FOOTER = ""; + + protected AbstractRestoreDriver() { + init(); + } + + protected void init() { + Log4jUtils.disableZkAndClientLoggers(); + } + + protected abstract int executeRestore(boolean check, TableName[] fromTables, TableName[] toTables, + boolean isOverwrite); + + private int parseAndRun() throws IOException { + if (!BackupManager.isBackupEnabled(getConf())) { + System.err.println(BackupRestoreConstants.ENABLE_BACKUP); + return -1; + } + + if (cmd.hasOption(OPTION_DEBUG)) { + Log4jUtils.setLogLevel("org.apache.hadoop.hbase.backup", "DEBUG"); + } + + boolean overwrite = cmd.hasOption(OPTION_OVERWRITE); + if (overwrite) { + LOG.debug("Found overwrite option (-{}) in restore command, " + + "will overwrite to existing table if any in the restore target", OPTION_OVERWRITE); + } + + boolean check = cmd.hasOption(OPTION_CHECK); + if (check) { + LOG.debug( + "Found check option (-{}) in restore command, will check and verify the dependencies", + OPTION_CHECK); + } + + if (cmd.hasOption(OPTION_SET) && cmd.hasOption(OPTION_TABLE)) { + System.err.printf( + "Set name (-%s) and table list (-%s) are mutually exclusive, you can not specify both " + + "of them.%n", + OPTION_SET, OPTION_TABLE); + printToolUsage(); + return -1; + } + + if (!cmd.hasOption(OPTION_SET) && !cmd.hasOption(OPTION_TABLE)) { + System.err.printf( + "You have to specify either set name (-%s) or table list (-%s) to " + "restore%n", + OPTION_SET, OPTION_TABLE); + printToolUsage(); + return -1; + } + + if (cmd.hasOption(OPTION_YARN_QUEUE_NAME)) { + String queueName = cmd.getOptionValue(OPTION_YARN_QUEUE_NAME); + // Set MR job queuename to configuration + getConf().set("mapreduce.job.queuename", queueName); + } + + String tables; + TableName[] sTableArray; + TableName[] tTableArray; + + String tableMapping = cmd.getOptionValue(OPTION_TABLE_MAPPING); + + try (final Connection conn = ConnectionFactory.createConnection(conf)) { + // Check backup set + if (cmd.hasOption(OPTION_SET)) { + String setName = cmd.getOptionValue(OPTION_SET); + try { + tables = getTablesForSet(conn, setName); + } catch (IOException e) { + System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName); + printToolUsage(); + return -2; + } + if (tables == null) { + System.out + .println("ERROR: Backup set '" + setName + "' is either empty or does not exist"); + printToolUsage(); + return -3; + } + } else { + tables = cmd.getOptionValue(OPTION_TABLE); + } + + sTableArray = BackupUtils.parseTableNames(tables); + tTableArray = BackupUtils.parseTableNames(tableMapping); + + if ( + sTableArray != null && tTableArray != null && (sTableArray.length != tTableArray.length) + ) { + System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping); + printToolUsage(); + return -4; + } + } + + return executeRestore(check, sTableArray, tTableArray, overwrite); + } + + @Override + protected void addOptions() { + addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC); + addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC); + addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC); + addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC); + addOptWithArg(OPTION_TABLE, OPTION_TABLE_LIST_DESC); + addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC); + addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_RESTORE_DESC); + } + + @Override + protected void processOptions(CommandLine cmd) { + this.cmd = cmd; + } + + @Override + protected int doWork() throws Exception { + return parseAndRun(); + } + + @Override + public int run(String[] args) { + Objects.requireNonNull(conf, "Tool configuration is not initialized"); + + try { + cmd = parseArgs(args); + } catch (Exception e) { + System.out.println("Error parsing command-line arguments: " + e.getMessage()); + printToolUsage(); + return EXIT_FAILURE; + } + + if (cmd.hasOption(SHORT_HELP_OPTION) || cmd.hasOption(LONG_HELP_OPTION)) { + printToolUsage(); + return EXIT_FAILURE; + } + + processOptions(cmd); + + try { + return doWork(); + } catch (Exception e) { + LOG.error("Error running restore tool", e); + return EXIT_FAILURE; + } + } + + protected void printToolUsage() { + System.out.println(getUsageString()); + HelpFormatter helpFormatter = new HelpFormatter(); + helpFormatter.setLeftPadding(2); + helpFormatter.setDescPadding(8); + helpFormatter.setWidth(100); + helpFormatter.setSyntaxPrefix("Options:"); + helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); + System.out.println(BackupRestoreConstants.VERIFY_BACKUP); + } + + protected abstract String getUsageString(); + + private String getTablesForSet(Connection conn, String name) throws IOException { + try (final BackupSystemTable table = new BackupSystemTable(conn)) { + List tables = table.describeBackupSet(name); + + if (tables == null) { + return null; + } + + return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java index 25055fd5e8e6..7fdf1d9bfbff 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java @@ -51,6 +51,13 @@ public interface BackupAdmin extends Closeable { */ void restore(RestoreRequest request) throws IOException; + /** + * Restore the tables to specific time + * @param request Point in Time restore request + * @throws IOException exception + */ + void pointInTimeRestore(PointInTimeRestoreRequest request) throws IOException; + /** * Describe backup image command * @param backupId backup id diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java index 3df67ac1aeff..be6e4c2686d0 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java @@ -109,6 +109,16 @@ public interface BackupRestoreConstants { String OPTION_FORCE_DELETE_DESC = "Flag to forcefully delete the backup, even if it may be required for Point-in-Time Restore"; + String OPTION_TO_DATETIME = "td"; + String LONG_OPTION_TO_DATETIME = "to-datetime"; + String OPTION_TO_DATETIME_DESC = "Target date and time up to which data should be restored"; + + String OPTION_PITR_BACKUP_PATH = "bp"; + String LONG_OPTION_PITR_BACKUP_PATH = "backup-path"; + String OPTION_PITR_BACKUP_PATH_DESC = + "Specifies a custom backup location for Point-In-Time Recovery (PITR). " + + "If provided, this location will be used exclusively instead of deriving the path from the system table."; + String JOB_NAME_CONF_KEY = "mapreduce.job.name"; String BACKUP_CONFIG_STRING = BackupRestoreConstants.BACKUP_ENABLE_KEY + "=true\n" diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java index d5fd9aaf4c34..0ad5b3f0e05c 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java @@ -17,13 +17,21 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUPID_PREFIX; + import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupManifest; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -135,4 +143,36 @@ public static BackupManifest getManifest(Configuration conf, Path backupRootPath new BackupManifest(conf, getManifestPath(conf, backupRootPath, backupId)); return manifest; } + + public static List getAllBackupImages(Configuration conf, Path backupRootPath) + throws IOException { + FileSystem fs = FileSystem.get(backupRootPath.toUri(), conf); + RemoteIterator it = fs.listLocatedStatus(backupRootPath); + + List images = new ArrayList<>(); + + while (it.hasNext()) { + LocatedFileStatus lfs = it.next(); + if (!lfs.isDirectory()) { + continue; + } + + String backupId = lfs.getPath().getName(); + try { + BackupManifest manifest = getManifest(conf, backupRootPath, backupId); + images.add(manifest.getBackupImage()); + } catch (IOException e) { + LOG.error("Cannot load backup manifest from: " + lfs.getPath(), e); + } + } + + // Sort images by timestamp in descending order + images.sort(Comparator.comparingLong(m -> -getTimestamp(m.getBackupId()))); + + return images; + } + + private static long getTimestamp(String backupId) { + return Long.parseLong(backupId.substring(BACKUPID_PREFIX.length())); + } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java new file mode 100644 index 000000000000..abdf52f14302 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java @@ -0,0 +1,137 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.LONG_OPTION_PITR_BACKUP_PATH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.LONG_OPTION_TO_DATETIME; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PITR_BACKUP_PATH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PITR_BACKUP_PATH_DESC; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TO_DATETIME; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TO_DATETIME_DESC; + +import java.net.URI; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.base.Strings; + +/** + * Command-line entry point for restore operation + */ +@InterfaceAudience.Private +public class PointInTimeRestoreDriver extends AbstractRestoreDriver { + private static final String USAGE_STRING = """ + Usage: hbase pitr [options] + Backup Path to use for Point in Time Restore + table(s) Comma-separated list of tables to restore + """; + + @Override + protected int executeRestore(boolean check, TableName[] fromTables, TableName[] toTables, + boolean isOverwrite) { + String walBackupDir = getConf().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + if (Strings.isNullOrEmpty(walBackupDir)) { + System.err.printf( + "Point-in-Time Restore requires the WAL backup directory (%s) to replay logs after full and incremental backups. " + + "Set this property if you need Point-in-Time Restore. Otherwise, use the normal restore process with the appropriate backup ID.%n", + CONF_CONTINUOUS_BACKUP_WAL_DIR); + return -1; + } + + String[] remainArgs = cmd.getArgs(); + if (remainArgs.length != 0) { + printToolUsage(); + return -1; + } + + String backupRootDir = cmd.getOptionValue(OPTION_PITR_BACKUP_PATH); + + try (final Connection conn = ConnectionFactory.createConnection(conf); + BackupAdmin client = new BackupAdminImpl(conn)) { + // Get the replication checkpoint (last known safe point for Continuous Backup) + long replicationCheckpoint = BackupUtils.getReplicationCheckpoint(conn); + long endTime = replicationCheckpoint; + + if (cmd.hasOption(OPTION_TO_DATETIME)) { + String time = cmd.getOptionValue(OPTION_TO_DATETIME); + try { + endTime = Long.parseLong(time); + // Convert seconds to milliseconds if input is in seconds + if (endTime < 10_000_000_000L) { + endTime *= 1000; + } + } catch (NumberFormatException e) { + System.out.println("ERROR: Invalid timestamp format for --to-datetime: " + time); + printToolUsage(); + return -5; + } + } + + // Ensure the requested restore time does not exceed the replication checkpoint + if (endTime > replicationCheckpoint) { + LOG.error( + "ERROR: Requested restore time ({}) exceeds the last known safe replication checkpoint ({}). " + + "Please choose a time before this checkpoint to ensure data consistency.", + endTime, replicationCheckpoint); + return -5; + } + + PointInTimeRestoreRequest pointInTimeRestoreRequest = new PointInTimeRestoreRequest.Builder() + .withBackupRootDir(backupRootDir).withCheck(check).withFromTables(fromTables) + .withToTables(toTables).withOverwrite(isOverwrite).withToDateTime(endTime).build(); + + client.pointInTimeRestore(pointInTimeRestoreRequest); + } catch (Exception e) { + LOG.error("Error while running restore backup", e); + return -5; + } + return 0; + } + + @Override + protected void addOptions() { + super.addOptions(); + addOptWithArg(OPTION_TO_DATETIME, LONG_OPTION_TO_DATETIME, OPTION_TO_DATETIME_DESC); + addOptWithArg(OPTION_PITR_BACKUP_PATH, LONG_OPTION_PITR_BACKUP_PATH, + OPTION_PITR_BACKUP_PATH_DESC); + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + Path rootDir = CommonFSUtils.getRootDir(conf); + URI defaultFs = rootDir.getFileSystem(conf).getUri(); + CommonFSUtils.setFsDefault(conf, new Path(defaultFs)); + int ret = ToolRunner.run(conf, new PointInTimeRestoreDriver(), args); + System.exit(ret); + } + + @Override + protected String getUsageString() { + return USAGE_STRING; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java new file mode 100644 index 000000000000..f2462a1cfd18 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java @@ -0,0 +1,111 @@ +/* + * 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; + +import org.apache.hadoop.hbase.TableName; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * POJO class for Point In Time Restore request + */ +@InterfaceAudience.Private +public final class PointInTimeRestoreRequest { + + private final String backupRootDir; + private final boolean check; + private final TableName[] fromTables; + private final TableName[] toTables; + private final boolean overwrite; + private final long toDateTime; + + private PointInTimeRestoreRequest(Builder builder) { + this.backupRootDir = builder.backupRootDir; + this.check = builder.check; + this.fromTables = builder.fromTables; + this.toTables = builder.toTables; + this.overwrite = builder.overwrite; + this.toDateTime = builder.toDateTime; + } + + public String getBackupRootDir() { + return backupRootDir; + } + + public boolean isCheck() { + return check; + } + + public TableName[] getFromTables() { + return fromTables; + } + + public TableName[] getToTables() { + return toTables; + } + + public boolean isOverwrite() { + return overwrite; + } + + public long getToDateTime() { + return toDateTime; + } + + public static class Builder { + private String backupRootDir; + private boolean check = false; + private TableName[] fromTables; + private TableName[] toTables; + private boolean overwrite = false; + private long toDateTime; + + public Builder withBackupRootDir(String backupRootDir) { + this.backupRootDir = backupRootDir; + return this; + } + + public Builder withCheck(boolean check) { + this.check = check; + return this; + } + + public Builder withFromTables(TableName[] fromTables) { + this.fromTables = fromTables; + return this; + } + + public Builder withToTables(TableName[] toTables) { + this.toTables = toTables; + return this; + } + + public Builder withOverwrite(boolean overwrite) { + this.overwrite = overwrite; + return this; + } + + public Builder withToDateTime(long dateTime) { + this.toDateTime = dateTime; + return this; + } + + public PointInTimeRestoreRequest build() { + return new PointInTimeRestoreRequest(this); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java index 38b767ecf67e..efc4b0df9a03 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java @@ -17,117 +17,34 @@ */ package org.apache.hadoop.hbase.backup; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK_DESC; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE_DESC; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_RESTORE_DESC; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING_DESC; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_RESTORE_DESC; - -import java.io.IOException; import java.net.URI; -import java.util.List; -import java.util.Objects; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; -import org.apache.hadoop.hbase.backup.impl.BackupManager; -import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.logging.Log4jUtils; -import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.util.ToolRunner; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; -import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; /** * Command-line entry point for restore operation */ @InterfaceAudience.Private -public class RestoreDriver extends AbstractHBaseTool { - private static final Logger LOG = LoggerFactory.getLogger(RestoreDriver.class); - private CommandLine cmd; - - private static final String USAGE_STRING = - "Usage: hbase restore [options]\n" - + " backup_path Path to a backup destination root\n" - + " backup_id Backup image ID to restore\n" - + " table(s) Comma-separated list of tables to restore\n"; - - private static final String USAGE_FOOTER = ""; - - protected RestoreDriver() throws IOException { - init(); - } - - protected void init() { - // disable irrelevant loggers to avoid it mess up command output - Log4jUtils.disableZkAndClientLoggers(); - } - - private int parseAndRun() throws IOException { - // Check if backup is enabled - if (!BackupManager.isBackupEnabled(getConf())) { - System.err.println(BackupRestoreConstants.ENABLE_BACKUP); - return -1; - } - - // enable debug logging - if (cmd.hasOption(OPTION_DEBUG)) { - Log4jUtils.setLogLevel("org.apache.hadoop.hbase.backup", "DEBUG"); - } - - // whether to overwrite to existing table if any, false by default - boolean overwrite = cmd.hasOption(OPTION_OVERWRITE); - if (overwrite) { - LOG.debug("Found -overwrite option in restore command, " - + "will overwrite to existing table if any in the restore target"); - } - - // whether to only check the dependencies, false by default - boolean check = cmd.hasOption(OPTION_CHECK); - if (check) { - LOG.debug( - "Found -check option in restore command, " + "will check and verify the dependencies"); - } - - if (cmd.hasOption(OPTION_SET) && cmd.hasOption(OPTION_TABLE)) { - System.err.println( - "Options -s and -t are mutaully exclusive," + " you can not specify both of them."); - printToolUsage(); - return -1; - } - - if (!cmd.hasOption(OPTION_SET) && !cmd.hasOption(OPTION_TABLE)) { - System.err.println("You have to specify either set name or table list to restore"); - printToolUsage(); - return -1; - } - - if (cmd.hasOption(OPTION_YARN_QUEUE_NAME)) { - String queueName = cmd.getOptionValue(OPTION_YARN_QUEUE_NAME); - // Set MR job queuename to configuration - getConf().set("mapreduce.job.queuename", queueName); - } +public class RestoreDriver extends AbstractRestoreDriver { + private static final String USAGE_STRING = """ + Usage: hbase restore [options] + backup_path Path to a backup destination root + backup_id Backup image ID to restore + table(s) Comma-separated list of tables to restore + """; + @Override + protected int executeRestore(boolean check, TableName[] fromTables, TableName[] toTables, + boolean isOverwrite) { // parse main restore command options String[] remainArgs = cmd.getArgs(); if (remainArgs.length != 2) { @@ -137,44 +54,11 @@ private int parseAndRun() throws IOException { String backupRootDir = remainArgs[0]; String backupId = remainArgs[1]; - String tables; - String tableMapping = - cmd.hasOption(OPTION_TABLE_MAPPING) ? cmd.getOptionValue(OPTION_TABLE_MAPPING) : null; + try (final Connection conn = ConnectionFactory.createConnection(conf); BackupAdmin client = new BackupAdminImpl(conn)) { - // Check backup set - if (cmd.hasOption(OPTION_SET)) { - String setName = cmd.getOptionValue(OPTION_SET); - try { - tables = getTablesForSet(conn, setName); - } catch (IOException e) { - System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName); - printToolUsage(); - return -2; - } - if (tables == null) { - System.out - .println("ERROR: Backup set '" + setName + "' is either empty or does not exist"); - printToolUsage(); - return -3; - } - } else { - tables = cmd.getOptionValue(OPTION_TABLE); - } - - TableName[] sTableArray = BackupUtils.parseTableNames(tables); - TableName[] tTableArray = BackupUtils.parseTableNames(tableMapping); - - if ( - sTableArray != null && tTableArray != null && (sTableArray.length != tTableArray.length) - ) { - System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping); - printToolUsage(); - return -4; - } - - client.restore(BackupUtils.createRestoreRequest(backupRootDir, backupId, check, sTableArray, - tTableArray, overwrite)); + client.restore(BackupUtils.createRestoreRequest(backupRootDir, backupId, check, fromTables, + toTables, isOverwrite)); } catch (Exception e) { LOG.error("Error while running restore backup", e); return -5; @@ -182,40 +66,6 @@ private int parseAndRun() throws IOException { return 0; } - private String getTablesForSet(Connection conn, String name) throws IOException { - try (final BackupSystemTable table = new BackupSystemTable(conn)) { - List tables = table.describeBackupSet(name); - - if (tables == null) { - return null; - } - - return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND); - } - } - - @Override - protected void addOptions() { - // define supported options - addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC); - addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC); - addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC); - addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC); - addOptWithArg(OPTION_TABLE, OPTION_TABLE_LIST_DESC); - addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC); - addOptWithArg(OPTION_YARN_QUEUE_NAME, OPTION_YARN_QUEUE_NAME_RESTORE_DESC); - } - - @Override - protected void processOptions(CommandLine cmd) { - this.cmd = cmd; - } - - @Override - protected int doWork() throws Exception { - return parseAndRun(); - } - public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); Path hbasedir = CommonFSUtils.getRootDir(conf); @@ -226,45 +76,7 @@ public static void main(String[] args) throws Exception { } @Override - public int run(String[] args) { - Objects.requireNonNull(conf, "Tool configuration is not initialized"); - - CommandLine cmd; - try { - // parse the command line arguments - cmd = parseArgs(args); - cmdLineArgs = args; - } catch (Exception e) { - System.out.println("Error when parsing command-line arguments: " + e.getMessage()); - printToolUsage(); - return EXIT_FAILURE; - } - - if (cmd.hasOption(SHORT_HELP_OPTION) || cmd.hasOption(LONG_HELP_OPTION)) { - printToolUsage(); - return EXIT_FAILURE; - } - - processOptions(cmd); - - int ret = EXIT_FAILURE; - try { - ret = doWork(); - } catch (Exception e) { - LOG.error("Error running command-line tool", e); - return EXIT_FAILURE; - } - return ret; - } - - protected void printToolUsage() { - System.out.println(USAGE_STRING); - HelpFormatter helpFormatter = new HelpFormatter(); - helpFormatter.setLeftPadding(2); - helpFormatter.setDescPadding(8); - helpFormatter.setWidth(100); - helpFormatter.setSyntaxPrefix("Options:"); - helpFormatter.printHelp(" ", null, options, USAGE_FOOTER); - System.out.println(BackupRestoreConstants.VERIFY_BACKUP); + protected String getUsageString() { + return USAGE_STRING; } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java index 1e745c69cdad..e94389d69386 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -17,18 +17,32 @@ */ package org.apache.hadoop.hbase.backup.impl; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; + import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Date; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupAdmin; import org.apache.hadoop.hbase.backup.BackupClientFactory; @@ -40,12 +54,16 @@ import org.apache.hadoop.hbase.backup.BackupRestoreFactory; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest; import org.apache.hadoop.hbase.backup.RestoreRequest; import org.apache.hadoop.hbase.backup.util.BackupSet; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.Tool; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -490,15 +508,8 @@ private String[] toStringArray(TableName[] list) { @Override public void restore(RestoreRequest request) throws IOException { if (request.isCheck()) { - // check and load backup image manifest for the tables - Path rootPath = new Path(request.getBackupRootDir()); - String backupId = request.getBackupId(); - TableName[] sTableArray = request.getFromTables(); - BackupManifest manifest = - HBackupFileSystem.getManifest(conn.getConfiguration(), rootPath, backupId); - - // Check and validate the backup image and its dependencies - if (BackupUtils.validate(Arrays.asList(sTableArray), manifest, conn.getConfiguration())) { + boolean isValid = validateRequest(request); + if (isValid) { LOG.info(CHECK_OK); } else { LOG.error(CHECK_FAILED); @@ -509,6 +520,337 @@ public void restore(RestoreRequest request) throws IOException { new RestoreTablesClient(conn, request).execute(); } + private boolean validateRequest(RestoreRequest request) throws IOException { + // check and load backup image manifest for the tables + Path rootPath = new Path(request.getBackupRootDir()); + String backupId = request.getBackupId(); + TableName[] sTableArray = request.getFromTables(); + BackupManifest manifest = + HBackupFileSystem.getManifest(conn.getConfiguration(), rootPath, backupId); + + // Validate the backup image and its dependencies + return BackupUtils.validate(Arrays.asList(sTableArray), manifest, conn.getConfiguration()); + } + + @Override + public void pointInTimeRestore(PointInTimeRestoreRequest request) throws IOException { + if (request.getBackupRootDir() == null) { + defaultPointInTimeRestore(request); + } else { + // TODO: special case, not supported at the moment + throw new IOException("Custom backup location for Point-In-Time Recovery Not supported!"); + } + LOG.info("Successfully completed Point In Time Restore for all tables."); + } + + /** + * Performs a default Point-In-Time Restore (PITR) by restoring the latest valid backup and + * replaying the WALs to bring the table to the desired state. PITR requires: 1. A valid backup + * available before the end time. 2. Write-Ahead Logs (WALs) covering the remaining duration up to + * the end time. + * @param request PointInTimeRestoreRequest containing restore parameters. + * @throws IOException If no valid backup or WALs are found, or if an error occurs during + * restoration. + */ + private void defaultPointInTimeRestore(PointInTimeRestoreRequest request) throws IOException { + long endTime = request.getToDateTime(); + validateRequestToTime(endTime); + + TableName[] sTableArray = request.getFromTables(); + TableName[] tTableArray = resolveTargetTables(sTableArray, request.getToTables()); + + // Validate PITR requirements + validatePitr(endTime, sTableArray, tTableArray); + + // If only validation is required, log and return + if (request.isCheck()) { + LOG.info("PITR can be successfully executed"); + return; + } + + // Execute PITR process + try (BackupSystemTable table = new BackupSystemTable(conn)) { + Map continuousBackupTables = table.getContinuousBackupTableSet(); + List backupInfos = table.getBackupInfos(BackupState.COMPLETE); + + for (int i = 0; i < sTableArray.length; i++) { + restoreTableWithWalReplay(sTableArray[i], tTableArray[i], endTime, continuousBackupTables, + backupInfos, request); + } + } + } + + /** + * Validates whether the requested end time falls within the allowed PITR recovery window. + * @param endTime The target recovery time. + * @throws IOException If the requested recovery time is outside the allowed window. + */ + private void validateRequestToTime(long endTime) throws IOException { + long pitrWindowDays = conn.getConfiguration().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, + DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS); + long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); + long pitrMaxStartTime = currentTime - TimeUnit.DAYS.toMillis(pitrWindowDays); + + if (endTime < pitrMaxStartTime) { + String errorMsg = String.format( + "Requested recovery time (%d) is out of the allowed PITR window (last %d days).", endTime, + pitrWindowDays); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + + if (endTime > currentTime) { + String errorMsg = String.format( + "Requested recovery time (%d) is in the future. Current time: %d.", endTime, currentTime); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + } + + /** + * Resolves the target table array. If null or empty, defaults to the source table array. + */ + private TableName[] resolveTargetTables(TableName[] sourceTables, TableName[] targetTables) { + return (targetTables == null || targetTables.length == 0) ? sourceTables : targetTables; + } + + /** + * Validates whether Point-In-Time Recovery (PITR) is possible for the given tables at the + * specified time. + *

+ * PITR requires: + *

    + *
  • Continuous backup to be enabled for the source tables.
  • + *
  • A valid backup image and corresponding WALs to be available.
  • + *
+ * @param endTime The target recovery time. + * @param sTableArray The source tables to restore. + * @param tTableArray The target tables where the restore will be performed. + * @throws IOException If PITR is not possible due to missing continuous backup or backup images. + */ + private void validatePitr(long endTime, TableName[] sTableArray, TableName[] tTableArray) + throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + // Retrieve the set of tables with continuous backup enabled + Map continuousBackupTables = table.getContinuousBackupTableSet(); + + // Ensure all source tables have continuous backup enabled + validateContinuousBackup(sTableArray, continuousBackupTables); + + // Fetch completed backup information + List backupInfos = table.getBackupInfos(BackupState.COMPLETE); + + // Ensure a valid backup and WALs exist for PITR + validateBackupAvailability(sTableArray, tTableArray, endTime, continuousBackupTables, + backupInfos); + } + } + + /** + * Ensures that all source tables have continuous backup enabled. + */ + private void validateContinuousBackup(TableName[] tables, + Map continuousBackupTables) throws IOException { + List missingTables = + Arrays.stream(tables).filter(table -> !continuousBackupTables.containsKey(table)).toList(); + + if (!missingTables.isEmpty()) { + String errorMsg = "Continuous Backup is not enabled for the following tables: " + + missingTables.stream().map(TableName::getNameAsString).collect(Collectors.joining(", ")); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + } + + /** + * Ensures that a valid backup and corresponding WALs exist for PITR for each source table. PITR + * requires: 1. A valid backup available before the end time. 2. Write-Ahead Logs (WALs) covering + * the remaining duration up to the end time. + */ + private void validateBackupAvailability(TableName[] sTableArray, TableName[] tTableArray, + long endTime, Map continuousBackupTables, List backupInfos) + throws IOException { + for (int i = 0; i < sTableArray.length; i++) { + if ( + !canPerformPitr(sTableArray[i], tTableArray[i], endTime, continuousBackupTables, + backupInfos) + ) { + String errorMsg = "Could not find a valid backup and WALs for PITR for table: " + + sTableArray[i].getNameAsString(); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + } + } + + /** + * Checks whether PITR can be performed for a given source-target table pair. + */ + private boolean canPerformPitr(TableName stableName, TableName tTableName, long endTime, + Map continuousBackupTables, List backupInfos) { + return getValidBackupInfo(stableName, tTableName, endTime, continuousBackupTables, backupInfos) + != null; + } + + /** + * Finds a valid backup for PITR that meets the required conditions. + */ + private BackupInfo getValidBackupInfo(TableName sTableName, TableName tTablename, long endTime, + Map continuousBackupTables, List backupInfos) { + for (BackupInfo info : backupInfos) { + if (isValidBackupForPitr(info, sTableName, endTime, continuousBackupTables)) { + + RestoreRequest restoreRequest = + BackupUtils.createRestoreRequest(info.getBackupRootDir(), info.getBackupId(), true, + new TableName[] { sTableName }, new TableName[] { tTablename }, false); + + try { + if (validateRequest(restoreRequest)) { + return info; + } + } catch (IOException e) { + LOG.warn("Exception occurred while testing the backup : {} for restore ", info, e); + } + } + } + return null; + } + + /** + * Determines if the given backup is valid for PITR. + *

+ * A backup is valid if: + *

    + *
  • It contains the source table.
  • + *
  • It was completed before the end time.
  • + *
  • The start timestamp of the backup is after the continuous backup start time for the + * table.
  • + *
+ * @param info Backup information object. + * @param tableName Table to check. + * @param endTime The target recovery time. + * @param continuousBackupTables Map of tables with continuous backup enabled. + * @return true if the backup is valid for PITR, false otherwise. + */ + private boolean isValidBackupForPitr(BackupInfo info, TableName tableName, long endTime, + Map continuousBackupTables) { + return info.getTableNames().contains(tableName) && info.getCompleteTs() <= endTime + && continuousBackupTables.getOrDefault(tableName, 0L) <= info.getStartTs(); + } + + /** + * Restores a table from a valid backup and replays WALs to reach the desired PITR state. + */ + private void restoreTableWithWalReplay(TableName sourceTable, TableName targetTable, long endTime, + Map continuousBackupTables, List backupInfos, + PointInTimeRestoreRequest request) throws IOException { + BackupInfo backupInfo = + getValidBackupInfo(sourceTable, targetTable, endTime, continuousBackupTables, backupInfos); + if (backupInfo == null) { + String errorMsg = "Could not find a valid backup and WALs for PITR for table: " + + sourceTable.getNameAsString(); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + + RestoreRequest restoreRequest = BackupUtils.createRestoreRequest(backupInfo.getBackupRootDir(), + backupInfo.getBackupId(), false, new TableName[] { sourceTable }, + new TableName[] { targetTable }, request.isOverwrite()); + + restore(restoreRequest); + replayWal(sourceTable, targetTable, backupInfo.getStartTs(), endTime); + } + + /** + * Replays WALs to bring the table to the desired state. + */ + private void replayWal(TableName sourceTable, TableName targetTable, long startTime, long endTime) + throws IOException { + String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + Path walDirPath = new Path(walBackupDir); + LOG.info( + "Starting WAL replay for source: {}, target: {}, time range: {} - {}, WAL backup dir: {}", + sourceTable, targetTable, startTime, endTime, walDirPath); + + List validDirs = + getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); + if (validDirs.isEmpty()) { + LOG.warn("No valid WAL directories found for range {} - {}. Skipping WAL replay.", startTime, + endTime); + return; + } + + executeWalReplay(validDirs, sourceTable, targetTable, startTime, endTime); + } + + /** + * Fetches valid WAL directories based on the given time range. + */ + private List getValidWalDirs(Configuration conf, Path walBackupDir, long startTime, + long endTime) throws IOException { + FileSystem backupFs = FileSystem.get(walBackupDir.toUri(), conf); + FileStatus[] dayDirs = backupFs.listStatus(new Path(walBackupDir, WALS_DIR)); + + List validDirs = new ArrayList<>(); + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + + for (FileStatus dayDir : dayDirs) { + if (!dayDir.isDirectory()) { + continue; // Skip files, only process directories + } + + String dirName = dayDir.getPath().getName(); + try { + Date dirDate = dateFormat.parse(dirName); + long dirStartTime = dirDate.getTime(); // Start of that day (00:00:00) + long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) + + // Check if this day's WAL files overlap with the required time range + if (dirEndTime >= startTime && dirStartTime <= endTime) { + validDirs.add(dayDir.getPath().toString()); + } + } catch (ParseException e) { + LOG.warn("Skipping invalid directory name: " + dirName, e); + } + } + return validDirs; + } + + /** + * Executes WAL replay using WALPlayer. + */ + private void executeWalReplay(List walDirs, TableName sourceTable, TableName targetTable, + long startTime, long endTime) throws IOException { + Tool walPlayer = initializeWalPlayer(startTime, endTime); + String[] args = + { String.join(",", walDirs), sourceTable.getNameAsString(), targetTable.getNameAsString() }; + + try { + LOG.info("Executing WALPlayer with args: {}", Arrays.toString(args)); + int exitCode = walPlayer.run(args); + if (exitCode == 0) { + LOG.info("WAL replay completed successfully for {}", targetTable); + } else { + throw new IOException("WAL replay failed with exit code: " + exitCode); + } + } catch (Exception e) { + LOG.error("Error during WAL replay for {}: {}", targetTable, e.getMessage(), e); + throw new IOException("Exception during WAL replay", e); + } + } + + /** + * Initializes and configures WALPlayer. + */ + private Tool initializeWalPlayer(long startTime, long endTime) { + Configuration conf = HBaseConfiguration.create(conn.getConfiguration()); + conf.setLong(WALInputFormat.START_TIME_KEY, startTime); + conf.setLong(WALInputFormat.END_TIME_KEY, endTime); + Tool walPlayer = new WALPlayer(); + walPlayer.setConf(conf); + return walPlayer; + } + @Override public String backupTables(BackupRequest request) throws IOException { BackupType type = request.getBackupType(); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 7ea78cedfcbc..3166ddf3ef28 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -170,6 +170,12 @@ public String toString() { private final static String INCR_BACKUP_SET = "incrbackupset:"; private final static String CONTINUOUS_BACKUP_SET = "continuousbackupset"; + /** + * Row key identifier for storing the last replicated WAL timestamp in the backup system table for + * continuous backup. + */ + private static final String CONTINUOUS_BACKUP_REPLICATION_TIMESTAMP_ROW = + "continuous_backup_last_replicated"; private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm:"; private final static String RS_LOG_TS_PREFIX = "rslogts:"; @@ -1003,6 +1009,86 @@ public void removeContinuousBackupTableSet(Set tables) throws IOExcep } } + /** + * Updates the latest replicated WAL timestamp for a region server in the backup system table. + * This is used to track the replication checkpoint for continuous backup and PITR (Point-in-Time + * Restore). + * @param serverName the server for which the latest WAL timestamp is being recorded + * @param timestamp the timestamp (in milliseconds) of the last WAL entry replicated + * @throws IOException if an error occurs while writing to the backup system table + */ + public void updateBackupCheckpointTimestamp(ServerName serverName, long timestamp) + throws IOException { + + HBaseProtos.ServerName.Builder serverProto = + HBaseProtos.ServerName.newBuilder().setHostName(serverName.getHostname()) + .setPort(serverName.getPort()).setStartCode(serverName.getStartCode()); + + try (Table table = connection.getTable(tableName)) { + Put put = createPutForBackupCheckpoint(serverProto.build().toByteArray(), timestamp); + if (!put.isEmpty()) { + table.put(put); + } + } + } + + /** + * Retrieves the latest replicated WAL timestamps for all region servers from the backup system + * table. This is used to track the replication checkpoint state for continuous backup and PITR + * (Point-in-Time Restore). + * @return a map where the key is {@link ServerName} and the value is the latest replicated WAL + * timestamp in milliseconds + * @throws IOException if an error occurs while reading from the backup system table + */ + public Map getBackupCheckpointTimestamps() throws IOException { + LOG.trace("Fetching latest backup checkpoint timestamps for all region servers."); + + Map checkpointMap = new HashMap<>(); + + byte[] rowKey = rowkey(CONTINUOUS_BACKUP_REPLICATION_TIMESTAMP_ROW); + Get get = new Get(rowKey); + get.addFamily(BackupSystemTable.META_FAMILY); + + try (Table table = connection.getTable(tableName)) { + Result result = table.get(get); + + if (result.isEmpty()) { + LOG.debug("No checkpoint timestamps found in backup system table."); + return checkpointMap; + } + + List cells = result.listCells(); + for (Cell cell : cells) { + try { + HBaseProtos.ServerName protoServer = + HBaseProtos.ServerName.parseFrom(CellUtil.cloneQualifier(cell)); + ServerName serverName = ServerName.valueOf(protoServer.getHostName(), + protoServer.getPort(), protoServer.getStartCode()); + + long timestamp = Bytes.toLong(CellUtil.cloneValue(cell)); + checkpointMap.put(serverName, timestamp); + } catch (IllegalArgumentException e) { + LOG.warn("Failed to parse server name or timestamp from cell: {}", cell, e); + } + } + } + + return checkpointMap; + } + + /** + * Constructs a {@link Put} operation to update the last replicated WAL timestamp for a given + * server in the backup system table. + * @param serverNameBytes the serialized server name as bytes + * @param timestamp the WAL entry timestamp to store + * @return a {@link Put} object ready to be written to the system table + */ + private Put createPutForBackupCheckpoint(byte[] serverNameBytes, long timestamp) { + Put put = new Put(rowkey(CONTINUOUS_BACKUP_REPLICATION_TIMESTAMP_ROW)); + put.addColumn(BackupSystemTable.META_FAMILY, serverNameBytes, Bytes.toBytes(timestamp)); + return put; + } + /** * Deletes incremental backup set for a backup destination * @param backupRoot backup root diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java index de47e8f33915..907400aa1129 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -172,10 +172,18 @@ public void execute() throws IOException { private void handleContinuousBackup(Admin admin) throws IOException { backupInfo.setPhase(BackupInfo.BackupPhase.SETUP_WAL_REPLICATION); long startTimestamp = startContinuousWALBackup(admin); + backupManager.addContinuousBackupTableSet(backupInfo.getTables(), startTimestamp); - performBackupSnapshots(admin); + // Updating the start time of this backup to reflect the actual beginning of the full backup. + // So far, we have only set up continuous WAL replication, but the full backup has not yet + // started. + // Setting the correct start time is crucial for Point-In-Time Recovery (PITR). + // When selecting a backup for PITR, we must ensure that the backup started **on or after** the + // starting time of the WALs. If WAL streaming began later, we couldn't guarantee that WALs + // exist for the entire period between the backup's start time and the desired PITR timestamp. + backupInfo.setStartTs(startTimestamp); - backupManager.addContinuousBackupTableSet(backupInfo.getTables(), startTimestamp); + performBackupSnapshots(admin); // set overall backup status: complete. Here we make sure to complete the backup. // After this checkpoint, even if entering cancel process, will let the backup finished diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java index c973af8102e7..34fcd76bf9c5 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -36,6 +36,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; @@ -82,6 +85,8 @@ public class ContinuousBackupReplicationEndpoint extends BaseReplicationEndpoint private String peerId; private ScheduledExecutorService flushExecutor; + private long latestWALEntryTimestamp = -1L; + public static final long ONE_DAY_IN_MILLISECONDS = TimeUnit.DAYS.toMillis(1); public static final String WAL_FILE_PREFIX = "wal_file."; public static final String DATE_FORMAT = "yyyy-MM-dd"; @@ -174,6 +179,13 @@ private void flushWriters() throws IOException { } } walWriters.clear(); + + // All received WAL entries have been flushed and persisted successfully. + // At this point, it's safe to record the latest replicated timestamp, + // as we are guaranteed that all entries up to that timestamp are durably stored. + // This checkpoint is essential for enabling consistent Point-in-Time Restore (PITR). + updateLastReplicatedTimestampForContinuousBackup(); + LOG.info("{} WAL writers flushed and cleared", Utils.logPeerId(peerId)); } @@ -218,6 +230,12 @@ public ReplicationResult replicate(ReplicateContext replicateContext) { backupWalEntries(entry.getKey(), entry.getValue()); } + // Capture the timestamp of the last WAL entry processed. This is used as the replication + // checkpoint so that point-in-time restores know the latest consistent time up to which + // replication has + // occurred. + latestWALEntryTimestamp = entries.get(entries.size() - 1).getKey().getWriteTime(); + if (isAnyWriterFull()) { LOG.debug("{} Some WAL writers reached max size, triggering flush", Utils.logPeerId(peerId)); @@ -237,6 +255,21 @@ public ReplicationResult replicate(ReplicateContext replicateContext) { } } + /** + * Persists the latest replicated WAL entry timestamp in the backup system table. This checkpoint + * is critical for Continuous Backup and Point-in-Time Restore (PITR) to ensure restore operations + * only go up to a known safe point. The value is stored per region server using its ServerName as + * the key. + * @throws IOException if the checkpoint update fails + */ + private void updateLastReplicatedTimestampForContinuousBackup() throws IOException { + try (final Connection conn = ConnectionFactory.createConnection(conf); + BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { + backupSystemTable.updateBackupCheckpointTimestamp(replicationSource.getServerWALsBelongTo(), + latestWALEntryTimestamp); + } + } + private Map> groupEntriesByDay(List entries) { return entries.stream().collect( Collectors.groupingBy(entry -> (entry.getKey().getWriteTime() / ONE_DAY_IN_MILLISECONDS) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java index 183cc2054f1a..d8a033a1f077 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.hbase.backup.util; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; +import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_DEFAULT; +import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY; + import java.io.FileNotFoundException; import java.io.IOException; import java.net.URLDecoder; @@ -56,6 +60,11 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.master.region.MasterRegionFactory; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationGroupOffset; +import org.apache.hadoop.hbase.replication.ReplicationQueueId; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.tool.BulkLoadHFiles; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -821,4 +830,157 @@ private static void logRollV2(Connection conn, String backupRootDir) throws IOEx } } } + + /** + * Calculates the replication checkpoint timestamp used for continuous backup. + *

+ * A replication checkpoint is the earliest timestamp across all region servers such that every + * WAL entry before that point is known to be replicated to the target system. This is essential + * for features like Point-in-Time Restore (PITR) and incremental backups, where we want to + * confidently restore data to a consistent state without missing updates. + *

+ * The checkpoint is calculated using a combination of: + *

    + *
  • The start timestamps of WAL files currently being replicated for each server.
  • + *
  • The latest successfully replicated timestamp recorded by the replication marker chore.
  • + *
+ *

+ * We combine these two sources to handle the following challenges: + *

    + *
  • Stale WAL start times: If replication traffic is low or WALs are long-lived, the + * replication offset may point to the same WAL for a long time, resulting in stale timestamps + * that underestimate progress. This could delay PITR unnecessarily.
  • + *
  • Limitations of marker-only tracking: The replication marker chore stores the last + * successfully replicated timestamp per region server in a system table. However, this data may + * become stale if the server goes offline or region ownership changes. For example, if a region + * initially belonged to rs1 and was later moved to rs4 due to re-balancing, rs1’s marker would + * persist even though it no longer holds any regions. Relying solely on these stale markers could + * lead to incorrect or outdated checkpoints.
  • + *
+ *

+ * To handle these limitations, the method: + *

    + *
  1. Verifies that the continuous backup peer exists to ensure replication is enabled.
  2. + *
  3. Retrieves WAL replication queue information for the peer, collecting WAL start times per + * region server. This gives us a lower bound for replication progress.
  4. + *
  5. Reads the marker chore's replicated timestamps from the backup system table.
  6. + *
  7. For servers found in both sources, if the marker timestamp is more recent than the WAL's + * start timestamp, we use the marker (since replication has progressed beyond the WAL).
  8. + *
  9. We discard marker entries for region servers that are not present in WAL queues, assuming + * those servers are no longer relevant (e.g., decommissioned or reassigned).
  10. + *
  11. The checkpoint is the minimum of all chosen timestamps — i.e., the slowest replicating + * region server.
  12. + *
  13. Finally, we persist the updated marker information to include any newly participating + * region servers.
  14. + *
+ *

+ * Note: If the replication marker chore is disabled, we fall back to using only the WAL start + * times. This ensures correctness but may lead to conservative checkpoint estimates during idle + * periods. + * @param conn the HBase connection + * @return the calculated replication checkpoint timestamp + * @throws IOException if reading replication queues or updating the backup system table fails + */ + public static long getReplicationCheckpoint(Connection conn) throws IOException { + Configuration conf = conn.getConfiguration(); + long checkpoint = EnvironmentEdgeManager.getDelegate().currentTime(); + + // Step 1: Ensure the continuous backup replication peer exists + if (!continuousBackupReplicationPeerExists(conn.getAdmin())) { + String msg = "Replication peer '" + CONTINUOUS_BACKUP_REPLICATION_PEER + + "' not found. Continuous backup not enabled."; + LOG.error(msg); + throw new IOException(msg); + } + + // Step 2: Get all replication queues for the continuous backup peer + ReplicationQueueStorage queueStorage = + ReplicationStorageFactory.getReplicationQueueStorage(conn, conf); + + List queueIds; + try { + queueIds = queueStorage.listAllQueueIds(CONTINUOUS_BACKUP_REPLICATION_PEER); + } catch (ReplicationException e) { + String msg = "Failed to retrieve replication queue IDs for peer '" + + CONTINUOUS_BACKUP_REPLICATION_PEER + "'"; + LOG.error(msg, e); + throw new IOException(msg, e); + } + + if (queueIds.isEmpty()) { + String msg = "Replication peer '" + CONTINUOUS_BACKUP_REPLICATION_PEER + "' has no queues. " + + "This may indicate that continuous backup replication is not initialized correctly."; + LOG.error(msg); + throw new IOException(msg); + } + + // Step 3: Build a map of ServerName -> WAL start timestamp (lowest seen per server) + Map serverToCheckpoint = new HashMap<>(); + for (ReplicationQueueId queueId : queueIds) { + Map offsets; + try { + offsets = queueStorage.getOffsets(queueId); + } catch (ReplicationException e) { + String msg = "Failed to fetch WAL offsets for replication queue: " + queueId; + LOG.error(msg, e); + throw new IOException(msg, e); + } + + for (ReplicationGroupOffset offset : offsets.values()) { + String walFile = offset.getWal(); + long ts = AbstractFSWALProvider.getTimestamp(walFile); // WAL creation time + ServerName server = queueId.getServerName(); + // Store the minimum timestamp per server (ts - 1 to avoid edge boundary issues) + serverToCheckpoint.merge(server, ts - 1, Math::min); + } + } + + // Step 4: If replication markers are enabled, overlay fresher timestamps from backup system + // table + boolean replicationMarkerEnabled = + conf.getBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); + if (replicationMarkerEnabled) { + try (BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { + Map markerTimestamps = backupSystemTable.getBackupCheckpointTimestamps(); + + for (Map.Entry entry : markerTimestamps.entrySet()) { + ServerName server = entry.getKey(); + long markerTs = entry.getValue(); + + // If marker timestamp is newer, override + if (serverToCheckpoint.containsKey(server)) { + long current = serverToCheckpoint.get(server); + if (markerTs > current) { + serverToCheckpoint.put(server, markerTs); + } + } else { + // This server is no longer active (e.g., RS moved or removed); skip + if (LOG.isDebugEnabled()) { + LOG.debug("Skipping replication marker timestamp for inactive server: {}", server); + } + } + } + + // Step 5: Persist current server timestamps into backup system table + for (Map.Entry entry : serverToCheckpoint.entrySet()) { + backupSystemTable.updateBackupCheckpointTimestamp(entry.getKey(), entry.getValue()); + } + } + } else { + LOG.warn( + "Replication marker chore is disabled. Using WAL-based timestamps only for checkpoint calculation."); + } + + // Step 6: Calculate final checkpoint as minimum timestamp across all active servers + for (long ts : serverToCheckpoint.values()) { + checkpoint = Math.min(checkpoint, ts); + } + + return checkpoint; + } + + private static boolean continuousBackupReplicationPeerExists(Admin admin) throws IOException { + return admin.listReplicationPeers().stream() + .anyMatch(peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER)); + } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index bfc61010257e..f85171c735a0 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_MAX_WAL_SIZE; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INITIAL_DELAY; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INTERVAL; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -291,6 +295,10 @@ public static void setUpHelper() throws Exception { BACKUP_ROOT_DIR = Path.SEPARATOR + "backupUT"; BACKUP_REMOTE_ROOT_DIR = Path.SEPARATOR + "backupUT"; + conf1.set(CONF_BACKUP_MAX_WAL_SIZE, "10240"); + conf1.set(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, "10"); + conf1.set(CONF_STAGED_WAL_FLUSH_INTERVAL, "10"); + if (secure) { // set the always on security provider UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(), diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java new file mode 100644 index 000000000000..fb37977c4eee --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java @@ -0,0 +1,277 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TO_DATETIME; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import java.io.IOException; +import java.util.Arrays; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.ToolRunner; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(LargeTests.class) +public class TestPointInTimeRestore extends TestBackupBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPointInTimeRestore.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestPointInTimeRestore.class); + + private static final String backupWalDirName = "TestPointInTimeRestoreWalDir"; + private static final int WAIT_FOR_REPLICATION_MS = 30_000; + static Path backupWalDir; + static FileSystem fs; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + backupWalDir = new Path(root, backupWalDirName); + fs = FileSystem.get(conf1); + fs.mkdirs(backupWalDir); + conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); + + setUpBackups(); + } + + /** + * Sets up multiple backups at different timestamps by: 1. Adjusting the system time to simulate + * past backup points. 2. Loading data into tables to create meaningful snapshots. 3. Running full + * backups with or without continuous backup enabled. 4. Ensuring replication is complete before + * proceeding. + */ + private static void setUpBackups() throws Exception { + // Simulate a backup taken 20 days ago + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - 20 * ONE_DAY_IN_MILLISECONDS); + loadRandomData(table1, 1000); // Insert initial data into table1 + + // Perform a full backup for table1 with continuous backup enabled + String[] args = buildBackupArgs("full", new TableName[] { table1 }, true); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + // Move time forward to simulate 15 days ago + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - 15 * ONE_DAY_IN_MILLISECONDS); + loadRandomData(table1, 1000); // Add more data to table1 + loadRandomData(table2, 500); // Insert data into table2 + + waitForReplication(); // Ensure replication is complete + + // Perform a full backup for table2 with continuous backup enabled + args = buildBackupArgs("full", new TableName[] { table2 }, true); + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + // Move time forward to simulate 10 days ago + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - 10 * ONE_DAY_IN_MILLISECONDS); + loadRandomData(table2, 500); // Add more data to table2 + loadRandomData(table3, 500); // Insert data into table3 + + // Perform a full backup for table3 and table4 (without continuous backup) + args = buildBackupArgs("full", new TableName[] { table3, table4 }, false); + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + waitForReplication(); // Ensure replication is complete before concluding setup + + // Reset time mocking to avoid affecting other tests + EnvironmentEdgeManager.reset(); + } + + @AfterClass + public static void setupAfterClass() throws IOException { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, backupWalDirName); + FileSystem fs = FileSystem.get(conf1); + + if (fs.exists(backupWalDir)) { + fs.delete(backupWalDir, true); + } + + conf1.unset(CONF_CONTINUOUS_BACKUP_WAL_DIR); + } + + /** + * Verifies that PITR (Point-in-Time Restore) fails when the requested restore time is either in + * the future or outside the allowed retention window. + */ + @Test + public void testPITR_FailsOutsideWindow() throws Exception { + // Case 1: Requested restore time is in the future (should fail) + String[] args = buildPITRArgs(new TableName[] { table1 }, + new TableName[] { TableName.valueOf("restoredTable1") }, + EnvironmentEdgeManager.currentTime() + ONE_DAY_IN_MILLISECONDS); + + int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertNotEquals("Restore should fail since the requested restore time is in the future", 0, + ret); + + // Case 2: Requested restore time is too old (beyond the retention window, should fail) + args = buildPITRArgs(new TableName[] { table1 }, + new TableName[] { TableName.valueOf("restoredTable1") }, + EnvironmentEdgeManager.currentTime() - 40 * ONE_DAY_IN_MILLISECONDS); + + ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertNotEquals( + "Restore should fail since the requested restore time is outside the retention window", 0, + ret); + } + + /** + * Ensures that PITR fails when attempting to restore tables where continuous backup was not + * enabled. + */ + @Test + public void testPointInTimeRestore_ContinuousBackupNotEnabledTables() throws Exception { + String[] args = buildPITRArgs(new TableName[] { table3 }, + new TableName[] { TableName.valueOf("restoredTable1") }, + EnvironmentEdgeManager.currentTime() - 10 * ONE_DAY_IN_MILLISECONDS); + + int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertNotEquals("Restore should fail since continuous backup is not enabled for the table", 0, + ret); + } + + /** + * Ensures that PITR fails when trying to restore from a point before continuous backup started. + */ + @Test + public void testPointInTimeRestore_TablesWithNoProperBackupOrWals() throws Exception { + String[] args = buildPITRArgs(new TableName[] { table2 }, + new TableName[] { TableName.valueOf("restoredTable1") }, + EnvironmentEdgeManager.currentTime() - 16 * ONE_DAY_IN_MILLISECONDS); + + int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertNotEquals( + "Restore should fail since the requested restore point is before the start of continuous backup", + 0, ret); + } + + /** + * Verifies that PITR successfully restores data for a single table. + */ + @Test + public void testPointInTimeRestore_SuccessfulRestoreForOneTable() throws Exception { + TableName restoredTable = TableName.valueOf("restoredTable"); + + // Perform restore operation + String[] args = buildPITRArgs(new TableName[] { table1 }, new TableName[] { restoredTable }, + EnvironmentEdgeManager.currentTime() - 5 * ONE_DAY_IN_MILLISECONDS); + + int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertEquals("Restore should succeed", 0, ret); + + // Validate that the restored table contains the same number of rows as the original table + assertEquals("Restored table should have the same row count as the original", + getRowCount(table1), getRowCount(restoredTable)); + } + + /** + * Verifies that PITR successfully restores multiple tables at once. + */ + @Test + public void testPointInTimeRestore_SuccessfulRestoreForMultipleTables() throws Exception { + TableName restoredTable1 = TableName.valueOf("restoredTable1"); + TableName restoredTable2 = TableName.valueOf("restoredTable2"); + + // Perform restore operation for multiple tables + String[] args = buildPITRArgs(new TableName[] { table1, table2 }, + new TableName[] { restoredTable1, restoredTable2 }, + EnvironmentEdgeManager.currentTime() - 5 * ONE_DAY_IN_MILLISECONDS); + + int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertEquals("Restore should succeed", 0, ret); + + // Validate that the restored tables contain the same number of rows as the originals + assertEquals("Restored table1 should have the same row count as the original", + getRowCount(table1), getRowCount(restoredTable1)); + assertEquals("Restored table2 should have the same row count as the original", + getRowCount(table2), getRowCount(restoredTable2)); + } + + private String[] buildPITRArgs(TableName[] sourceTables, TableName[] targetTables, long endTime) { + String sourceTableNames = + Arrays.stream(sourceTables).map(TableName::getNameAsString).collect(Collectors.joining(",")); + + String targetTableNames = + Arrays.stream(targetTables).map(TableName::getNameAsString).collect(Collectors.joining(",")); + + return new String[] { "-" + OPTION_TABLE, sourceTableNames, "-" + OPTION_TABLE_MAPPING, + targetTableNames, "-" + OPTION_TO_DATETIME, String.valueOf(endTime) }; + } + + private static String[] buildBackupArgs(String backupType, TableName[] tables, + boolean continuousEnabled) { + String tableNames = + Arrays.stream(tables).map(TableName::getNameAsString).collect(Collectors.joining(",")); + + if (continuousEnabled) { + return new String[] { "create", backupType, BACKUP_ROOT_DIR, "-" + OPTION_TABLE, tableNames, + "-" + OPTION_ENABLE_CONTINUOUS_BACKUP }; + } else { + return new String[] { "create", backupType, BACKUP_ROOT_DIR, "-" + OPTION_TABLE, tableNames }; + } + } + + private static void loadRandomData(TableName tableName, int totalRows) throws IOException { + int rowSize = 32; + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + TEST_UTIL.loadRandomRows(table, famName, rowSize, totalRows); + } + } + + private static void waitForReplication() { + LOG.info("Waiting for replication to complete for {} ms", WAIT_FOR_REPLICATION_MS); + try { + Thread.sleep(WAIT_FOR_REPLICATION_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Thread was interrupted while waiting", e); + } + } + + private int getRowCount(TableName tableName) throws IOException { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + return HBaseTestingUtil.countRows(table); + } + } +} From 716dab83d91d5b9510dde65344a42d21fd473c8a Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Wed, 11 Jun 2025 19:55:48 +0530 Subject: [PATCH 06/24] HBASE-29255: Integrate backup WAL cleanup logic with the delete command (#7007) * Store bulkload files in daywise bucket as well * Integrate backup WAL cleanup logic with the delete command * address the review comments * address the review comments * address the review comments * add more unit tests to cover all cases * address the review comments --- hbase-backup/pom.xml | 5 + .../hbase/backup/impl/BackupCommands.java | 147 ++++++++++++++ .../hbase/backup/impl/BackupSystemTable.java | 26 +++ .../ContinuousBackupReplicationEndpoint.java | 24 ++- .../backup/TestBackupDeleteWithCleanup.java | 184 ++++++++++++++++++ .../hbase/backup/impl/TestBackupCommands.java | 177 +++++++++++++++++ ...stContinuousBackupReplicationEndpoint.java | 20 +- 7 files changed, 575 insertions(+), 8 deletions(-) create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java diff --git a/hbase-backup/pom.xml b/hbase-backup/pom.xml index 7cac75f88d2b..0bb4e4e03ca4 100644 --- a/hbase-backup/pom.xml +++ b/hbase-backup/pom.xml @@ -182,6 +182,11 @@ junit-vintage-engine test + + org.mockito + mockito-inline + test + diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 804dc7141a19..11b6890ed038 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.backup.impl; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BACKUP_LIST_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH; @@ -47,18 +49,26 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_DESC; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; import java.io.IOException; import java.net.URI; +import java.text.ParseException; +import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.TimeZone; import java.util.concurrent.TimeUnit; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -71,6 +81,7 @@ import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager; import org.apache.hadoop.hbase.backup.util.BackupSet; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Connection; @@ -80,6 +91,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.base.Splitter; +import org.apache.hbase.thirdparty.com.google.common.base.Strings; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; @@ -649,6 +661,8 @@ public void execute() throws IOException { } else if (cmdline.hasOption(OPTION_LIST)) { executeDeleteListOfBackups(cmdline, isForceDelete); } + + cleanUpUnusedBackupWALs(); } private void executeDeleteOlderThan(CommandLine cmdline, boolean isForceDelete) @@ -876,6 +890,139 @@ private boolean canAnyOtherBackupCover(List allBackups, BackupInfo c return false; } + /** + * Cleans up Write-Ahead Logs (WALs) that are no longer required for PITR after a successful + * backup deletion. + */ + private void cleanUpUnusedBackupWALs() throws IOException { + Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); + String backupWalDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + + if (Strings.isNullOrEmpty(backupWalDir)) { + System.out.println("No WAL directory specified for continuous backup. Skipping cleanup."); + return; + } + + try (BackupSystemTable sysTable = new BackupSystemTable(conn)) { + // Get list of tables under continuous backup + Map continuousBackupTables = sysTable.getContinuousBackupTableSet(); + if (continuousBackupTables.isEmpty()) { + System.out.println("No continuous backups configured. Skipping WAL cleanup."); + return; + } + + // Find the earliest timestamp after which WALs are still needed + long cutoffTimestamp = determineWALCleanupCutoffTime(sysTable); + if (cutoffTimestamp == 0) { + System.err.println("ERROR: No valid full backup found. Skipping WAL cleanup."); + return; + } + + // Update metadata before actual cleanup to avoid inconsistencies + updateBackupTableStartTimes(sysTable, cutoffTimestamp); + + // Delete WAL files older than cutoff timestamp + deleteOldWALFiles(conf, backupWalDir, cutoffTimestamp); + + } + } + + /** + * Determines the cutoff time for cleaning WAL files. + * @param sysTable Backup system table + * @return cutoff timestamp or 0 if not found + */ + long determineWALCleanupCutoffTime(BackupSystemTable sysTable) throws IOException { + List backupInfos = sysTable.getBackupInfos(BackupState.COMPLETE); + Collections.reverse(backupInfos); // Start from oldest + + for (BackupInfo backupInfo : backupInfos) { + if (BackupType.FULL.equals(backupInfo.getType())) { + return backupInfo.getStartTs(); + } + } + return 0; + } + + /** + * Updates the start time for continuous backups if older than cutoff timestamp. + * @param sysTable Backup system table + * @param cutoffTimestamp Timestamp before which WALs are no longer needed + */ + void updateBackupTableStartTimes(BackupSystemTable sysTable, long cutoffTimestamp) + throws IOException { + + Map backupTables = sysTable.getContinuousBackupTableSet(); + Set tablesToUpdate = new HashSet<>(); + + for (Map.Entry entry : backupTables.entrySet()) { + if (entry.getValue() < cutoffTimestamp) { + tablesToUpdate.add(entry.getKey()); + } + } + + if (!tablesToUpdate.isEmpty()) { + sysTable.updateContinuousBackupTableSet(tablesToUpdate, cutoffTimestamp); + } + } + + /** + * Cleans up old WAL and bulk-loaded files based on the determined cutoff timestamp. + */ + void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) + throws IOException { + System.out.println("Starting WAL cleanup in backup directory: " + backupWalDir + + " with cutoff time: " + cutoffTime); + + BackupFileSystemManager manager = + new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); + FileSystem fs = manager.getBackupFs(); + Path walDir = manager.getWalsDir(); + Path bulkloadDir = manager.getBulkLoadFilesDir(); + + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); + + System.out.println("Listing directories under: " + walDir); + + FileStatus[] directories = fs.listStatus(walDir); + + for (FileStatus dirStatus : directories) { + if (!dirStatus.isDirectory()) { + continue; // Skip files, we only want directories + } + + Path dirPath = dirStatus.getPath(); + String dirName = dirPath.getName(); + + try { + long dayStart = parseDayDirectory(dirName, dateFormat); + System.out + .println("Checking WAL directory: " + dirName + " (Start Time: " + dayStart + ")"); + + // If WAL files of that day are older than cutoff time, delete them + if (dayStart + ONE_DAY_IN_MILLISECONDS - 1 < cutoffTime) { + System.out.println("Deleting outdated WAL directory: " + dirPath); + fs.delete(dirPath, true); + fs.delete(new Path(bulkloadDir, dirName), true); + } + } catch (ParseException e) { + System.out.println("WARNING: Failed to parse directory name '" + dirName + + "'. Skipping. Error: " + e.getMessage()); + } catch (IOException e) { + System.out.println("WARNING: Failed to delete directory '" + dirPath + + "'. Skipping. Error: " + e.getMessage()); + } + } + + System.out.println("Completed WAL cleanup for backup directory: " + backupWalDir); + } + + private long parseDayDirectory(String dayDir, SimpleDateFormat dateFormat) + throws ParseException { + return dateFormat.parse(dayDir).getTime(); + } + @Override protected void printUsage() { System.out.println(DELETE_CMD_USAGE); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 3166ddf3ef28..24bd0888f9cb 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -979,6 +979,32 @@ public void addContinuousBackupTableSet(Set tables, long startTimesta } } + /** + * Updates the system table with the new start timestamps for continuous backup tables. + * @param tablesToUpdate The set of tables that need their start timestamps updated. + * @param newStartTimestamp The new start timestamp to be set. + */ + public void updateContinuousBackupTableSet(Set tablesToUpdate, long newStartTimestamp) + throws IOException { + if (tablesToUpdate == null || tablesToUpdate.isEmpty()) { + LOG.warn("No tables provided for updating start timestamps."); + return; + } + + try (Table table = connection.getTable(tableName)) { + Put put = new Put(rowkey(CONTINUOUS_BACKUP_SET)); + + for (TableName tableName : tablesToUpdate) { + put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes(tableName.getNameAsString()), + Bytes.toBytes(newStartTimestamp)); + } + + table.put(put); + LOG.info("Successfully updated start timestamps for {} tables in the backup system table.", + tablesToUpdate.size()); + } + } + /** * Removes tables from the global continuous backup set. Only removes entries that currently exist * in the backup system table. diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java index 34fcd76bf9c5..eeacc8fbf34c 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -23,6 +23,7 @@ import java.util.Date; import java.util.List; import java.util.Map; +import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; @@ -304,7 +305,7 @@ private void backupWalEntries(long day, List walEntries) throws IOExc walWriter.append(entry); } walWriter.sync(true); - uploadBulkLoadFiles(bulkLoadFiles); + uploadBulkLoadFiles(day, bulkLoadFiles); } catch (UncheckedIOException e) { String errorMsg = Utils.logPeerId(peerId) + " Failed to get or create WAL Writer for " + day; LOG.error("{} Backup failed for day {}. Error: {}", Utils.logPeerId(peerId), day, @@ -314,9 +315,7 @@ private void backupWalEntries(long day, List walEntries) throws IOExc } private FSHLogProvider.Writer createWalWriter(long dayInMillis) { - // Convert dayInMillis to "yyyy-MM-dd" format - SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); - String dayDirectoryName = dateFormat.format(new Date(dayInMillis)); + String dayDirectoryName = formatToDateString(dayInMillis); FileSystem fs = backupFileSystemManager.getBackupFs(); Path walsDir = backupFileSystemManager.getWalsDir(); @@ -376,7 +375,7 @@ private void close() { } } - private void uploadBulkLoadFiles(List bulkLoadFiles) throws IOException { + private void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) throws IOException { LOG.debug("{} Starting upload of {} bulk load files", Utils.logPeerId(peerId), bulkLoadFiles.size()); @@ -384,9 +383,13 @@ private void uploadBulkLoadFiles(List bulkLoadFiles) throws IOException { LOG.trace("{} Bulk load files to upload: {}", Utils.logPeerId(peerId), bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); } + String dayDirectoryName = formatToDateString(dayInMillis); + Path bulkloadDir = new Path(backupFileSystemManager.getBulkLoadFilesDir(), dayDirectoryName); + backupFileSystemManager.getBackupFs().mkdirs(bulkloadDir); + for (Path file : bulkLoadFiles) { Path sourcePath = getBulkLoadFileStagingPath(file); - Path destPath = new Path(backupFileSystemManager.getBulkLoadFilesDir(), file); + Path destPath = new Path(bulkloadDir, file); try { LOG.debug("{} Copying bulk load file from {} to {}", Utils.logPeerId(peerId), sourcePath, @@ -407,6 +410,15 @@ private void uploadBulkLoadFiles(List bulkLoadFiles) throws IOException { LOG.debug("{} Completed upload of bulk load files", Utils.logPeerId(peerId)); } + /** + * Convert dayInMillis to "yyyy-MM-dd" format + */ + private String formatToDateString(long dayInMillis) { + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); + return dateFormat.format(new Date(dayInMillis)); + } + private Path getBulkLoadFileStagingPath(Path relativePathFromNamespace) throws IOException { FileSystem rootFs = CommonFSUtils.getRootDirFileSystem(conf); Path rootDir = CommonFSUtils.getRootDir(conf); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java new file mode 100644 index 000000000000..6d76ac4e89bf --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java @@ -0,0 +1,184 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestBackupDeleteWithCleanup extends TestBackupBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupDeleteWithCleanup.class); + + String backupWalDirName = "TestBackupDeleteWithCleanup"; + + @Test + public void testBackupDeleteWithCleanupLogic() throws Exception { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, backupWalDirName); + conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); + FileSystem fs = FileSystem.get(conf1); + fs.mkdirs(backupWalDir); + + // Step 1: Setup Backup Folders + long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); + setupBackupFolders(fs, backupWalDir, currentTime); + + // Log the directory structure before cleanup + logDirectoryStructure(fs, backupWalDir, "Directory structure BEFORE cleanup:"); + + // Step 2: Simulate Backup Creation + BackupSystemTable backupSystemTable = new BackupSystemTable(TEST_UTIL.getConnection()); + backupSystemTable.addContinuousBackupTableSet(Set.of(table1), + currentTime - (2 * ONE_DAY_IN_MILLISECONDS)); + + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - (2 * ONE_DAY_IN_MILLISECONDS)); + String backupId = fullTableBackup(Lists.newArrayList(table1)); + assertTrue(checkSucceeded(backupId)); + + String anotherBackupId = fullTableBackup(Lists.newArrayList(table1)); + assertTrue(checkSucceeded(anotherBackupId)); + + // Step 3: Run Delete Command + int ret = + ToolRunner.run(conf1, new BackupDriver(), new String[] { "delete", "-l", backupId, "-fd" }); + assertEquals(0, ret); + + // Log the directory structure after cleanup + logDirectoryStructure(fs, backupWalDir, "Directory structure AFTER cleanup:"); + + // Step 4: Verify Cleanup + verifyBackupCleanup(fs, backupWalDir, currentTime); + + // Step 5: Verify System Table Update + verifySystemTableUpdate(backupSystemTable, currentTime); + } + + public static void setupBackupFolders(FileSystem fs, Path backupWalDir, long currentTime) + throws IOException { + Path walsDir = new Path(backupWalDir, WALS_DIR); + Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); + + fs.mkdirs(walsDir); + fs.mkdirs(bulkLoadDir); + + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + + for (int i = 0; i < 5; i++) { + String dateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); + fs.mkdirs(new Path(walsDir, dateStr)); + fs.mkdirs(new Path(bulkLoadDir, dateStr)); + } + } + + private static void verifyBackupCleanup(FileSystem fs, Path backupWalDir, long currentTime) + throws IOException { + Path walsDir = new Path(backupWalDir, WALS_DIR); + Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + + // Expect folders older than 3 days to be deleted + for (int i = 3; i < 5; i++) { + String oldDateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); + Path walPath = new Path(walsDir, oldDateStr); + Path bulkLoadPath = new Path(bulkLoadDir, oldDateStr); + assertFalse("Old WAL directory (" + walPath + ") should be deleted, but it exists!", + fs.exists(walPath)); + assertFalse("Old BulkLoad directory (" + bulkLoadPath + ") should be deleted, but it exists!", + fs.exists(bulkLoadPath)); + } + + // Expect folders within the last 3 days to exist + for (int i = 0; i < 3; i++) { + String recentDateStr = + dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); + Path walPath = new Path(walsDir, recentDateStr); + Path bulkLoadPath = new Path(bulkLoadDir, recentDateStr); + + assertTrue("Recent WAL directory (" + walPath + ") should exist, but it is missing!", + fs.exists(walPath)); + assertTrue( + "Recent BulkLoad directory (" + bulkLoadPath + ") should exist, but it is missing!", + fs.exists(bulkLoadPath)); + } + } + + private void verifySystemTableUpdate(BackupSystemTable backupSystemTable, long currentTime) + throws IOException { + Map updatedTables = backupSystemTable.getContinuousBackupTableSet(); + + for (Map.Entry entry : updatedTables.entrySet()) { + long updatedStartTime = entry.getValue(); + + // Ensure that the updated start time is not earlier than the expected cutoff time + assertTrue("System table update failed!", + updatedStartTime >= (currentTime - (3 * ONE_DAY_IN_MILLISECONDS))); + } + } + + public static void logDirectoryStructure(FileSystem fs, Path dir, String message) + throws IOException { + System.out.println(message); + listDirectory(fs, dir, " "); + } + + public static void listDirectory(FileSystem fs, Path dir, String indent) throws IOException { + if (!fs.exists(dir)) { + System.out.println(indent + "[Missing] " + dir); + return; + } + FileStatus[] files = fs.listStatus(dir); + System.out.println(indent + dir); + for (FileStatus file : files) { + if (file.isDirectory()) { + listDirectory(fs, file.getPath(), indent + " "); + } else { + System.out.println(indent + " " + file.getPath()); + } + } + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java new file mode 100644 index 000000000000..b2ebbd640bbd --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java @@ -0,0 +1,177 @@ +/* + * 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.impl; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.logDirectoryStructure; +import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.setupBackupFolders; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TimeZone; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.TestBackupBase; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestBackupCommands extends TestBackupBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupCommands.class); + + String backupWalDirName = "TestBackupWalDir"; + + /** + * Tests whether determineWALCleanupCutoffTime returns the correct FULL backup start timestamp. + */ + @Test + public void testDetermineWALCleanupCutoffTimeOfCleanupCommand() throws IOException { + // GIVEN + BackupSystemTable sysTable = mock(BackupSystemTable.class); + + BackupInfo full1 = new BackupInfo(); + full1.setType(BackupType.FULL); + full1.setStartTs(1111L); + full1.setState(BackupInfo.BackupState.COMPLETE); + + BackupInfo inc = new BackupInfo(); + inc.setType(BackupType.INCREMENTAL); + inc.setStartTs(2222L); + inc.setState(BackupInfo.BackupState.COMPLETE); + + BackupInfo full2 = new BackupInfo(); + full2.setType(BackupType.FULL); + full2.setStartTs(3333L); + full2.setState(BackupInfo.BackupState.COMPLETE); + + // Ordered as newest to oldest, will be reversed in the method + List backupInfos = List.of(full2, inc, full1); + when(sysTable.getBackupInfos(BackupInfo.BackupState.COMPLETE)) + .thenReturn(new ArrayList<>(backupInfos)); + + // WHEN + BackupCommands.DeleteCommand command = new BackupCommands.DeleteCommand(conf1, null); + long cutoff = command.determineWALCleanupCutoffTime(sysTable); + + // THEN + assertEquals("Expected oldest FULL backup timestamp", 1111L, cutoff); + } + + @Test + public void testUpdateBackupTableStartTimesOfCleanupCommand() throws IOException { + // GIVEN + BackupSystemTable mockSysTable = mock(BackupSystemTable.class); + + TableName tableA = TableName.valueOf("ns", "tableA"); + TableName tableB = TableName.valueOf("ns", "tableB"); + TableName tableC = TableName.valueOf("ns", "tableC"); + + long cutoffTimestamp = 1_000_000L; + + // Simulate current table start times + Map tableSet = Map.of(tableA, 900_000L, // Before cutoff → should be updated + tableB, 1_100_000L, // After cutoff → should NOT be updated + tableC, 800_000L // Before cutoff → should be updated + ); + + when(mockSysTable.getContinuousBackupTableSet()).thenReturn(tableSet); + + // WHEN + BackupCommands.DeleteCommand command = new BackupCommands.DeleteCommand(conf1, null); + command.updateBackupTableStartTimes(mockSysTable, cutoffTimestamp); + + // THEN + Set expectedUpdated = Set.of(tableA, tableC); + verify(mockSysTable).updateContinuousBackupTableSet(expectedUpdated, cutoffTimestamp); + } + + @Test + public void testDeleteOldWALFilesOfCleanupCommand() throws IOException { + // GIVEN + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, backupWalDirName); + conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); + + FileSystem fs = FileSystem.get(conf1); + fs.mkdirs(backupWalDir); + + long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); + setupBackupFolders(fs, backupWalDir, currentTime); // Create 5 days of WAL/bulk folders + + logDirectoryStructure(fs, backupWalDir, "Before cleanup:"); + + // Delete files older than 2 days from current time + long cutoffTime = currentTime - (2 * ONE_DAY_IN_MILLISECONDS); + + // WHEN + BackupCommands.DeleteCommand command = new BackupCommands.DeleteCommand(conf1, null); + command.deleteOldWALFiles(conf1, backupWalDir.toString(), cutoffTime); + + logDirectoryStructure(fs, backupWalDir, "After cleanup:"); + + // THEN + verifyCleanupOutcome(fs, backupWalDir, currentTime, cutoffTime); + } + + private static void verifyCleanupOutcome(FileSystem fs, Path backupWalDir, long currentTime, + long cutoffTime) throws IOException { + Path walsDir = new Path(backupWalDir, WALS_DIR); + Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); + + for (int i = 0; i < 5; i++) { + long dayTime = currentTime - (i * ONE_DAY_IN_MILLISECONDS); + String dayDir = dateFormat.format(new Date(dayTime)); + Path walPath = new Path(walsDir, dayDir); + Path bulkPath = new Path(bulkLoadDir, dayDir); + + if (dayTime + ONE_DAY_IN_MILLISECONDS - 1 < cutoffTime) { + assertFalse("Old WAL dir should be deleted: " + walPath, fs.exists(walPath)); + assertFalse("Old BulkLoad dir should be deleted: " + bulkPath, fs.exists(bulkPath)); + } else { + assertTrue("Recent WAL dir should exist: " + walPath, fs.exists(walPath)); + assertTrue("Recent BulkLoad dir should exist: " + bulkPath, fs.exists(bulkPath)); + } + } + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java index cd1f758f7607..253675f85d97 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java @@ -44,6 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; @@ -282,6 +283,7 @@ public void testDayWiseWALBackup() throws IOException { long oneDayBackTime = currentTime - ONE_DAY_IN_MILLISECONDS; SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); String expectedPrevDayDir = dateFormat.format(new Date(oneDayBackTime)); String expectedCurrentDayDir = dateFormat.format(new Date(currentTime)); @@ -437,8 +439,22 @@ private void verifyBackup(String backupRootDir, boolean hasBulkLoadFiles, assertEquals(0, getRowCount(tableName)); replayWALs(new Path(backupRootDir, WALS_DIR).toString(), tableName); - replayBulkLoadHFilesIfPresent(new Path(backupRootDir, BULKLOAD_FILES_DIR).toString(), - tableName); + + // replay Bulk loaded HFiles if Present + try { + Path bulkloadDir = new Path(backupRootDir, BULKLOAD_FILES_DIR); + if (fs.exists(bulkloadDir)) { + FileStatus[] directories = fs.listStatus(bulkloadDir); + for (FileStatus dirStatus : directories) { + if (dirStatus.isDirectory()) { + replayBulkLoadHFilesIfPresent(dirStatus.getPath().toString(), tableName); + } + } + } + } catch (Exception e) { + fail("Failed to replay BulkLoad HFiles properly: " + e.getMessage()); + } + assertEquals(expectedRows, getRowCount(tableName)); } } From b54da1baea4ed6b936a997c60b382f15c75e582f Mon Sep 17 00:00:00 2001 From: asolomon Date: Sat, 21 Jun 2025 00:32:14 +0530 Subject: [PATCH 07/24] HBASE-28990 Modify Incremental Backup for Continuous Backup (#6788) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Tak Lon (Stephen) Wu Signed-off-by: Andor Molnár andor@apache.org Reviewed by: Kota-SH Reviewed by: Vinayak Hegde Reviewed by: Kevin Geiszler --- .../hadoop/hbase/backup/BackupInfo.java | 14 + .../hbase/backup/impl/BackupAdminImpl.java | 58 ++-- .../impl/IncrementalTableBackupClient.java | 166 +++++++++--- .../hadoop/hbase/backup/TestBackupBase.java | 29 +- .../hbase/backup/TestBackupDescribe.java | 1 + .../hbase/backup/TestContinuousBackup.java | 15 +- .../TestIncrementalBackupWithContinuous.java | 254 ++++++++++++++++++ 7 files changed, 472 insertions(+), 65 deletions(-) create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java index 862a9cbad107..0997aec19ecf 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java @@ -124,6 +124,11 @@ public enum BackupPhase { */ private long completeTs; + /** + * Committed WAL timestamp for incremental backup + */ + private long incrCommittedWalTs; + /** * Total bytes of incremental logs copied */ @@ -293,6 +298,14 @@ public void setCompleteTs(long endTs) { this.completeTs = endTs; } + public long getIncrCommittedWalTs() { + return incrCommittedWalTs; + } + + public void setIncrCommittedWalTs(long timestamp) { + this.incrCommittedWalTs = timestamp; + } + public long getTotalBytesCopied() { return totalBytesCopied; } @@ -549,6 +562,7 @@ public String getShortDescription() { sb.append("{"); sb.append("ID=" + backupId).append(","); sb.append("Type=" + getType()).append(","); + sb.append("IsContinuous=" + isContinuousBackupEnabled()).append(","); sb.append("Tables=" + getTableListAsString()).append(","); sb.append("State=" + getState()).append(","); Calendar cal = Calendar.getInstance(); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java index e94389d69386..1e91258ba6cc 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -859,28 +859,47 @@ public String backupTables(BackupRequest request) throws IOException { String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime(); if (type == BackupType.INCREMENTAL) { - Set incrTableSet; - try (BackupSystemTable table = new BackupSystemTable(conn)) { - incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); - } + if (request.isContinuousBackupEnabled()) { + Set continuousBackupTableSet; + try (BackupSystemTable table = new BackupSystemTable(conn)) { + continuousBackupTableSet = table.getContinuousBackupTableSet().keySet(); + } + if (continuousBackupTableSet.isEmpty()) { + String msg = "Continuous backup table set contains no tables. " + + "You need to run Continuous backup first " + + (tableList != null ? "on " + StringUtils.join(tableList, ",") : ""); + throw new IOException(msg); + } + if (!continuousBackupTableSet.containsAll(tableList)) { + String extraTables = StringUtils.join(tableList, ","); + String msg = "Some tables (" + extraTables + ") haven't gone through Continuous backup. " + + "Perform Continuous backup on " + extraTables + " first, then retry the command"; + throw new IOException(msg); + } + } else { + Set incrTableSet; + try (BackupSystemTable table = new BackupSystemTable(conn)) { + incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); + } - if (incrTableSet.isEmpty()) { - String msg = - "Incremental backup table set contains no tables. " + "You need to run full backup first " + if (incrTableSet.isEmpty()) { + String msg = "Incremental backup table set contains no tables. " + + "You need to run full backup first " + (tableList != null ? "on " + StringUtils.join(tableList, ",") : ""); - throw new IOException(msg); - } - if (tableList != null) { - tableList.removeAll(incrTableSet); - if (!tableList.isEmpty()) { - String extraTables = StringUtils.join(tableList, ","); - String msg = "Some tables (" + extraTables + ") haven't gone through full backup. " - + "Perform full backup on " + extraTables + " first, " + "then retry the command"; throw new IOException(msg); } + if (tableList != null) { + tableList.removeAll(incrTableSet); + if (!tableList.isEmpty()) { + String extraTables = StringUtils.join(tableList, ","); + String msg = "Some tables (" + extraTables + ") haven't gone through full backup. " + + "Perform full backup on " + extraTables + " first, then retry the command"; + throw new IOException(msg); + } + } + tableList = Lists.newArrayList(incrTableSet); } - tableList = Lists.newArrayList(incrTableSet); } if (tableList != null && !tableList.isEmpty()) { for (TableName table : tableList) { @@ -907,7 +926,12 @@ public String backupTables(BackupRequest request) throws IOException { } } if (nonExistingTableList != null) { - if (type == BackupType.INCREMENTAL) { + // Non-continuous incremental backup is controlled by 'incremental backup table set' + // and not by user provided backup table list. This is an optimization to avoid copying + // the same set of WALs for incremental backups of different tables at different times + // HBASE-14038. Since continuous incremental backup and full backup backs-up user provided + // table list, we should inform use about non-existence of input table(s) + if (type == BackupType.INCREMENTAL && !request.isContinuousBackupEnabled()) { // Update incremental backup set tableList = excludeNonExistingTables(tableList, nonExistingTableList); } else { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index 4fac0ca3c93c..34f9a4e29866 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -17,18 +17,28 @@ */ package org.apache.hadoop.hbase.backup.impl; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; +import java.text.ParseException; +import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TimeZone; import org.apache.commons.io.FilenameUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -48,6 +58,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat; import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2; import org.apache.hadoop.hbase.mapreduce.WALPlayer; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; @@ -63,6 +74,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Strings; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -292,9 +304,19 @@ public void execute() throws IOException, ColumnFamilyMismatchException { // case PREPARE_INCREMENTAL: beginBackup(backupManager, backupInfo); backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL); - LOG.debug("For incremental backup, current table set is " - + backupManager.getIncrementalBackupTableSet()); - newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap(); + // Non-continuous Backup incremental backup is controlled by 'incremental backup table set' + // and not by user provided backup table list. This is an optimization to avoid copying + // the same set of WALs for incremental backups of different tables at different times + // HBASE-14038 + // Continuous-incremental backup backs up user provided table list/set + Set currentTableSet; + if (backupInfo.isContinuousBackupEnabled()) { + currentTableSet = backupInfo.getTables(); + } else { + currentTableSet = backupManager.getIncrementalBackupTableSet(); + newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap(); + } + LOG.debug("For incremental backup, the current table set is {}", currentTableSet); } catch (Exception e) { // fail the overall backup and return failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ", @@ -321,21 +343,24 @@ public void execute() throws IOException, ColumnFamilyMismatchException { // set overall backup status: complete. Here we make sure to complete the backup. // After this checkpoint, even if entering cancel process, will let the backup finished try { - // Set the previousTimestampMap which is before this current log roll to the manifest. - Map> previousTimestampMap = backupManager.readLogTimestampMap(); - backupInfo.setIncrTimestampMap(previousTimestampMap); - - // The table list in backupInfo is good for both full backup and incremental backup. - // For incremental backup, it contains the incremental backup table set. - backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); - - Map> newTableSetTimestampMap = - backupManager.readLogTimestampMap(); - - backupInfo.setTableSetTimestampMap(newTableSetTimestampMap); - Long newStartCode = - BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); - backupManager.writeBackupStartCode(newStartCode); + if (!backupInfo.isContinuousBackupEnabled()) { + // Set the previousTimestampMap which is before this current log roll to the manifest. + Map> previousTimestampMap = + backupManager.readLogTimestampMap(); + backupInfo.setIncrTimestampMap(previousTimestampMap); + + // The table list in backupInfo is good for both full backup and incremental backup. + // For incremental backup, it contains the incremental backup table set. + backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps); + + Map> newTableSetTimestampMap = + backupManager.readLogTimestampMap(); + + backupInfo.setTableSetTimestampMap(newTableSetTimestampMap); + Long newStartCode = + BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); + backupManager.writeBackupStartCode(newStartCode); + } List bulkLoads = handleBulkLoad(backupInfo.getTableNames()); @@ -396,23 +421,88 @@ protected void deleteBulkLoadDirectory() throws IOException { } protected void convertWALsToHFiles() throws IOException { - // get incremental backup file list and prepare parameters for DistCp - List incrBackupFileList = backupInfo.getIncrBackupFileList(); - // Get list of tables in incremental backup set - Set tableSet = backupManager.getIncrementalBackupTableSet(); - // filter missing files out (they have been copied by previous backups) - incrBackupFileList = filterMissingFiles(incrBackupFileList); - List tableList = new ArrayList(); - for (TableName table : tableSet) { - // Check if table exists - if (tableExists(table, conn)) { - tableList.add(table.getNameAsString()); - } else { - LOG.warn("Table " + table + " does not exists. Skipping in WAL converter"); + long previousBackupTs = 0L; + if (backupInfo.isContinuousBackupEnabled()) { + Set tableSet = backupInfo.getTables(); + List backupInfos = backupManager.getBackupHistory(true); + for (TableName table : tableSet) { + for (BackupInfo backup : backupInfos) { + // find previous backup for this table + if (backup.getTables().contains(table)) { + LOG.info("Found previous backup of type {} with id {} for table {}", backup.getType(), + backup.getBackupId(), table.getNameAsString()); + List walBackupFileList; + if (backup.getType() == BackupType.FULL) { + previousBackupTs = backup.getStartTs(); + } else { + previousBackupTs = backup.getIncrCommittedWalTs(); + } + walBackupFileList = getBackupLogs(previousBackupTs); + walToHFiles(walBackupFileList, Arrays.asList(table.getNameAsString()), + previousBackupTs); + break; + } + } } + } else { + // get incremental backup file list and prepare parameters for DistCp + List incrBackupFileList = backupInfo.getIncrBackupFileList(); + // Get list of tables in incremental backup set + Set tableSet = backupManager.getIncrementalBackupTableSet(); + // filter missing files out (they have been copied by previous backups) + incrBackupFileList = filterMissingFiles(incrBackupFileList); + List tableList = new ArrayList(); + for (TableName table : tableSet) { + // Check if table exists + if (tableExists(table, conn)) { + tableList.add(table.getNameAsString()); + } else { + LOG.warn("Table " + table + " does not exists. Skipping in WAL converter"); + } + } + walToHFiles(incrBackupFileList, tableList, previousBackupTs); + } + } + + private List getBackupLogs(long startTs) throws IOException { + // get log files from backup dir + String walBackupDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + if (Strings.isNullOrEmpty(walBackupDir)) { + throw new IOException( + "Incremental backup requires the WAL backup directory " + CONF_CONTINUOUS_BACKUP_WAL_DIR); } - walToHFiles(incrBackupFileList, tableList); + List resultLogFiles = new ArrayList<>(); + Path walBackupPath = new Path(walBackupDir); + FileSystem backupFs = FileSystem.get(walBackupPath.toUri(), conf); + FileStatus[] dayDirs = backupFs.listStatus(new Path(walBackupDir, WALS_DIR)); + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); + + for (FileStatus dayDir : dayDirs) { + if (!dayDir.isDirectory()) { + continue; // Skip files, only process directories + } + String dirName = dayDir.getPath().getName(); + try { + Date dirDate = dateFormat.parse(dirName); + long dirStartTime = dirDate.getTime(); // Start of that day (00:00:00) + long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) + + if (dirEndTime >= startTs) { + Path dirPath = dayDir.getPath(); + FileStatus[] logs = backupFs.listStatus(dirPath); + for (FileStatus log : logs) { + String filepath = log.getPath().toString(); + LOG.debug("Found WAL file: {}", filepath); + resultLogFiles.add(filepath); + } + } + } catch (ParseException e) { + LOG.warn("Skipping invalid directory name: " + dirName, e); + } + } + return resultLogFiles; } protected boolean tableExists(TableName table, Connection conn) throws IOException { @@ -421,7 +511,8 @@ protected boolean tableExists(TableName table, Connection conn) throws IOExcepti } } - protected void walToHFiles(List dirPaths, List tableList) throws IOException { + protected void walToHFiles(List dirPaths, List tableList, long previousBackupTs) + throws IOException { Tool player = new WALPlayer(); // Player reads all files in arbitrary directory structure and creates @@ -435,9 +526,16 @@ protected void walToHFiles(List dirPaths, List tableList) throws conf.set(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ";"); conf.setBoolean(WALPlayer.MULTI_TABLES_SUPPORT, true); conf.set(JOB_NAME_CONF_KEY, jobname); - boolean diskBasedSortingEnabledOriginalValue = HFileOutputFormat2.diskBasedSortingEnabled(conf); conf.setBoolean(HFileOutputFormat2.DISK_BASED_SORTING_ENABLED_KEY, true); + if (backupInfo.isContinuousBackupEnabled()) { + conf.set(WALInputFormat.START_TIME_KEY, Long.toString(previousBackupTs)); + // committedWALsTs is needed only for Incremental backups with continuous backup + // since these do not depend on log roll ts + long committedWALsTs = BackupUtils.getReplicationCheckpoint(conn); + backupInfo.setIncrCommittedWalTs(committedWALsTs); + conf.set(WALInputFormat.END_TIME_KEY, Long.toString(committedWALsTs)); + } String[] playerArgs = { dirs, StringUtils.join(tableList, ",") }; try { diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index f85171c735a0..5f28c7dfe869 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_MAX_WAL_SIZE; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INITIAL_DELAY; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INTERVAL; @@ -409,15 +410,31 @@ protected BackupRequest createBackupRequest(BackupType type, List tab return request; } + protected BackupRequest createBackupRequest(BackupType type, List tables, String path, + boolean noChecksumVerify, boolean continuousBackupEnabled) { + BackupRequest.Builder builder = new BackupRequest.Builder(); + BackupRequest request = builder.withBackupType(type).withTableList(tables) + .withTargetRootDir(path).withNoChecksumVerify(noChecksumVerify) + .withContinuousBackupEnabled(continuousBackupEnabled).build(); + return request; + } + protected String backupTables(BackupType type, List tables, String path) throws IOException { + return backupTables(type, tables, path, false); + } + + protected String backupTables(BackupType type, List tables, String path, + boolean isContinuousBackup) throws IOException { Connection conn = null; BackupAdmin badmin = null; String backupId; try { conn = ConnectionFactory.createConnection(conf1); badmin = new BackupAdminImpl(conn); - BackupRequest request = createBackupRequest(type, new ArrayList<>(tables), path); + + BackupRequest request = + createBackupRequest(type, new ArrayList<>(tables), path, false, isContinuousBackup); backupId = badmin.backupTables(request); } finally { if (badmin != null) { @@ -549,4 +566,14 @@ protected void dumpBackupDir() throws IOException { LOG.debug(Objects.toString(it.next().getPath())); } } + + void deleteContinuousBackupReplicationPeerIfExists(Admin admin) throws IOException { + if ( + admin.listReplicationPeers().stream() + .anyMatch(peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER)) + ) { + admin.disableReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); + admin.removeReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); + } + } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java index 7ce039fd6668..6084dc730ee8 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java @@ -94,6 +94,7 @@ public void testBackupDescribeCommand() throws Exception { System.setOut(new PrintStream(baos)); String[] args = new String[] { "describe", backupId }; + // Run backup int ret = ToolRunner.run(conf1, new BackupDriver(), args); assertTrue(ret == 0); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java index fe44ebf420df..0cc34ed63eb0 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java @@ -259,8 +259,7 @@ private void verifyReplicationPeerSubscription(TableName table) throws IOExcepti } } - private String[] buildBackupArgs(String backupType, TableName[] tables, - boolean continuousEnabled) { + String[] buildBackupArgs(String backupType, TableName[] tables, boolean continuousEnabled) { String tableNames = Arrays.stream(tables).map(TableName::getNameAsString).collect(Collectors.joining(",")); @@ -272,7 +271,7 @@ private String[] buildBackupArgs(String backupType, TableName[] tables, } } - private BackupManifest getLatestBackupManifest(List backups) throws IOException { + BackupManifest getLatestBackupManifest(List backups) throws IOException { BackupInfo newestBackup = backups.get(0); return HBackupFileSystem.getManifest(conf1, new Path(BACKUP_ROOT_DIR), newestBackup.getBackupId()); @@ -289,14 +288,4 @@ private void verifyTableInBackupSystemTable(TableName table) throws IOException } } - private void deleteContinuousBackupReplicationPeerIfExists(Admin admin) throws IOException { - if ( - admin.listReplicationPeers().stream() - .anyMatch(peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER)) - ) { - admin.disableReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); - admin.removeReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); - } - } - } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java new file mode 100644 index 000000000000..79d1df645b95 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java @@ -0,0 +1,254 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_DEFAULT; +import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; +import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.impl.BulkLoad; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.tool.BulkLoadHFiles; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.HFileTestUtil; +import org.apache.hadoop.util.ToolRunner; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + +@Category(LargeTests.class) +public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIncrementalBackupWithContinuous.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestIncrementalBackupWithContinuous.class); + + private byte[] ROW = Bytes.toBytes("row1"); + private final byte[] FAMILY = Bytes.toBytes("family"); + private final byte[] COLUMN = Bytes.toBytes("col"); + private static final int ROWS_IN_BULK_LOAD = 100; + + @Test + public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception { + LOG.info("Testing incremental backup with continuous backup"); + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + Table t1 = TEST_UTIL.createTable(tableName, FAMILY); + + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + int before = table.getBackupHistory().size(); + + // Run continuous backup + String[] args = buildBackupArgs("full", new TableName[] { tableName }, true); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Full Backup should succeed", 0, ret); + + // Verify backup history increased and all the backups are succeeded + LOG.info("Verify backup history increased and all the backups are succeeded"); + List backups = table.getBackupHistory(); + assertEquals("Backup history should increase", before + 1, backups.size()); + for (BackupInfo data : List.of(backups.get(0))) { + String backupId = data.getBackupId(); + assertTrue(checkSucceeded(backupId)); + } + + // Verify backup manifest contains the correct tables + LOG.info("Verify backup manifest contains the correct tables"); + BackupManifest manifest = getLatestBackupManifest(backups); + assertEquals("Backup should contain the expected tables", Sets.newHashSet(tableName), + new HashSet<>(manifest.getTableList())); + + Put p = new Put(ROW); + p.addColumn(FAMILY, COLUMN, COLUMN); + t1.put(p); + Thread.sleep(5000); + + // Run incremental backup + LOG.info("Run incremental backup now"); + before = table.getBackupHistory().size(); + args = buildBackupArgs("incremental", new TableName[] { tableName }, false); + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Incremental Backup should succeed", 0, ret); + LOG.info("Incremental backup completed"); + + // Verify backup history increased and all the backups are succeeded + backups = table.getBackupHistory(); + String incrementalBackupid = null; + assertEquals("Backup history should increase", before + 1, backups.size()); + for (BackupInfo data : List.of(backups.get(0))) { + String backupId = data.getBackupId(); + incrementalBackupid = backupId; + assertTrue(checkSucceeded(backupId)); + } + + TEST_UTIL.truncateTable(tableName); + // Restore incremental backup + TableName[] tables = new TableName[] { tableName }; + BackupAdminImpl client = new BackupAdminImpl(TEST_UTIL.getConnection()); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, incrementalBackupid, false, + tables, tables, true)); + + verifyTable(t1); + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); + } + } + + @Test + public void testContinuousBackupWithIncrementalBackupAndBulkloadSuccess() throws Exception { + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { + // The test starts with some data, and no bulk loaded rows. + int expectedRowCount = NB_ROWS_IN_BATCH; + assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); + assertTrue(systemTable.readBulkloadRows(List.of(table1)).isEmpty()); + + // Bulk loads aren't tracked if the table isn't backed up yet + performBulkLoad("bulk1", methodName); + expectedRowCount += ROWS_IN_BULK_LOAD; + assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); + assertEquals(0, systemTable.readBulkloadRows(List.of(table1)).size()); + + // Create a backup, bulk loads are now being tracked + String backup1 = backupTables(BackupType.FULL, List.of(table1), BACKUP_ROOT_DIR, true); + assertTrue(checkSucceeded(backup1)); + + loadTable(TEST_UTIL.getConnection().getTable(table1)); + assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); + performBulkLoad("bulk2", methodName); + expectedRowCount += ROWS_IN_BULK_LOAD; + assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); + assertEquals(1, systemTable.readBulkloadRows(List.of(table1)).size()); + + // Creating an incremental backup clears the bulk loads + performBulkLoad("bulk4", methodName); + performBulkLoad("bulk5", methodName); + performBulkLoad("bulk6", methodName); + expectedRowCount += 3 * ROWS_IN_BULK_LOAD; + assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); + assertEquals(4, systemTable.readBulkloadRows(List.of(table1)).size()); + String backup2 = backupTables(BackupType.INCREMENTAL, List.of(table1), BACKUP_ROOT_DIR, true); + assertTrue(checkSucceeded(backup2)); + assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); + assertEquals(0, systemTable.readBulkloadRows(List.of(table1)).size()); + int rowCountAfterBackup2 = expectedRowCount; + + // Doing another bulk load, to check that this data will disappear after a restore operation + performBulkLoad("bulk7", methodName); + expectedRowCount += ROWS_IN_BULK_LOAD; + assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); + List bulkloadsTemp = systemTable.readBulkloadRows(List.of(table1)); + assertEquals(1, bulkloadsTemp.size()); + BulkLoad bulk7 = bulkloadsTemp.get(0); + + // Doing a restore. Overwriting the table implies clearing the bulk loads, + // but the loading of restored data involves loading bulk data, we expect 2 bulk loads + // associated with backup 3 (loading of full backup, loading of incremental backup). + BackupAdmin client = getBackupAdmin(); + client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backup2, false, + new TableName[] { table1 }, new TableName[] { table1 }, true)); + assertEquals(rowCountAfterBackup2, TEST_UTIL.countRows(table1)); + List bulkLoads = systemTable.readBulkloadRows(List.of(table1)); + assertEquals(3, bulkLoads.size()); + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); + } + } + + private void verifyTable(Table t1) throws IOException { + Get g = new Get(ROW); + Result r = t1.get(g); + assertEquals(1, r.size()); + assertTrue(CellUtil.matchingQualifier(r.rawCells()[0], COLUMN)); + } + + private void performBulkLoad(String keyPrefix, String testDir) throws IOException { + FileSystem fs = TEST_UTIL.getTestFileSystem(); + Path baseDirectory = TEST_UTIL.getDataTestDirOnTestFS(testDir); + Path hfilePath = + new Path(baseDirectory, Bytes.toString(famName) + Path.SEPARATOR + "hfile_" + keyPrefix); + + HFileTestUtil.createHFile(TEST_UTIL.getConfiguration(), fs, hfilePath, famName, qualName, + Bytes.toBytes(keyPrefix), Bytes.toBytes(keyPrefix + "z"), ROWS_IN_BULK_LOAD); + + listFiles(fs, baseDirectory, baseDirectory); + + Map result = + BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(table1, baseDirectory); + assertFalse(result.isEmpty()); + } + + private static Set listFiles(final FileSystem fs, final Path root, final Path dir) + throws IOException { + Set files = new HashSet<>(); + FileStatus[] list = CommonFSUtils.listStatus(fs, dir); + if (list != null) { + for (FileStatus fstat : list) { + if (fstat.isDirectory()) { + LOG.info("Found directory {}", Objects.toString(fstat.getPath())); + files.addAll(listFiles(fs, root, fstat.getPath())); + } else { + LOG.info("Found file {}", Objects.toString(fstat.getPath())); + String file = fstat.getPath().makeQualified(fs).toString(); + files.add(file); + } + } + } + return files; + } + + protected static void loadTable(Table table) throws Exception { + Put p; // 100 + 1 row to t1_syncup + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p = new Put(Bytes.toBytes("row" + i)); + p.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + table.put(p); + } + } +} From 393602dc301d0bd286a31d74ac2573758288aaa2 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Mon, 23 Jun 2025 22:34:58 +0530 Subject: [PATCH 08/24] HBASE-29350: Ensure Cleanup of Continuous Backup WALs After Last Backup is Force Deleted (#7090) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kevin Geiszler --- .../hbase/backup/impl/BackupCommands.java | 73 ++++++++- .../hbase/backup/impl/BackupSystemTable.java | 2 +- .../backup/impl/FullTableBackupClient.java | 3 + .../hadoop/hbase/backup/TestBackupBase.java | 11 +- .../backup/TestBackupDeleteWithCleanup.java | 150 ++++++++++++++++-- 5 files changed, 218 insertions(+), 21 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 11b6890ed038..2020b84bc1cb 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -84,8 +84,10 @@ import org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager; import org.apache.hadoop.hbase.backup.util.BackupSet; import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; @@ -892,7 +894,8 @@ private boolean canAnyOtherBackupCover(List allBackups, BackupInfo c /** * Cleans up Write-Ahead Logs (WALs) that are no longer required for PITR after a successful - * backup deletion. + * backup deletion. If no full backups are present, all WALs are deleted, tables are removed + * from continuous backup metadata, and the associated replication peer is disabled. */ private void cleanUpUnusedBackupWALs() throws IOException { Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create(); @@ -903,7 +906,8 @@ private void cleanUpUnusedBackupWALs() throws IOException { return; } - try (BackupSystemTable sysTable = new BackupSystemTable(conn)) { + try (Admin admin = conn.getAdmin(); + BackupSystemTable sysTable = new BackupSystemTable(conn)) { // Get list of tables under continuous backup Map continuousBackupTables = sysTable.getContinuousBackupTableSet(); if (continuousBackupTables.isEmpty()) { @@ -914,7 +918,15 @@ private void cleanUpUnusedBackupWALs() throws IOException { // Find the earliest timestamp after which WALs are still needed long cutoffTimestamp = determineWALCleanupCutoffTime(sysTable); if (cutoffTimestamp == 0) { - System.err.println("ERROR: No valid full backup found. Skipping WAL cleanup."); + // No full backup exists. PITR cannot function without a base full backup. + // Clean up all WALs, remove tables from backup metadata, and disable the replication + // peer. + System.out + .println("No full backups found. Cleaning up all WALs and disabling replication peer."); + + disableContinuousBackupReplicationPeer(admin); + removeAllTablesFromContinuousBackup(sysTable); + deleteAllBackupWALFiles(conf, backupWalDir); return; } @@ -944,6 +956,16 @@ long determineWALCleanupCutoffTime(BackupSystemTable sysTable) throws IOExceptio return 0; } + private void disableContinuousBackupReplicationPeer(Admin admin) throws IOException { + for (ReplicationPeerDescription peer : admin.listReplicationPeers()) { + if (peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER) && peer.isEnabled()) { + admin.disableReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); + System.out.println("Disabled replication peer: " + CONTINUOUS_BACKUP_REPLICATION_PEER); + break; + } + } + } + /** * Updates the start time for continuous backups if older than cutoff timestamp. * @param sysTable Backup system table @@ -966,6 +988,49 @@ void updateBackupTableStartTimes(BackupSystemTable sysTable, long cutoffTimestam } } + private void removeAllTablesFromContinuousBackup(BackupSystemTable sysTable) + throws IOException { + Map allTables = sysTable.getContinuousBackupTableSet(); + if (!allTables.isEmpty()) { + sysTable.removeContinuousBackupTableSet(allTables.keySet()); + System.out.println("Removed all tables from continuous backup metadata."); + } + } + + private void deleteAllBackupWALFiles(Configuration conf, String backupWalDir) + throws IOException { + try { + BackupFileSystemManager manager = + new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); + FileSystem fs = manager.getBackupFs(); + Path walDir = manager.getWalsDir(); + Path bulkloadDir = manager.getBulkLoadFilesDir(); + + // Delete contents under WAL directory + if (fs.exists(walDir)) { + FileStatus[] walContents = fs.listStatus(walDir); + for (FileStatus item : walContents) { + fs.delete(item.getPath(), true); // recursive delete of each child + } + System.out.println("Deleted all contents under WAL directory: " + walDir); + } + + // Delete contents under bulk load directory + if (fs.exists(bulkloadDir)) { + FileStatus[] bulkContents = fs.listStatus(bulkloadDir); + for (FileStatus item : bulkContents) { + fs.delete(item.getPath(), true); // recursive delete of each child + } + System.out.println("Deleted all contents under Bulk Load directory: " + bulkloadDir); + } + + } catch (IOException e) { + System.out.println("WARNING: Failed to delete contents under backup directories: " + + backupWalDir + ". Error: " + e.getMessage()); + throw e; + } + } + /** * Cleans up old WAL and bulk-loaded files based on the determined cutoff timestamp. */ @@ -1010,7 +1075,7 @@ void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) System.out.println("WARNING: Failed to parse directory name '" + dirName + "'. Skipping. Error: " + e.getMessage()); } catch (IOException e) { - System.out.println("WARNING: Failed to delete directory '" + dirPath + System.err.println("WARNING: Failed to delete directory '" + dirPath + "'. Skipping. Error: " + e.getMessage()); } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 24bd0888f9cb..098946dab38a 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -1510,7 +1510,7 @@ private Delete createDeleteForIncrBackupTableSet(String backupRoot) { private Delete createDeleteForContinuousBackupTableSet(Set tables) { Delete delete = new Delete(rowkey(CONTINUOUS_BACKUP_SET)); for (TableName tableName : tables) { - delete.addColumns(META_FAMILY, Bytes.toBytes(tableName.getNameAsString())); + delete.addColumn(META_FAMILY, Bytes.toBytes(tableName.getNameAsString())); } return delete; } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java index 907400aa1129..f76dbda60abd 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -299,6 +299,9 @@ private void updateContinuousBackupReplicationPeer(Admin admin) throws IOExcepti .collect(Collectors.toMap(tableName -> tableName, tableName -> new ArrayList<>())); try { + if (!admin.isReplicationPeerEnabled(CONTINUOUS_BACKUP_REPLICATION_PEER)) { + admin.enableReplicationPeer(CONTINUOUS_BACKUP_REPLICATION_PEER); + } admin.appendReplicationPeerTableCFs(CONTINUOUS_BACKUP_REPLICATION_PEER, tableMap); LOG.info("Updated replication peer {} with table and column family map.", CONTINUOUS_BACKUP_REPLICATION_PEER); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index 5f28c7dfe869..4fb9f209b760 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -410,13 +410,12 @@ protected BackupRequest createBackupRequest(BackupType type, List tab return request; } - protected BackupRequest createBackupRequest(BackupType type, List tables, String path, - boolean noChecksumVerify, boolean continuousBackupEnabled) { + protected BackupRequest createBackupRequest(BackupType type, List tables, + String rootDir, boolean noChecksumVerify, boolean isContinuousBackupEnabled) { BackupRequest.Builder builder = new BackupRequest.Builder(); - BackupRequest request = builder.withBackupType(type).withTableList(tables) - .withTargetRootDir(path).withNoChecksumVerify(noChecksumVerify) - .withContinuousBackupEnabled(continuousBackupEnabled).build(); - return request; + return builder.withBackupType(type).withTableList(tables).withTargetRootDir(rootDir) + .withNoChecksumVerify(noChecksumVerify).withContinuousBackupEnabled(isContinuousBackupEnabled) + .build(); } protected String backupTables(BackupType type, List tables, String path) diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java index 6d76ac4e89bf..07c9110072b2 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.backup; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; @@ -28,18 +29,25 @@ import java.io.IOException; import java.text.SimpleDateFormat; +import java.util.ArrayList; import java.util.Date; +import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.ToolRunner; +import org.junit.After; +import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,38 +63,55 @@ public class TestBackupDeleteWithCleanup extends TestBackupBase { String backupWalDirName = "TestBackupDeleteWithCleanup"; - @Test - public void testBackupDeleteWithCleanupLogic() throws Exception { + private FileSystem fs; + private Path backupWalDir; + private BackupSystemTable backupSystemTable; + + @Before + public void setUpTest() throws Exception { Path root = TEST_UTIL.getDataTestDirOnTestFS(); - Path backupWalDir = new Path(root, backupWalDirName); + backupWalDir = new Path(root, backupWalDirName); conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); - FileSystem fs = FileSystem.get(conf1); + fs = FileSystem.get(conf1); fs.mkdirs(backupWalDir); + backupSystemTable = new BackupSystemTable(TEST_UTIL.getConnection()); + } + + @After + public void tearDownTest() throws Exception { + if (backupSystemTable != null) { + backupSystemTable.close(); + } + if (fs != null && backupWalDir != null) { + fs.delete(backupWalDir, true); + } + + EnvironmentEdgeManager.reset(); + } + @Test + public void testBackupDeleteWithCleanupLogic() throws Exception { // Step 1: Setup Backup Folders long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); - setupBackupFolders(fs, backupWalDir, currentTime); + setupBackupFolders(currentTime); // Log the directory structure before cleanup logDirectoryStructure(fs, backupWalDir, "Directory structure BEFORE cleanup:"); // Step 2: Simulate Backup Creation - BackupSystemTable backupSystemTable = new BackupSystemTable(TEST_UTIL.getConnection()); backupSystemTable.addContinuousBackupTableSet(Set.of(table1), currentTime - (2 * ONE_DAY_IN_MILLISECONDS)); EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - (2 * ONE_DAY_IN_MILLISECONDS)); + String backupId = fullTableBackup(Lists.newArrayList(table1)); assertTrue(checkSucceeded(backupId)); - String anotherBackupId = fullTableBackup(Lists.newArrayList(table1)); assertTrue(checkSucceeded(anotherBackupId)); // Step 3: Run Delete Command - int ret = - ToolRunner.run(conf1, new BackupDriver(), new String[] { "delete", "-l", backupId, "-fd" }); - assertEquals(0, ret); + deleteBackup(backupId); // Log the directory structure after cleanup logDirectoryStructure(fs, backupWalDir, "Directory structure AFTER cleanup:"); @@ -96,6 +121,70 @@ public void testBackupDeleteWithCleanupLogic() throws Exception { // Step 5: Verify System Table Update verifySystemTableUpdate(backupSystemTable, currentTime); + + // Cleanup + deleteBackup(anotherBackupId); + } + + @Test + public void testSingleBackupForceDelete() throws Exception { + // Step 1: Setup Backup Folders + long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); + setupBackupFolders(currentTime); + + // Log the directory structure before cleanup + logDirectoryStructure(fs, backupWalDir, "Directory structure BEFORE cleanup:"); + + // Step 2: Simulate Backup Creation + backupSystemTable.addContinuousBackupTableSet(Set.of(table1), + currentTime - (2 * ONE_DAY_IN_MILLISECONDS)); + + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - (2 * ONE_DAY_IN_MILLISECONDS)); + + String backupId = fullTableBackupWithContinuous(Lists.newArrayList(table1)); + assertTrue(checkSucceeded(backupId)); + + assertTrue("Backup replication peer should be enabled after the backup", + continuousBackupReplicationPeerExistsAndEnabled()); + + // Step 3: Run Delete Command + deleteBackup(backupId); + + // Log the directory structure after cleanup + logDirectoryStructure(fs, backupWalDir, "Directory structure AFTER cleanup:"); + + // Step 4: Verify CONTINUOUS_BACKUP_REPLICATION_PEER is disabled + assertFalse("Backup replication peer should be disabled or removed", + continuousBackupReplicationPeerExistsAndEnabled()); + + // Step 5: Verify that system table is updated to remove all the tables + Set remainingTables = backupSystemTable.getContinuousBackupTableSet().keySet(); + assertTrue("System table should have no tables after all full backups are clear", + remainingTables.isEmpty()); + + // Step 6: Verify that the backup WAL directory is empty + assertTrue("WAL backup directory should be empty after force delete", + areWalAndBulkloadDirsEmpty(conf1, backupWalDir.toString())); + + // Step 7: Take new full backup with continuous backup enabled + String backupIdContinuous = fullTableBackupWithContinuous(Lists.newArrayList(table1)); + + // Step 8: Verify CONTINUOUS_BACKUP_REPLICATION_PEER is enabled again + assertTrue("Backup replication peer should be re-enabled after new backup", + continuousBackupReplicationPeerExistsAndEnabled()); + + // And system table has new entry + Set newTables = backupSystemTable.getContinuousBackupTableSet().keySet(); + assertTrue("System table should contain the table after new backup", + newTables.contains(table1)); + + // Cleanup + deleteBackup(backupIdContinuous); + } + + private void setupBackupFolders(long currentTime) throws IOException { + setupBackupFolders(fs, backupWalDir, currentTime); } public static void setupBackupFolders(FileSystem fs, Path backupWalDir, long currentTime) @@ -181,4 +270,45 @@ public static void listDirectory(FileSystem fs, Path dir, String indent) throws } } } + + private boolean continuousBackupReplicationPeerExistsAndEnabled() throws IOException { + return TEST_UTIL.getAdmin().listReplicationPeers().stream().anyMatch( + peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER) && peer.isEnabled()); + } + + private static boolean areWalAndBulkloadDirsEmpty(Configuration conf, String backupWalDir) + throws IOException { + BackupFileSystemManager manager = + new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); + + FileSystem fs = manager.getBackupFs(); + Path walDir = manager.getWalsDir(); + Path bulkloadDir = manager.getBulkLoadFilesDir(); + + return isDirectoryEmpty(fs, walDir) && isDirectoryEmpty(fs, bulkloadDir); + } + + private static boolean isDirectoryEmpty(FileSystem fs, Path dirPath) throws IOException { + if (!fs.exists(dirPath)) { + // Directory doesn't exist — treat as empty + return true; + } + FileStatus[] entries = fs.listStatus(dirPath); + return entries == null || entries.length == 0; + } + + private static void deleteBackup(String backupId) throws Exception { + int ret = + ToolRunner.run(conf1, new BackupDriver(), new String[] { "delete", "-l", backupId, "-fd" }); + assertEquals(0, ret); + } + + private String fullTableBackupWithContinuous(List tables) throws IOException { + try (BackupAdmin admin = new BackupAdminImpl(TEST_UTIL.getConnection())) { + BackupRequest request = + createBackupRequest(BackupType.FULL, new ArrayList<>(tables), BACKUP_ROOT_DIR, false, true); + return admin.backupTables(request); + } + } + } From 1a4c61024be1b195dcd4dbef1bd817098bfd5495 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Wed, 25 Jun 2025 01:28:51 +0530 Subject: [PATCH 09/24] HBASE-29219 Ignore Empty WAL Files While Consuming Backed-Up WAL Files (#7106) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kota-SH Reviewed by: Kevin Geiszler --- .../hbase/backup/impl/BackupAdminImpl.java | 2 + .../hbase/mapreduce/WALInputFormat.java | 11 ++++- .../hadoop/hbase/mapreduce/WALPlayer.java | 18 ++++++++ .../hbase/mapreduce/TestWALInputFormat.java | 43 +++++++++++++++++ .../hadoop/hbase/mapreduce/TestWALPlayer.java | 46 +++++++++++++++++++ 5 files changed, 118 insertions(+), 2 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java index 1e91258ba6cc..e82d9804f9dc 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES; import java.io.IOException; import java.text.ParseException; @@ -846,6 +847,7 @@ private Tool initializeWalPlayer(long startTime, long endTime) { Configuration conf = HBaseConfiguration.create(conn.getConfiguration()); conf.setLong(WALInputFormat.START_TIME_KEY, startTime); conf.setLong(WALInputFormat.END_TIME_KEY, endTime); + conf.setBoolean(IGNORE_EMPTY_FILES, true); Tool walPlayer = new WALPlayer(); walPlayer.setConf(conf); return walPlayer; diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java index badf581efefc..b5c1d39a550c 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java @@ -328,14 +328,21 @@ List getSplits(final JobContext context, final String startKey, fina throw e; } } + + boolean ignoreEmptyFiles = + conf.getBoolean(WALPlayer.IGNORE_EMPTY_FILES, WALPlayer.DEFAULT_IGNORE_EMPTY_FILES); List splits = new ArrayList(allFiles.size()); for (FileStatus file : allFiles) { + if (ignoreEmptyFiles && file.getLen() == 0) { + LOG.warn("Ignoring empty file: " + file.getPath()); + continue; + } splits.add(new WALSplit(file.getPath().toString(), file.getLen(), startTime, endTime)); } return splits; } - private Path[] getInputPaths(Configuration conf) { + Path[] getInputPaths(Configuration conf) { String inpDirs = conf.get(FileInputFormat.INPUT_DIR); return StringUtils .stringToPath(inpDirs.split(conf.get(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ","))); @@ -349,7 +356,7 @@ private Path[] getInputPaths(Configuration conf) { * equal to this value else we will filter out the file. If name does not seem to * have a timestamp, we will just return it w/o filtering. */ - private List getFiles(FileSystem fs, Path dir, long startTime, long endTime, + List getFiles(FileSystem fs, Path dir, long startTime, long endTime, Configuration conf) throws IOException { List result = new ArrayList<>(); LOG.debug("Scanning " + dir.toString() + " for WAL files"); diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java index 9813118e2502..4c0b12ef7333 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java @@ -83,6 +83,24 @@ public class WALPlayer extends Configured implements Tool { public final static String IGNORE_MISSING_FILES = "wal.input.ignore.missing.files"; public final static String MULTI_TABLES_SUPPORT = "wal.multi.tables.support"; + /** + * Configuration flag that controls how the WALPlayer handles empty input WAL files. + *

+ * If set to {@code true}, the WALPlayer will silently ignore empty files that cannot be parsed as + * valid WAL files. This is useful in scenarios where such files are expected (e.g., due to + * partial writes or cleanup operations). + *

+ *

+ * If set to {@code false} (default), the WALPlayer will throw an exception when it encounters an + * empty or un-parsable WAL file. This is useful for catching unexpected data issues early. + *

+ *

+ * Default value: {@link #DEFAULT_IGNORE_EMPTY_FILES} ({@code false}) + *

+ */ + public final static String IGNORE_EMPTY_FILES = "wal.input.ignore.empty.files"; + public final static boolean DEFAULT_IGNORE_EMPTY_FILES = false; + protected static final String tableSeparator = ";"; private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name"; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java index 930c8d11375f..0b2d66bd0b8f 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java @@ -18,11 +18,15 @@ package org.apache.hadoop.hbase.mapreduce; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -125,4 +129,43 @@ public void testHandlesArchivedWALFiles() throws Exception { assertEquals(archiveWal.toString(), split.getLogFileName()); } + @Test + public void testEmptyFileIsIgnoredWhenConfigured() throws IOException, InterruptedException { + List splits = getSplitsForEmptyFile(true); + assertTrue("Empty file should be ignored when IGNORE_EMPTY_FILES is true", splits.isEmpty()); + } + + @Test + public void testEmptyFileIsIncludedWhenNotIgnored() throws IOException, InterruptedException { + List splits = getSplitsForEmptyFile(false); + assertEquals("Empty file should be included when IGNORE_EMPTY_FILES is false", 1, + splits.size()); + } + + private List getSplitsForEmptyFile(boolean ignoreEmptyFiles) + throws IOException, InterruptedException { + Configuration conf = new Configuration(); + conf.setBoolean(WALPlayer.IGNORE_EMPTY_FILES, ignoreEmptyFiles); + + JobContext jobContext = Mockito.mock(JobContext.class); + Mockito.when(jobContext.getConfiguration()).thenReturn(conf); + + LocatedFileStatus emptyFile = Mockito.mock(LocatedFileStatus.class); + Mockito.when(emptyFile.getLen()).thenReturn(0L); + Mockito.when(emptyFile.getPath()).thenReturn(new Path("/empty.wal")); + + WALInputFormat inputFormat = new WALInputFormat() { + @Override + Path[] getInputPaths(Configuration conf) { + return new Path[] { new Path("/input") }; + } + + @Override + List getFiles(FileSystem fs, Path inputPath, long startTime, long endTime) { + return Collections.singletonList(emptyFile); + } + }; + + return inputFormat.getSplits(jobContext, "", ""); + } } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java index 220e9a3793cd..bbadabab69bf 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java @@ -22,6 +22,7 @@ import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; @@ -31,10 +32,12 @@ import java.io.ByteArrayOutputStream; import java.io.File; +import java.io.IOException; import java.io.PrintStream; import java.util.ArrayList; import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -376,4 +379,47 @@ private interface TestMethod { void run() throws Exception; } + @Test + public void testIgnoreEmptyWALFiles() throws Exception { + Path inputDir = createEmptyWALFile("empty-wal-dir"); + FileSystem dfs = TEST_UTIL.getDFSCluster().getFileSystem(); + Path emptyWAL = new Path(inputDir, "empty.wal"); + + assertTrue("Empty WAL file should exist", dfs.exists(emptyWAL)); + assertEquals("WAL file should be 0 bytes", 0, dfs.getFileStatus(emptyWAL).getLen()); + + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.setBoolean(WALPlayer.IGNORE_EMPTY_FILES, true); + + int exitCode = ToolRunner.run(conf, new WALPlayer(conf), new String[] { inputDir.toString() }); + assertEquals("WALPlayer should exit cleanly even with empty files", 0, exitCode); + } + + @Test + public void testFailOnEmptyWALFilesWhenNotIgnored() throws Exception { + Path inputDir = createEmptyWALFile("fail-empty-wal-dir"); + FileSystem dfs = TEST_UTIL.getDFSCluster().getFileSystem(); + Path emptyWAL = new Path(inputDir, "empty.wal"); + + assertTrue("Empty WAL file should exist", dfs.exists(emptyWAL)); + assertEquals("WAL file should be 0 bytes", 0, dfs.getFileStatus(emptyWAL).getLen()); + + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.setBoolean(WALPlayer.IGNORE_EMPTY_FILES, false); + + int exitCode = ToolRunner.run(conf, new WALPlayer(conf), new String[] { inputDir.toString() }); + assertNotEquals("WALPlayer should fail on empty files when not ignored", 0, exitCode); + } + + private Path createEmptyWALFile(String walDir) throws IOException { + FileSystem dfs = TEST_UTIL.getDFSCluster().getFileSystem(); + Path inputDir = new Path("/" + walDir); + dfs.mkdirs(inputDir); + + Path emptyWAL = new Path(inputDir, "empty.wal"); + FSDataOutputStream out = dfs.create(emptyWAL); + out.close(); // Explicitly closing the stream + + return inputDir; + } } From 1a2ff7bddd6880b300b561f8457886bffd6d0de5 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Fri, 27 Jun 2025 23:53:52 +0530 Subject: [PATCH 10/24] HBASE-29406: Skip Copying Bulkloaded Files to Backup Location in Continuous Backup (#7119) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kevin Geiszler --- .../hbase/backup/impl/BackupCommands.java | 14 +-- .../backup/impl/FullTableBackupClient.java | 8 -- .../replication/BackupFileSystemManager.java | 11 +- .../backup/replication/BulkLoadProcessor.java | 96 -------------- .../ContinuousBackupReplicationEndpoint.java | 93 +------------- .../backup/TestBackupDeleteWithCleanup.java | 19 +-- .../hbase/backup/impl/TestBackupCommands.java | 7 +- ...stContinuousBackupReplicationEndpoint.java | 117 ++---------------- 8 files changed, 18 insertions(+), 347 deletions(-) delete mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 2020b84bc1cb..3ae97c487ef3 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -1004,7 +1004,6 @@ private void deleteAllBackupWALFiles(Configuration conf, String backupWalDir) new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); FileSystem fs = manager.getBackupFs(); Path walDir = manager.getWalsDir(); - Path bulkloadDir = manager.getBulkLoadFilesDir(); // Delete contents under WAL directory if (fs.exists(walDir)) { @@ -1015,15 +1014,6 @@ private void deleteAllBackupWALFiles(Configuration conf, String backupWalDir) System.out.println("Deleted all contents under WAL directory: " + walDir); } - // Delete contents under bulk load directory - if (fs.exists(bulkloadDir)) { - FileStatus[] bulkContents = fs.listStatus(bulkloadDir); - for (FileStatus item : bulkContents) { - fs.delete(item.getPath(), true); // recursive delete of each child - } - System.out.println("Deleted all contents under Bulk Load directory: " + bulkloadDir); - } - } catch (IOException e) { System.out.println("WARNING: Failed to delete contents under backup directories: " + backupWalDir + ". Error: " + e.getMessage()); @@ -1032,7 +1022,7 @@ private void deleteAllBackupWALFiles(Configuration conf, String backupWalDir) } /** - * Cleans up old WAL and bulk-loaded files based on the determined cutoff timestamp. + * Cleans up old WAL files based on the determined cutoff timestamp. */ void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) throws IOException { @@ -1043,7 +1033,6 @@ void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); FileSystem fs = manager.getBackupFs(); Path walDir = manager.getWalsDir(); - Path bulkloadDir = manager.getBulkLoadFilesDir(); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); @@ -1069,7 +1058,6 @@ void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) if (dayStart + ONE_DAY_IN_MILLISECONDS - 1 < cutoffTime) { System.out.println("Deleting outdated WAL directory: " + dirPath); fs.delete(dirPath, true); - fs.delete(new Path(bulkloadDir, dirName), true); } } catch (ParseException e) { System.out.println("WARNING: Failed to parse directory name '" + dirName diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java index f76dbda60abd..f431506fc0e4 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -188,14 +188,6 @@ private void handleContinuousBackup(Admin admin) throws IOException { // set overall backup status: complete. Here we make sure to complete the backup. // After this checkpoint, even if entering cancel process, will let the backup finished backupInfo.setState(BackupState.COMPLETE); - - if (!conf.getBoolean("hbase.replication.bulkload.enabled", false)) { - System.out.println("NOTE: Bulkload replication is not enabled. " - + "Bulk loaded files will not be backed up as part of continuous backup. " - + "To ensure bulk loaded files are included in the backup, please enable bulkload replication " - + "(hbase.replication.bulkload.enabled=true) and configure other necessary settings " - + "to properly enable bulkload replication."); - } } private void handleNonContinuousBackup(Admin admin) throws IOException { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java index 225d32172766..9d1d818c207e 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java @@ -26,20 +26,18 @@ import org.slf4j.LoggerFactory; /** - * Initializes and organizes backup directories for continuous Write-Ahead Logs (WALs) and - * bulk-loaded files within the specified backup root directory. + * Initializes and organizes backup directories for continuous Write-Ahead Logs (WALs) files within + * the specified backup root directory. */ @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 final Path walsDir; - private final Path bulkLoadFilesDir; public BackupFileSystemManager(String peerId, Configuration conf, String backupRootDirStr) throws IOException { @@ -47,7 +45,6 @@ public BackupFileSystemManager(String peerId, Configuration conf, String backupR this.backupRootDir = new Path(backupRootDirStr); this.backupFs = FileSystem.get(backupRootDir.toUri(), conf); this.walsDir = createDirectory(WALS_DIR); - this.bulkLoadFilesDir = createDirectory(BULKLOAD_FILES_DIR); } private Path createDirectory(String dirName) throws IOException { @@ -61,10 +58,6 @@ public Path getWalsDir() { return walsDir; } - public Path getBulkLoadFilesDir() { - return bulkLoadFilesDir; - } - public FileSystem getBackupFs() { return backupFs; } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java deleted file mode 100644 index 6e1271313bcd..000000000000 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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. - *

- * 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. - *

- * The class is designed for scenarios where replicable bulk load operations need to be parsed and - * their file paths need to be determined programmatically. - *

- */ -@InterfaceAudience.Private -public final class BulkLoadProcessor { - private BulkLoadProcessor() { - } - - public static List processBulkLoadFiles(List walEntries) throws IOException { - List bulkLoadFilePaths = new ArrayList<>(); - - for (WAL.Entry entry : walEntries) { - WALEdit edit = entry.getEdit(); - for (Cell cell : edit.getCells()) { - if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) { - TableName tableName = entry.getKey().getTableName(); - String namespace = tableName.getNamespaceAsString(); - String table = tableName.getQualifierAsString(); - bulkLoadFilePaths.addAll(processBulkLoadDescriptor(cell, namespace, table)); - } - } - } - return bulkLoadFilePaths; - } - - private static List processBulkLoadDescriptor(Cell cell, String namespace, String table) - throws IOException { - List 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 processStoreDescriptor(WALProtos.StoreDescriptor storeDescriptor, - String namespace, String table, String regionName) { - List 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; - } -} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java index eeacc8fbf34c..bf3fbd531bfe 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -33,10 +33,8 @@ import java.util.stream.Collectors; 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.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -45,7 +43,6 @@ import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface; -import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WAL; @@ -56,8 +53,8 @@ /** * ContinuousBackupReplicationEndpoint is responsible for replicating WAL entries to a backup * storage. It organizes WAL entries by day and periodically flushes the data, ensuring that WAL - * files do not exceed the configured size. The class includes mechanisms for handling the WAL - * files, performing bulk load backups, and ensuring that the replication process is safe. + * files do not exceed the configured size. The class includes mechanisms for handling the WAL files + * and ensuring that the replication process is safe. */ @InterfaceAudience.Private public class ContinuousBackupReplicationEndpoint extends BaseReplicationEndpoint { @@ -292,20 +289,11 @@ private void backupWalEntries(long day, List walEntries) throws IOExc try { FSHLogProvider.Writer walWriter = walWriters.computeIfAbsent(day, this::createWalWriter); - List bulkLoadFiles = BulkLoadProcessor.processBulkLoadFiles(walEntries); - - if (LOG.isTraceEnabled()) { - LOG.trace("{} Processed {} bulk load files for WAL entries", Utils.logPeerId(peerId), - bulkLoadFiles.size()); - LOG.trace("{} Bulk load files: {}", Utils.logPeerId(peerId), - bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); - } for (WAL.Entry entry : walEntries) { walWriter.append(entry); } walWriter.sync(true); - uploadBulkLoadFiles(day, bulkLoadFiles); } catch (UncheckedIOException e) { String errorMsg = Utils.logPeerId(peerId) + " Failed to get or create WAL Writer for " + day; LOG.error("{} Backup failed for day {}. Error: {}", Utils.logPeerId(peerId), day, @@ -375,41 +363,6 @@ private void close() { } } - private void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) throws IOException { - LOG.debug("{} Starting upload of {} bulk load files", Utils.logPeerId(peerId), - bulkLoadFiles.size()); - - if (LOG.isTraceEnabled()) { - LOG.trace("{} Bulk load files to upload: {}", Utils.logPeerId(peerId), - bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); - } - String dayDirectoryName = formatToDateString(dayInMillis); - Path bulkloadDir = new Path(backupFileSystemManager.getBulkLoadFilesDir(), dayDirectoryName); - backupFileSystemManager.getBackupFs().mkdirs(bulkloadDir); - - for (Path file : bulkLoadFiles) { - Path sourcePath = getBulkLoadFileStagingPath(file); - Path destPath = new Path(bulkloadDir, file); - - try { - LOG.debug("{} Copying bulk load file from {} to {}", Utils.logPeerId(peerId), sourcePath, - destPath); - - FileUtil.copy(CommonFSUtils.getRootDirFileSystem(conf), 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.getMessage(), e); - throw e; - } - } - - LOG.debug("{} Completed upload of bulk load files", Utils.logPeerId(peerId)); - } - /** * Convert dayInMillis to "yyyy-MM-dd" format */ @@ -419,48 +372,6 @@ private String formatToDateString(long dayInMillis) { return dateFormat.format(new Date(dayInMillis)); } - private Path getBulkLoadFileStagingPath(Path relativePathFromNamespace) throws IOException { - FileSystem rootFs = CommonFSUtils.getRootDirFileSystem(conf); - Path rootDir = CommonFSUtils.getRootDir(conf); - Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR); - Path baseNamespaceDir = new Path(rootDir, baseNSDir); - Path hFileArchiveDir = - new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir)); - - LOG.debug("{} Searching for bulk load file: {} in paths: {}, {}", Utils.logPeerId(peerId), - relativePathFromNamespace, baseNamespaceDir, hFileArchiveDir); - - Path result = - findExistingPath(rootFs, baseNamespaceDir, hFileArchiveDir, relativePathFromNamespace); - - if (result == null) { - LOG.error("{} No bulk loaded file found in relative path: {}", Utils.logPeerId(peerId), - relativePathFromNamespace); - throw new IOException( - "No Bulk loaded file found in relative path: " + relativePathFromNamespace); - } - - LOG.debug("{} Bulk load file found at {}", Utils.logPeerId(peerId), result); - return result; - } - - private static Path findExistingPath(FileSystem rootFs, Path baseNamespaceDir, - Path hFileArchiveDir, Path filePath) throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace("Checking for bulk load file at: {} and {}", new Path(baseNamespaceDir, filePath), - new Path(hFileArchiveDir, filePath)); - } - - for (Path candidate : new Path[] { new Path(baseNamespaceDir, filePath), - new Path(hFileArchiveDir, filePath) }) { - if (rootFs.exists(candidate)) { - LOG.debug("Found bulk load file at: {}", candidate); - return candidate; - } - } - return null; - } - private void shutdownFlushExecutor() { if (flushExecutor != null) { LOG.info("{} Initiating WAL flush executor shutdown.", Utils.logPeerId(peerId)); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java index 07c9110072b2..d22f4c9cda98 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java @@ -19,7 +19,6 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; @@ -165,7 +164,7 @@ public void testSingleBackupForceDelete() throws Exception { // Step 6: Verify that the backup WAL directory is empty assertTrue("WAL backup directory should be empty after force delete", - areWalAndBulkloadDirsEmpty(conf1, backupWalDir.toString())); + isWalDirsEmpty(conf1, backupWalDir.toString())); // Step 7: Take new full backup with continuous backup enabled String backupIdContinuous = fullTableBackupWithContinuous(Lists.newArrayList(table1)); @@ -190,35 +189,28 @@ private void setupBackupFolders(long currentTime) throws IOException { public static void setupBackupFolders(FileSystem fs, Path backupWalDir, long currentTime) throws IOException { Path walsDir = new Path(backupWalDir, WALS_DIR); - Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); fs.mkdirs(walsDir); - fs.mkdirs(bulkLoadDir); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); for (int i = 0; i < 5; i++) { String dateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); fs.mkdirs(new Path(walsDir, dateStr)); - fs.mkdirs(new Path(bulkLoadDir, dateStr)); } } private static void verifyBackupCleanup(FileSystem fs, Path backupWalDir, long currentTime) throws IOException { Path walsDir = new Path(backupWalDir, WALS_DIR); - Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); // Expect folders older than 3 days to be deleted for (int i = 3; i < 5; i++) { String oldDateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); Path walPath = new Path(walsDir, oldDateStr); - Path bulkLoadPath = new Path(bulkLoadDir, oldDateStr); assertFalse("Old WAL directory (" + walPath + ") should be deleted, but it exists!", fs.exists(walPath)); - assertFalse("Old BulkLoad directory (" + bulkLoadPath + ") should be deleted, but it exists!", - fs.exists(bulkLoadPath)); } // Expect folders within the last 3 days to exist @@ -226,13 +218,9 @@ private static void verifyBackupCleanup(FileSystem fs, Path backupWalDir, long c String recentDateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); Path walPath = new Path(walsDir, recentDateStr); - Path bulkLoadPath = new Path(bulkLoadDir, recentDateStr); assertTrue("Recent WAL directory (" + walPath + ") should exist, but it is missing!", fs.exists(walPath)); - assertTrue( - "Recent BulkLoad directory (" + bulkLoadPath + ") should exist, but it is missing!", - fs.exists(bulkLoadPath)); } } @@ -276,16 +264,15 @@ private boolean continuousBackupReplicationPeerExistsAndEnabled() throws IOExcep peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER) && peer.isEnabled()); } - private static boolean areWalAndBulkloadDirsEmpty(Configuration conf, String backupWalDir) + private static boolean isWalDirsEmpty(Configuration conf, String backupWalDir) throws IOException { BackupFileSystemManager manager = new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); FileSystem fs = manager.getBackupFs(); Path walDir = manager.getWalsDir(); - Path bulkloadDir = manager.getBulkLoadFilesDir(); - return isDirectoryEmpty(fs, walDir) && isDirectoryEmpty(fs, bulkloadDir); + return isDirectoryEmpty(fs, walDir); } private static boolean isDirectoryEmpty(FileSystem fs, Path dirPath) throws IOException { diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java index b2ebbd640bbd..e00ebd6099f5 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java @@ -20,7 +20,6 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.logDirectoryStructure; import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.setupBackupFolders; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; @@ -135,7 +134,7 @@ public void testDeleteOldWALFilesOfCleanupCommand() throws IOException { fs.mkdirs(backupWalDir); long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); - setupBackupFolders(fs, backupWalDir, currentTime); // Create 5 days of WAL/bulk folders + setupBackupFolders(fs, backupWalDir, currentTime); // Create 5 days of WALs folders logDirectoryStructure(fs, backupWalDir, "Before cleanup:"); @@ -155,7 +154,6 @@ public void testDeleteOldWALFilesOfCleanupCommand() throws IOException { private static void verifyCleanupOutcome(FileSystem fs, Path backupWalDir, long currentTime, long cutoffTime) throws IOException { Path walsDir = new Path(backupWalDir, WALS_DIR); - Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); @@ -163,14 +161,11 @@ private static void verifyCleanupOutcome(FileSystem fs, Path backupWalDir, long long dayTime = currentTime - (i * ONE_DAY_IN_MILLISECONDS); String dayDir = dateFormat.format(new Date(dayTime)); Path walPath = new Path(walsDir, dayDir); - Path bulkPath = new Path(bulkLoadDir, dayDir); if (dayTime + ONE_DAY_IN_MILLISECONDS - 1 < cutoffTime) { assertFalse("Old WAL dir should be deleted: " + walPath, fs.exists(walPath)); - assertFalse("Old BulkLoad dir should be deleted: " + bulkPath, fs.exists(bulkPath)); } else { assertTrue("Recent WAL dir should exist: " + walPath, fs.exists(walPath)); - assertTrue("Recent BulkLoad dir should exist: " + bulkPath, fs.exists(bulkPath)); } } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java index 253675f85d97..3919746d3b7b 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java @@ -17,9 +17,7 @@ */ package org.apache.hadoop.hbase.backup.replication; -import static org.apache.hadoop.hbase.HConstants.REPLICATION_BULKLOAD_ENABLE_KEY; import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_MAX_WAL_SIZE; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_ROOT_DIR; @@ -66,11 +64,8 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.hadoop.hbase.tool.BulkLoadHFiles; -import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; -import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; @@ -96,14 +91,12 @@ public class TestContinuousBackupReplicationEndpoint { private final String replicationEndpoint = ContinuousBackupReplicationEndpoint.class.getName(); private static final String CF_NAME = "cf"; - private static final byte[] QUALIFIER = Bytes.toBytes("my-qualifier"); static FileSystem fs = null; static Path root; @BeforeClass public static void setUpBeforeClass() throws Exception { // Set the configuration properties as required - conf.setBoolean(REPLICATION_BULKLOAD_ENABLE_KEY, true); conf.set(REPLICATION_CLUSTER_ID, "clusterId1"); TEST_UTIL.startMiniZKCluster(); @@ -122,7 +115,7 @@ public static void tearDownAfterClass() throws Exception { } @Test - public void testWALAndBulkLoadFileBackup() throws IOException { + public void testWALBackup() throws IOException { String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); TableName tableName = TableName.valueOf("table_" + methodName); String peerId = "peerId"; @@ -140,15 +133,10 @@ public void testWALAndBulkLoadFileBackup() throws IOException { loadRandomData(tableName, 100); assertEquals(100, getRowCount(tableName)); - Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoadByFamily"); - generateHFiles(dir); - bulkLoadHFiles(tableName, dir); - assertEquals(1100, getRowCount(tableName)); - waitForReplication(15000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), true, Map.of(tableName, 1100)); + verifyBackup(backupRootDir.toString(), Map.of(tableName, 100)); deleteTable(tableName); } @@ -196,7 +184,7 @@ public void testMultiTableWALBackup() throws IOException { waitForReplication(15000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), false, Map.of(table1, 100, table2, 100, table3, 50)); + verifyBackup(backupRootDir.toString(), Map.of(table1, 100, table2, 100, table3, 50)); for (TableName table : List.of(table1, table2, table3)) { deleteTable(table); @@ -254,7 +242,7 @@ public void testWALBackupWithPeerRestart() throws IOException, InterruptedExcept waitForReplication(20000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), false, Map.of(tableName, getRowCount(tableName))); + verifyBackup(backupRootDir.toString(), Map.of(tableName, getRowCount(tableName))); deleteTable(tableName); } @@ -301,7 +289,7 @@ public void testDayWiseWALBackup() throws IOException { waitForReplication(15000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), false, Map.of(tableName, 200)); + verifyBackup(backupRootDir.toString(), Map.of(tableName, 200)); // Verify that WALs are stored in two directories, one for each day Path walDir = new Path(backupRootDir, WALS_DIR); @@ -370,42 +358,6 @@ private void loadRandomData(TableName tableName, int totalRows) throws IOExcepti } } - private void bulkLoadHFiles(TableName tableName, Path inputDir) throws IOException { - TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true); - - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { - BulkLoadHFiles loader = new BulkLoadHFilesTool(TEST_UTIL.getConfiguration()); - loader.bulkLoad(table.getName(), inputDir); - } finally { - TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false); - } - } - - private void bulkLoadHFiles(TableName tableName, Map> family2Files) - throws IOException { - TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true); - - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { - BulkLoadHFiles loader = new BulkLoadHFilesTool(TEST_UTIL.getConfiguration()); - loader.bulkLoad(table.getName(), family2Files); - } finally { - TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false); - } - } - - private void generateHFiles(Path outputDir) throws IOException { - String hFileName = "MyHFile"; - int numRows = 1000; - outputDir = outputDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()); - - byte[] from = Bytes.toBytes(CF_NAME + "begin"); - byte[] to = Bytes.toBytes(CF_NAME + "end"); - - Path familyDir = new Path(outputDir, CF_NAME); - HFileTestUtil.createHFile(TEST_UTIL.getConfiguration(), fs, new Path(familyDir, hFileName), - Bytes.toBytes(CF_NAME), QUALIFIER, from, to, numRows); - } - private void waitForReplication(int durationInMillis) { LOG.info("Waiting for replication to complete for {} ms", durationInMillis); try { @@ -418,17 +370,12 @@ private void waitForReplication(int durationInMillis) { /** * Verifies the backup process by: 1. Checking whether any WAL (Write-Ahead Log) files were - * generated in the backup directory. 2. Checking whether any bulk-loaded files were generated in - * the backup directory. 3. Replaying the WAL and bulk-loaded files (if present) to restore data - * and check consistency by verifying that the restored data matches the expected row count for - * each table. + * generated in the backup directory. 2. Replaying the WAL files to restore data and check + * consistency by verifying that the restored data matches the expected row count for each table. */ - private void verifyBackup(String backupRootDir, boolean hasBulkLoadFiles, - Map tablesWithExpectedRows) throws IOException { + private void verifyBackup(String backupRootDir, Map tablesWithExpectedRows) + throws IOException { verifyWALBackup(backupRootDir); - if (hasBulkLoadFiles) { - verifyBulkLoadBackup(backupRootDir); - } for (Map.Entry entry : tablesWithExpectedRows.entrySet()) { TableName tableName = entry.getKey(); @@ -440,21 +387,6 @@ private void verifyBackup(String backupRootDir, boolean hasBulkLoadFiles, replayWALs(new Path(backupRootDir, WALS_DIR).toString(), tableName); - // replay Bulk loaded HFiles if Present - try { - Path bulkloadDir = new Path(backupRootDir, BULKLOAD_FILES_DIR); - if (fs.exists(bulkloadDir)) { - FileStatus[] directories = fs.listStatus(bulkloadDir); - for (FileStatus dirStatus : directories) { - if (dirStatus.isDirectory()) { - replayBulkLoadHFilesIfPresent(dirStatus.getPath().toString(), tableName); - } - } - } - } catch (Exception e) { - fail("Failed to replay BulkLoad HFiles properly: " + e.getMessage()); - } - assertEquals(expectedRows, getRowCount(tableName)); } } @@ -480,15 +412,6 @@ private void verifyWALBackup(String backupRootDir) throws IOException { assertFalse("Expected some WAL files but found none!", walFiles.isEmpty()); } - private void verifyBulkLoadBackup(String backupRootDir) throws IOException { - Path bulkLoadFilesDir = new Path(backupRootDir, BULKLOAD_FILES_DIR); - assertTrue("BulkLoad Files directory does not exist!", fs.exists(bulkLoadFilesDir)); - - FileStatus[] bulkLoadFiles = fs.listStatus(bulkLoadFilesDir); - assertNotNull("No Bulk load files found!", bulkLoadFiles); - assertTrue("Expected some Bulk load files but found none!", bulkLoadFiles.length > 0); - } - private void replayWALs(String walDir, TableName tableName) { WALPlayer player = new WALPlayer(); try { @@ -499,28 +422,6 @@ private void replayWALs(String walDir, TableName tableName) { } } - private void replayBulkLoadHFilesIfPresent(String bulkLoadDir, TableName tableName) { - try { - Path tableBulkLoadDir = new Path(bulkLoadDir + "/default/" + tableName); - if (fs.exists(tableBulkLoadDir)) { - RemoteIterator fileStatusIterator = fs.listFiles(tableBulkLoadDir, true); - List bulkLoadFiles = new ArrayList<>(); - - while (fileStatusIterator.hasNext()) { - LocatedFileStatus fileStatus = fileStatusIterator.next(); - Path filePath = fileStatus.getPath(); - - if (!fileStatus.isDirectory()) { - bulkLoadFiles.add(filePath); - } - } - bulkLoadHFiles(tableName, Map.of(Bytes.toBytes(CF_NAME), bulkLoadFiles)); - } - } catch (Exception e) { - fail("Failed to replay BulkLoad HFiles properly: " + e.getMessage()); - } - } - private int getRowCount(TableName tableName) throws IOException { try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { return HBaseTestingUtil.countRows(table); From 27ea7b34f30054b9a1c06f632040202a8c8b2ff2 Mon Sep 17 00:00:00 2001 From: asolomon Date: Wed, 16 Jul 2025 00:18:40 +0530 Subject: [PATCH 11/24] HBASE-29449 Update backup describe command for continuous backup (#7045) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kevin Geiszler --- .../hadoop/hbase/backup/BackupInfo.java | 10 +++ .../hbase/backup/TestBackupDescribe.java | 87 +++++++++++++++++++ .../src/main/protobuf/Backup.proto | 2 + 3 files changed, 99 insertions(+) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java index 0997aec19ecf..47731b935efe 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java @@ -440,6 +440,8 @@ public BackupProtos.BackupInfo toProtosBackupInfo() { builder.setBackupType(BackupProtos.BackupType.valueOf(getType().name())); builder.setWorkersNumber(workers); builder.setBandwidth(bandwidth); + builder.setContinuousBackupEnabled(isContinuousBackupEnabled()); + builder.setIncrCommittedWalTs(getIncrCommittedWalTs()); return builder.build(); } @@ -535,6 +537,8 @@ public static BackupInfo fromProto(BackupProtos.BackupInfo proto) { context.setType(BackupType.valueOf(proto.getBackupType().name())); context.setWorkers(proto.getWorkersNumber()); context.setBandwidth(proto.getBandwidth()); + context.setContinuousBackupEnabled(proto.getContinuousBackupEnabled()); + context.setIncrCommittedWalTs(proto.getIncrCommittedWalTs()); return context; } @@ -578,6 +582,12 @@ public String getShortDescription() { cal.setTimeInMillis(getCompleteTs()); date = cal.getTime(); sb.append("End time=" + date).append(","); + if (getType() == BackupType.INCREMENTAL) { + cal = Calendar.getInstance(); + cal.setTimeInMillis(getIncrCommittedWalTs()); + date = cal.getTime(); + sb.append("Committed WAL time for incremental backup=" + date).append(","); + } } sb.append("Progress=" + getProgress() + "%"); sb.append("}"); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java index 6084dc730ee8..54be17f94dae 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java @@ -17,18 +17,27 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; +import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_DEFAULT; +import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import java.io.ByteArrayOutputStream; import java.io.PrintStream; import java.util.List; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupCommands; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.ToolRunner; import org.junit.ClassRule; import org.junit.Test; @@ -101,6 +110,7 @@ public void testBackupDescribeCommand() throws Exception { String response = baos.toString(); assertTrue(response.indexOf(backupId) > 0); assertTrue(response.indexOf("COMPLETE") > 0); + assertTrue(response.contains("IsContinuous=false")); BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection()); BackupInfo status = table.readBackupInfo(backupId); @@ -108,4 +118,81 @@ public void testBackupDescribeCommand() throws Exception { table.close(); assertTrue(response.indexOf(desc) >= 0); } + + @Test + public void testBackupDescribeCommandForContinuousBackup() throws Exception { + LOG.info("test backup describe on a single table with data: command-line"); + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, "testBackupDescribeCommand"); + FileSystem fs = FileSystem.get(conf1); + fs.mkdirs(backupWalDir); + conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); + + try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { + // Continuous backup + String[] backupArgs = new String[] { "create", BackupType.FULL.name(), BACKUP_ROOT_DIR, "-t", + table1.getNameAsString(), "-" + OPTION_ENABLE_CONTINUOUS_BACKUP }; + int ret = ToolRunner.run(conf1, new BackupDriver(), backupArgs); + assertEquals("Backup should succeed", 0, ret); + List backups = table.getBackupHistory(); + String backupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(backupId)); + LOG.info("backup complete"); + + BackupInfo info = getBackupAdmin().getBackupInfo(backupId); + assertTrue(info.getState() == BackupState.COMPLETE); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baos)); + + // Run backup describe + String[] args = new String[] { "describe", backupId }; + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + String response = baos.toString(); + assertTrue(response.contains(backupId)); + assertTrue(response.contains("COMPLETE")); + assertTrue(response.contains("IsContinuous=true")); + BackupInfo status = table.readBackupInfo(backupId); + String desc = status.getShortDescription(); + assertTrue(response.contains(desc)); + + // load table + Put p; + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + p = new Put(Bytes.toBytes("row" + i)); + p.addColumn(famName, qualName, Bytes.toBytes("val" + i)); + TEST_UTIL.getConnection().getTable(table1).put(p); + } + Thread.sleep(5000); + + // Incremental backup + backupArgs = new String[] { "create", BackupType.INCREMENTAL.name(), BACKUP_ROOT_DIR, "-t", + table1.getNameAsString() }; + ret = ToolRunner.run(conf1, new BackupDriver(), backupArgs); + assertEquals("Incremental Backup should succeed", 0, ret); + backups = table.getBackupHistory(); + String incrBackupId = backups.get(0).getBackupId(); + assertTrue(checkSucceeded(incrBackupId)); + LOG.info("Incremental backup complete"); + + // Run backup describe + args = new String[] { "describe", incrBackupId }; + ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertTrue(ret == 0); + response = baos.toString(); + assertTrue(response.contains(incrBackupId)); + assertTrue(response.contains("COMPLETE")); + assertTrue(response.contains("Committed WAL time for incremental backup=")); + status = table.readBackupInfo(incrBackupId); + desc = status.getShortDescription(); + assertTrue(response.contains(desc)); + } finally { + if (fs.exists(backupWalDir)) { + fs.delete(backupWalDir, true); + } + conf1.unset(CONF_CONTINUOUS_BACKUP_WAL_DIR); + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); + } + } } diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto index 95a298673251..0ad1f5ba6191 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Backup.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto @@ -93,6 +93,8 @@ message BackupInfo { optional uint32 workers_number = 11; optional uint64 bandwidth = 12; map table_set_timestamp = 13; + optional bool continuous_backup_enabled = 14; + optional uint64 incr_committed_wal_ts = 15; message RSTimestampMap { map rs_timestamp = 1; From 37e195a32edb48dfab1c7c63df814317c03a1812 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Wed, 16 Jul 2025 22:43:59 +0530 Subject: [PATCH 12/24] HBASE-29445 Add Option to Specify Custom Backup Location in PITR (#7153) Signed-off-by: Tak Lon (Stephen) Wu --- .../impl/AbstractPitrRestoreHandler.java | 403 ++++++++++++++++++ .../hbase/backup/impl/BackupAdminImpl.java | 348 +-------------- .../hbase/backup/impl/BackupImageAdapter.java | 60 +++ .../hbase/backup/impl/BackupInfoAdapter.java | 60 +++ ...ustomBackupLocationPitrRestoreHandler.java | 57 +++ .../impl/DefaultPitrRestoreHandler.java | 55 +++ .../hbase/backup/impl/PitrBackupMetadata.java | 50 +++ .../hadoop/hbase/backup/PITRTestUtil.java | 107 +++++ .../hbase/backup/TestPointInTimeRestore.java | 117 ++--- ...ointInTimeRestoreWithCustomBackupPath.java | 121 ++++++ 10 files changed, 962 insertions(+), 416 deletions(-) create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/CustomBackupLocationPitrRestoreHandler.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/DefaultPitrRestoreHandler.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestoreWithCustomBackupPath.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java new file mode 100644 index 000000000000..b2edce6b0fd1 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java @@ -0,0 +1,403 @@ +/* + * 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.impl; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES; + +import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest; +import org.apache.hadoop.hbase.backup.RestoreRequest; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.Tool; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Abstract base class for handling Point-In-Time Restore (PITR). + *

+ * Defines the common PITR algorithm using the Template Method Pattern. Subclasses provide the + * metadata source (e.g., backup system table or a custom backup location). + *

+ * The PITR flow includes: + *

    + *
  • Validating recovery time within the PITR window
  • + *
  • Checking for continuous backup and valid backup availability
  • + *
  • Restoring the backup
  • + *
  • Replaying WALs to bring tables to the target state
  • + *
+ *

+ * Subclasses must implement {@link #getBackupMetadata(PointInTimeRestoreRequest)} to supply the + * list of completed backups. + */ +@InterfaceAudience.Private +public abstract class AbstractPitrRestoreHandler { + private static final Logger LOG = LoggerFactory.getLogger(AbstractPitrRestoreHandler.class); + + protected final Connection conn; + protected final BackupAdminImpl backupAdmin; + protected final PointInTimeRestoreRequest request; + + AbstractPitrRestoreHandler(Connection conn, PointInTimeRestoreRequest request) { + this.conn = conn; + this.backupAdmin = new BackupAdminImpl(conn); + this.request = request; + } + + /** + * Validates the PITR request and performs the restore if valid. This is the main entry point for + * the PITR process and should be called by clients. + */ + public final void validateAndRestore() throws IOException { + long endTime = request.getToDateTime(); + validateRequestToTime(endTime); + + TableName[] sourceTableArray = request.getFromTables(); + TableName[] targetTableArray = resolveTargetTables(sourceTableArray, request.getToTables()); + + // Validate PITR requirements + validatePitr(endTime, sourceTableArray, targetTableArray); + + // If only validation is required, log and return + if (request.isCheck()) { + LOG.info("PITR can be successfully executed"); + return; + } + + // Execute PITR process + try (BackupSystemTable table = new BackupSystemTable(conn)) { + Map continuousBackupTables = table.getContinuousBackupTableSet(); + List backupMetadataList = getBackupMetadata(request); + + for (int i = 0; i < sourceTableArray.length; i++) { + restoreTableWithWalReplay(sourceTableArray[i], targetTableArray[i], endTime, + continuousBackupTables, backupMetadataList, request); + } + } + } + + /** + * Validates whether the requested end time falls within the allowed PITR recovery window. + * @param endTime The target recovery time. + * @throws IOException If the requested recovery time is outside the allowed window. + */ + private void validateRequestToTime(long endTime) throws IOException { + long pitrWindowDays = conn.getConfiguration().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, + DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS); + long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); + long pitrMaxStartTime = currentTime - TimeUnit.DAYS.toMillis(pitrWindowDays); + + if (endTime < pitrMaxStartTime) { + String errorMsg = String.format( + "Requested recovery time (%d) is out of the allowed PITR window (last %d days).", endTime, + pitrWindowDays); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + + if (endTime > currentTime) { + String errorMsg = String.format( + "Requested recovery time (%d) is in the future. Current time: %d.", endTime, currentTime); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + } + + /** + * Resolves the target table array. If null or empty, defaults to the source table array. + */ + private TableName[] resolveTargetTables(TableName[] sourceTables, TableName[] targetTables) { + return (targetTables == null || targetTables.length == 0) ? sourceTables : targetTables; + } + + /** + * Validates whether Point-In-Time Recovery (PITR) is possible for the given tables at the + * specified time. + *

+ * PITR requires: + *

    + *
  • Continuous backup to be enabled for the source tables.
  • + *
  • A valid backup image and corresponding WALs to be available.
  • + *
+ * @param endTime The target recovery time. + * @param sTableArray The source tables to restore. + * @param tTableArray The target tables where the restore will be performed. + * @throws IOException If PITR is not possible due to missing continuous backup or backup images. + */ + private void validatePitr(long endTime, TableName[] sTableArray, TableName[] tTableArray) + throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + // Retrieve the set of tables with continuous backup enabled + Map continuousBackupTables = table.getContinuousBackupTableSet(); + + // Ensure all source tables have continuous backup enabled + validateContinuousBackup(sTableArray, continuousBackupTables); + + // Fetch completed backup information + List backupMetadataList = getBackupMetadata(request); + + // Ensure a valid backup and WALs exist for PITR + validateBackupAvailability(sTableArray, tTableArray, endTime, continuousBackupTables, + backupMetadataList); + } + } + + /** + * Ensures that all source tables have continuous backup enabled. + */ + private void validateContinuousBackup(TableName[] tables, + Map continuousBackupTables) throws IOException { + List missingTables = + Arrays.stream(tables).filter(table -> !continuousBackupTables.containsKey(table)).toList(); + + if (!missingTables.isEmpty()) { + String errorMsg = "Continuous Backup is not enabled for the following tables: " + + missingTables.stream().map(TableName::getNameAsString).collect(Collectors.joining(", ")); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + } + + /** + * Ensures that a valid backup and corresponding WALs exist for PITR for each source table. PITR + * requires: 1. A valid backup available before the end time. 2. Write-Ahead Logs (WALs) covering + * the remaining duration up to the end time. + */ + private void validateBackupAvailability(TableName[] sTableArray, TableName[] tTableArray, + long endTime, Map continuousBackupTables, List backups) + throws IOException { + for (int i = 0; i < sTableArray.length; i++) { + if ( + !canPerformPitr(sTableArray[i], tTableArray[i], endTime, continuousBackupTables, backups) + ) { + String errorMsg = String.format( + "PITR failed: No valid backup/WALs found for source table %s (target: %s) before time %d", + sTableArray[i].getNameAsString(), tTableArray[i].getNameAsString(), endTime); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + } + } + + /** + * Checks whether PITR can be performed for a given source-target table pair. + */ + private boolean canPerformPitr(TableName stableName, TableName tTableName, long endTime, + Map continuousBackupTables, List backups) { + return getValidBackup(stableName, tTableName, endTime, continuousBackupTables, backups) != null; + } + + /** + * Finds and returns the first valid backup metadata entry that can be used to restore the given + * source table up to the specified end time. A backup is considered valid if: + *
    + *
  • It contains the source table
  • + *
  • It was completed before the requested end time
  • + *
  • Its start time is after the table's continuous backup start time
  • + *
  • It passes the restore request validation
  • + *
+ */ + private PitrBackupMetadata getValidBackup(TableName sTableName, TableName tTablename, + long endTime, Map continuousBackupTables, List backups) { + for (PitrBackupMetadata backup : backups) { + if (isValidBackupForPitr(backup, sTableName, endTime, continuousBackupTables)) { + + RestoreRequest restoreRequest = + BackupUtils.createRestoreRequest(backup.getRootDir(), backup.getBackupId(), true, + new TableName[] { sTableName }, new TableName[] { tTablename }, false); + + try { + if (backupAdmin.validateRequest(restoreRequest)) { + return backup; + } + } catch (IOException e) { + LOG.warn("Exception occurred while testing the backup : {} for restore ", + backup.getBackupId(), e); + } + } + } + return null; + } + + /** + * Determines if the given backup is valid for PITR. + *

+ * A backup is valid if: + *

    + *
  • It contains the source table.
  • + *
  • It was completed before the end time.
  • + *
  • The start timestamp of the backup is after the continuous backup start time for the + * table.
  • + *
+ * @param backupMetadata Backup information object. + * @param tableName Table to check. + * @param endTime The target recovery time. + * @param continuousBackupTables Map of tables with continuous backup enabled. + * @return true if the backup is valid for PITR, false otherwise. + */ + private boolean isValidBackupForPitr(PitrBackupMetadata backupMetadata, TableName tableName, + long endTime, Map continuousBackupTables) { + return backupMetadata.getTableNames().contains(tableName) + && backupMetadata.getCompleteTs() <= endTime + && continuousBackupTables.getOrDefault(tableName, 0L) <= backupMetadata.getStartTs(); + } + + /** + * Restores the table using the selected backup and replays WALs from the backup start time to the + * requested end time. + * @throws IOException if no valid backup is found or WAL replay fails + */ + private void restoreTableWithWalReplay(TableName sourceTable, TableName targetTable, long endTime, + Map continuousBackupTables, List backupMetadataList, + PointInTimeRestoreRequest request) throws IOException { + PitrBackupMetadata backupMetadata = + getValidBackup(sourceTable, targetTable, endTime, continuousBackupTables, backupMetadataList); + if (backupMetadata == null) { + String errorMsg = "Could not find a valid backup and WALs for PITR for table: " + + sourceTable.getNameAsString(); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + + RestoreRequest restoreRequest = BackupUtils.createRestoreRequest(backupMetadata.getRootDir(), + backupMetadata.getBackupId(), false, new TableName[] { sourceTable }, + new TableName[] { targetTable }, request.isOverwrite()); + + backupAdmin.restore(restoreRequest); + replayWal(sourceTable, targetTable, backupMetadata.getStartTs(), endTime); + } + + /** + * Replays WALs to bring the table to the desired state. + */ + private void replayWal(TableName sourceTable, TableName targetTable, long startTime, long endTime) + throws IOException { + String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + Path walDirPath = new Path(walBackupDir); + LOG.info( + "Starting WAL replay for source: {}, target: {}, time range: {} - {}, WAL backup dir: {}", + sourceTable, targetTable, startTime, endTime, walDirPath); + + List validDirs = + getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); + if (validDirs.isEmpty()) { + LOG.warn("No valid WAL directories found for range {} - {}. Skipping WAL replay.", startTime, + endTime); + return; + } + + executeWalReplay(validDirs, sourceTable, targetTable, startTime, endTime); + } + + /** + * Fetches valid WAL directories based on the given time range. + */ + private List getValidWalDirs(Configuration conf, Path walBackupDir, long startTime, + long endTime) throws IOException { + FileSystem backupFs = FileSystem.get(walBackupDir.toUri(), conf); + FileStatus[] dayDirs = backupFs.listStatus(new Path(walBackupDir, WALS_DIR)); + + List validDirs = new ArrayList<>(); + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + + for (FileStatus dayDir : dayDirs) { + if (!dayDir.isDirectory()) { + continue; // Skip files, only process directories + } + + String dirName = dayDir.getPath().getName(); + try { + Date dirDate = dateFormat.parse(dirName); + long dirStartTime = dirDate.getTime(); // Start of that day (00:00:00) + long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) + + // Check if this day's WAL files overlap with the required time range + if (dirEndTime >= startTime && dirStartTime <= endTime) { + validDirs.add(dayDir.getPath().toString()); + } + } catch (ParseException e) { + LOG.warn("Skipping invalid directory name: " + dirName, e); + } + } + return validDirs; + } + + /** + * Executes WAL replay using WALPlayer. + */ + private void executeWalReplay(List walDirs, TableName sourceTable, TableName targetTable, + long startTime, long endTime) throws IOException { + Tool walPlayer = initializeWalPlayer(startTime, endTime); + String[] args = + { String.join(",", walDirs), sourceTable.getNameAsString(), targetTable.getNameAsString() }; + + try { + LOG.info("Executing WALPlayer with args: {}", Arrays.toString(args)); + int exitCode = walPlayer.run(args); + if (exitCode == 0) { + LOG.info("WAL replay completed successfully for {}", targetTable); + } else { + throw new IOException("WAL replay failed with exit code: " + exitCode); + } + } catch (Exception e) { + LOG.error("Error during WAL replay for {}: {}", targetTable, e.getMessage(), e); + throw new IOException("Exception during WAL replay", e); + } + } + + /** + * Initializes and configures WALPlayer. + */ + private Tool initializeWalPlayer(long startTime, long endTime) { + Configuration conf = HBaseConfiguration.create(conn.getConfiguration()); + conf.setLong(WALInputFormat.START_TIME_KEY, startTime); + conf.setLong(WALInputFormat.END_TIME_KEY, endTime); + conf.setBoolean(IGNORE_EMPTY_FILES, true); + Tool walPlayer = new WALPlayer(); + walPlayer.setConf(conf); + return walPlayer; + } + + protected abstract List getBackupMetadata(PointInTimeRestoreRequest request) + throws IOException; +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java index e82d9804f9dc..75a2a6343a58 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -17,33 +17,18 @@ */ package org.apache.hadoop.hbase.backup.impl; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; -import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES; - import java.io.IOException; -import java.text.ParseException; -import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Date; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupAdmin; import org.apache.hadoop.hbase.backup.BackupClientFactory; @@ -61,10 +46,7 @@ import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.mapreduce.WALInputFormat; -import org.apache.hadoop.hbase.mapreduce.WALPlayer; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.util.Tool; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -521,7 +503,7 @@ public void restore(RestoreRequest request) throws IOException { new RestoreTablesClient(conn, request).execute(); } - private boolean validateRequest(RestoreRequest request) throws IOException { + public boolean validateRequest(RestoreRequest request) throws IOException { // check and load backup image manifest for the tables Path rootPath = new Path(request.getBackupRootDir()); String backupId = request.getBackupId(); @@ -533,324 +515,28 @@ private boolean validateRequest(RestoreRequest request) throws IOException { return BackupUtils.validate(Arrays.asList(sTableArray), manifest, conn.getConfiguration()); } - @Override - public void pointInTimeRestore(PointInTimeRestoreRequest request) throws IOException { - if (request.getBackupRootDir() == null) { - defaultPointInTimeRestore(request); - } else { - // TODO: special case, not supported at the moment - throw new IOException("Custom backup location for Point-In-Time Recovery Not supported!"); - } - LOG.info("Successfully completed Point In Time Restore for all tables."); - } - - /** - * Performs a default Point-In-Time Restore (PITR) by restoring the latest valid backup and - * replaying the WALs to bring the table to the desired state. PITR requires: 1. A valid backup - * available before the end time. 2. Write-Ahead Logs (WALs) covering the remaining duration up to - * the end time. - * @param request PointInTimeRestoreRequest containing restore parameters. - * @throws IOException If no valid backup or WALs are found, or if an error occurs during - * restoration. - */ - private void defaultPointInTimeRestore(PointInTimeRestoreRequest request) throws IOException { - long endTime = request.getToDateTime(); - validateRequestToTime(endTime); - - TableName[] sTableArray = request.getFromTables(); - TableName[] tTableArray = resolveTargetTables(sTableArray, request.getToTables()); - - // Validate PITR requirements - validatePitr(endTime, sTableArray, tTableArray); - - // If only validation is required, log and return - if (request.isCheck()) { - LOG.info("PITR can be successfully executed"); - return; - } - - // Execute PITR process - try (BackupSystemTable table = new BackupSystemTable(conn)) { - Map continuousBackupTables = table.getContinuousBackupTableSet(); - List backupInfos = table.getBackupInfos(BackupState.COMPLETE); - - for (int i = 0; i < sTableArray.length; i++) { - restoreTableWithWalReplay(sTableArray[i], tTableArray[i], endTime, continuousBackupTables, - backupInfos, request); - } - } - } - - /** - * Validates whether the requested end time falls within the allowed PITR recovery window. - * @param endTime The target recovery time. - * @throws IOException If the requested recovery time is outside the allowed window. - */ - private void validateRequestToTime(long endTime) throws IOException { - long pitrWindowDays = conn.getConfiguration().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS, - DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS); - long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); - long pitrMaxStartTime = currentTime - TimeUnit.DAYS.toMillis(pitrWindowDays); - - if (endTime < pitrMaxStartTime) { - String errorMsg = String.format( - "Requested recovery time (%d) is out of the allowed PITR window (last %d days).", endTime, - pitrWindowDays); - LOG.error(errorMsg); - throw new IOException(errorMsg); - } - - if (endTime > currentTime) { - String errorMsg = String.format( - "Requested recovery time (%d) is in the future. Current time: %d.", endTime, currentTime); - LOG.error(errorMsg); - throw new IOException(errorMsg); - } - } - - /** - * Resolves the target table array. If null or empty, defaults to the source table array. - */ - private TableName[] resolveTargetTables(TableName[] sourceTables, TableName[] targetTables) { - return (targetTables == null || targetTables.length == 0) ? sourceTables : targetTables; - } - /** - * Validates whether Point-In-Time Recovery (PITR) is possible for the given tables at the - * specified time. + * Initiates Point-In-Time Restore (PITR) for the given request. *

- * PITR requires: - *

    - *
  • Continuous backup to be enabled for the source tables.
  • - *
  • A valid backup image and corresponding WALs to be available.
  • - *
- * @param endTime The target recovery time. - * @param sTableArray The source tables to restore. - * @param tTableArray The target tables where the restore will be performed. - * @throws IOException If PITR is not possible due to missing continuous backup or backup images. - */ - private void validatePitr(long endTime, TableName[] sTableArray, TableName[] tTableArray) - throws IOException { - try (BackupSystemTable table = new BackupSystemTable(conn)) { - // Retrieve the set of tables with continuous backup enabled - Map continuousBackupTables = table.getContinuousBackupTableSet(); - - // Ensure all source tables have continuous backup enabled - validateContinuousBackup(sTableArray, continuousBackupTables); - - // Fetch completed backup information - List backupInfos = table.getBackupInfos(BackupState.COMPLETE); - - // Ensure a valid backup and WALs exist for PITR - validateBackupAvailability(sTableArray, tTableArray, endTime, continuousBackupTables, - backupInfos); - } - } - - /** - * Ensures that all source tables have continuous backup enabled. - */ - private void validateContinuousBackup(TableName[] tables, - Map continuousBackupTables) throws IOException { - List missingTables = - Arrays.stream(tables).filter(table -> !continuousBackupTables.containsKey(table)).toList(); - - if (!missingTables.isEmpty()) { - String errorMsg = "Continuous Backup is not enabled for the following tables: " - + missingTables.stream().map(TableName::getNameAsString).collect(Collectors.joining(", ")); - LOG.error(errorMsg); - throw new IOException(errorMsg); - } - } - - /** - * Ensures that a valid backup and corresponding WALs exist for PITR for each source table. PITR - * requires: 1. A valid backup available before the end time. 2. Write-Ahead Logs (WALs) covering - * the remaining duration up to the end time. + * If {@code backupRootDir} is specified in the request, performs PITR using metadata from the + * provided custom backup location. Otherwise, defaults to using metadata from the backup system + * table. + * @param request PointInTimeRestoreRequest containing PITR parameters. + * @throws IOException if validation fails or restore cannot be completed. */ - private void validateBackupAvailability(TableName[] sTableArray, TableName[] tTableArray, - long endTime, Map continuousBackupTables, List backupInfos) - throws IOException { - for (int i = 0; i < sTableArray.length; i++) { - if ( - !canPerformPitr(sTableArray[i], tTableArray[i], endTime, continuousBackupTables, - backupInfos) - ) { - String errorMsg = "Could not find a valid backup and WALs for PITR for table: " - + sTableArray[i].getNameAsString(); - LOG.error(errorMsg); - throw new IOException(errorMsg); - } - } - } - - /** - * Checks whether PITR can be performed for a given source-target table pair. - */ - private boolean canPerformPitr(TableName stableName, TableName tTableName, long endTime, - Map continuousBackupTables, List backupInfos) { - return getValidBackupInfo(stableName, tTableName, endTime, continuousBackupTables, backupInfos) - != null; - } - - /** - * Finds a valid backup for PITR that meets the required conditions. - */ - private BackupInfo getValidBackupInfo(TableName sTableName, TableName tTablename, long endTime, - Map continuousBackupTables, List backupInfos) { - for (BackupInfo info : backupInfos) { - if (isValidBackupForPitr(info, sTableName, endTime, continuousBackupTables)) { - - RestoreRequest restoreRequest = - BackupUtils.createRestoreRequest(info.getBackupRootDir(), info.getBackupId(), true, - new TableName[] { sTableName }, new TableName[] { tTablename }, false); - - try { - if (validateRequest(restoreRequest)) { - return info; - } - } catch (IOException e) { - LOG.warn("Exception occurred while testing the backup : {} for restore ", info, e); - } - } - } - return null; - } - - /** - * Determines if the given backup is valid for PITR. - *

- * A backup is valid if: - *

    - *
  • It contains the source table.
  • - *
  • It was completed before the end time.
  • - *
  • The start timestamp of the backup is after the continuous backup start time for the - * table.
  • - *
- * @param info Backup information object. - * @param tableName Table to check. - * @param endTime The target recovery time. - * @param continuousBackupTables Map of tables with continuous backup enabled. - * @return true if the backup is valid for PITR, false otherwise. - */ - private boolean isValidBackupForPitr(BackupInfo info, TableName tableName, long endTime, - Map continuousBackupTables) { - return info.getTableNames().contains(tableName) && info.getCompleteTs() <= endTime - && continuousBackupTables.getOrDefault(tableName, 0L) <= info.getStartTs(); - } - - /** - * Restores a table from a valid backup and replays WALs to reach the desired PITR state. - */ - private void restoreTableWithWalReplay(TableName sourceTable, TableName targetTable, long endTime, - Map continuousBackupTables, List backupInfos, - PointInTimeRestoreRequest request) throws IOException { - BackupInfo backupInfo = - getValidBackupInfo(sourceTable, targetTable, endTime, continuousBackupTables, backupInfos); - if (backupInfo == null) { - String errorMsg = "Could not find a valid backup and WALs for PITR for table: " - + sourceTable.getNameAsString(); - LOG.error(errorMsg); - throw new IOException(errorMsg); - } - - RestoreRequest restoreRequest = BackupUtils.createRestoreRequest(backupInfo.getBackupRootDir(), - backupInfo.getBackupId(), false, new TableName[] { sourceTable }, - new TableName[] { targetTable }, request.isOverwrite()); - - restore(restoreRequest); - replayWal(sourceTable, targetTable, backupInfo.getStartTs(), endTime); - } - - /** - * Replays WALs to bring the table to the desired state. - */ - private void replayWal(TableName sourceTable, TableName targetTable, long startTime, long endTime) - throws IOException { - String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); - Path walDirPath = new Path(walBackupDir); - LOG.info( - "Starting WAL replay for source: {}, target: {}, time range: {} - {}, WAL backup dir: {}", - sourceTable, targetTable, startTime, endTime, walDirPath); - - List validDirs = - getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); - if (validDirs.isEmpty()) { - LOG.warn("No valid WAL directories found for range {} - {}. Skipping WAL replay.", startTime, - endTime); - return; - } - - executeWalReplay(validDirs, sourceTable, targetTable, startTime, endTime); - } - - /** - * Fetches valid WAL directories based on the given time range. - */ - private List getValidWalDirs(Configuration conf, Path walBackupDir, long startTime, - long endTime) throws IOException { - FileSystem backupFs = FileSystem.get(walBackupDir.toUri(), conf); - FileStatus[] dayDirs = backupFs.listStatus(new Path(walBackupDir, WALS_DIR)); - - List validDirs = new ArrayList<>(); - SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); - - for (FileStatus dayDir : dayDirs) { - if (!dayDir.isDirectory()) { - continue; // Skip files, only process directories - } - - String dirName = dayDir.getPath().getName(); - try { - Date dirDate = dateFormat.parse(dirName); - long dirStartTime = dirDate.getTime(); // Start of that day (00:00:00) - long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) - - // Check if this day's WAL files overlap with the required time range - if (dirEndTime >= startTime && dirStartTime <= endTime) { - validDirs.add(dayDir.getPath().toString()); - } - } catch (ParseException e) { - LOG.warn("Skipping invalid directory name: " + dirName, e); - } - } - return validDirs; - } - - /** - * Executes WAL replay using WALPlayer. - */ - private void executeWalReplay(List walDirs, TableName sourceTable, TableName targetTable, - long startTime, long endTime) throws IOException { - Tool walPlayer = initializeWalPlayer(startTime, endTime); - String[] args = - { String.join(",", walDirs), sourceTable.getNameAsString(), targetTable.getNameAsString() }; + @Override + public void pointInTimeRestore(PointInTimeRestoreRequest request) throws IOException { + AbstractPitrRestoreHandler handler; - try { - LOG.info("Executing WALPlayer with args: {}", Arrays.toString(args)); - int exitCode = walPlayer.run(args); - if (exitCode == 0) { - LOG.info("WAL replay completed successfully for {}", targetTable); - } else { - throw new IOException("WAL replay failed with exit code: " + exitCode); - } - } catch (Exception e) { - LOG.error("Error during WAL replay for {}: {}", targetTable, e.getMessage(), e); - throw new IOException("Exception during WAL replay", e); + // Choose the appropriate handler based on whether a custom backup location is provided + if (request.getBackupRootDir() == null) { + handler = new DefaultPitrRestoreHandler(conn, request); + } else { + handler = new CustomBackupLocationPitrRestoreHandler(conn, request); } - } + handler.validateAndRestore(); - /** - * Initializes and configures WALPlayer. - */ - private Tool initializeWalPlayer(long startTime, long endTime) { - Configuration conf = HBaseConfiguration.create(conn.getConfiguration()); - conf.setLong(WALInputFormat.START_TIME_KEY, startTime); - conf.setLong(WALInputFormat.END_TIME_KEY, endTime); - conf.setBoolean(IGNORE_EMPTY_FILES, true); - Tool walPlayer = new WALPlayer(); - walPlayer.setConf(conf); - return walPlayer; + LOG.info("Successfully completed Point In Time Restore for all tables."); } @Override diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java new file mode 100644 index 000000000000..8b785a0f0504 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java @@ -0,0 +1,60 @@ +/* + * 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.impl; + +import java.util.List; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Adapter that wraps a {@link BackupImage} to expose it as {@link PitrBackupMetadata}. + */ +@InterfaceAudience.Private +public class BackupImageAdapter implements PitrBackupMetadata { + private final BackupImage image; + + public BackupImageAdapter(BackupImage image) { + this.image = image; + } + + @Override + public List getTableNames() { + return image.getTableNames(); + } + + @Override + public long getStartTs() { + return image.getStartTs(); + } + + @Override + public long getCompleteTs() { + return image.getCompleteTs(); + } + + @Override + public String getBackupId() { + return image.getBackupId(); + } + + @Override + public String getRootDir() { + return image.getRootDir(); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java new file mode 100644 index 000000000000..967fae551cb5 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java @@ -0,0 +1,60 @@ +/* + * 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.impl; + +import java.util.List; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Adapter that wraps a {@link BackupInfo} to expose it as {@link PitrBackupMetadata}. + */ +@InterfaceAudience.Private +public class BackupInfoAdapter implements PitrBackupMetadata { + private final BackupInfo info; + + public BackupInfoAdapter(BackupInfo info) { + this.info = info; + } + + @Override + public List getTableNames() { + return info.getTableNames(); + } + + @Override + public long getStartTs() { + return info.getStartTs(); + } + + @Override + public long getCompleteTs() { + return info.getCompleteTs(); + } + + @Override + public String getBackupId() { + return info.getBackupId(); + } + + @Override + public String getRootDir() { + return info.getBackupRootDir(); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/CustomBackupLocationPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/CustomBackupLocationPitrRestoreHandler.java new file mode 100644 index 000000000000..1657b68d0234 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/CustomBackupLocationPitrRestoreHandler.java @@ -0,0 +1,57 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.backup.HBackupFileSystem; +import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * PITR restore handler that retrieves backup metadata from a custom backup root directory. + *

+ * This implementation is used when the PITR request specifies a custom backup location via + * {@code backupRootDir}. + */ +@InterfaceAudience.Private +public class CustomBackupLocationPitrRestoreHandler extends AbstractPitrRestoreHandler { + + public CustomBackupLocationPitrRestoreHandler(Connection conn, + PointInTimeRestoreRequest request) { + super(conn, request); + } + + /** + * Retrieves completed backup entries from the given custom backup root directory and converts + * them into {@link PitrBackupMetadata} using {@link BackupImageAdapter}. + * @param request the PITR request + * @return list of completed backup metadata entries from the custom location + * @throws IOException if reading from the custom backup directory fails + */ + @Override + protected List getBackupMetadata(PointInTimeRestoreRequest request) + throws IOException { + return HBackupFileSystem + .getAllBackupImages(conn.getConfiguration(), new Path(request.getBackupRootDir())).stream() + .map(BackupImageAdapter::new).collect(Collectors.toList()); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/DefaultPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/DefaultPitrRestoreHandler.java new file mode 100644 index 000000000000..c6844ba96bd3 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/DefaultPitrRestoreHandler.java @@ -0,0 +1,55 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Default PITR restore handler that retrieves backup metadata from the system table. + *

+ * This implementation is used when no custom backup root directory is specified in the request. + */ +@InterfaceAudience.Private +public class DefaultPitrRestoreHandler extends AbstractPitrRestoreHandler { + + public DefaultPitrRestoreHandler(Connection conn, PointInTimeRestoreRequest request) { + super(conn, request); + } + + /** + * Retrieves completed backup entries from the BackupSystemTable and converts them into + * {@link PitrBackupMetadata} using {@link BackupInfoAdapter}. + * @param request the PITR request + * @return list of completed backup metadata entries + * @throws IOException if reading from the backup system table fails + */ + @Override + protected List getBackupMetadata(PointInTimeRestoreRequest request) + throws IOException { + try (BackupSystemTable table = new BackupSystemTable(conn)) { + return table.getBackupInfos(BackupInfo.BackupState.COMPLETE).stream() + .map(BackupInfoAdapter::new).collect(Collectors.toList()); + } + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java new file mode 100644 index 000000000000..dc135ce79c08 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java @@ -0,0 +1,50 @@ +/* + * 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.impl; + +import java.util.List; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A unified abstraction over backup metadata used during Point-In-Time Restore (PITR). + *

+ * This interface allows the PITR algorithm to operate uniformly over different types of backup + * metadata sources, such as {@link BackupInfo} (system table) and {@link BackupImage} (custom + * backup location), without knowing their specific implementations. + */ +@InterfaceAudience.Private +public interface PitrBackupMetadata { + + /** Returns List of table names included in the backup */ + List getTableNames(); + + /** Returns Start timestamp of the backup */ + long getStartTs(); + + /** Returns Completion timestamp of the backup */ + long getCompleteTs(); + + /** Returns Unique identifier for the backup */ + String getBackupId(); + + /** Returns Root directory where the backup is stored */ + String getRootDir(); +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java new file mode 100644 index 000000000000..ae26cf960501 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java @@ -0,0 +1,107 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PITR_BACKUP_PATH; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TO_DATETIME; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Table; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public final class PITRTestUtil { + private static final Logger LOG = LoggerFactory.getLogger(PITRTestUtil.class); + private static final int DEFAULT_WAIT_FOR_REPLICATION_MS = 30_000; + + private PITRTestUtil() { + // Utility class + } + + public static String[] buildPITRArgs(TableName[] sourceTables, TableName[] targetTables, + long endTime, String backupRootDir) { + String sourceTableNames = + Arrays.stream(sourceTables).map(TableName::getNameAsString).collect(Collectors.joining(",")); + String targetTableNames = + Arrays.stream(targetTables).map(TableName::getNameAsString).collect(Collectors.joining(",")); + + List args = new ArrayList<>(); + args.add("-" + OPTION_TABLE); + args.add(sourceTableNames); + args.add("-" + OPTION_TABLE_MAPPING); + args.add(targetTableNames); + args.add("-" + OPTION_TO_DATETIME); + args.add(String.valueOf(endTime)); + + if (backupRootDir != null) { + args.add("-" + OPTION_PITR_BACKUP_PATH); + args.add(backupRootDir); + } + + return args.toArray(new String[0]); + } + + public static String[] buildBackupArgs(String backupType, TableName[] tables, + boolean continuousEnabled, String backupRootDir) { + String tableNames = + Arrays.stream(tables).map(TableName::getNameAsString).collect(Collectors.joining(",")); + + List args = new ArrayList<>( + Arrays.asList("create", backupType, backupRootDir, "-" + OPTION_TABLE, tableNames)); + + if (continuousEnabled) { + args.add("-" + OPTION_ENABLE_CONTINUOUS_BACKUP); + } + + return args.toArray(new String[0]); + } + + public static void loadRandomData(HBaseTestingUtil testUtil, TableName tableName, byte[] family, + int totalRows) throws IOException { + try (Table table = testUtil.getConnection().getTable(tableName)) { + testUtil.loadRandomRows(table, family, 32, totalRows); + } + } + + public static void waitForReplication() { + try { + LOG.info("Waiting for replication to complete for {} ms", DEFAULT_WAIT_FOR_REPLICATION_MS); + Thread.sleep(DEFAULT_WAIT_FOR_REPLICATION_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for replication", e); + } + } + + public static int getRowCount(HBaseTestingUtil testUtil, TableName tableName) throws IOException { + try (Table table = testUtil.getConnection().getTable(tableName)) { + return HBaseTestingUtil.countRows(table); + } + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java index fb37977c4eee..a1ce9c97a687 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java @@ -18,23 +18,15 @@ package org.apache.hadoop.hbase.backup; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TO_DATETIME; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import java.io.IOException; -import java.util.Arrays; -import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.ToolRunner; @@ -43,8 +35,6 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; @Category(LargeTests.class) public class TestPointInTimeRestore extends TestBackupBase { @@ -52,10 +42,7 @@ public class TestPointInTimeRestore extends TestBackupBase { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestPointInTimeRestore.class); - private static final Logger LOG = LoggerFactory.getLogger(TestPointInTimeRestore.class); - private static final String backupWalDirName = "TestPointInTimeRestoreWalDir"; - private static final int WAIT_FOR_REPLICATION_MS = 30_000; static Path backupWalDir; static FileSystem fs; @@ -80,38 +67,41 @@ private static void setUpBackups() throws Exception { // Simulate a backup taken 20 days ago EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - 20 * ONE_DAY_IN_MILLISECONDS); - loadRandomData(table1, 1000); // Insert initial data into table1 + PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Insert initial data into + // table1 // Perform a full backup for table1 with continuous backup enabled - String[] args = buildBackupArgs("full", new TableName[] { table1 }, true); + String[] args = + PITRTestUtil.buildBackupArgs("full", new TableName[] { table1 }, true, BACKUP_ROOT_DIR); int ret = ToolRunner.run(conf1, new BackupDriver(), args); assertEquals("Backup should succeed", 0, ret); // Move time forward to simulate 15 days ago EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - 15 * ONE_DAY_IN_MILLISECONDS); - loadRandomData(table1, 1000); // Add more data to table1 - loadRandomData(table2, 500); // Insert data into table2 + PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Add more data to table1 + PITRTestUtil.loadRandomData(TEST_UTIL, table2, famName, 500); // Insert data into table2 - waitForReplication(); // Ensure replication is complete + PITRTestUtil.waitForReplication(); // Ensure replication is complete // Perform a full backup for table2 with continuous backup enabled - args = buildBackupArgs("full", new TableName[] { table2 }, true); + args = PITRTestUtil.buildBackupArgs("full", new TableName[] { table2 }, true, BACKUP_ROOT_DIR); ret = ToolRunner.run(conf1, new BackupDriver(), args); assertEquals("Backup should succeed", 0, ret); // Move time forward to simulate 10 days ago EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - 10 * ONE_DAY_IN_MILLISECONDS); - loadRandomData(table2, 500); // Add more data to table2 - loadRandomData(table3, 500); // Insert data into table3 + PITRTestUtil.loadRandomData(TEST_UTIL, table2, famName, 500); // Add more data to table2 + PITRTestUtil.loadRandomData(TEST_UTIL, table3, famName, 500); // Insert data into table3 // Perform a full backup for table3 and table4 (without continuous backup) - args = buildBackupArgs("full", new TableName[] { table3, table4 }, false); + args = PITRTestUtil.buildBackupArgs("full", new TableName[] { table3, table4 }, false, + BACKUP_ROOT_DIR); ret = ToolRunner.run(conf1, new BackupDriver(), args); assertEquals("Backup should succeed", 0, ret); - waitForReplication(); // Ensure replication is complete before concluding setup + PITRTestUtil.waitForReplication(); // Ensure replication is complete before concluding setup // Reset time mocking to avoid affecting other tests EnvironmentEdgeManager.reset(); @@ -137,18 +127,18 @@ public static void setupAfterClass() throws IOException { @Test public void testPITR_FailsOutsideWindow() throws Exception { // Case 1: Requested restore time is in the future (should fail) - String[] args = buildPITRArgs(new TableName[] { table1 }, + String[] args = PITRTestUtil.buildPITRArgs(new TableName[] { table1 }, new TableName[] { TableName.valueOf("restoredTable1") }, - EnvironmentEdgeManager.currentTime() + ONE_DAY_IN_MILLISECONDS); + EnvironmentEdgeManager.currentTime() + ONE_DAY_IN_MILLISECONDS, null); int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); assertNotEquals("Restore should fail since the requested restore time is in the future", 0, ret); // Case 2: Requested restore time is too old (beyond the retention window, should fail) - args = buildPITRArgs(new TableName[] { table1 }, + args = PITRTestUtil.buildPITRArgs(new TableName[] { table1 }, new TableName[] { TableName.valueOf("restoredTable1") }, - EnvironmentEdgeManager.currentTime() - 40 * ONE_DAY_IN_MILLISECONDS); + EnvironmentEdgeManager.currentTime() - 40 * ONE_DAY_IN_MILLISECONDS, null); ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); assertNotEquals( @@ -162,9 +152,9 @@ public void testPITR_FailsOutsideWindow() throws Exception { */ @Test public void testPointInTimeRestore_ContinuousBackupNotEnabledTables() throws Exception { - String[] args = buildPITRArgs(new TableName[] { table3 }, + String[] args = PITRTestUtil.buildPITRArgs(new TableName[] { table3 }, new TableName[] { TableName.valueOf("restoredTable1") }, - EnvironmentEdgeManager.currentTime() - 10 * ONE_DAY_IN_MILLISECONDS); + EnvironmentEdgeManager.currentTime() - 10 * ONE_DAY_IN_MILLISECONDS, null); int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); assertNotEquals("Restore should fail since continuous backup is not enabled for the table", 0, @@ -176,9 +166,9 @@ public void testPointInTimeRestore_ContinuousBackupNotEnabledTables() throws Exc */ @Test public void testPointInTimeRestore_TablesWithNoProperBackupOrWals() throws Exception { - String[] args = buildPITRArgs(new TableName[] { table2 }, + String[] args = PITRTestUtil.buildPITRArgs(new TableName[] { table2 }, new TableName[] { TableName.valueOf("restoredTable1") }, - EnvironmentEdgeManager.currentTime() - 16 * ONE_DAY_IN_MILLISECONDS); + EnvironmentEdgeManager.currentTime() - 16 * ONE_DAY_IN_MILLISECONDS, null); int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); assertNotEquals( @@ -194,15 +184,17 @@ public void testPointInTimeRestore_SuccessfulRestoreForOneTable() throws Excepti TableName restoredTable = TableName.valueOf("restoredTable"); // Perform restore operation - String[] args = buildPITRArgs(new TableName[] { table1 }, new TableName[] { restoredTable }, - EnvironmentEdgeManager.currentTime() - 5 * ONE_DAY_IN_MILLISECONDS); + String[] args = + PITRTestUtil.buildPITRArgs(new TableName[] { table1 }, new TableName[] { restoredTable }, + EnvironmentEdgeManager.currentTime() - 5 * ONE_DAY_IN_MILLISECONDS, null); int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); assertEquals("Restore should succeed", 0, ret); // Validate that the restored table contains the same number of rows as the original table assertEquals("Restored table should have the same row count as the original", - getRowCount(table1), getRowCount(restoredTable)); + PITRTestUtil.getRowCount(TEST_UTIL, table1), + PITRTestUtil.getRowCount(TEST_UTIL, restoredTable)); } /** @@ -214,64 +206,19 @@ public void testPointInTimeRestore_SuccessfulRestoreForMultipleTables() throws E TableName restoredTable2 = TableName.valueOf("restoredTable2"); // Perform restore operation for multiple tables - String[] args = buildPITRArgs(new TableName[] { table1, table2 }, + String[] args = PITRTestUtil.buildPITRArgs(new TableName[] { table1, table2 }, new TableName[] { restoredTable1, restoredTable2 }, - EnvironmentEdgeManager.currentTime() - 5 * ONE_DAY_IN_MILLISECONDS); + EnvironmentEdgeManager.currentTime() - 5 * ONE_DAY_IN_MILLISECONDS, null); int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); assertEquals("Restore should succeed", 0, ret); // Validate that the restored tables contain the same number of rows as the originals assertEquals("Restored table1 should have the same row count as the original", - getRowCount(table1), getRowCount(restoredTable1)); + PITRTestUtil.getRowCount(TEST_UTIL, table1), + PITRTestUtil.getRowCount(TEST_UTIL, restoredTable1)); assertEquals("Restored table2 should have the same row count as the original", - getRowCount(table2), getRowCount(restoredTable2)); - } - - private String[] buildPITRArgs(TableName[] sourceTables, TableName[] targetTables, long endTime) { - String sourceTableNames = - Arrays.stream(sourceTables).map(TableName::getNameAsString).collect(Collectors.joining(",")); - - String targetTableNames = - Arrays.stream(targetTables).map(TableName::getNameAsString).collect(Collectors.joining(",")); - - return new String[] { "-" + OPTION_TABLE, sourceTableNames, "-" + OPTION_TABLE_MAPPING, - targetTableNames, "-" + OPTION_TO_DATETIME, String.valueOf(endTime) }; - } - - private static String[] buildBackupArgs(String backupType, TableName[] tables, - boolean continuousEnabled) { - String tableNames = - Arrays.stream(tables).map(TableName::getNameAsString).collect(Collectors.joining(",")); - - if (continuousEnabled) { - return new String[] { "create", backupType, BACKUP_ROOT_DIR, "-" + OPTION_TABLE, tableNames, - "-" + OPTION_ENABLE_CONTINUOUS_BACKUP }; - } else { - return new String[] { "create", backupType, BACKUP_ROOT_DIR, "-" + OPTION_TABLE, tableNames }; - } - } - - private static void loadRandomData(TableName tableName, int totalRows) throws IOException { - int rowSize = 32; - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { - TEST_UTIL.loadRandomRows(table, famName, rowSize, totalRows); - } - } - - private static void waitForReplication() { - LOG.info("Waiting for replication to complete for {} ms", WAIT_FOR_REPLICATION_MS); - try { - Thread.sleep(WAIT_FOR_REPLICATION_MS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Thread was interrupted while waiting", e); - } - } - - private int getRowCount(TableName tableName) throws IOException { - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { - return HBaseTestingUtil.countRows(table); - } + PITRTestUtil.getRowCount(TEST_UTIL, table2), + PITRTestUtil.getRowCount(TEST_UTIL, restoredTable2)); } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestoreWithCustomBackupPath.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestoreWithCustomBackupPath.java new file mode 100644 index 000000000000..78c5ac94ba00 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestoreWithCustomBackupPath.java @@ -0,0 +1,121 @@ +/* + * 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; + +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.ToolRunner; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(LargeTests.class) +public class TestPointInTimeRestoreWithCustomBackupPath extends TestBackupBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPointInTimeRestoreWithCustomBackupPath.class); + + private static final String backupWalDirName = "TestCustomBackupWalDir"; + private static final String customBackupDirName = "CustomBackupRoot"; + + private static Path backupWalDir; + private static Path customBackupDir; + private static FileSystem fs; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + backupWalDir = new Path(root, backupWalDirName); + customBackupDir = new Path(root, customBackupDirName); + + fs = FileSystem.get(conf1); + fs.mkdirs(backupWalDir); + fs.mkdirs(customBackupDir); + + conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); + + createAndCopyBackupData(); + } + + private static void createAndCopyBackupData() throws Exception { + // Simulate time 10 days ago + EnvironmentEdgeManager + .injectEdge(() -> System.currentTimeMillis() - 10 * ONE_DAY_IN_MILLISECONDS); + PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); + + // Perform backup with continuous backup enabled + String[] args = + PITRTestUtil.buildBackupArgs("full", new TableName[] { table1 }, true, BACKUP_ROOT_DIR); + int ret = ToolRunner.run(conf1, new BackupDriver(), args); + assertEquals("Backup should succeed", 0, ret); + + PITRTestUtil.waitForReplication(); + + // Copy the contents of BACKUP_ROOT_DIR to the new customBackupDir + Path defaultBackupDir = new Path(BACKUP_ROOT_DIR); + for (FileStatus status : fs.listStatus(defaultBackupDir)) { + Path dst = new Path(customBackupDir, status.getPath().getName()); + FileUtil.copy(fs, status, fs, dst, true, false, conf1); + } + + EnvironmentEdgeManager.reset(); + } + + @AfterClass + public static void cleanupAfterClass() throws IOException { + if (fs.exists(backupWalDir)) { + fs.delete(backupWalDir, true); + } + if (fs.exists(customBackupDir)) { + fs.delete(customBackupDir, true); + } + + conf1.unset(CONF_CONTINUOUS_BACKUP_WAL_DIR); + } + + @Test + public void testPITR_FromCustomBackupRootDir() throws Exception { + TableName restoredTable = TableName.valueOf("restoredTableCustomPath"); + + long restoreTime = EnvironmentEdgeManager.currentTime() - 2 * ONE_DAY_IN_MILLISECONDS; + + String[] args = PITRTestUtil.buildPITRArgs(new TableName[] { table1 }, + new TableName[] { restoredTable }, restoreTime, customBackupDir.toString()); + + int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertEquals("PITR should succeed with custom backup root dir", 0, ret); + + // Validate that the restored table has same row count + assertEquals("Restored table should match row count", + PITRTestUtil.getRowCount(TEST_UTIL, table1), + PITRTestUtil.getRowCount(TEST_UTIL, restoredTable)); + } +} From aa69616e8bb64de138f4f6502a7d5a29b6d904be Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Wed, 16 Jul 2025 23:11:35 +0530 Subject: [PATCH 13/24] HBASE-29441 ReplicationSourceShipper should delegate the empty wal entries handling to ReplicationEndpoint (#7145) Signed-off-by: Tak Lon (Stephen) Wu --- .../ContinuousBackupReplicationEndpoint.java | 9 +++ .../hbase/replication/EmptyEntriesPolicy.java | 34 ++++++++++ .../replication/ReplicationEndpoint.java | 18 ++++++ .../ReplicationSourceShipper.java | 33 ++++++++-- .../regionserver/TestReplicationSource.java | 63 +++++++++++++++++++ 5 files changed, 153 insertions(+), 4 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/replication/EmptyEntriesPolicy.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java index bf3fbd531bfe..2442e0789a8d 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; +import org.apache.hadoop.hbase.replication.EmptyEntriesPolicy; import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -205,6 +206,14 @@ protected void doStart() { notifyStarted(); } + @Override + public EmptyEntriesPolicy getEmptyEntriesPolicy() { + // Since this endpoint writes to S3 asynchronously, an empty entry batch + // does not guarantee that all previously submitted entries were persisted. + // Hence, avoid committing the WAL position. + return EmptyEntriesPolicy.SUBMIT; + } + @Override public ReplicationResult replicate(ReplicateContext replicateContext) { final List entries = replicateContext.getEntries(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/EmptyEntriesPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/EmptyEntriesPolicy.java new file mode 100644 index 000000000000..5a5d8ab754c3 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/EmptyEntriesPolicy.java @@ -0,0 +1,34 @@ +/* + * 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.replication; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Policy that defines what a replication endpoint should do when the entry batch is empty. This is + * used to determine whether the replication source should consider an empty batch as: - + * {@code COMMIT}: Consider the position as fully committed, and update the WAL position. - + * {@code SUBMIT}: Treat it as submitted but not committed, i.e., do not advance the WAL position. + * Some endpoints may buffer entries (e.g., in open files on S3) and delay actual persistence. In + * such cases, an empty batch should not result in WAL position commit. + */ +@InterfaceAudience.Private +public enum EmptyEntriesPolicy { + COMMIT, + SUBMIT +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java index fc5c2bf62659..fbb6b6b9ef10 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationEndpoint.java @@ -291,4 +291,22 @@ public int getTimeout() { * @throws IllegalStateException if this service's state isn't FAILED. */ Throwable failureCause(); + + /** + * Defines the behavior when the replication source encounters an empty entry batch. + *

+ * By default, this method returns {@link EmptyEntriesPolicy#COMMIT}, meaning the replication + * source can safely consider the WAL position as committed and move on. + *

+ *

+ * However, certain endpoints like backup or asynchronous S3 writers may delay persistence (e.g., + * writing to temporary files or buffers). In those cases, returning + * {@link EmptyEntriesPolicy#SUBMIT} avoids incorrectly advancing WAL position and risking data + * loss. + *

+ * @return the {@link EmptyEntriesPolicy} to apply for empty entry batches. + */ + default EmptyEntriesPolicy getEmptyEntriesPolicy() { + return EmptyEntriesPolicy.COMMIT; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java index ee819faa77b8..f45c8762683a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.replication.ReplicationUtils.getAdaptiveTimeout; import static org.apache.hadoop.hbase.replication.ReplicationUtils.sleepForRetries; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.util.List; import org.apache.hadoop.conf.Configuration; @@ -27,6 +28,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.replication.EmptyEntriesPolicy; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -150,13 +152,25 @@ protected void postFinish() { } /** - * Do the shipping logic + * Do the shipping logic. */ - private void shipEdits(WALEntryBatch entryBatch) { + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java)") + void shipEdits(WALEntryBatch entryBatch) { List entries = entryBatch.getWalEntries(); int sleepMultiplier = 0; if (entries.isEmpty()) { - updateLogPosition(entryBatch, ReplicationResult.COMMITTED); + /* + * Delegate to the endpoint to decide how to treat empty entry batches. In most replication + * flows, receiving an empty entry batch means that everything so far has been successfully + * replicated and committed — so it's safe to mark the WAL position as committed (COMMIT). + * However, some endpoints (e.g., asynchronous S3 backups) may buffer writes and delay actual + * persistence. In such cases, we must avoid committing the WAL position prematurely. + */ + final ReplicationResult result = getReplicationResult(); + updateLogPosition(entryBatch, result); return; } int currentSize = (int) entryBatch.getHeapSize(); @@ -232,6 +246,13 @@ private void shipEdits(WALEntryBatch entryBatch) { } } + private ReplicationResult getReplicationResult() { + EmptyEntriesPolicy policy = source.getReplicationEndpoint().getEmptyEntriesPolicy(); + return (policy == EmptyEntriesPolicy.COMMIT) + ? ReplicationResult.COMMITTED + : ReplicationResult.SUBMITTED; + } + private void cleanUpHFileRefs(WALEdit edit) throws IOException { String peerId = source.getPeerId(); if (peerId.contains("-")) { @@ -256,7 +277,11 @@ private void cleanUpHFileRefs(WALEdit edit) throws IOException { } } - private boolean updateLogPosition(WALEntryBatch batch, ReplicationResult replicated) { + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java)") + boolean updateLogPosition(WALEntryBatch batch, ReplicationResult replicated) { boolean updated = false; // if end of file is true, then the logPositionAndCleanOldLogs method will remove the file // record on zk, so let's call it. The last wal position maybe zero if end of file is true and diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index 37af52eb93b9..25eef51ff681 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -53,11 +53,13 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.hadoop.hbase.replication.EmptyEntriesPolicy; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationPeer; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationQueueData; import org.apache.hadoop.hbase.replication.ReplicationQueueId; +import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.WALEntryFilter; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; @@ -492,6 +494,67 @@ public synchronized UUID getPeerUUID() { } + /** + * Custom ReplicationEndpoint that simulates an asynchronous target like S3 or cloud storage. In + * this case, empty entry batches should not cause WAL position to be committed immediately. + */ + public static class AsyncReplicationEndpoint extends DoNothingReplicationEndpoint { + @Override + public EmptyEntriesPolicy getEmptyEntriesPolicy() { + return EmptyEntriesPolicy.SUBMIT; + } + } + + /** + * Default synchronous ReplicationEndpoint that treats empty entry batches as a signal to commit + * WAL position, assuming all entries pushed before were safely replicated. + */ + public static class SyncReplicationEndpoint extends DoNothingReplicationEndpoint { + // Inherits default COMMIT behavior + } + + /** + * Verifies that ReplicationSourceShipper commits the WAL position when using a synchronous + * endpoint and the entry batch is empty. + */ + @Test + public void testEmptyBatchCommitsPositionForCommitEndpoint() { + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + ReplicationSource source = Mockito.mock(ReplicationSource.class); + Mockito.when(source.getReplicationEndpoint()).thenReturn(new SyncReplicationEndpoint()); + + ReplicationSourceShipper shipper = + Mockito.spy(new ReplicationSourceShipper(conf, "testGroup", source, null)); + + WALEntryBatch emptyBatch = new WALEntryBatch(0, new Path("test-wal")); + + shipper.shipEdits(emptyBatch); + + // With default (COMMIT) policy, empty entry batch should advance WAL position + Mockito.verify(shipper).updateLogPosition(emptyBatch, ReplicationResult.COMMITTED); + } + + /** + * Verifies that ReplicationSourceShipper does NOT commit the WAL position when using an + * asynchronous endpoint and the entry batch is empty. + */ + @Test + public void testEmptyBatchSubmitsPositionForSubmitEndpoint() { + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + ReplicationSource source = Mockito.mock(ReplicationSource.class); + Mockito.when(source.getReplicationEndpoint()).thenReturn(new AsyncReplicationEndpoint()); + + ReplicationSourceShipper shipper = + Mockito.spy(new ReplicationSourceShipper(conf, "testGroup", source, null)); + + WALEntryBatch emptyBatch = new WALEntryBatch(0, new Path("test-wal")); + + shipper.shipEdits(emptyBatch); + + // With SUBMIT policy, empty entry batch should NOT advance WAL position + Mockito.verify(shipper).updateLogPosition(emptyBatch, ReplicationResult.SUBMITTED); + } + private RegionServerServices setupForAbortTests(ReplicationSource rs, Configuration conf, String endpointName) throws IOException { conf.setInt("replication.source.maxretriesmultiplier", 1); From a4cd71a74aaf4de44aa83030179ce0564f6b852f Mon Sep 17 00:00:00 2001 From: asolomon Date: Tue, 22 Jul 2025 22:13:49 +0530 Subject: [PATCH 14/24] HBASE-29459 Capture bulkload files only till IncrCommittedWalTs during Incremental Backup (#7166) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kevin Geiszler --- .../hbase/backup/impl/BackupManager.java | 5 + .../hbase/backup/impl/BackupSystemTable.java | 29 +++-- .../hadoop/hbase/backup/impl/BulkLoad.java | 15 ++- .../impl/IncrementalTableBackupClient.java | 8 +- .../TestIncrementalBackupWithContinuous.java | 100 ++++++++---------- 5 files changed, 88 insertions(+), 69 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java index 8b17e93868be..c2ed4f7fa1fc 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java @@ -363,6 +363,11 @@ public HashMap readRegionServerLastLogRollResult() throws IOExcept return systemTable.readRegionServerLastLogRollResult(backupInfo.getBackupRootDir()); } + public List readBulkloadRows(List tableList, long endTimestamp) + throws IOException { + return systemTable.readBulkloadRows(tableList, endTimestamp); + } + public List readBulkloadRows(List tableList) throws IOException { return systemTable.readBulkloadRows(tableList); } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 098946dab38a..30307c43bb88 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -380,26 +380,36 @@ public void deleteBulkLoadedRows(List rows) throws IOException { } /** - * Reads all registered bulk loads. + * Reads the rows from backup table recording bulk loaded hfiles */ public List readBulkloadRows() throws IOException { Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(null); - return processBulkLoadRowScan(scan); + return processBulkLoadRowScan(scan, Long.MAX_VALUE); } /** - * Reads the registered bulk loads for the given tables. + * Reads the rows from backup table recording bulk loaded hfiles + * @param tableList list of table names */ public List readBulkloadRows(Collection tableList) throws IOException { + return readBulkloadRows(tableList, Long.MAX_VALUE); + } + + /** + * Reads the rows from backup table recording bulk loaded hfiles + * @param tableList list of table names + * @param endTimestamp upper bound timestamp for bulkload entries retrieval + */ + public List readBulkloadRows(Collection tableList, long endTimestamp) throws IOException { List result = new ArrayList<>(); for (TableName table : tableList) { Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(table); - result.addAll(processBulkLoadRowScan(scan)); + result.addAll(processBulkLoadRowScan(scan, endTimestamp)); } return result; } - private List processBulkLoadRowScan(Scan scan) throws IOException { + private List processBulkLoadRowScan(Scan scan, long endTimestamp) throws IOException { List result = new ArrayList<>(); try (Table bulkLoadTable = connection.getTable(bulkLoadTableName); ResultScanner scanner = bulkLoadTable.getScanner(scan)) { @@ -411,8 +421,10 @@ private List processBulkLoadRowScan(Scan scan) throws IOException { String path = null; String region = null; byte[] row = null; + long timestamp = 0L; for (Cell cell : res.listCells()) { row = CellUtil.cloneRow(cell); + timestamp = cell.getTimestamp(); String rowStr = Bytes.toString(row); region = BackupSystemTable.getRegionNameFromOrigBulkLoadRow(rowStr); if ( @@ -432,8 +444,11 @@ private List processBulkLoadRowScan(Scan scan) throws IOException { path = Bytes.toString(CellUtil.cloneValue(cell)); } } - result.add(new BulkLoad(table, region, fam, path, row)); - LOG.debug("Found bulk load entry for table {}, family {}: {}", table, fam, path); + LOG.debug("Found orig path {} for family {} of table {} and region {} with timestamp {}", + path, fam, table, region, timestamp); + if (timestamp <= endTimestamp) { + result.add(new BulkLoad(table, region, fam, path, row, timestamp)); + } } } return result; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BulkLoad.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BulkLoad.java index 0f1e79c976bb..1befe7c469cc 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BulkLoad.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BulkLoad.java @@ -34,14 +34,16 @@ public class BulkLoad { private final String columnFamily; private final String hfilePath; private final byte[] rowKey; + private final long timestamp; public BulkLoad(TableName tableName, String region, String columnFamily, String hfilePath, - byte[] rowKey) { + byte[] rowKey, long timestamp) { this.tableName = tableName; this.region = region; this.columnFamily = columnFamily; this.hfilePath = hfilePath; this.rowKey = rowKey; + this.timestamp = timestamp; } public TableName getTableName() { @@ -64,6 +66,10 @@ public byte[] getRowKey() { return rowKey; } + public long getTimestamp() { + return timestamp; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -75,19 +81,20 @@ public boolean equals(Object o) { BulkLoad that = (BulkLoad) o; return new EqualsBuilder().append(tableName, that.tableName).append(region, that.region) .append(columnFamily, that.columnFamily).append(hfilePath, that.hfilePath) - .append(rowKey, that.rowKey).isEquals(); + .append(rowKey, that.rowKey).append(timestamp, that.timestamp).isEquals(); } @Override public int hashCode() { return new HashCodeBuilder().append(tableName).append(region).append(columnFamily) - .append(hfilePath).append(rowKey).toHashCode(); + .append(hfilePath).append(rowKey).append(timestamp).toHashCode(); } @Override public String toString() { return new ToStringBuilder(this, ToStringStyle.NO_CLASS_NAME_STYLE) .append("tableName", tableName).append("region", region).append("columnFamily", columnFamily) - .append("hfilePath", hfilePath).append("rowKey", rowKey).toString(); + .append("hfilePath", hfilePath).append("rowKey", rowKey).append("timestamp", timestamp) + .toString(); } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index 34f9a4e29866..4d39d11a36d7 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -139,7 +139,13 @@ protected static int getIndex(TableName tbl, List sTableList) { */ protected List handleBulkLoad(List tablesToBackup) throws IOException { Map toBulkload = new HashMap<>(); - List bulkLoads = backupManager.readBulkloadRows(tablesToBackup); + List bulkLoads; + if (backupInfo.isContinuousBackupEnabled()) { + bulkLoads = + backupManager.readBulkloadRows(tablesToBackup, backupInfo.getIncrCommittedWalTs()); + } else { + bulkLoads = backupManager.readBulkloadRows(tablesToBackup); + } FileSystem tgtFs; try { tgtFs = FileSystem.get(new URI(backupInfo.getBackupRootDir()), conf); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java index 79d1df645b95..170cc8665687 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupManifest; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; -import org.apache.hadoop.hbase.backup.impl.BulkLoad; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -70,7 +69,6 @@ public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { LoggerFactory.getLogger(TestIncrementalBackupWithContinuous.class); private byte[] ROW = Bytes.toBytes("row1"); - private final byte[] FAMILY = Bytes.toBytes("family"); private final byte[] COLUMN = Bytes.toBytes("col"); private static final int ROWS_IN_BULK_LOAD = 100; @@ -80,7 +78,7 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); TableName tableName = TableName.valueOf("table_" + methodName); - Table t1 = TEST_UTIL.createTable(tableName, FAMILY); + Table t1 = TEST_UTIL.createTable(tableName, famName); try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) { int before = table.getBackupHistory().size(); @@ -105,10 +103,8 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception assertEquals("Backup should contain the expected tables", Sets.newHashSet(tableName), new HashSet<>(manifest.getTableList())); - Put p = new Put(ROW); - p.addColumn(FAMILY, COLUMN, COLUMN); - t1.put(p); - Thread.sleep(5000); + loadTable(t1); + Thread.sleep(10000); // Run incremental backup LOG.info("Run incremental backup now"); @@ -135,68 +131,57 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, incrementalBackupid, false, tables, tables, true)); - verifyTable(t1); + assertEquals(NB_ROWS_IN_BATCH, TEST_UTIL.countRows(tableName)); + } finally { conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); } } @Test - public void testContinuousBackupWithIncrementalBackupAndBulkloadSuccess() throws Exception { + public void testIncrementalBackupCopyingBulkloadTillIncrCommittedWalTs() throws Exception { conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName1 = TableName.valueOf("table_" + methodName); + TEST_UTIL.createTable(tableName1, famName); try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { - // The test starts with some data, and no bulk loaded rows. - int expectedRowCount = NB_ROWS_IN_BATCH; - assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); - assertTrue(systemTable.readBulkloadRows(List.of(table1)).isEmpty()); - // Bulk loads aren't tracked if the table isn't backed up yet - performBulkLoad("bulk1", methodName); - expectedRowCount += ROWS_IN_BULK_LOAD; - assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); - assertEquals(0, systemTable.readBulkloadRows(List.of(table1)).size()); + // The test starts with no data, and no bulk loaded rows. + int expectedRowCount = 0; + assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); + assertTrue(systemTable.readBulkloadRows(List.of(tableName1)).isEmpty()); - // Create a backup, bulk loads are now being tracked - String backup1 = backupTables(BackupType.FULL, List.of(table1), BACKUP_ROOT_DIR, true); + // Create continuous backup, bulk loads are now being tracked + String backup1 = backupTables(BackupType.FULL, List.of(tableName1), BACKUP_ROOT_DIR, true); assertTrue(checkSucceeded(backup1)); - loadTable(TEST_UTIL.getConnection().getTable(table1)); - assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); - performBulkLoad("bulk2", methodName); + loadTable(TEST_UTIL.getConnection().getTable(tableName1)); + expectedRowCount = expectedRowCount + NB_ROWS_IN_BATCH; + performBulkLoad("bulkPreIncr", methodName, tableName1); expectedRowCount += ROWS_IN_BULK_LOAD; - assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); - assertEquals(1, systemTable.readBulkloadRows(List.of(table1)).size()); - - // Creating an incremental backup clears the bulk loads - performBulkLoad("bulk4", methodName); - performBulkLoad("bulk5", methodName); - performBulkLoad("bulk6", methodName); - expectedRowCount += 3 * ROWS_IN_BULK_LOAD; - assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); - assertEquals(4, systemTable.readBulkloadRows(List.of(table1)).size()); - String backup2 = backupTables(BackupType.INCREMENTAL, List.of(table1), BACKUP_ROOT_DIR, true); + assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); + assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); + loadTable(TEST_UTIL.getConnection().getTable(tableName1)); + Thread.sleep(10000); + + performBulkLoad("bulkPostIncr", methodName, tableName1); + assertEquals(2, systemTable.readBulkloadRows(List.of(tableName1)).size()); + + // Incremental backup + String backup2 = + backupTables(BackupType.INCREMENTAL, List.of(tableName1), BACKUP_ROOT_DIR, true); assertTrue(checkSucceeded(backup2)); - assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); - assertEquals(0, systemTable.readBulkloadRows(List.of(table1)).size()); - int rowCountAfterBackup2 = expectedRowCount; - // Doing another bulk load, to check that this data will disappear after a restore operation - performBulkLoad("bulk7", methodName); - expectedRowCount += ROWS_IN_BULK_LOAD; - assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); - List bulkloadsTemp = systemTable.readBulkloadRows(List.of(table1)); - assertEquals(1, bulkloadsTemp.size()); - BulkLoad bulk7 = bulkloadsTemp.get(0); - - // Doing a restore. Overwriting the table implies clearing the bulk loads, - // but the loading of restored data involves loading bulk data, we expect 2 bulk loads - // associated with backup 3 (loading of full backup, loading of incremental backup). - BackupAdmin client = getBackupAdmin(); - client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backup2, false, - new TableName[] { table1 }, new TableName[] { table1 }, true)); - assertEquals(rowCountAfterBackup2, TEST_UTIL.countRows(table1)); - List bulkLoads = systemTable.readBulkloadRows(List.of(table1)); - assertEquals(3, bulkLoads.size()); + // bulkPostIncr Bulkload entry should not be deleted post incremental backup + assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); + + TEST_UTIL.truncateTable(tableName1); + // Restore incremental backup + TableName[] tables = new TableName[] { tableName1 }; + BackupAdminImpl client = new BackupAdminImpl(TEST_UTIL.getConnection()); + client.restore( + BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backup2, false, tables, tables, true)); + assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); + } finally { conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); } } @@ -208,7 +193,8 @@ private void verifyTable(Table t1) throws IOException { assertTrue(CellUtil.matchingQualifier(r.rawCells()[0], COLUMN)); } - private void performBulkLoad(String keyPrefix, String testDir) throws IOException { + private void performBulkLoad(String keyPrefix, String testDir, TableName tableName) + throws IOException { FileSystem fs = TEST_UTIL.getTestFileSystem(); Path baseDirectory = TEST_UTIL.getDataTestDirOnTestFS(testDir); Path hfilePath = @@ -220,7 +206,7 @@ private void performBulkLoad(String keyPrefix, String testDir) throws IOExceptio listFiles(fs, baseDirectory, baseDirectory); Map result = - BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(table1, baseDirectory); + BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tableName, baseDirectory); assertFalse(result.isEmpty()); } @@ -246,7 +232,7 @@ private static Set listFiles(final FileSystem fs, final Path root, final protected static void loadTable(Table table) throws Exception { Put p; // 100 + 1 row to t1_syncup for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { - p = new Put(Bytes.toBytes("row" + i)); + p = new Put(Bytes.toBytes("rowLoad" + i)); p.addColumn(famName, qualName, Bytes.toBytes("val" + i)); table.put(p); } From 3c5c99944f001375c92e30066df967bf10d1acee Mon Sep 17 00:00:00 2001 From: asolomon Date: Wed, 23 Jul 2025 21:49:35 +0530 Subject: [PATCH 15/24] HBASE-29310 Handle Bulk Load Operations in Continuous Backup (#7150) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kevin Geiszler --- .../impl/AbstractPitrRestoreHandler.java | 28 +++++++++ .../hbase/backup/impl/BackupImageAdapter.java | 11 ++++ .../hbase/backup/impl/BackupInfoAdapter.java | 11 ++++ .../hbase/backup/impl/BackupManifest.java | 22 ++++++- .../hbase/backup/impl/PitrBackupMetadata.java | 7 +++ .../TestIncrementalBackupWithContinuous.java | 60 +++++++++++++++---- .../hbase/backup/TestPointInTimeRestore.java | 4 +- .../src/main/protobuf/Backup.proto | 1 + .../hadoop/hbase/tool/BulkLoadHFilesTool.java | 5 ++ 9 files changed, 135 insertions(+), 14 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java index b2edce6b0fd1..8072277bf684 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest; import org.apache.hadoop.hbase.backup.RestoreRequest; import org.apache.hadoop.hbase.backup.util.BackupUtils; @@ -248,6 +249,8 @@ private PitrBackupMetadata getValidBackup(TableName sTableName, TableName tTable try { if (backupAdmin.validateRequest(restoreRequest)) { + // check if any bulkload entry exists post this backup time and before "endtime" + checkBulkLoadAfterBackup(conn, sTableName, backup, endTime); return backup; } } catch (IOException e) { @@ -259,6 +262,31 @@ private PitrBackupMetadata getValidBackup(TableName sTableName, TableName tTable return null; } + /** + * Checks if any bulk load operation occurred for the specified table post last successful backup + * and before restore time. + * @param conn Active HBase connection + * @param sTableName Table for which to check bulk load history + * @param backup Last successful backup before the target recovery time + * @param endTime Target recovery time + * @throws IOException if a bulkload entry is found in between backup time and endtime + */ + private void checkBulkLoadAfterBackup(Connection conn, TableName sTableName, + PitrBackupMetadata backup, long endTime) throws IOException { + try (BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { + List bulkLoads = backupSystemTable.readBulkloadRows(List.of(sTableName)); + for (BulkLoad load : bulkLoads) { + long lastBackupTs = (backup.getType() == BackupType.FULL) + ? backup.getStartTs() + : backup.getIncrCommittedWalTs(); + if (lastBackupTs < load.getTimestamp() && load.getTimestamp() < endTime) { + throw new IOException("Bulk load operation detected after last successful backup for " + + "table: " + sTableName); + } + } + } + } + /** * Determines if the given backup is valid for PITR. *

diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java index 8b785a0f0504..b6d8d4901a22 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java @@ -19,6 +19,7 @@ import java.util.List; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; import org.apache.yetus.audience.InterfaceAudience; @@ -57,4 +58,14 @@ public String getBackupId() { public String getRootDir() { return image.getRootDir(); } + + @Override + public BackupType getType() { + return image.getType(); + } + + @Override + public long getIncrCommittedWalTs() { + return image.getIncrCommittedWalTs(); + } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java index 967fae551cb5..34d812121e02 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupType; import org.apache.yetus.audience.InterfaceAudience; /** @@ -57,4 +58,14 @@ public String getBackupId() { public String getRootDir() { return info.getBackupRootDir(); } + + @Override + public BackupType getType() { + return info.getType(); + } + + @Override + public long getIncrCommittedWalTs() { + return info.getIncrCommittedWalTs(); + } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java index 59ae3857f2ec..f35755d24512 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java @@ -101,6 +101,11 @@ Builder withCompleteTime(long completeTime) { return this; } + Builder withIncrCommittedWalTs(long incrCommittedWalTs) { + image.setIncrCommittedWalTs(incrCommittedWalTs); + return this; + } + BackupImage build() { return image; } @@ -115,6 +120,7 @@ BackupImage build() { private long completeTs; private ArrayList ancestors; private Map> incrTimeRanges; + private long incrCommittedWalTs; static Builder newBuilder() { return new Builder(); @@ -125,13 +131,14 @@ public BackupImage() { } private BackupImage(String backupId, BackupType type, String rootDir, List tableList, - long startTs, long completeTs) { + long startTs, long completeTs, long incrCommittedWalTs) { this.backupId = backupId; this.type = type; this.rootDir = rootDir; this.tableList = tableList; this.startTs = startTs; this.completeTs = completeTs; + this.incrCommittedWalTs = incrCommittedWalTs; } static BackupImage fromProto(BackupProtos.BackupImage im) { @@ -139,6 +146,7 @@ static BackupImage fromProto(BackupProtos.BackupImage im) { String rootDir = im.getBackupRootDir(); long startTs = im.getStartTs(); long completeTs = im.getCompleteTs(); + long incrCommittedWalTs = im.getIncrCommittedWalTs(); List tableListList = im.getTableListList(); List tableList = new ArrayList<>(); for (HBaseProtos.TableName tn : tableListList) { @@ -151,7 +159,8 @@ static BackupImage fromProto(BackupProtos.BackupImage im) { ? BackupType.FULL : BackupType.INCREMENTAL; - BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs); + BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs, + incrCommittedWalTs); for (BackupProtos.BackupImage img : ancestorList) { image.addAncestor(fromProto(img)); } @@ -170,6 +179,7 @@ BackupProtos.BackupImage toProto() { builder.setBackupId(backupId); builder.setCompleteTs(completeTs); builder.setStartTs(startTs); + builder.setIncrCommittedWalTs(incrCommittedWalTs); if (type == BackupType.FULL) { builder.setBackupType(BackupProtos.BackupType.FULL); } else { @@ -287,6 +297,14 @@ public long getCompleteTs() { return completeTs; } + public long getIncrCommittedWalTs() { + return incrCommittedWalTs; + } + + public void setIncrCommittedWalTs(long incrCommittedWalTs) { + this.incrCommittedWalTs = incrCommittedWalTs; + } + private void setCompleteTs(long completeTs) { this.completeTs = completeTs; } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java index dc135ce79c08..3d143b336573 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; import org.apache.yetus.audience.InterfaceAudience; @@ -47,4 +48,10 @@ public interface PitrBackupMetadata { /** Returns Root directory where the backup is stored */ String getRootDir(); + + /** Returns backup type */ + BackupType getType(); + + /** Returns incrCommittedWalTs */ + long getIncrCommittedWalTs(); } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java index 170cc8665687..0978ff3ebef5 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java @@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -33,16 +34,13 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupManifest; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.backup.util.BackupUtils; -import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.tool.BulkLoadHFiles; @@ -68,8 +66,6 @@ public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { private static final Logger LOG = LoggerFactory.getLogger(TestIncrementalBackupWithContinuous.class); - private byte[] ROW = Bytes.toBytes("row1"); - private final byte[] COLUMN = Bytes.toBytes("col"); private static final int ROWS_IN_BULK_LOAD = 100; @Test @@ -186,11 +182,55 @@ public void testIncrementalBackupCopyingBulkloadTillIncrCommittedWalTs() throws } } - private void verifyTable(Table t1) throws IOException { - Get g = new Get(ROW); - Result r = t1.get(g); - assertEquals(1, r.size()); - assertTrue(CellUtil.matchingQualifier(r.rawCells()[0], COLUMN)); + @Test + public void testPitrFailureDueToMissingBackupPostBulkload() throws Exception { + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName1 = TableName.valueOf("table_" + methodName); + TEST_UTIL.createTable(tableName1, famName); + try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { + + // The test starts with no data, and no bulk loaded rows. + int expectedRowCount = 0; + assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); + assertTrue(systemTable.readBulkloadRows(List.of(tableName1)).isEmpty()); + + // Create continuous backup, bulk loads are now being tracked + String backup1 = backupTables(BackupType.FULL, List.of(tableName1), BACKUP_ROOT_DIR, true); + assertTrue(checkSucceeded(backup1)); + + loadTable(TEST_UTIL.getConnection().getTable(tableName1)); + expectedRowCount = expectedRowCount + NB_ROWS_IN_BATCH; + performBulkLoad("bulkPreIncr", methodName, tableName1); + expectedRowCount += ROWS_IN_BULK_LOAD; + assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); + assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); + + loadTable(TEST_UTIL.getConnection().getTable(tableName1)); + Thread.sleep(5000); + + // Incremental backup + String backup2 = + backupTables(BackupType.INCREMENTAL, List.of(tableName1), BACKUP_ROOT_DIR, true); + assertTrue(checkSucceeded(backup2)); + assertEquals(0, systemTable.readBulkloadRows(List.of(tableName1)).size()); + + performBulkLoad("bulkPostIncr", methodName, tableName1); + assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); + + loadTable(TEST_UTIL.getConnection().getTable(tableName1)); + Thread.sleep(10000); + long restoreTs = BackupUtils.getReplicationCheckpoint(TEST_UTIL.getConnection()); + + // expect restore failure due to no backup post bulkPostIncr bulkload + TableName restoredTable = TableName.valueOf("restoredTable"); + String[] args = PITRTestUtil.buildPITRArgs(new TableName[] { tableName1 }, + new TableName[] { restoredTable }, restoreTs, null); + int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); + assertNotEquals("Restore should fail since there is one bulkload without any backup", 0, ret); + } finally { + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); + } } private void performBulkLoad(String keyPrefix, String testDir, TableName tableName) diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java index a1ce9c97a687..e9a0b50abcfa 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java @@ -67,8 +67,8 @@ private static void setUpBackups() throws Exception { // Simulate a backup taken 20 days ago EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - 20 * ONE_DAY_IN_MILLISECONDS); - PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Insert initial data into - // table1 + // Insert initial data into table1 + PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Perform a full backup for table1 with continuous backup enabled String[] args = diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto index 0ad1f5ba6191..b173848cd09d 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Backup.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto @@ -65,6 +65,7 @@ message BackupImage { optional uint64 complete_ts = 6; repeated BackupImage ancestors = 7; repeated TableServerTimestamp tst_map = 8; + optional uint64 incr_committed_wal_ts = 9; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java index 4d6f57e22edc..cd9bc32e3e18 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java @@ -1197,6 +1197,11 @@ public int run(String[] args) throws Exception { public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); int ret = ToolRunner.run(conf, new BulkLoadHFilesTool(conf), args); + if (ret == 0) { + System.out.println("Bulk load completed successfully."); + System.out.println("IMPORTANT: Please take a backup of the table immediately if this table " + + "is part of continuous backup"); + } System.exit(ret); } From fa6b83f521445d82309a581f103dc32570d8f413 Mon Sep 17 00:00:00 2001 From: Vinayak Hegde Date: Tue, 29 Jul 2025 14:31:12 +0530 Subject: [PATCH 16/24] HBASE-28957 spotless apply after rebase --- .../org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java | 3 ++- .../org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java index 30307c43bb88..2f9c3171346a 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java @@ -400,7 +400,8 @@ public List readBulkloadRows(Collection tableList) throws I * @param tableList list of table names * @param endTimestamp upper bound timestamp for bulkload entries retrieval */ - public List readBulkloadRows(Collection tableList, long endTimestamp) throws IOException { + public List readBulkloadRows(Collection tableList, long endTimestamp) + throws IOException { List result = new ArrayList<>(); for (TableName table : tableList) { Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(table); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java index 0b2d66bd0b8f..903b03694e4a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALInputFormat.java @@ -161,7 +161,8 @@ Path[] getInputPaths(Configuration conf) { } @Override - List getFiles(FileSystem fs, Path inputPath, long startTime, long endTime) { + List getFiles(FileSystem fs, Path inputPath, long startTime, long endTime, + Configuration conf) { return Collections.singletonList(emptyFile); } }; From 3044b114b80d93fe44ceafc68a83d88ae09343ef Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Tue, 29 Jul 2025 23:18:47 +0530 Subject: [PATCH 17/24] HBASE-29375 Add Unit Tests for BackupAdminImpl and Improve Test Granularity (#7171) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kevin Geiszler --- .../hbase/backup/impl/BackupAdminImpl.java | 55 +- .../backup/impl/TestBackupAdminImpl.java | 750 ++++++++++++++++++ 2 files changed, 795 insertions(+), 10 deletions(-) create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupAdminImpl.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java index 75a2a6343a58..2122ef9378fb 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.backup.impl; +import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -175,8 +176,11 @@ public int deleteBackups(String[] backupIds) throws IOException { * @param table backup system table * @throws IOException if a table operation fails */ - private void finalizeDelete(List backupRoots, BackupSystemTable table) - throws IOException { + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + void finalizeDelete(List backupRoots, BackupSystemTable table) throws IOException { for (String backupRoot : backupRoots) { Set incrTableSet = table.getIncrementalBackupTableSet(backupRoot); Map> tableMap = @@ -212,7 +216,11 @@ private void finalizeDelete(List backupRoots, BackupSystemTable table) * @return total number of deleted backup images * @throws IOException if deleting the backup fails */ - private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException { + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException { BackupInfo backupInfo = sysTable.readBackupInfo(backupId); int totalDeleted = 0; @@ -274,7 +282,11 @@ private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOE return totalDeleted; } - private void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable) + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable) throws IOException { List tables = info.getTableNames(); LOG.debug( @@ -297,7 +309,11 @@ private void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSys } } - private List getAffectedBackupSessions(BackupInfo backupInfo, TableName tn, + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + List getAffectedBackupSessions(BackupInfo backupInfo, TableName tn, BackupSystemTable table) throws IOException { LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn); long ts = backupInfo.getStartTs(); @@ -329,7 +345,11 @@ private List getAffectedBackupSessions(BackupInfo backupInfo, TableN * Clean up the data at target directory * @throws IOException if cleaning up the backup directory fails */ - private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) throws IOException { try { // clean up the data at target directory @@ -339,7 +359,7 @@ private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configurat return; } - FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf); + FileSystem outputFs = getFileSystem(new Path(backupInfo.getBackupRootDir()), conf); Path targetDirPath = new Path(BackupUtils.getTableBackupDir(backupInfo.getBackupRootDir(), backupInfo.getBackupId(), table)); @@ -355,7 +375,19 @@ private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configurat } } - private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime) + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + FileSystem getFileSystem(Path path, Configuration conf) throws IOException { + return FileSystem.get(path.toUri(), conf); + } + + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime) throws IOException { List history = table.getBackupHistory(); for (BackupInfo info : history) { @@ -682,8 +714,11 @@ public void mergeBackups(String[] backupIds) throws IOException { * @param table backup system table * @throws IOException if the backup image is not valid for merge */ - private void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) - throws IOException { + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java)") + void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) throws IOException { String backupRoot = null; final Set allTables = new HashSet<>(); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupAdminImpl.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupAdminImpl.java new file mode 100644 index 000000000000..b78d44c144b5 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupAdminImpl.java @@ -0,0 +1,750 @@ +/* + * 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.impl; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anySet; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupInfo; +import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; +import org.apache.hadoop.hbase.backup.BackupType; +import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.ArgumentCaptor; +import org.mockito.MockedStatic; + +/** + * Unit tests for {@link BackupAdminImpl}. + *

+ * This class improves test coverage by validating the behavior of key methods in BackupAdminImpl. + * Some methods are made package-private to enable testing. + */ +@Category(SmallTests.class) +public class TestBackupAdminImpl { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupAdminImpl.class); + + private BackupAdminImpl backupAdminImpl; + private BackupSystemTable mockTable; + + @Before + public void setUp() { + backupAdminImpl = new BackupAdminImpl(null); + mockTable = mock(BackupSystemTable.class); + } + + /** + * Scenario: - The initial incremental table set contains "table1" and "table2" - Only "table1" + * still exists in backup history Expectation: - The backup system should delete the existing set + * - Then re-add a filtered set that includes only "table1" + */ + @Test + public void testFinalizeDelete_addsRetainedTablesBack() throws IOException { + String backupRoot = "backupRoot1"; + List backupRoots = Collections.singletonList(backupRoot); + + Set initialTableSet = + new HashSet<>(Arrays.asList(TableName.valueOf("ns:table1"), TableName.valueOf("ns:table2"))); + + Map> backupHistory = new HashMap<>(); + backupHistory.put(TableName.valueOf("ns:table1"), List.of(new BackupInfo())); // Only table1 + // retained + + when(mockTable.getIncrementalBackupTableSet(backupRoot)) + .thenReturn(new HashSet<>(initialTableSet)); + when(mockTable.getBackupHistoryForTableSet(initialTableSet, backupRoot)) + .thenReturn(backupHistory); + + backupAdminImpl.finalizeDelete(backupRoots, mockTable); + + // Always remove existing backup metadata + verify(mockTable).deleteIncrementalBackupTableSet(backupRoot); + + // Re-add only retained tables (should be just table1) + @SuppressWarnings("unchecked") + ArgumentCaptor> captor = + (ArgumentCaptor>) (ArgumentCaptor) ArgumentCaptor.forClass(Set.class); + verify(mockTable).addIncrementalBackupTableSet(captor.capture(), eq(backupRoot)); + + Set retained = captor.getValue(); + assertEquals(1, retained.size()); + assertTrue(retained.contains(TableName.valueOf("ns:table1"))); + } + + /** + * Scenario: - The incremental table set has "tableX" - No backups exist for this table + * Expectation: - Backup metadata should be deleted - Nothing should be re-added since no tables + * are retained + */ + @Test + public void testFinalizeDelete_retainedSetEmpty_doesNotAddBack() throws IOException { + String backupRoot = "backupRoot2"; + List backupRoots = List.of(backupRoot); + + Set initialTableSet = Set.of(TableName.valueOf("ns:tableX")); + Map> backupHistory = Map.of(); // No overlap + + when(mockTable.getIncrementalBackupTableSet(backupRoot)) + .thenReturn(new HashSet<>(initialTableSet)); + when(mockTable.getBackupHistoryForTableSet(initialTableSet, backupRoot)) + .thenReturn(backupHistory); + + backupAdminImpl.finalizeDelete(backupRoots, mockTable); + + // Delete should be called + verify(mockTable).deleteIncrementalBackupTableSet(backupRoot); + // No add back since retained set is empty + verify(mockTable, never()).addIncrementalBackupTableSet(any(), eq(backupRoot)); + } + + /** + * Scenario: - Two backup roots: - root1: one table with valid backup history → should be retained + * - root2: one table with no history → should not be retained Expectation: - root1 metadata + * deleted and re-added - root2 metadata only deleted + */ + @Test + public void testFinalizeDelete_multipleRoots() throws IOException { + String root1 = "root1"; + String root2 = "root2"; + List roots = List.of(root1, root2); + + TableName t1 = TableName.valueOf("ns:table1"); + TableName t2 = TableName.valueOf("ns:table2"); + + // root1 setup + when(mockTable.getIncrementalBackupTableSet(root1)).thenReturn(new HashSet<>(List.of(t1))); + when(mockTable.getBackupHistoryForTableSet(Set.of(t1), root1)) + .thenReturn(Map.of(t1, List.of(new BackupInfo()))); + + // root2 setup + when(mockTable.getIncrementalBackupTableSet(root2)).thenReturn(new HashSet<>(List.of(t2))); + when(mockTable.getBackupHistoryForTableSet(Set.of(t2), root2)).thenReturn(Map.of()); // empty + // history + + backupAdminImpl.finalizeDelete(roots, mockTable); + + // root1: should delete and re-add table + verify(mockTable).deleteIncrementalBackupTableSet(root1); + verify(mockTable).addIncrementalBackupTableSet(Set.of(t1), root1); + + // root2: delete only + verify(mockTable).deleteIncrementalBackupTableSet(root2); + verify(mockTable, never()).addIncrementalBackupTableSet(anySet(), eq(root2)); + } + + /** + * Verifies that {@code cleanupBackupDir} correctly deletes the target backup directory for a + * given table and backup ID using the mocked FileSystem. + *

+ * This test ensures: - The correct path is constructed using BackupUtils. - FileSystem#delete is + * invoked with that path. + */ + @Test + public void testCleanupBackupDir_deletesTargetDirSuccessfully() throws Exception { + // Setup test input + String backupId = "backup_001"; + String backupRootDir = "/backup/root"; + TableName table = TableName.valueOf("test_table"); + + BackupInfo mockBackupInfo = mock(BackupInfo.class); + when(mockBackupInfo.getBackupRootDir()).thenReturn(backupRootDir); + when(mockBackupInfo.getBackupId()).thenReturn(backupId); + + Configuration conf = new Configuration(); + + // Spy BackupAdminImpl to mock getFileSystem behavior + backupAdminImpl = spy(backupAdminImpl); + + FileSystem mockFs = mock(FileSystem.class); + Path expectedPath = new Path(BackupUtils.getTableBackupDir(backupRootDir, backupId, table)); + + // Mock getFileSystem to return our mock FileSystem + doReturn(mockFs).when(backupAdminImpl).getFileSystem(any(Path.class), eq(conf)); + when(mockFs.delete(expectedPath, true)).thenReturn(true); + + // Call the method under test + backupAdminImpl.cleanupBackupDir(mockBackupInfo, table, conf); + + // Verify the FileSystem delete call with correct path + verify(mockFs).delete(expectedPath, true); + } + + /** + * Verifies that {@code cleanupBackupDir} throws an {@link IOException} if the FileSystem + * retrieval fails. + *

+ * This test simulates an exception while trying to obtain the FileSystem, and expects the method + * to propagate the exception. + */ + @Test(expected = IOException.class) + public void testCleanupBackupDir_throwsIOException() throws Exception { + // Setup test input + String backupId = "backup_003"; + String backupRootDir = "/backup/root"; + TableName table = TableName.valueOf("test_table"); + + BackupInfo mockBackupInfo = mock(BackupInfo.class); + when(mockBackupInfo.getBackupRootDir()).thenReturn(backupRootDir); + when(mockBackupInfo.getBackupId()).thenReturn(backupId); + + Configuration conf = new Configuration(); + + // Spy BackupAdminImpl to inject failure in getFileSystem + backupAdminImpl = spy(backupAdminImpl); + doThrow(new IOException("FS error")).when(backupAdminImpl).getFileSystem(any(Path.class), + eq(conf)); + + // Call method and expect IOException + backupAdminImpl.cleanupBackupDir(mockBackupInfo, table, conf); + } + + /** + * Tests that when a current incremental backup is found in the history, all later incremental + * backups for the same table are returned. This simulates rolling forward from the current backup + * timestamp, capturing newer incremental backups that depend on it. + */ + @Test + public void testGetAffectedBackupSessions() throws IOException { + BackupInfo current = mock(BackupInfo.class); + TableName table = TableName.valueOf("test_table"); + + when(current.getStartTs()).thenReturn(2000L); + when(current.getBackupId()).thenReturn("backup_002"); + when(current.getBackupRootDir()).thenReturn("/backup/root"); + + BackupInfo b0 = createBackupInfo("backup_000", 500L, BackupType.FULL, table); + BackupInfo b1 = createBackupInfo("backup_001", 1000L, BackupType.INCREMENTAL, table); + BackupInfo b2 = createBackupInfo("backup_002", 2000L, BackupType.INCREMENTAL, table); // current + BackupInfo b3 = createBackupInfo("backup_003", 3000L, BackupType.INCREMENTAL, table); + BackupInfo b4 = createBackupInfo("backup_004", 4000L, BackupType.INCREMENTAL, table); + + when(mockTable.getBackupHistory("/backup/root")).thenReturn(List.of(b4, b3, b2, b1, b0)); + + List result = backupAdminImpl.getAffectedBackupSessions(current, table, mockTable); + + assertEquals(2, result.size()); + assertTrue(result.containsAll(List.of(b3, b4))); + } + + /** + * Tests that if a full backup appears after the current backup, the affected list is reset and + * incremental backups following that full backup are not included. This ensures full backups act + * as a reset boundary. + */ + @Test + public void testGetAffectedBackupSessions_resetsOnFullBackup() throws IOException { + BackupInfo current = mock(BackupInfo.class); + TableName table = TableName.valueOf("test_table"); + + when(current.getStartTs()).thenReturn(1000L); + when(current.getBackupId()).thenReturn("backup_001"); + when(current.getBackupRootDir()).thenReturn("/backup/root"); + + BackupInfo b0 = createBackupInfo("backup_000", 500L, BackupType.FULL, table); + BackupInfo b1 = createBackupInfo("backup_001", 1000L, BackupType.INCREMENTAL, table); // current + BackupInfo b2 = createBackupInfo("backup_002", 2000L, BackupType.FULL, table); + BackupInfo b3 = createBackupInfo("backup_003", 3000L, BackupType.INCREMENTAL, table); + + when(mockTable.getBackupHistory("/backup/root")).thenReturn(List.of(b3, b2, b1, b0)); + + List result = backupAdminImpl.getAffectedBackupSessions(current, table, mockTable); + + assertTrue(result.isEmpty()); + } + + /** + * Tests that backups for other tables are ignored, even if they are incremental and fall after + * the current backup. Only backups affecting the specified table should be considered. + */ + @Test + public void testGetAffectedBackupSessions_skipsNonMatchingTable() throws IOException { + BackupInfo current = mock(BackupInfo.class); + TableName table = TableName.valueOf("test_table"); + + when(current.getStartTs()).thenReturn(1000L); + when(current.getBackupId()).thenReturn("backup_001"); + when(current.getBackupRootDir()).thenReturn("/backup/root"); + + BackupInfo b0 = createBackupInfo("backup_000", 500L, BackupType.FULL, table); + BackupInfo b1 = createBackupInfo("backup_001", 1000L, BackupType.INCREMENTAL, table); // current + BackupInfo b2 = createBackupInfo("backup_002", 2000L, BackupType.INCREMENTAL, table); + BackupInfo b3 = createBackupInfo("backup_003", 3000L, BackupType.INCREMENTAL, + TableName.valueOf("other_table")); + BackupInfo b4 = createBackupInfo("backup_004", 4000L, BackupType.INCREMENTAL, table); + + when(mockTable.getBackupHistory("/backup/root")).thenReturn(List.of(b4, b3, b2, b1, b0)); + + List result = backupAdminImpl.getAffectedBackupSessions(current, table, mockTable); + + assertEquals(2, result.size()); + assertTrue(result.containsAll(List.of(b2, b4))); + } + + /** + * Tests that a full backup for a different table is ignored and does not reset the affected list. + * Only full backups for the same table act as reset boundaries. + */ + @Test + public void testGetAffectedBackupSessions_ignoresFullBackupOfOtherTable() throws IOException { + BackupInfo current = mock(BackupInfo.class); + TableName table = TableName.valueOf("test_table"); + + when(current.getStartTs()).thenReturn(1000L); + when(current.getBackupId()).thenReturn("backup_001"); + when(current.getBackupRootDir()).thenReturn("/backup/root"); + + BackupInfo b0 = createBackupInfo("backup_000", 500L, BackupType.FULL, table); + BackupInfo b1 = createBackupInfo("backup_001", 1000L, BackupType.INCREMENTAL, table); // current + // Full backup for other table - should be ignored + BackupInfo b2 = + createBackupInfo("backup_002", 2000L, BackupType.FULL, TableName.valueOf("other_table")); + BackupInfo b3 = createBackupInfo("backup_003", 3000L, BackupType.INCREMENTAL, table); + BackupInfo b4 = createBackupInfo("backup_004", 4000L, BackupType.INCREMENTAL, table); + + when(mockTable.getBackupHistory("/backup/root")).thenReturn(List.of(b4, b3, b2, b1, b0)); + + List result = backupAdminImpl.getAffectedBackupSessions(current, table, mockTable); + + // Full backup of other table should not reset, so we expect both incremental backups after + // current + assertEquals(2, result.size()); + assertTrue(result.containsAll(List.of(b3, b4))); + } + + private BackupInfo createBackupInfo(String id, long ts, BackupType type, TableName... tables) { + BackupInfo info = mock(BackupInfo.class); + when(info.getBackupId()).thenReturn(id); + when(info.getStartTs()).thenReturn(ts); + when(info.getType()).thenReturn(type); + List tableList = Arrays.asList(tables); + when(info.getTableNames()).thenReturn(tableList); + when(info.getTableListAsString()).thenReturn(tableList.toString()); + return info; + } + + /** + * Tests that when a table is removed from a backup image that still contains other tables, it + * updates the BackupInfo correctly and does not delete the entire backup metadata. + */ + @Test + public void testRemoveTableFromBackupImage() throws IOException { + // Arrange + TableName tableToRemove = TableName.valueOf("ns", "t1"); + TableName remainingTable = TableName.valueOf("ns", "t2"); + + BackupInfo info = new BackupInfo(); + info.setBackupId("backup_001"); + info.setTables(List.of(tableToRemove, remainingTable)); + info.setBackupRootDir("/backup/root"); + + BackupSystemTable sysTable = mock(BackupSystemTable.class); + Configuration conf = new Configuration(); + + Connection mockConn = mock(Connection.class); + when(mockConn.getConfiguration()).thenReturn(conf); + backupAdminImpl = spy(new BackupAdminImpl(mockConn)); + + doNothing().when(backupAdminImpl).cleanupBackupDir(any(), any(), any()); + + try (MockedStatic mockedStatic = mockStatic(BackupUtils.class)) { + mockedStatic.when(() -> BackupUtils.cleanupBackupData(any(), any())) + .thenAnswer(invocation -> null); // no-op for safety + + // Act + backupAdminImpl.removeTableFromBackupImage(info, tableToRemove, sysTable); + + // Assert + assertEquals(1, info.getTableNames().size()); + assertTrue(info.getTableNames().contains(remainingTable)); + + verify(sysTable).updateBackupInfo(info); + verify(sysTable, never()).deleteBackupInfo(any()); + verify(backupAdminImpl).cleanupBackupDir(eq(info), eq(tableToRemove), eq(conf)); + + mockedStatic.verifyNoInteractions(); // should not call static cleanup for partial table + // removal + } + } + + /** + * Tests that when the last table in a backup image is removed, the backup metadata is deleted + * entirely and static cleanup is invoked. + */ + @Test + public void testRemoveTableFromBackupImageDeletesWhenLastTableRemoved() throws IOException { + // Arrange + TableName onlyTable = TableName.valueOf("ns", "t1"); + + BackupInfo info = new BackupInfo(); + info.setBackupId("backup_002"); + info.setTables(new ArrayList<>(List.of(onlyTable))); + info.setBackupRootDir("/backup/root"); + + BackupSystemTable sysTable = mock(BackupSystemTable.class); + Configuration conf = new Configuration(); + + Connection mockConn = mock(Connection.class); + when(mockConn.getConfiguration()).thenReturn(conf); + backupAdminImpl = spy(new BackupAdminImpl(mockConn)); + + doNothing().when(backupAdminImpl).cleanupBackupDir(any(), any(), any()); + + try (MockedStatic mockedStatic = mockStatic(BackupUtils.class)) { + mockedStatic.when(() -> BackupUtils.cleanupBackupData(any(), any())) + .thenAnswer(invocation -> null); // no-op for static void + + // Act + backupAdminImpl.removeTableFromBackupImage(info, onlyTable, sysTable); + + // Assert + verify(sysTable).deleteBackupInfo("backup_002"); + verify(sysTable, never()).updateBackupInfo(any()); + + mockedStatic.verify(() -> BackupUtils.cleanupBackupData(info, conf)); + } + } + + /** + * Tests that when a backup ID is not found, the method logs a warning and returns 0. + */ + @Test + public void testDeleteBackupWhenBackupInfoNotFound() throws IOException { + String backupId = "backup_missing"; + BackupSystemTable sysTable = mock(BackupSystemTable.class); + when(sysTable.readBackupInfo(backupId)).thenReturn(null); + + int result = backupAdminImpl.deleteBackup(backupId, sysTable); + + assertEquals(0, result); + verify(sysTable, never()).deleteBackupInfo(any()); + } + + /** + * Tests deleting a FULL backup when it is the last session for all its tables. Ensures cleanup is + * called and metadata is deleted, but no other backups are affected. + */ + @Test + public void testDeleteFullBackupWithLastSession() throws IOException { + TableName table = TableName.valueOf("ns", "t1"); + String backupId = "backup_full_001"; + BackupInfo info = new BackupInfo(); + info.setBackupId(backupId); + info.setBackupRootDir("/backup/root"); + info.setTables(List.of(table)); + info.setStartTs(1000L); + info.setType(BackupType.FULL); + + BackupSystemTable sysTable = mock(BackupSystemTable.class); + Configuration conf = new Configuration(); + Connection mockConn = mock(Connection.class); + when(mockConn.getConfiguration()).thenReturn(conf); + backupAdminImpl = spy(new BackupAdminImpl(mockConn)); + + when(sysTable.readBackupInfo(backupId)).thenReturn(info); + when(backupAdminImpl.isLastBackupSession(sysTable, table, 1000L)).thenReturn(true); + when(sysTable.readBulkLoadedFiles(backupId)).thenReturn(Map.of()); + + doNothing().when(sysTable).deleteBackupInfo(backupId); + + try (MockedStatic mockedStatic = mockStatic(BackupUtils.class)) { + mockedStatic.when(() -> BackupUtils.cleanupBackupData(eq(info), eq(conf))) + .thenAnswer(inv -> null); + + int result = backupAdminImpl.deleteBackup(backupId, sysTable); + + assertEquals(1, result); + verify(sysTable).deleteBackupInfo(backupId); + mockedStatic.verify(() -> BackupUtils.cleanupBackupData(eq(info), eq(conf))); + } + } + + /** + * Tests that deleteBackup will update other backups by removing the table when it's not the last + * session. + */ + @Test + public void testDeleteBackupWithAffectedSessions() throws IOException { + TableName table = TableName.valueOf("ns", "t1"); + String backupId = "backup_inc_001"; + BackupInfo current = new BackupInfo(); + current.setBackupId(backupId); + current.setBackupRootDir("/backup/root"); + current.setTables(List.of(table)); + current.setStartTs(2000L); + current.setType(BackupType.INCREMENTAL); + + BackupInfo dependent = new BackupInfo(); + dependent.setBackupId("backup_inc_002"); + dependent.setBackupRootDir("/backup/root"); + dependent.setTables(new ArrayList<>(List.of(table))); + dependent.setStartTs(3000L); + dependent.setType(BackupType.INCREMENTAL); + + BackupSystemTable sysTable = mock(BackupSystemTable.class); + Configuration conf = new Configuration(); + Connection mockConn = mock(Connection.class); + when(mockConn.getConfiguration()).thenReturn(conf); + backupAdminImpl = spy(new BackupAdminImpl(mockConn)); + + when(sysTable.readBackupInfo(backupId)).thenReturn(current); + when(sysTable.readBulkLoadedFiles(backupId)).thenReturn(Map.of()); + when(backupAdminImpl.isLastBackupSession(sysTable, table, 2000L)).thenReturn(false); + doReturn(List.of(dependent)).when(backupAdminImpl).getAffectedBackupSessions(current, table, + sysTable); + + doNothing().when(backupAdminImpl).removeTableFromBackupImage(eq(dependent), eq(table), + eq(sysTable)); + + try (MockedStatic mockedStatic = mockStatic(BackupUtils.class)) { + mockedStatic.when(() -> BackupUtils.cleanupBackupData(eq(current), eq(conf))) + .thenAnswer(inv -> null); + + int result = backupAdminImpl.deleteBackup(backupId, sysTable); + + assertEquals(1, result); + verify(backupAdminImpl).removeTableFromBackupImage(dependent, table, sysTable); + verify(sysTable).deleteBackupInfo(backupId); + mockedStatic.verify(() -> BackupUtils.cleanupBackupData(current, conf)); + } + } + + /** + * Tests that deleteBackup will remove bulk-loaded files and handle exceptions gracefully. + */ + @Test + public void testDeleteBackupWithBulkLoadedFiles() throws IOException { + // Set up test data + TableName table = TableName.valueOf("ns", "t1"); + String backupId = "backup_with_bulkload"; + Path dummyPath = new Path("/bulk/load/file1"); + Map bulkFiles = Map.of("k1".getBytes(), dummyPath.toString()); + + // BackupInfo mock + BackupInfo info = new BackupInfo(); + info.setBackupId(backupId); + info.setBackupRootDir("/backup/root"); + info.setTables(List.of(table)); + info.setStartTs(1500L); + info.setType(BackupType.FULL); + + // Create mock objects + Configuration conf = new Configuration(); + Connection conn = mock(Connection.class); + when(conn.getConfiguration()).thenReturn(conf); + + BackupSystemTable sysTable = mock(BackupSystemTable.class); + when(sysTable.readBackupInfo(backupId)).thenReturn(info); + when(sysTable.readBulkLoadedFiles(backupId)).thenReturn(bulkFiles); + + FileSystem fs = mock(FileSystem.class); + + try (MockedStatic fsStatic = mockStatic(FileSystem.class)) { + fsStatic.when(() -> FileSystem.get(conf)).thenReturn(fs); + when(fs.delete(dummyPath)).thenReturn(true); // Simulate successful delete + + // Create spy on BackupAdminImpl + BackupAdminImpl backupAdmin = spy(new BackupAdminImpl(conn)); + when(backupAdmin.isLastBackupSession(sysTable, table, 1500L)).thenReturn(true); + + // No-ops for cleanup + doNothing().when(sysTable).deleteBackupInfo(backupId); + doNothing().when(sysTable).deleteBulkLoadedRows(any()); + + try (MockedStatic staticMock = mockStatic(BackupUtils.class)) { + staticMock.when(() -> BackupUtils.cleanupBackupData(info, conf)) + .thenAnswer(invocation -> null); + + // Execute method + int result = backupAdmin.deleteBackup(backupId, sysTable); + + // Assertions + assertEquals(1, result); + verify(fs).delete(dummyPath); + verify(sysTable).deleteBulkLoadedRows(any()); + verify(sysTable).deleteBackupInfo(backupId); + staticMock.verify(() -> BackupUtils.cleanupBackupData(info, conf), times(1)); + } + } + } + + /** + * Verifies that checkIfValidForMerge succeeds with valid INCREMENTAL, COMPLETE images from the + * same destination and no holes in the backup sequence. + */ + @Test + public void testCheckIfValidForMerge_validCase() throws IOException { + String[] ids = { "b1", "b2" }; + TableName t1 = TableName.valueOf("ns", "t1"); + String dest = "/backup/root"; + + BackupInfo b1 = + createBackupInfo("b1", BackupType.INCREMENTAL, BackupState.COMPLETE, 1000L, dest, t1); + BackupInfo b2 = + createBackupInfo("b2", BackupType.INCREMENTAL, BackupState.COMPLETE, 2000L, dest, t1); + + BackupSystemTable table = mock(BackupSystemTable.class); + when(table.readBackupInfo("b1")).thenReturn(b1); + when(table.readBackupInfo("b2")).thenReturn(b2); + when(table.getBackupHistory(eq(-1), any(), any(), any(), any(), any())) + .thenReturn(List.of(b1, b2)); + + new BackupAdminImpl(mock(Connection.class)).checkIfValidForMerge(ids, table); + } + + /** + * Verifies that checkIfValidForMerge fails if a FULL backup is included. + */ + @Test(expected = IOException.class) + public void testCheckIfValidForMerge_failsWithFullBackup() throws IOException { + String[] ids = { "b1" }; + TableName t1 = TableName.valueOf("ns", "t1"); + + BackupInfo b1 = + createBackupInfo("b1", BackupType.FULL, BackupState.COMPLETE, 1000L, "/dest", t1); + + BackupSystemTable table = mock(BackupSystemTable.class); + when(table.readBackupInfo("b1")).thenReturn(b1); + + new BackupAdminImpl(mock(Connection.class)).checkIfValidForMerge(ids, table); + } + + /** + * Verifies that checkIfValidForMerge fails if one of the provided backup IDs is not found in the + * system table (i.e., null is returned). + */ + @Test(expected = IOException.class) + public void testCheckIfValidForMerge_failsWhenBackupInfoNotFound() throws IOException { + String[] ids = { "b_missing" }; + + BackupSystemTable table = mock(BackupSystemTable.class); + when(table.readBackupInfo("b_missing")).thenReturn(null); + + new BackupAdminImpl(mock(Connection.class)).checkIfValidForMerge(ids, table); + } + + /** + * Verifies that checkIfValidForMerge fails when backups come from different destinations. + */ + @Test(expected = IOException.class) + public void testCheckIfValidForMerge_failsWithDifferentDestinations() throws IOException { + String[] ids = { "b1", "b2" }; + TableName t1 = TableName.valueOf("ns", "t1"); + + BackupInfo b1 = + createBackupInfo("b1", BackupType.INCREMENTAL, BackupState.COMPLETE, 1000L, "/dest1", t1); + BackupInfo b2 = + createBackupInfo("b2", BackupType.INCREMENTAL, BackupState.COMPLETE, 2000L, "/dest2", t1); + + BackupSystemTable table = mock(BackupSystemTable.class); + when(table.readBackupInfo("b1")).thenReturn(b1); + when(table.readBackupInfo("b2")).thenReturn(b2); + + new BackupAdminImpl(mock(Connection.class)).checkIfValidForMerge(ids, table); + } + + /** + * Verifies that checkIfValidForMerge fails if any backup is not in COMPLETE state. + */ + @Test(expected = IOException.class) + public void testCheckIfValidForMerge_failsWithNonCompleteState() throws IOException { + String[] ids = { "b1" }; + TableName t1 = TableName.valueOf("ns", "t1"); + + BackupInfo b1 = + createBackupInfo("b1", BackupType.INCREMENTAL, BackupState.RUNNING, 1000L, "/dest", t1); + + BackupSystemTable table = mock(BackupSystemTable.class); + when(table.readBackupInfo("b1")).thenReturn(b1); + + new BackupAdminImpl(mock(Connection.class)).checkIfValidForMerge(ids, table); + } + + /** + * Verifies that checkIfValidForMerge fails when there is a "hole" in the backup sequence — i.e., + * a required image from the full backup history is missing in the input list. + */ + @Test(expected = IOException.class) + public void testCheckIfValidForMerge_failsWhenHoleInImages() throws IOException { + TableName t1 = TableName.valueOf("ns", "t1"); + String dest = "/backup/root"; + + BackupInfo b1 = + createBackupInfo("b1", BackupType.INCREMENTAL, BackupState.COMPLETE, 1000L, dest, t1); + BackupInfo b2 = + createBackupInfo("b2", BackupType.INCREMENTAL, BackupState.COMPLETE, 2000L, dest, t1); + BackupInfo b3 = + createBackupInfo("b3", BackupType.INCREMENTAL, BackupState.COMPLETE, 3000L, dest, t1); + + BackupSystemTable table = mock(BackupSystemTable.class); + when(table.readBackupInfo("b1")).thenReturn(b1); + when(table.readBackupInfo("b2")).thenReturn(b2); + when(table.readBackupInfo("b3")).thenReturn(b3); + + when(table.getBackupHistory(eq(-1), any(), any(), any(), any(), any())) + .thenReturn(List.of(b1, b2, b3)); + + // Simulate a "hole" by omitting b2 from images + String[] idsWithHole = { "b1", "b3" }; + new BackupAdminImpl(mock(Connection.class)).checkIfValidForMerge(idsWithHole, table); + } + + private BackupInfo createBackupInfo(String id, BackupType type, BackupInfo.BackupState state, + long ts, String dest, TableName... tables) { + BackupInfo info = new BackupInfo(); + info.setBackupId(id); + info.setType(type); + info.setState(state); + info.setStartTs(ts); + info.setBackupRootDir(dest); + info.setTables(List.of(tables)); + return info; + } +} From 176e8c640e5000840232b8102a39b1fbb253c7a1 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Wed, 20 Aug 2025 21:50:00 +0530 Subject: [PATCH 18/24] HBASE-29519 Copy Bulkloaded Files in Continuous Backup (#7222) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Tak Lon (Stephen) Wu Signed-off-by: Andor Molnár --- .../backup/impl/FullTableBackupClient.java | 10 + .../replication/BackupFileSystemManager.java | 11 +- .../backup/replication/BulkLoadProcessor.java | 96 +++++ .../replication/BulkLoadUploadException.java | 32 ++ .../ContinuousBackupReplicationEndpoint.java | 168 ++++++++- .../replication/TestBulkLoadProcessor.java | 166 +++++++++ ...stContinuousBackupReplicationEndpoint.java | 333 +++++++++++++++++- 7 files changed, 801 insertions(+), 15 deletions(-) create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadUploadException.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestBulkLoadProcessor.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java index f431506fc0e4..afba233c9a51 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.backup.impl; +import static org.apache.hadoop.hbase.HConstants.REPLICATION_BULKLOAD_ENABLE_KEY; import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_GLOBAL; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_ATTEMPTS_PAUSE_MS_KEY; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY; @@ -188,6 +189,15 @@ private void handleContinuousBackup(Admin admin) throws IOException { // set overall backup status: complete. Here we make sure to complete the backup. // After this checkpoint, even if entering cancel process, will let the backup finished backupInfo.setState(BackupState.COMPLETE); + + if (!conf.getBoolean(REPLICATION_BULKLOAD_ENABLE_KEY, false)) { + System.out.println("WARNING: Bulkload replication is not enabled. " + + "Since continuous backup is using HBase replication, bulk loaded files won't be backed up as part of continuous backup. " + + "To ensure bulk-loaded files are backed up, enable bulkload replication " + + "(hbase.replication.bulkload.enabled=true) and configure a unique cluster ID using " + + "hbase.replication.cluster.id. This cluster ID is required by the replication framework " + + "to uniquely identify clusters, even if continuous backup itself does not directly rely on it."); + } } private void handleNonContinuousBackup(Admin admin) throws IOException { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java index 9d1d818c207e..225d32172766 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java @@ -26,18 +26,20 @@ import org.slf4j.LoggerFactory; /** - * Initializes and organizes backup directories for continuous Write-Ahead Logs (WALs) files within - * the specified backup root directory. + * Initializes and organizes backup directories for continuous Write-Ahead Logs (WALs) and + * bulk-loaded files within the specified backup root directory. */ @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 final Path walsDir; + private final Path bulkLoadFilesDir; public BackupFileSystemManager(String peerId, Configuration conf, String backupRootDirStr) throws IOException { @@ -45,6 +47,7 @@ public BackupFileSystemManager(String peerId, Configuration conf, String backupR this.backupRootDir = new Path(backupRootDirStr); this.backupFs = FileSystem.get(backupRootDir.toUri(), conf); this.walsDir = createDirectory(WALS_DIR); + this.bulkLoadFilesDir = createDirectory(BULKLOAD_FILES_DIR); } private Path createDirectory(String dirName) throws IOException { @@ -58,6 +61,10 @@ public Path getWalsDir() { return walsDir; } + public Path getBulkLoadFilesDir() { + return bulkLoadFilesDir; + } + public FileSystem getBackupFs() { return backupFs; } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java new file mode 100644 index 000000000000..6e1271313bcd --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java @@ -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. + *

+ * 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. + *

+ * The class is designed for scenarios where replicable bulk load operations need to be parsed and + * their file paths need to be determined programmatically. + *

+ */ +@InterfaceAudience.Private +public final class BulkLoadProcessor { + private BulkLoadProcessor() { + } + + public static List processBulkLoadFiles(List walEntries) throws IOException { + List bulkLoadFilePaths = new ArrayList<>(); + + for (WAL.Entry entry : walEntries) { + WALEdit edit = entry.getEdit(); + for (Cell cell : edit.getCells()) { + if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) { + TableName tableName = entry.getKey().getTableName(); + String namespace = tableName.getNamespaceAsString(); + String table = tableName.getQualifierAsString(); + bulkLoadFilePaths.addAll(processBulkLoadDescriptor(cell, namespace, table)); + } + } + } + return bulkLoadFilePaths; + } + + private static List processBulkLoadDescriptor(Cell cell, String namespace, String table) + throws IOException { + List 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 processStoreDescriptor(WALProtos.StoreDescriptor storeDescriptor, + String namespace, String table, String regionName) { + List 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; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadUploadException.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadUploadException.java new file mode 100644 index 000000000000..91a46c77e319 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadUploadException.java @@ -0,0 +1,32 @@ +/* + * 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.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class BulkLoadUploadException extends IOException { + public BulkLoadUploadException(String message) { + super(message); + } + + public BulkLoadUploadException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java index 2442e0789a8d..69c445c484d8 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.backup.replication; +import com.google.errorprone.annotations.RestrictedApi; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.UncheckedIOException; import java.text.SimpleDateFormat; @@ -32,9 +34,12 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -44,6 +49,7 @@ import org.apache.hadoop.hbase.replication.EmptyEntriesPolicy; import org.apache.hadoop.hbase.replication.ReplicationResult; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface; +import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WAL; @@ -54,8 +60,8 @@ /** * ContinuousBackupReplicationEndpoint is responsible for replicating WAL entries to a backup * storage. It organizes WAL entries by day and periodically flushes the data, ensuring that WAL - * files do not exceed the configured size. The class includes mechanisms for handling the WAL files - * and ensuring that the replication process is safe. + * files do not exceed the configured size. The class includes mechanisms for handling the WAL + * files, performing bulk load backups, and ensuring that the replication process is safe. */ @InterfaceAudience.Private public class ContinuousBackupReplicationEndpoint extends BaseReplicationEndpoint { @@ -302,6 +308,7 @@ private void backupWalEntries(long day, List walEntries) throws IOExc for (WAL.Entry entry : walEntries) { walWriter.append(entry); } + walWriter.sync(true); } catch (UncheckedIOException e) { String errorMsg = Utils.logPeerId(peerId) + " Failed to get or create WAL Writer for " + day; @@ -309,6 +316,17 @@ private void backupWalEntries(long day, List walEntries) throws IOExc e.getMessage(), e); throw new IOException(errorMsg, e); } + + List bulkLoadFiles = BulkLoadProcessor.processBulkLoadFiles(walEntries); + + if (LOG.isTraceEnabled()) { + LOG.trace("{} Processed {} bulk load files for WAL entries", Utils.logPeerId(peerId), + bulkLoadFiles.size()); + LOG.trace("{} Bulk load files: {}", Utils.logPeerId(peerId), + bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); + } + + uploadBulkLoadFiles(day, bulkLoadFiles); } private FSHLogProvider.Writer createWalWriter(long dayInMillis) { @@ -372,15 +390,159 @@ private void close() { } } + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java)") + void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) + throws BulkLoadUploadException { + if (bulkLoadFiles.isEmpty()) { + LOG.debug("{} No bulk load files to upload for {}", Utils.logPeerId(peerId), dayInMillis); + return; + } + + LOG.debug("{} Starting upload of {} bulk load files", Utils.logPeerId(peerId), + bulkLoadFiles.size()); + + if (LOG.isTraceEnabled()) { + LOG.trace("{} Bulk load files to upload: {}", Utils.logPeerId(peerId), + bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); + } + String dayDirectoryName = formatToDateString(dayInMillis); + Path bulkloadDir = new Path(backupFileSystemManager.getBulkLoadFilesDir(), dayDirectoryName); + try { + backupFileSystemManager.getBackupFs().mkdirs(bulkloadDir); + } catch (IOException e) { + throw new BulkLoadUploadException( + String.format("%s Failed to create bulkload directory in backupFS: %s", + Utils.logPeerId(peerId), bulkloadDir), + e); + } + + for (Path file : bulkLoadFiles) { + Path sourcePath; + try { + sourcePath = getBulkLoadFileStagingPath(file); + } catch (FileNotFoundException fnfe) { + throw new BulkLoadUploadException( + String.format("%s Bulk load file not found: %s", Utils.logPeerId(peerId), file), fnfe); + } catch (IOException ioe) { + throw new BulkLoadUploadException( + String.format("%s Failed to resolve source path for: %s", Utils.logPeerId(peerId), file), + ioe); + } + + Path destPath = new Path(bulkloadDir, file); + + try { + LOG.debug("{} Copying bulk load file from {} to {}", Utils.logPeerId(peerId), sourcePath, + destPath); + + copyWithCleanup(CommonFSUtils.getRootDirFileSystem(conf), sourcePath, + backupFileSystemManager.getBackupFs(), destPath, conf); + + LOG.info("{} Bulk load file {} successfully backed up to {}", Utils.logPeerId(peerId), file, + destPath); + } catch (IOException e) { + throw new BulkLoadUploadException( + String.format("%s Failed to copy bulk load file %s to %s on day %s", + Utils.logPeerId(peerId), file, destPath, formatToDateString(dayInMillis)), + e); + } + } + + LOG.debug("{} Completed upload of bulk load files", Utils.logPeerId(peerId)); + } + + /** + * Copy a file with cleanup logic in case of failure. Always overwrite destination to avoid + * leaving corrupt partial files. + */ + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java)") + static void copyWithCleanup(FileSystem srcFS, Path src, FileSystem dstFS, Path dst, + Configuration conf) throws IOException { + try { + if (dstFS.exists(dst)) { + FileStatus srcStatus = srcFS.getFileStatus(src); + FileStatus dstStatus = dstFS.getFileStatus(dst); + + if (srcStatus.getLen() == dstStatus.getLen()) { + LOG.info("Destination file {} already exists with same length ({}). Skipping copy.", dst, + dstStatus.getLen()); + return; // Skip upload + } else { + LOG.warn( + "Destination file {} exists but length differs (src={}, dst={}). " + "Overwriting now.", + dst, srcStatus.getLen(), dstStatus.getLen()); + } + } + + // Always overwrite in case previous copy left partial data + FileUtil.copy(srcFS, src, dstFS, dst, false, true, conf); + } catch (IOException e) { + try { + if (dstFS.exists(dst)) { + dstFS.delete(dst, true); + LOG.warn("Deleted partial/corrupt destination file {} after copy failure", dst); + } + } catch (IOException cleanupEx) { + LOG.warn("Failed to cleanup destination file {} after copy failure", dst, cleanupEx); + } + throw e; + } + } + /** * Convert dayInMillis to "yyyy-MM-dd" format */ - private String formatToDateString(long dayInMillis) { + @RestrictedApi( + explanation = "Package-private for test visibility only. Do not use outside tests.", + link = "", + allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java)") + String formatToDateString(long dayInMillis) { SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); return dateFormat.format(new Date(dayInMillis)); } + private Path getBulkLoadFileStagingPath(Path relativePathFromNamespace) throws IOException { + FileSystem rootFs = CommonFSUtils.getRootDirFileSystem(conf); + Path rootDir = CommonFSUtils.getRootDir(conf); + Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR); + Path baseNamespaceDir = new Path(rootDir, baseNSDir); + Path hFileArchiveDir = + new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir)); + + LOG.debug("{} Searching for bulk load file: {} in paths: {}, {}", Utils.logPeerId(peerId), + relativePathFromNamespace, baseNamespaceDir, hFileArchiveDir); + + Path result = + findExistingPath(rootFs, baseNamespaceDir, hFileArchiveDir, relativePathFromNamespace); + LOG.debug("{} Bulk load file found at {}", Utils.logPeerId(peerId), result); + return result; + } + + private static Path findExistingPath(FileSystem rootFs, Path baseNamespaceDir, + Path hFileArchiveDir, Path filePath) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Checking for bulk load file at: {} and {}", new Path(baseNamespaceDir, filePath), + new Path(hFileArchiveDir, filePath)); + } + + for (Path candidate : new Path[] { new Path(baseNamespaceDir, filePath), + new Path(hFileArchiveDir, filePath) }) { + if (rootFs.exists(candidate)) { + return candidate; + } + } + + throw new FileNotFoundException("Bulk load file not found at either: " + + new Path(baseNamespaceDir, filePath) + " or " + new Path(hFileArchiveDir, filePath)); + } + private void shutdownFlushExecutor() { if (flushExecutor != null) { LOG.info("{} Initiating WAL flush executor shutdown.", Utils.logPeerId(peerId)); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestBulkLoadProcessor.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestBulkLoadProcessor.java new file mode 100644 index 000000000000..9837f9e926d2 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestBulkLoadProcessor.java @@ -0,0 +1,166 @@ +/* + * 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 static org.apache.hadoop.hbase.wal.WALEdit.METAFAMILY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellBuilderFactory; +import org.apache.hadoop.hbase.CellBuilderType; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; + +/** + * Unit tests for {@link BulkLoadProcessor}. + *

+ * These tests validate the extraction of bulk-loaded file paths from WAL entries under different + * scenarios, including: + *

    + *
  • Valid replicable bulk load entries
  • + *
  • Non-replicable bulk load entries
  • + *
  • Entries with no bulk load qualifier
  • + *
  • Entries containing multiple column families
  • + *
+ */ +@Category({ SmallTests.class }) +public class TestBulkLoadProcessor { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBulkLoadProcessor.class); + + /** + * Creates a WAL.Entry containing a {@link WALProtos.BulkLoadDescriptor} with the given + * parameters. + * @param tableName The table name + * @param regionName The encoded region name + * @param replicate Whether the bulk load is marked for replication + * @param family Column family name + * @param storeFiles One or more store file names to include + * @return A WAL.Entry representing the bulk load event + */ + private WAL.Entry createBulkLoadWalEntry(TableName tableName, String regionName, + boolean replicate, String family, String... storeFiles) { + + // Build StoreDescriptor + WALProtos.StoreDescriptor.Builder storeDescBuilder = + WALProtos.StoreDescriptor.newBuilder().setFamilyName(ByteString.copyFromUtf8(family)) + .setStoreHomeDir(family).addAllStoreFile(Arrays.asList(storeFiles)); + + // Build BulkLoadDescriptor + WALProtos.BulkLoadDescriptor.Builder bulkDescBuilder = WALProtos.BulkLoadDescriptor.newBuilder() + .setReplicate(replicate).setEncodedRegionName(ByteString.copyFromUtf8(regionName)) + .setTableName(ProtobufUtil.toProtoTableName(tableName)).setBulkloadSeqNum(1000) // Random + .addStores(storeDescBuilder); + + byte[] value = bulkDescBuilder.build().toByteArray(); + + // Build Cell with BULK_LOAD qualifier + Cell cell = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setType(Cell.Type.Put) + .setRow(new byte[] { 1 }).setFamily(METAFAMILY).setQualifier(WALEdit.BULK_LOAD) + .setValue(value).build(); + + WALEdit edit = new WALEdit(); + edit.add(cell); + + WALKeyImpl key = new WALKeyImpl(Bytes.toBytes(regionName), // region + tableName, 0L, 0L, null); + + return new WAL.Entry(key, edit); + } + + /** + * Verifies that a valid replicable bulk load WAL entry produces the correct number and structure + * of file paths. + */ + @Test + public void testProcessBulkLoadFiles_validEntry() throws IOException { + WAL.Entry entry = createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "region123", true, + "cf1", "file1", "file2"); + + List paths = BulkLoadProcessor.processBulkLoadFiles(Collections.singletonList(entry)); + + assertEquals(2, paths.size()); + assertTrue(paths.get(0).toString().contains("ns/tbl/region123/cf1/file1")); + assertTrue(paths.get(1).toString().contains("ns/tbl/region123/cf1/file2")); + } + + /** + * Verifies that a non-replicable bulk load entry is ignored. + */ + @Test + public void testProcessBulkLoadFiles_nonReplicableSkipped() throws IOException { + WAL.Entry entry = + createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "region123", false, "cf1", "file1"); + + List paths = BulkLoadProcessor.processBulkLoadFiles(Collections.singletonList(entry)); + + assertTrue(paths.isEmpty()); + } + + /** + * Verifies that entries without the BULK_LOAD qualifier are ignored. + */ + @Test + public void testProcessBulkLoadFiles_noBulkLoadQualifier() throws IOException { + WALEdit edit = new WALEdit(); + WALKeyImpl key = new WALKeyImpl(new byte[] {}, TableName.valueOf("ns", "tbl"), 0L, 0L, null); + WAL.Entry entry = new WAL.Entry(key, edit); + + List paths = BulkLoadProcessor.processBulkLoadFiles(Collections.singletonList(entry)); + + assertTrue(paths.isEmpty()); + } + + /** + * Verifies that multiple WAL entries with different column families produce the correct set of + * file paths. + */ + @Test + public void testProcessBulkLoadFiles_multipleFamilies() throws IOException { + WAL.Entry entry = + createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "regionXYZ", true, "cf1", "file1"); + WAL.Entry entry2 = + createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "regionXYZ", true, "cf2", "fileA"); + + List paths = BulkLoadProcessor.processBulkLoadFiles(Arrays.asList(entry, entry2)); + + assertEquals(2, paths.size()); + assertTrue(paths.stream().anyMatch(p -> p.toString().contains("cf1/file1"))); + assertTrue(paths.stream().anyMatch(p -> p.toString().contains("cf2/fileA"))); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java index 3919746d3b7b..8f8e83dbda6b 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java @@ -17,7 +17,9 @@ */ package org.apache.hadoop.hbase.backup.replication; +import static org.apache.hadoop.hbase.HConstants.REPLICATION_BULKLOAD_ENABLE_KEY; import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_MAX_WAL_SIZE; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_ROOT_DIR; @@ -27,11 +29,18 @@ import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.WAL_FILE_PREFIX; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.copyWithCleanup; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.IOException; import java.text.SimpleDateFormat; @@ -46,8 +55,10 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; @@ -64,8 +75,11 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.tool.BulkLoadHFiles; +import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; +import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; @@ -73,6 +87,7 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.mockito.MockedStatic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,12 +106,14 @@ public class TestContinuousBackupReplicationEndpoint { private final String replicationEndpoint = ContinuousBackupReplicationEndpoint.class.getName(); private static final String CF_NAME = "cf"; + private static final byte[] QUALIFIER = Bytes.toBytes("my-qualifier"); static FileSystem fs = null; static Path root; @BeforeClass public static void setUpBeforeClass() throws Exception { // Set the configuration properties as required + conf.setBoolean(REPLICATION_BULKLOAD_ENABLE_KEY, true); conf.set(REPLICATION_CLUSTER_ID, "clusterId1"); TEST_UTIL.startMiniZKCluster(); @@ -115,7 +132,7 @@ public static void tearDownAfterClass() throws Exception { } @Test - public void testWALBackup() throws IOException { + public void testWALAndBulkLoadFileBackup() throws IOException { String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); TableName tableName = TableName.valueOf("table_" + methodName); String peerId = "peerId"; @@ -133,10 +150,15 @@ public void testWALBackup() throws IOException { loadRandomData(tableName, 100); assertEquals(100, getRowCount(tableName)); + Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoadByFamily"); + generateHFiles(dir); + bulkLoadHFiles(tableName, dir); + assertEquals(1100, getRowCount(tableName)); + waitForReplication(15000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), Map.of(tableName, 100)); + verifyBackup(backupRootDir.toString(), true, Map.of(tableName, 1100)); deleteTable(tableName); } @@ -184,7 +206,7 @@ public void testMultiTableWALBackup() throws IOException { waitForReplication(15000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), Map.of(table1, 100, table2, 100, table3, 50)); + verifyBackup(backupRootDir.toString(), false, Map.of(table1, 100, table2, 100, table3, 50)); for (TableName table : List.of(table1, table2, table3)) { deleteTable(table); @@ -242,7 +264,7 @@ public void testWALBackupWithPeerRestart() throws IOException, InterruptedExcept waitForReplication(20000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), Map.of(tableName, getRowCount(tableName))); + verifyBackup(backupRootDir.toString(), false, Map.of(tableName, getRowCount(tableName))); deleteTable(tableName); } @@ -289,7 +311,7 @@ public void testDayWiseWALBackup() throws IOException { waitForReplication(15000); deleteReplicationPeer(peerId); - verifyBackup(backupRootDir.toString(), Map.of(tableName, 200)); + verifyBackup(backupRootDir.toString(), false, Map.of(tableName, 200)); // Verify that WALs are stored in two directories, one for each day Path walDir = new Path(backupRootDir, WALS_DIR); @@ -312,6 +334,204 @@ public void testDayWiseWALBackup() throws IOException { deleteTable(tableName); } + /** + * Simulates a one-time failure during bulk load file upload. This validates that the retry logic + * in the replication endpoint works as expected. + */ + @Test + public void testBulkLoadFileUploadRetry() throws IOException { + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + String peerId = "peerId"; + + // Reset static failure flag before test + FailingOnceContinuousBackupReplicationEndpoint.reset(); + + createTable(tableName); + + Path backupRootDir = new Path(root, methodName); + fs.mkdirs(backupRootDir); + + Map> tableMap = new HashMap<>(); + tableMap.put(tableName, new ArrayList<>()); + + addReplicationPeer(peerId, backupRootDir, tableMap, + FailingOnceContinuousBackupReplicationEndpoint.class.getName()); + + loadRandomData(tableName, 100); + assertEquals(100, getRowCount(tableName)); + + Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoadByFamily"); + generateHFiles(dir); + bulkLoadHFiles(tableName, dir); + assertEquals(1100, getRowCount(tableName)); + + // Replication: first attempt fails, second attempt succeeds + waitForReplication(15000); + deleteReplicationPeer(peerId); + + verifyBackup(backupRootDir.toString(), true, Map.of(tableName, 1100)); + + deleteTable(tableName); + } + + /** + * Replication endpoint that fails only once on first upload attempt, then succeeds on retry. + */ + public static class FailingOnceContinuousBackupReplicationEndpoint + extends ContinuousBackupReplicationEndpoint { + + private static boolean failedOnce = false; + + @Override + protected void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) + throws BulkLoadUploadException { + if (!failedOnce) { + failedOnce = true; + throw new BulkLoadUploadException("Simulated upload failure on first attempt"); + } + super.uploadBulkLoadFiles(dayInMillis, bulkLoadFiles); + } + + /** Reset failure state for new tests */ + public static void reset() { + failedOnce = false; + } + } + + /** + * Unit test for verifying cleanup of partial files. Simulates a failure during + * {@link FileUtil#copy(FileSystem, Path, FileSystem, Path, boolean, boolean, Configuration)} and + * checks that the destination file is deleted. + */ + @Test + public void testCopyWithCleanupDeletesPartialFile() throws Exception { + FileSystem srcFS = mock(FileSystem.class); + FileSystem dstFS = mock(FileSystem.class); + Path src = new Path("/src/file"); + Path dst = new Path("/dst/file"); + Configuration conf = new Configuration(); + + FileStatus srcStatus = mock(FileStatus.class); + FileStatus dstStatus = mock(FileStatus.class); + + when(srcFS.getFileStatus(src)).thenReturn(srcStatus); + when(dstFS.getFileStatus(dst)).thenReturn(dstStatus); + + // lengths differ -> should attempt to overwrite and then cleanup + when(srcStatus.getLen()).thenReturn(200L); + when(dstStatus.getLen()).thenReturn(100L); + + // Simulate FileUtil.copy failing + try (MockedStatic mockedFileUtil = mockStatic(FileUtil.class)) { + mockedFileUtil.when( + () -> FileUtil.copy(eq(srcFS), eq(src), eq(dstFS), eq(dst), eq(false), eq(true), eq(conf))) + .thenThrow(new IOException("simulated copy failure")); + + // Pretend partial file exists in destination + when(dstFS.exists(dst)).thenReturn(true); + + // Run the method under test + assertThrows(IOException.class, () -> copyWithCleanup(srcFS, src, dstFS, dst, conf)); + + // Verify cleanup happened + verify(dstFS).delete(dst, true); + } + } + + /** + * Simulates a stale/partial file left behind after a failed bulk load. On retry, the stale file + * should be overwritten and replication succeeds. + */ + @Test + public void testBulkLoadFileUploadWithStaleFileRetry() throws Exception { + String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); + TableName tableName = TableName.valueOf("table_" + methodName); + String peerId = "peerId"; + + // Reset static failure flag before test + PartiallyUploadedBulkloadFileEndpoint.reset(); + + createTable(tableName); + + Path backupRootDir = new Path(root, methodName); + fs.mkdirs(backupRootDir); + conf.set(CONF_BACKUP_ROOT_DIR, backupRootDir.toString()); + + Map> tableMap = new HashMap<>(); + tableMap.put(tableName, new ArrayList<>()); + + addReplicationPeer(peerId, backupRootDir, tableMap, + PartiallyUploadedBulkloadFileEndpoint.class.getName()); + + loadRandomData(tableName, 100); + assertEquals(100, getRowCount(tableName)); + + Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoadByFamily"); + generateHFiles(dir); + bulkLoadHFiles(tableName, dir); + assertEquals(1100, getRowCount(tableName)); + + // first attempt will fail leaving stale file, second attempt should overwrite and succeed + waitForReplication(15000); + deleteReplicationPeer(peerId); + + verifyBackup(backupRootDir.toString(), true, Map.of(tableName, 1100)); + + deleteTable(tableName); + } + + /** + * Replication endpoint that simulates leaving a partial file behind on first attempt, then + * succeeds on second attempt by overwriting it. + */ + public static class PartiallyUploadedBulkloadFileEndpoint + extends ContinuousBackupReplicationEndpoint { + + private static boolean firstAttempt = true; + + @Override + protected void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) + throws BulkLoadUploadException { + if (firstAttempt) { + firstAttempt = false; + try { + // Construct destination path and create a partial file + String dayDirectoryName = formatToDateString(dayInMillis); + BackupFileSystemManager backupFileSystemManager = + new BackupFileSystemManager("peer1", conf, conf.get(CONF_BACKUP_ROOT_DIR)); + Path bulkloadDir = + new Path(backupFileSystemManager.getBulkLoadFilesDir(), dayDirectoryName); + + FileSystem dstFs = backupFileSystemManager.getBackupFs(); + if (!dstFs.exists(bulkloadDir)) { + dstFs.mkdirs(bulkloadDir); + } + + for (Path file : bulkLoadFiles) { + Path destPath = new Path(bulkloadDir, file); + try (FSDataOutputStream out = dstFs.create(destPath, true)) { + out.writeBytes("partial-data"); // simulate incomplete upload + } + } + } catch (IOException e) { + throw new BulkLoadUploadException("Simulated failure while creating partial file", e); + } + + // Fail after leaving partial files behind + throw new BulkLoadUploadException("Simulated upload failure on first attempt"); + } + + // Retry succeeds, overwriting stale files + super.uploadBulkLoadFiles(dayInMillis, bulkLoadFiles); + } + + /** Reset for new tests */ + public static void reset() { + firstAttempt = true; + } + } + private void createTable(TableName tableName) throws IOException { ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CF_NAME)).setScope(1).build(); @@ -332,6 +552,12 @@ private void deleteTable(TableName tableName) throws IOException { private void addReplicationPeer(String peerId, Path backupRootDir, Map> tableMap) throws IOException { + addReplicationPeer(peerId, backupRootDir, tableMap, replicationEndpoint); + } + + private void addReplicationPeer(String peerId, Path backupRootDir, + Map> tableMap, String customReplicationEndpointImpl) + throws IOException { Map additionalArgs = new HashMap<>(); additionalArgs.put(CONF_PEER_UUID, UUID.randomUUID().toString()); additionalArgs.put(CONF_BACKUP_ROOT_DIR, backupRootDir.toString()); @@ -340,7 +566,7 @@ private void addReplicationPeer(String peerId, Path backupRootDir, additionalArgs.put(CONF_STAGED_WAL_FLUSH_INTERVAL, "10"); ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() - .setReplicationEndpointImpl(replicationEndpoint).setReplicateAllUserTables(false) + .setReplicationEndpointImpl(customReplicationEndpointImpl).setReplicateAllUserTables(false) .setTableCFsMap(tableMap).putAllConfiguration(additionalArgs).build(); admin.addReplicationPeer(peerId, peerConfig); @@ -358,6 +584,42 @@ private void loadRandomData(TableName tableName, int totalRows) throws IOExcepti } } + private void bulkLoadHFiles(TableName tableName, Path inputDir) throws IOException { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true); + + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + BulkLoadHFiles loader = new BulkLoadHFilesTool(TEST_UTIL.getConfiguration()); + loader.bulkLoad(table.getName(), inputDir); + } finally { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false); + } + } + + private void bulkLoadHFiles(TableName tableName, Map> family2Files) + throws IOException { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true); + + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + BulkLoadHFiles loader = new BulkLoadHFilesTool(TEST_UTIL.getConfiguration()); + loader.bulkLoad(table.getName(), family2Files); + } finally { + TEST_UTIL.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false); + } + } + + private void generateHFiles(Path outputDir) throws IOException { + String hFileName = "MyHFile"; + int numRows = 1000; + outputDir = outputDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()); + + byte[] from = Bytes.toBytes(CF_NAME + "begin"); + byte[] to = Bytes.toBytes(CF_NAME + "end"); + + Path familyDir = new Path(outputDir, CF_NAME); + HFileTestUtil.createHFile(TEST_UTIL.getConfiguration(), fs, new Path(familyDir, hFileName), + Bytes.toBytes(CF_NAME), QUALIFIER, from, to, numRows); + } + private void waitForReplication(int durationInMillis) { LOG.info("Waiting for replication to complete for {} ms", durationInMillis); try { @@ -370,12 +632,17 @@ private void waitForReplication(int durationInMillis) { /** * Verifies the backup process by: 1. Checking whether any WAL (Write-Ahead Log) files were - * generated in the backup directory. 2. Replaying the WAL files to restore data and check - * consistency by verifying that the restored data matches the expected row count for each table. + * generated in the backup directory. 2. Checking whether any bulk-loaded files were generated in + * the backup directory. 3. Replaying the WAL and bulk-loaded files (if present) to restore data + * and check consistency by verifying that the restored data matches the expected row count for + * each table. */ - private void verifyBackup(String backupRootDir, Map tablesWithExpectedRows) - throws IOException { + private void verifyBackup(String backupRootDir, boolean hasBulkLoadFiles, + Map tablesWithExpectedRows) throws IOException { verifyWALBackup(backupRootDir); + if (hasBulkLoadFiles) { + verifyBulkLoadBackup(backupRootDir); + } for (Map.Entry entry : tablesWithExpectedRows.entrySet()) { TableName tableName = entry.getKey(); @@ -387,6 +654,21 @@ private void verifyBackup(String backupRootDir, Map tablesWi replayWALs(new Path(backupRootDir, WALS_DIR).toString(), tableName); + // replay Bulk loaded HFiles if Present + try { + Path bulkloadDir = new Path(backupRootDir, BULKLOAD_FILES_DIR); + if (fs.exists(bulkloadDir)) { + FileStatus[] directories = fs.listStatus(bulkloadDir); + for (FileStatus dirStatus : directories) { + if (dirStatus.isDirectory()) { + replayBulkLoadHFilesIfPresent(dirStatus.getPath().toString(), tableName); + } + } + } + } catch (Exception e) { + fail("Failed to replay BulkLoad HFiles properly: " + e.getMessage()); + } + assertEquals(expectedRows, getRowCount(tableName)); } } @@ -412,6 +694,15 @@ private void verifyWALBackup(String backupRootDir) throws IOException { assertFalse("Expected some WAL files but found none!", walFiles.isEmpty()); } + private void verifyBulkLoadBackup(String backupRootDir) throws IOException { + Path bulkLoadFilesDir = new Path(backupRootDir, BULKLOAD_FILES_DIR); + assertTrue("BulkLoad Files directory does not exist!", fs.exists(bulkLoadFilesDir)); + + FileStatus[] bulkLoadFiles = fs.listStatus(bulkLoadFilesDir); + assertNotNull("No Bulk load files found!", bulkLoadFiles); + assertTrue("Expected some Bulk load files but found none!", bulkLoadFiles.length > 0); + } + private void replayWALs(String walDir, TableName tableName) { WALPlayer player = new WALPlayer(); try { @@ -422,6 +713,28 @@ private void replayWALs(String walDir, TableName tableName) { } } + private void replayBulkLoadHFilesIfPresent(String bulkLoadDir, TableName tableName) { + try { + Path tableBulkLoadDir = new Path(bulkLoadDir + "/default/" + tableName); + if (fs.exists(tableBulkLoadDir)) { + RemoteIterator fileStatusIterator = fs.listFiles(tableBulkLoadDir, true); + List bulkLoadFiles = new ArrayList<>(); + + while (fileStatusIterator.hasNext()) { + LocatedFileStatus fileStatus = fileStatusIterator.next(); + Path filePath = fileStatus.getPath(); + + if (!fileStatus.isDirectory()) { + bulkLoadFiles.add(filePath); + } + } + bulkLoadHFiles(tableName, Map.of(Bytes.toBytes(CF_NAME), bulkLoadFiles)); + } + } catch (Exception e) { + fail("Failed to replay BulkLoad HFiles properly: " + e.getMessage()); + } + } + private int getRowCount(TableName tableName) throws IOException { try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { return HBaseTestingUtil.countRows(table); From 5d815b82b5655a9f2aa9d39b9bcd5b601ba914cd Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Wed, 27 Aug 2025 03:55:29 +0530 Subject: [PATCH 19/24] HBASE-29524 Handle bulk-loaded HFiles in delete and cleanup process (#7239) Signed-off-by: Tak Lon (Stephen) Wu Reviewed by: Kota-SH --- .../hbase/backup/impl/BackupCommands.java | 16 +++++++++++++++- .../backup/TestBackupDeleteWithCleanup.java | 19 ++++++++++++++++--- .../hbase/backup/impl/TestBackupCommands.java | 7 ++++++- 3 files changed, 37 insertions(+), 5 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 3ae97c487ef3..0b02a5edd89d 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -1004,6 +1004,7 @@ private void deleteAllBackupWALFiles(Configuration conf, String backupWalDir) new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); FileSystem fs = manager.getBackupFs(); Path walDir = manager.getWalsDir(); + Path bulkloadDir = manager.getBulkLoadFilesDir(); // Delete contents under WAL directory if (fs.exists(walDir)) { @@ -1014,6 +1015,15 @@ private void deleteAllBackupWALFiles(Configuration conf, String backupWalDir) System.out.println("Deleted all contents under WAL directory: " + walDir); } + // Delete contents under bulk load directory + if (fs.exists(bulkloadDir)) { + FileStatus[] bulkContents = fs.listStatus(bulkloadDir); + for (FileStatus item : bulkContents) { + fs.delete(item.getPath(), true); // recursive delete of each child + } + System.out.println("Deleted all contents under Bulk Load directory: " + bulkloadDir); + } + } catch (IOException e) { System.out.println("WARNING: Failed to delete contents under backup directories: " + backupWalDir + ". Error: " + e.getMessage()); @@ -1022,7 +1032,7 @@ private void deleteAllBackupWALFiles(Configuration conf, String backupWalDir) } /** - * Cleans up old WAL files based on the determined cutoff timestamp. + * Cleans up old WAL and bulk-loaded files based on the determined cutoff timestamp. */ void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) throws IOException { @@ -1033,6 +1043,7 @@ void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); FileSystem fs = manager.getBackupFs(); Path walDir = manager.getWalsDir(); + Path bulkloadDir = manager.getBulkLoadFilesDir(); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); @@ -1058,6 +1069,9 @@ void deleteOldWALFiles(Configuration conf, String backupWalDir, long cutoffTime) if (dayStart + ONE_DAY_IN_MILLISECONDS - 1 < cutoffTime) { System.out.println("Deleting outdated WAL directory: " + dirPath); fs.delete(dirPath, true); + Path bulkloadPath = new Path(bulkloadDir, dirName); + System.out.println("Deleting corresponding bulk-load directory: " + bulkloadPath); + fs.delete(bulkloadPath, true); } } catch (ParseException e) { System.out.println("WARNING: Failed to parse directory name '" + dirName diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java index d22f4c9cda98..07c9110072b2 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java @@ -19,6 +19,7 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; @@ -164,7 +165,7 @@ public void testSingleBackupForceDelete() throws Exception { // Step 6: Verify that the backup WAL directory is empty assertTrue("WAL backup directory should be empty after force delete", - isWalDirsEmpty(conf1, backupWalDir.toString())); + areWalAndBulkloadDirsEmpty(conf1, backupWalDir.toString())); // Step 7: Take new full backup with continuous backup enabled String backupIdContinuous = fullTableBackupWithContinuous(Lists.newArrayList(table1)); @@ -189,28 +190,35 @@ private void setupBackupFolders(long currentTime) throws IOException { public static void setupBackupFolders(FileSystem fs, Path backupWalDir, long currentTime) throws IOException { Path walsDir = new Path(backupWalDir, WALS_DIR); + Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); fs.mkdirs(walsDir); + fs.mkdirs(bulkLoadDir); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); for (int i = 0; i < 5; i++) { String dateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); fs.mkdirs(new Path(walsDir, dateStr)); + fs.mkdirs(new Path(bulkLoadDir, dateStr)); } } private static void verifyBackupCleanup(FileSystem fs, Path backupWalDir, long currentTime) throws IOException { Path walsDir = new Path(backupWalDir, WALS_DIR); + Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); // Expect folders older than 3 days to be deleted for (int i = 3; i < 5; i++) { String oldDateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); Path walPath = new Path(walsDir, oldDateStr); + Path bulkLoadPath = new Path(bulkLoadDir, oldDateStr); assertFalse("Old WAL directory (" + walPath + ") should be deleted, but it exists!", fs.exists(walPath)); + assertFalse("Old BulkLoad directory (" + bulkLoadPath + ") should be deleted, but it exists!", + fs.exists(bulkLoadPath)); } // Expect folders within the last 3 days to exist @@ -218,9 +226,13 @@ private static void verifyBackupCleanup(FileSystem fs, Path backupWalDir, long c String recentDateStr = dateFormat.format(new Date(currentTime - (i * ONE_DAY_IN_MILLISECONDS))); Path walPath = new Path(walsDir, recentDateStr); + Path bulkLoadPath = new Path(bulkLoadDir, recentDateStr); assertTrue("Recent WAL directory (" + walPath + ") should exist, but it is missing!", fs.exists(walPath)); + assertTrue( + "Recent BulkLoad directory (" + bulkLoadPath + ") should exist, but it is missing!", + fs.exists(bulkLoadPath)); } } @@ -264,15 +276,16 @@ private boolean continuousBackupReplicationPeerExistsAndEnabled() throws IOExcep peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER) && peer.isEnabled()); } - private static boolean isWalDirsEmpty(Configuration conf, String backupWalDir) + private static boolean areWalAndBulkloadDirsEmpty(Configuration conf, String backupWalDir) throws IOException { BackupFileSystemManager manager = new BackupFileSystemManager(CONTINUOUS_BACKUP_REPLICATION_PEER, conf, backupWalDir); FileSystem fs = manager.getBackupFs(); Path walDir = manager.getWalsDir(); + Path bulkloadDir = manager.getBulkLoadFilesDir(); - return isDirectoryEmpty(fs, walDir); + return isDirectoryEmpty(fs, walDir) && isDirectoryEmpty(fs, bulkloadDir); } private static boolean isDirectoryEmpty(FileSystem fs, Path dirPath) throws IOException { diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java index e00ebd6099f5..be14227e4ccb 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.logDirectoryStructure; import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.setupBackupFolders; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; @@ -134,7 +135,7 @@ public void testDeleteOldWALFilesOfCleanupCommand() throws IOException { fs.mkdirs(backupWalDir); long currentTime = EnvironmentEdgeManager.getDelegate().currentTime(); - setupBackupFolders(fs, backupWalDir, currentTime); // Create 5 days of WALs folders + setupBackupFolders(fs, backupWalDir, currentTime); // Create 5 days of WAL/bulkload-files folder logDirectoryStructure(fs, backupWalDir, "Before cleanup:"); @@ -154,6 +155,7 @@ public void testDeleteOldWALFilesOfCleanupCommand() throws IOException { private static void verifyCleanupOutcome(FileSystem fs, Path backupWalDir, long currentTime, long cutoffTime) throws IOException { Path walsDir = new Path(backupWalDir, WALS_DIR); + Path bulkLoadDir = new Path(backupWalDir, BULKLOAD_FILES_DIR); SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); @@ -161,11 +163,14 @@ private static void verifyCleanupOutcome(FileSystem fs, Path backupWalDir, long long dayTime = currentTime - (i * ONE_DAY_IN_MILLISECONDS); String dayDir = dateFormat.format(new Date(dayTime)); Path walPath = new Path(walsDir, dayDir); + Path bulkPath = new Path(bulkLoadDir, dayDir); if (dayTime + ONE_DAY_IN_MILLISECONDS - 1 < cutoffTime) { assertFalse("Old WAL dir should be deleted: " + walPath, fs.exists(walPath)); + assertFalse("Old BulkLoad dir should be deleted: " + bulkPath, fs.exists(bulkPath)); } else { assertTrue("Recent WAL dir should exist: " + walPath, fs.exists(walPath)); + assertTrue("Recent BulkLoad dir should exist: " + bulkPath, fs.exists(bulkPath)); } } } From 29c228a85a21e5faae0235057dd90f12dba5d58b Mon Sep 17 00:00:00 2001 From: asolomon Date: Tue, 9 Sep 2025 03:18:01 +0530 Subject: [PATCH 20/24] [HBASE-29520] Utilize Backed-up Bulkloaded Files in Incremental Backup (#7246) Signed-off-by: Tak Lon (Stephen) Wu --- .../impl/AbstractPitrRestoreHandler.java | 2 +- .../hbase/backup/impl/BackupCommands.java | 2 +- .../impl/IncrementalTableBackupClient.java | 23 ++++++- .../ContinuousBackupReplicationEndpoint.java | 24 ++----- .../hadoop/hbase/backup/util/BackupUtils.java | 13 ++++ .../hadoop/hbase/backup/TestBackupBase.java | 13 ++++ .../backup/TestBackupDeleteWithCleanup.java | 2 +- .../hbase/backup/TestContinuousBackup.java | 6 -- .../TestIncrementalBackupWithContinuous.java | 67 ++++++++++--------- .../hbase/backup/impl/TestBackupCommands.java | 2 +- ...stContinuousBackupReplicationEndpoint.java | 5 +- 11 files changed, 95 insertions(+), 64 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java index 8072277bf684..048ed882fe8d 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java @@ -21,8 +21,8 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES; import java.io.IOException; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index 0b02a5edd89d..a30530a98fc6 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -49,8 +49,8 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_QUEUE_NAME_DESC; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import java.io.IOException; import java.net.URI; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index 4d39d11a36d7..8a893994616c 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -19,9 +19,10 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; +import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import java.io.IOException; import java.net.URI; @@ -170,6 +171,26 @@ protected List handleBulkLoad(List tablesToBackup) throws I Path tblDir = CommonFSUtils.getTableDir(rootdir, srcTable); Path p = new Path(tblDir, regionName + Path.SEPARATOR + fam + Path.SEPARATOR + filename); + // For continuous backup: bulkload files are copied from backup directory defined by + // CONF_CONTINUOUS_BACKUP_WAL_DIR instead of source cluster. + String backupRootDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + if (backupInfo.isContinuousBackupEnabled() && !Strings.isNullOrEmpty(backupRootDir)) { + String dayDirectoryName = BackupUtils.formatToDateString(bulkLoad.getTimestamp()); + Path bulkLoadBackupPath = + new Path(backupRootDir, BULKLOAD_FILES_DIR + Path.SEPARATOR + dayDirectoryName); + Path bulkLoadDir = new Path(bulkLoadBackupPath, + srcTable.getNamespaceAsString() + Path.SEPARATOR + srcTable.getNameAsString()); + FileSystem backupFs = FileSystem.get(bulkLoadDir.toUri(), conf); + Path fullBulkLoadBackupPath = + new Path(bulkLoadDir, regionName + Path.SEPARATOR + fam + Path.SEPARATOR + filename); + if (backupFs.exists(fullBulkLoadBackupPath)) { + LOG.debug("Backup bulkload file found {}", fullBulkLoadBackupPath); + p = fullBulkLoadBackupPath; + } else { + LOG.warn("Backup bulkload file not found {}", fullBulkLoadBackupPath); + } + } + String srcTableQualifier = srcTable.getQualifierAsString(); String srcTableNs = srcTable.getNamespaceAsString(); Path tgtFam = new Path(tgtRoot, srcTableNs + Path.SEPARATOR + srcTableQualifier diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java index 69c445c484d8..19624d04c23d 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -21,11 +21,8 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.UncheckedIOException; -import java.text.SimpleDateFormat; -import java.util.Date; import java.util.List; import java.util.Map; -import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; @@ -41,6 +38,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; @@ -94,7 +92,6 @@ public class ContinuousBackupReplicationEndpoint extends BaseReplicationEndpoint public static final long ONE_DAY_IN_MILLISECONDS = TimeUnit.DAYS.toMillis(1); public static final String WAL_FILE_PREFIX = "wal_file."; - public static final String DATE_FORMAT = "yyyy-MM-dd"; @Override public void init(Context context) throws IOException { @@ -330,7 +327,7 @@ private void backupWalEntries(long day, List walEntries) throws IOExc } private FSHLogProvider.Writer createWalWriter(long dayInMillis) { - String dayDirectoryName = formatToDateString(dayInMillis); + String dayDirectoryName = BackupUtils.formatToDateString(dayInMillis); FileSystem fs = backupFileSystemManager.getBackupFs(); Path walsDir = backupFileSystemManager.getWalsDir(); @@ -408,7 +405,7 @@ void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) LOG.trace("{} Bulk load files to upload: {}", Utils.logPeerId(peerId), bulkLoadFiles.stream().map(Path::toString).collect(Collectors.joining(", "))); } - String dayDirectoryName = formatToDateString(dayInMillis); + String dayDirectoryName = BackupUtils.formatToDateString(dayInMillis); Path bulkloadDir = new Path(backupFileSystemManager.getBulkLoadFilesDir(), dayDirectoryName); try { backupFileSystemManager.getBackupFs().mkdirs(bulkloadDir); @@ -446,7 +443,7 @@ void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) } catch (IOException e) { throw new BulkLoadUploadException( String.format("%s Failed to copy bulk load file %s to %s on day %s", - Utils.logPeerId(peerId), file, destPath, formatToDateString(dayInMillis)), + Utils.logPeerId(peerId), file, destPath, BackupUtils.formatToDateString(dayInMillis)), e); } } @@ -495,19 +492,6 @@ static void copyWithCleanup(FileSystem srcFS, Path src, FileSystem dstFS, Path d } } - /** - * Convert dayInMillis to "yyyy-MM-dd" format - */ - @RestrictedApi( - explanation = "Package-private for test visibility only. Do not use outside tests.", - link = "", - allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java)") - String formatToDateString(long dayInMillis) { - SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); - dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); - return dateFormat.format(new Date(dayInMillis)); - } - private Path getBulkLoadFileStagingPath(Path relativePathFromNamespace) throws IOException { FileSystem rootFs = CommonFSUtils.getRootDirFileSystem(conf); Path rootDir = CommonFSUtils.getRootDir(conf); diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java index d8a033a1f077..f76fa7353219 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java @@ -24,14 +24,17 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.URLDecoder; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.Date; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.TimeZone; import java.util.TreeMap; import java.util.TreeSet; import org.apache.hadoop.conf.Configuration; @@ -88,6 +91,7 @@ public final class BackupUtils { private static final Logger LOG = LoggerFactory.getLogger(BackupUtils.class); public static final String LOGNAME_SEPARATOR = "."; public static final int MILLISEC_IN_HOUR = 3600000; + public static final String DATE_FORMAT = "yyyy-MM-dd"; private BackupUtils() { throw new AssertionError("Instantiating utility class..."); @@ -983,4 +987,13 @@ private static boolean continuousBackupReplicationPeerExists(Admin admin) throws return admin.listReplicationPeers().stream() .anyMatch(peer -> peer.getPeerId().equals(CONTINUOUS_BACKUP_REPLICATION_PEER)); } + + /** + * Convert dayInMillis to "yyyy-MM-dd" format + */ + public static String formatToDateString(long dayInMillis) { + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); + return dateFormat.format(new Date(dayInMillis)); + } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index 4fb9f209b760..aae88c1bb9a2 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -17,10 +17,13 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.HConstants.REPLICATION_BULKLOAD_ENABLE_KEY; +import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_MAX_WAL_SIZE; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INITIAL_DELAY; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INTERVAL; +import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES; import java.io.IOException; import java.util.ArrayList; @@ -45,6 +48,7 @@ import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupManager; +import org.apache.hadoop.hbase.backup.impl.BackupManifest; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient; import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager; @@ -299,6 +303,9 @@ public static void setUpHelper() throws Exception { conf1.set(CONF_BACKUP_MAX_WAL_SIZE, "10240"); conf1.set(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, "10"); conf1.set(CONF_STAGED_WAL_FLUSH_INTERVAL, "10"); + conf1.setBoolean(REPLICATION_BULKLOAD_ENABLE_KEY, true); + conf1.set(REPLICATION_CLUSTER_ID, "clusterId1"); + conf1.setBoolean(IGNORE_EMPTY_FILES, true); if (secure) { // set the always on security provider @@ -566,6 +573,12 @@ protected void dumpBackupDir() throws IOException { } } + BackupManifest getLatestBackupManifest(List backups) throws IOException { + BackupInfo newestBackup = backups.get(0); + return HBackupFileSystem.getManifest(conf1, new Path(BACKUP_ROOT_DIR), + newestBackup.getBackupId()); + } + void deleteContinuousBackupReplicationPeerIfExists(Admin admin) throws IOException { if ( admin.listReplicationPeers().stream() diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java index 07c9110072b2..8bd2fe4cc78c 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java @@ -21,8 +21,8 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java index 0cc34ed63eb0..2fdfa8b73f8b 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestContinuousBackup.java @@ -271,12 +271,6 @@ String[] buildBackupArgs(String backupType, TableName[] tables, boolean continuo } } - BackupManifest getLatestBackupManifest(List backups) throws IOException { - BackupInfo newestBackup = backups.get(0); - return HBackupFileSystem.getManifest(conf1, new Path(BACKUP_ROOT_DIR), - newestBackup.getBackupId()); - } - private void verifyTableInBackupSystemTable(TableName table) throws IOException { try (BackupSystemTable backupTable = new BackupSystemTable(TEST_UTIL.getConnection())) { Map tableBackupMap = backupTable.getContinuousBackupTableSet(); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java index 0978ff3ebef5..e67e50ebee36 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_DEFAULT; import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY; import static org.junit.Assert.assertEquals; @@ -48,6 +49,8 @@ import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.util.ToolRunner; +import org.junit.After; +import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -57,7 +60,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @Category(LargeTests.class) -public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { +public class TestIncrementalBackupWithContinuous extends TestBackupBase { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -67,11 +70,31 @@ public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { LoggerFactory.getLogger(TestIncrementalBackupWithContinuous.class); private static final int ROWS_IN_BULK_LOAD = 100; + private static final String backupWalDirName = "TestContinuousBackupWalDir"; + + @Before + public void beforeTest() throws IOException { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, backupWalDirName); + conf1.set(CONF_CONTINUOUS_BACKUP_WAL_DIR, backupWalDir.toString()); + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); + } + + @After + public void afterTest() throws IOException { + Path root = TEST_UTIL.getDataTestDirOnTestFS(); + Path backupWalDir = new Path(root, backupWalDirName); + FileSystem fs = FileSystem.get(conf1); + if (fs.exists(backupWalDir)) { + fs.delete(backupWalDir, true); + } + conf1.unset(CONF_CONTINUOUS_BACKUP_WAL_DIR); + conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); + deleteContinuousBackupReplicationPeerIfExists(TEST_UTIL.getAdmin()); + } @Test public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception { - LOG.info("Testing incremental backup with continuous backup"); - conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); TableName tableName = TableName.valueOf("table_" + methodName); Table t1 = TEST_UTIL.createTable(tableName, famName); @@ -80,18 +103,13 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception int before = table.getBackupHistory().size(); // Run continuous backup - String[] args = buildBackupArgs("full", new TableName[] { tableName }, true); - int ret = ToolRunner.run(conf1, new BackupDriver(), args); - assertEquals("Full Backup should succeed", 0, ret); + String backup1 = backupTables(BackupType.FULL, List.of(tableName), BACKUP_ROOT_DIR, true); + assertTrue(checkSucceeded(backup1)); // Verify backup history increased and all the backups are succeeded LOG.info("Verify backup history increased and all the backups are succeeded"); List backups = table.getBackupHistory(); assertEquals("Backup history should increase", before + 1, backups.size()); - for (BackupInfo data : List.of(backups.get(0))) { - String backupId = data.getBackupId(); - assertTrue(checkSucceeded(backupId)); - } // Verify backup manifest contains the correct tables LOG.info("Verify backup manifest contains the correct tables"); @@ -105,42 +123,34 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception // Run incremental backup LOG.info("Run incremental backup now"); before = table.getBackupHistory().size(); - args = buildBackupArgs("incremental", new TableName[] { tableName }, false); - ret = ToolRunner.run(conf1, new BackupDriver(), args); - assertEquals("Incremental Backup should succeed", 0, ret); + String backup2 = + backupTables(BackupType.INCREMENTAL, List.of(tableName), BACKUP_ROOT_DIR, true); + assertTrue(checkSucceeded(backup2)); LOG.info("Incremental backup completed"); // Verify backup history increased and all the backups are succeeded backups = table.getBackupHistory(); - String incrementalBackupid = null; assertEquals("Backup history should increase", before + 1, backups.size()); - for (BackupInfo data : List.of(backups.get(0))) { - String backupId = data.getBackupId(); - incrementalBackupid = backupId; - assertTrue(checkSucceeded(backupId)); - } TEST_UTIL.truncateTable(tableName); + // Restore incremental backup TableName[] tables = new TableName[] { tableName }; BackupAdminImpl client = new BackupAdminImpl(TEST_UTIL.getConnection()); - client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, incrementalBackupid, false, - tables, tables, true)); + client.restore( + BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backup2, false, tables, tables, true)); assertEquals(NB_ROWS_IN_BATCH, TEST_UTIL.countRows(tableName)); - } finally { - conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); } } @Test public void testIncrementalBackupCopyingBulkloadTillIncrCommittedWalTs() throws Exception { - conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); TableName tableName1 = TableName.valueOf("table_" + methodName); TEST_UTIL.createTable(tableName1, famName); - try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { + try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { // The test starts with no data, and no bulk loaded rows. int expectedRowCount = 0; assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); @@ -157,7 +167,7 @@ public void testIncrementalBackupCopyingBulkloadTillIncrCommittedWalTs() throws assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); loadTable(TEST_UTIL.getConnection().getTable(tableName1)); - Thread.sleep(10000); + Thread.sleep(15000); performBulkLoad("bulkPostIncr", methodName, tableName1); assertEquals(2, systemTable.readBulkloadRows(List.of(tableName1)).size()); @@ -177,14 +187,11 @@ public void testIncrementalBackupCopyingBulkloadTillIncrCommittedWalTs() throws client.restore( BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backup2, false, tables, tables, true)); assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); - } finally { - conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); } } @Test public void testPitrFailureDueToMissingBackupPostBulkload() throws Exception { - conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, true); String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); TableName tableName1 = TableName.valueOf("table_" + methodName); TEST_UTIL.createTable(tableName1, famName); @@ -228,8 +235,6 @@ public void testPitrFailureDueToMissingBackupPostBulkload() throws Exception { new TableName[] { restoredTable }, restoreTs, null); int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); assertNotEquals("Restore should fail since there is one bulkload without any backup", 0, ret); - } finally { - conf1.setBoolean(REPLICATION_MARKER_ENABLED_KEY, REPLICATION_MARKER_ENABLED_DEFAULT); } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java index be14227e4ccb..15ab2b2bdbe1 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java @@ -22,8 +22,8 @@ import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.setupBackupFolders; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java index 8f8e83dbda6b..cc9200882e3d 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java @@ -26,10 +26,10 @@ import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_PEER_UUID; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INITIAL_DELAY; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_STAGED_WAL_FLUSH_INTERVAL; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.DATE_FORMAT; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.WAL_FILE_PREFIX; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.copyWithCleanup; +import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -497,7 +498,7 @@ protected void uploadBulkLoadFiles(long dayInMillis, List bulkLoadFiles) firstAttempt = false; try { // Construct destination path and create a partial file - String dayDirectoryName = formatToDateString(dayInMillis); + String dayDirectoryName = BackupUtils.formatToDateString(dayInMillis); BackupFileSystemManager backupFileSystemManager = new BackupFileSystemManager("peer1", conf, conf.get(CONF_BACKUP_ROOT_DIR)); Path bulkloadDir = From 9bd36d0d3e63fa903fc31d715f4e291edf7d2440 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andor=20Moln=C3=A1r?= Date: Thu, 11 Sep 2025 10:21:10 -0500 Subject: [PATCH 21/24] Revert "HBASE-29310 Handle Bulk Load Operations in Continuous Backup (#7150)" (#7290) This reverts commit 5ac2a7345fdc80bc5da7d0883a46b651ced2dd17. --- .../impl/AbstractPitrRestoreHandler.java | 28 ----------- .../hbase/backup/impl/BackupImageAdapter.java | 11 ---- .../hbase/backup/impl/BackupInfoAdapter.java | 11 ---- .../hbase/backup/impl/BackupManifest.java | 22 +------- .../hbase/backup/impl/PitrBackupMetadata.java | 7 --- .../TestIncrementalBackupWithContinuous.java | 50 ------------------- .../hbase/backup/TestPointInTimeRestore.java | 4 +- .../src/main/protobuf/Backup.proto | 1 - .../hadoop/hbase/tool/BulkLoadHFilesTool.java | 5 -- 9 files changed, 4 insertions(+), 135 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java index 048ed882fe8d..d7ffbb580938 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java @@ -41,7 +41,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest; import org.apache.hadoop.hbase.backup.RestoreRequest; import org.apache.hadoop.hbase.backup.util.BackupUtils; @@ -249,8 +248,6 @@ private PitrBackupMetadata getValidBackup(TableName sTableName, TableName tTable try { if (backupAdmin.validateRequest(restoreRequest)) { - // check if any bulkload entry exists post this backup time and before "endtime" - checkBulkLoadAfterBackup(conn, sTableName, backup, endTime); return backup; } } catch (IOException e) { @@ -262,31 +259,6 @@ private PitrBackupMetadata getValidBackup(TableName sTableName, TableName tTable return null; } - /** - * Checks if any bulk load operation occurred for the specified table post last successful backup - * and before restore time. - * @param conn Active HBase connection - * @param sTableName Table for which to check bulk load history - * @param backup Last successful backup before the target recovery time - * @param endTime Target recovery time - * @throws IOException if a bulkload entry is found in between backup time and endtime - */ - private void checkBulkLoadAfterBackup(Connection conn, TableName sTableName, - PitrBackupMetadata backup, long endTime) throws IOException { - try (BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { - List bulkLoads = backupSystemTable.readBulkloadRows(List.of(sTableName)); - for (BulkLoad load : bulkLoads) { - long lastBackupTs = (backup.getType() == BackupType.FULL) - ? backup.getStartTs() - : backup.getIncrCommittedWalTs(); - if (lastBackupTs < load.getTimestamp() && load.getTimestamp() < endTime) { - throw new IOException("Bulk load operation detected after last successful backup for " - + "table: " + sTableName); - } - } - } - } - /** * Determines if the given backup is valid for PITR. *

diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java index b6d8d4901a22..8b785a0f0504 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupImageAdapter.java @@ -19,7 +19,6 @@ import java.util.List; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; import org.apache.yetus.audience.InterfaceAudience; @@ -58,14 +57,4 @@ public String getBackupId() { public String getRootDir() { return image.getRootDir(); } - - @Override - public BackupType getType() { - return image.getType(); - } - - @Override - public long getIncrCommittedWalTs() { - return image.getIncrCommittedWalTs(); - } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java index 34d812121e02..967fae551cb5 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupInfoAdapter.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo; -import org.apache.hadoop.hbase.backup.BackupType; import org.apache.yetus.audience.InterfaceAudience; /** @@ -58,14 +57,4 @@ public String getBackupId() { public String getRootDir() { return info.getBackupRootDir(); } - - @Override - public BackupType getType() { - return info.getType(); - } - - @Override - public long getIncrCommittedWalTs() { - return info.getIncrCommittedWalTs(); - } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java index f35755d24512..59ae3857f2ec 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java @@ -101,11 +101,6 @@ Builder withCompleteTime(long completeTime) { return this; } - Builder withIncrCommittedWalTs(long incrCommittedWalTs) { - image.setIncrCommittedWalTs(incrCommittedWalTs); - return this; - } - BackupImage build() { return image; } @@ -120,7 +115,6 @@ BackupImage build() { private long completeTs; private ArrayList ancestors; private Map> incrTimeRanges; - private long incrCommittedWalTs; static Builder newBuilder() { return new Builder(); @@ -131,14 +125,13 @@ public BackupImage() { } private BackupImage(String backupId, BackupType type, String rootDir, List tableList, - long startTs, long completeTs, long incrCommittedWalTs) { + long startTs, long completeTs) { this.backupId = backupId; this.type = type; this.rootDir = rootDir; this.tableList = tableList; this.startTs = startTs; this.completeTs = completeTs; - this.incrCommittedWalTs = incrCommittedWalTs; } static BackupImage fromProto(BackupProtos.BackupImage im) { @@ -146,7 +139,6 @@ static BackupImage fromProto(BackupProtos.BackupImage im) { String rootDir = im.getBackupRootDir(); long startTs = im.getStartTs(); long completeTs = im.getCompleteTs(); - long incrCommittedWalTs = im.getIncrCommittedWalTs(); List tableListList = im.getTableListList(); List tableList = new ArrayList<>(); for (HBaseProtos.TableName tn : tableListList) { @@ -159,8 +151,7 @@ static BackupImage fromProto(BackupProtos.BackupImage im) { ? BackupType.FULL : BackupType.INCREMENTAL; - BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs, - incrCommittedWalTs); + BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs); for (BackupProtos.BackupImage img : ancestorList) { image.addAncestor(fromProto(img)); } @@ -179,7 +170,6 @@ BackupProtos.BackupImage toProto() { builder.setBackupId(backupId); builder.setCompleteTs(completeTs); builder.setStartTs(startTs); - builder.setIncrCommittedWalTs(incrCommittedWalTs); if (type == BackupType.FULL) { builder.setBackupType(BackupProtos.BackupType.FULL); } else { @@ -297,14 +287,6 @@ public long getCompleteTs() { return completeTs; } - public long getIncrCommittedWalTs() { - return incrCommittedWalTs; - } - - public void setIncrCommittedWalTs(long incrCommittedWalTs) { - this.incrCommittedWalTs = incrCommittedWalTs; - } - private void setCompleteTs(long completeTs) { this.completeTs = completeTs; } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java index 3d143b336573..dc135ce79c08 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/PitrBackupMetadata.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupInfo; -import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; import org.apache.yetus.audience.InterfaceAudience; @@ -48,10 +47,4 @@ public interface PitrBackupMetadata { /** Returns Root directory where the backup is stored */ String getRootDir(); - - /** Returns backup type */ - BackupType getType(); - - /** Returns incrCommittedWalTs */ - long getIncrCommittedWalTs(); } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java index e67e50ebee36..54f3842f463b 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java @@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -48,7 +47,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.HFileTestUtil; -import org.apache.hadoop.util.ToolRunner; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; @@ -190,54 +188,6 @@ public void testIncrementalBackupCopyingBulkloadTillIncrCommittedWalTs() throws } } - @Test - public void testPitrFailureDueToMissingBackupPostBulkload() throws Exception { - String methodName = Thread.currentThread().getStackTrace()[1].getMethodName(); - TableName tableName1 = TableName.valueOf("table_" + methodName); - TEST_UTIL.createTable(tableName1, famName); - try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) { - - // The test starts with no data, and no bulk loaded rows. - int expectedRowCount = 0; - assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); - assertTrue(systemTable.readBulkloadRows(List.of(tableName1)).isEmpty()); - - // Create continuous backup, bulk loads are now being tracked - String backup1 = backupTables(BackupType.FULL, List.of(tableName1), BACKUP_ROOT_DIR, true); - assertTrue(checkSucceeded(backup1)); - - loadTable(TEST_UTIL.getConnection().getTable(tableName1)); - expectedRowCount = expectedRowCount + NB_ROWS_IN_BATCH; - performBulkLoad("bulkPreIncr", methodName, tableName1); - expectedRowCount += ROWS_IN_BULK_LOAD; - assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); - assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); - - loadTable(TEST_UTIL.getConnection().getTable(tableName1)); - Thread.sleep(5000); - - // Incremental backup - String backup2 = - backupTables(BackupType.INCREMENTAL, List.of(tableName1), BACKUP_ROOT_DIR, true); - assertTrue(checkSucceeded(backup2)); - assertEquals(0, systemTable.readBulkloadRows(List.of(tableName1)).size()); - - performBulkLoad("bulkPostIncr", methodName, tableName1); - assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); - - loadTable(TEST_UTIL.getConnection().getTable(tableName1)); - Thread.sleep(10000); - long restoreTs = BackupUtils.getReplicationCheckpoint(TEST_UTIL.getConnection()); - - // expect restore failure due to no backup post bulkPostIncr bulkload - TableName restoredTable = TableName.valueOf("restoredTable"); - String[] args = PITRTestUtil.buildPITRArgs(new TableName[] { tableName1 }, - new TableName[] { restoredTable }, restoreTs, null); - int ret = ToolRunner.run(conf1, new PointInTimeRestoreDriver(), args); - assertNotEquals("Restore should fail since there is one bulkload without any backup", 0, ret); - } - } - private void performBulkLoad(String keyPrefix, String testDir, TableName tableName) throws IOException { FileSystem fs = TEST_UTIL.getTestFileSystem(); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java index e9a0b50abcfa..a1ce9c97a687 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java @@ -67,8 +67,8 @@ private static void setUpBackups() throws Exception { // Simulate a backup taken 20 days ago EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - 20 * ONE_DAY_IN_MILLISECONDS); - // Insert initial data into table1 - PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); + PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Insert initial data into + // table1 // Perform a full backup for table1 with continuous backup enabled String[] args = diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto index b173848cd09d..0ad1f5ba6191 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Backup.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto @@ -65,7 +65,6 @@ message BackupImage { optional uint64 complete_ts = 6; repeated BackupImage ancestors = 7; repeated TableServerTimestamp tst_map = 8; - optional uint64 incr_committed_wal_ts = 9; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java index cd9bc32e3e18..4d6f57e22edc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java @@ -1197,11 +1197,6 @@ public int run(String[] args) throws Exception { public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); int ret = ToolRunner.run(conf, new BulkLoadHFilesTool(conf), args); - if (ret == 0) { - System.out.println("Bulk load completed successfully."); - System.out.println("IMPORTANT: Please take a backup of the table immediately if this table " - + "is part of continuous backup"); - } System.exit(ret); } From 26f51a05430732b31f5ccb18d7ee60148841bc04 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Thu, 25 Sep 2025 22:27:31 +0530 Subject: [PATCH 22/24] HBASE-29521: Update Restore Command to Handle Bulkloaded Files (#7300) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Tak Lon (Stephen) Wu Signed-off-by: Andor Molnár andor@apache.org Reviewed by: Kevin Geiszler Reviewed by: Kota-SH --- .../backup/PointInTimeRestoreDriver.java | 13 +- .../backup/PointInTimeRestoreRequest.java | 24 ++ .../impl/AbstractPitrRestoreHandler.java | 88 +++- .../hbase/backup/impl/BackupCommands.java | 2 +- .../impl/IncrementalTableBackupClient.java | 4 +- .../mapreduce/BulkLoadCollectorJob.java | 399 ++++++++++++++++++ .../backup/mapreduce/MapReduceRestoreJob.java | 9 +- .../replication/BackupFileSystemManager.java | 71 ---- .../ContinuousBackupReplicationEndpoint.java | 2 + .../backup/util/BackupFileSystemManager.java | 154 +++++++ .../hbase/backup/util/BulkFilesCollector.java | 226 ++++++++++ .../BulkLoadProcessor.java | 50 ++- .../hadoop/hbase/backup/PITRTestUtil.java | 36 ++ .../backup/TestBackupDeleteWithCleanup.java | 6 +- .../hbase/backup/TestPointInTimeRestore.java | 26 +- .../hbase/backup/impl/TestBackupCommands.java | 4 +- .../mapreduce/TestBulkLoadCollectorJob.java | 220 ++++++++++ .../TestBulkLoadCollectorJobIntegration.java | 252 +++++++++++ ...stContinuousBackupReplicationEndpoint.java | 5 +- .../util/TestBackupFileSystemManager.java | 164 +++++++ .../TestBulkLoadProcessor.java | 68 ++- .../hbase/mapreduce/WALInputFormat.java | 2 +- 22 files changed, 1714 insertions(+), 111 deletions(-) create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java delete mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupFileSystemManager.java create mode 100644 hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BulkFilesCollector.java rename hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/{replication => util}/BulkLoadProcessor.java (63%) create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJob.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJobIntegration.java create mode 100644 hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/util/TestBackupFileSystemManager.java rename hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/{replication => util}/TestBulkLoadProcessor.java (69%) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java index abdf52f14302..19159eeba921 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreDriver.java @@ -27,6 +27,7 @@ import java.net.URI; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; @@ -101,9 +102,15 @@ protected int executeRestore(boolean check, TableName[] fromTables, TableName[] return -5; } - PointInTimeRestoreRequest pointInTimeRestoreRequest = new PointInTimeRestoreRequest.Builder() - .withBackupRootDir(backupRootDir).withCheck(check).withFromTables(fromTables) - .withToTables(toTables).withOverwrite(isOverwrite).withToDateTime(endTime).build(); + // TODO: Currently hardcoding keepOriginalSplits=false and restoreRootDir via tmp dir. + // These should come from user input (same issue exists in normal restore). + // Expose them as configurable options in future. + PointInTimeRestoreRequest pointInTimeRestoreRequest = + new PointInTimeRestoreRequest.Builder().withBackupRootDir(backupRootDir).withCheck(check) + .withFromTables(fromTables).withToTables(toTables).withOverwrite(isOverwrite) + .withToDateTime(endTime).withKeepOriginalSplits(false).withRestoreRootDir( + BackupUtils.getTmpRestoreOutputDir(FileSystem.get(conf), conf).toString()) + .build(); client.pointInTimeRestore(pointInTimeRestoreRequest); } catch (Exception e) { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java index f2462a1cfd18..d7f69c05b683 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/PointInTimeRestoreRequest.java @@ -27,25 +27,33 @@ public final class PointInTimeRestoreRequest { private final String backupRootDir; + private final String restoreRootDir; private final boolean check; private final TableName[] fromTables; private final TableName[] toTables; private final boolean overwrite; private final long toDateTime; + private final boolean isKeepOriginalSplits; private PointInTimeRestoreRequest(Builder builder) { this.backupRootDir = builder.backupRootDir; + this.restoreRootDir = builder.restoreRootDir; this.check = builder.check; this.fromTables = builder.fromTables; this.toTables = builder.toTables; this.overwrite = builder.overwrite; this.toDateTime = builder.toDateTime; + this.isKeepOriginalSplits = builder.isKeepOriginalSplits; } public String getBackupRootDir() { return backupRootDir; } + public String getRestoreRootDir() { + return restoreRootDir; + } + public boolean isCheck() { return check; } @@ -66,19 +74,30 @@ public long getToDateTime() { return toDateTime; } + public boolean isKeepOriginalSplits() { + return isKeepOriginalSplits; + } + public static class Builder { private String backupRootDir; + private String restoreRootDir; private boolean check = false; private TableName[] fromTables; private TableName[] toTables; private boolean overwrite = false; private long toDateTime; + private boolean isKeepOriginalSplits; public Builder withBackupRootDir(String backupRootDir) { this.backupRootDir = backupRootDir; return this; } + public Builder withRestoreRootDir(String restoreRootDir) { + this.restoreRootDir = restoreRootDir; + return this; + } + public Builder withCheck(boolean check) { this.check = check; return this; @@ -104,6 +123,11 @@ public Builder withToDateTime(long dateTime) { return this; } + public Builder withKeepOriginalSplits(boolean isKeepOriginalSplits) { + this.isKeepOriginalSplits = isKeepOriginalSplits; + return this; + } + public PointInTimeRestoreRequest build() { return new PointInTimeRestoreRequest(this); } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java index d7ffbb580938..ce6c4d4dc683 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java @@ -20,8 +20,8 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES; @@ -30,6 +30,7 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Date; import java.util.List; import java.util.Map; @@ -41,9 +42,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.BackupRestoreFactory; import org.apache.hadoop.hbase.backup.PointInTimeRestoreRequest; +import org.apache.hadoop.hbase.backup.RestoreJob; import org.apache.hadoop.hbase.backup.RestoreRequest; import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.backup.util.BulkFilesCollector; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.mapreduce.WALInputFormat; import org.apache.hadoop.hbase.mapreduce.WALPlayer; @@ -305,6 +309,63 @@ private void restoreTableWithWalReplay(TableName sourceTable, TableName targetTa backupAdmin.restore(restoreRequest); replayWal(sourceTable, targetTable, backupMetadata.getStartTs(), endTime); + + reBulkloadFiles(sourceTable, targetTable, backupMetadata.getStartTs(), endTime, + request.isKeepOriginalSplits(), request.getRestoreRootDir()); + } + + /** + * Re-applies/re-bulkloads store files discovered from WALs into the target table. + *

+ * Note: this method re-uses the same {@link RestoreJob} MapReduce job that we originally + * implemented for performing full and incremental backup restores. The MR job (obtained via + * {@link BackupRestoreFactory#getRestoreJob(Configuration)}) is used here to perform an HFile + * bulk-load of the discovered store files into {@code targetTable}. + * @param sourceTable source table name (used for locating bulk files and logging) + * @param targetTable destination table to bulk-load the HFiles into + * @param startTime start of WAL range (ms) + * @param endTime end of WAL range (ms) + * @param keepOriginalSplits pass-through flag to control whether original region splits are + * preserved + * @param restoreRootDir local/DFS path under which temporary and output dirs are created + * @throws IOException on IO or job failure + */ + private void reBulkloadFiles(TableName sourceTable, TableName targetTable, long startTime, + long endTime, boolean keepOriginalSplits, String restoreRootDir) throws IOException { + + Configuration conf = HBaseConfiguration.create(conn.getConfiguration()); + conf.setBoolean(RestoreJob.KEEP_ORIGINAL_SPLITS_KEY, keepOriginalSplits); + + String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + Path walDirPath = new Path(walBackupDir); + conf.set(RestoreJob.BACKUP_ROOT_PATH_KEY, walDirPath.toString()); + + RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf); + + List bulkloadFiles = + collectBulkFiles(sourceTable, targetTable, startTime, endTime, new Path(restoreRootDir)); + + if (bulkloadFiles.isEmpty()) { + LOG.info("No bulk-load files found for {} in time range {}-{}. Skipping bulkload restore.", + sourceTable, startTime, endTime); + return; + } + + Path[] pathsArray = bulkloadFiles.toArray(new Path[0]); + + try { + // Use the existing RestoreJob MR job (the same MapReduce job used for full/incremental + // restores) + // to perform the HFile bulk-load of the discovered store files into `targetTable`. + restoreService.run(pathsArray, new TableName[] { sourceTable }, new Path(restoreRootDir), + new TableName[] { targetTable }, false); + LOG.info("Re-bulkload completed for {}", targetTable); + } catch (Exception e) { + String errorMessage = + String.format("Re-bulkload failed for %s: %s", targetTable, e.getMessage()); + LOG.error(errorMessage, e); + throw new IOException(errorMessage, e); + } } /** @@ -329,6 +390,29 @@ private void replayWal(TableName sourceTable, TableName targetTable, long startT executeWalReplay(validDirs, sourceTable, targetTable, startTime, endTime); } + private List collectBulkFiles(TableName sourceTable, TableName targetTable, long startTime, + long endTime, Path restoreRootDir) throws IOException { + + String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + Path walDirPath = new Path(walBackupDir); + LOG.info( + "Starting WAL bulk-file collection for source: {}, target: {}, time range: {} - {}, WAL backup dir: {}, restore root: {}", + sourceTable, targetTable, startTime, endTime, walDirPath, restoreRootDir); + + List validDirs = + getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); + if (validDirs.isEmpty()) { + LOG.warn("No valid WAL directories found for range {} - {}. Skipping bulk-file collection.", + startTime, endTime); + return Collections.emptyList(); + } + + String walDirsCsv = String.join(",", validDirs); + + return BulkFilesCollector.collectFromWalDirs(HBaseConfiguration.create(conn.getConfiguration()), + walDirsCsv, restoreRootDir, sourceTable, targetTable, startTime, endTime); + } + /** * Fetches valid WAL directories based on the given time range. */ @@ -356,7 +440,7 @@ private List getValidWalDirs(Configuration conf, Path walBackupDir, long validDirs.add(dayDir.getPath().toString()); } } catch (ParseException e) { - LOG.warn("Skipping invalid directory name: " + dirName, e); + LOG.warn("Skipping invalid directory name: {}", dirName, e); } } return validDirs; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java index a30530a98fc6..f70bf627d176 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java @@ -81,7 +81,7 @@ import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; -import org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager; +import org.apache.hadoop.hbase.backup.util.BackupFileSystemManager; import org.apache.hadoop.hbase.backup.util.BackupSet; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index 8a893994616c..a78f6c929c45 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -19,9 +19,9 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.BULKLOAD_FILES_DIR; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import java.io.IOException; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java new file mode 100644 index 000000000000..b752c7f78e01 --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java @@ -0,0 +1,399 @@ +/* + * 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.mapreduce; + +import static org.apache.hadoop.hbase.mapreduce.WALPlayer.TABLES_KEY; +import static org.apache.hadoop.hbase.mapreduce.WALPlayer.TABLE_MAP_KEY; + +import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupFileSystemManager; +import org.apache.hadoop.hbase.backup.util.BulkLoadProcessor; +import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat; +import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * MapReduce job that scans WAL backups and extracts referenced bulk-load store-file paths. + *

+ * This job is intended to be used when you want a list of HFiles / store-files referenced by WAL + * bulk-load descriptors. It emits a de-duplicated list of full paths (one per line) by default + * using the {@link DedupReducer}. + *

+ *

+ * Usage (CLI): + * {@code BulkLoadCollector [ []]} + *

+ */ +@InterfaceAudience.Private +public class BulkLoadCollectorJob extends Configured implements Tool { + private static final Logger LOG = LoggerFactory.getLogger(BulkLoadCollectorJob.class); + + public static final String NAME = "BulkLoadCollector"; + public static final String DEFAULT_REDUCERS = "1"; + + public BulkLoadCollectorJob() { + } + + protected BulkLoadCollectorJob(final Configuration c) { + super(c); + } + + /** + * Mapper that extracts relative bulk-load paths from a WAL entry (via {@code BulkLoadProcessor}), + * resolves them to full paths (via + * {@code BackupFileSystemManager#resolveBulkLoadFullPath(Path, Path)}), and emits each full path + * as the map key (Text). Uses the same table-filtering semantics as other WAL mappers: if no + * tables are configured, all tables are processed; otherwise only the configured table set is + * processed. Map output: (Text fullPathString, NullWritable) + */ + public static class BulkLoadCollectorMapper extends Mapper { + private final Map tables = new TreeMap<>(); + private final Text out = new Text(); + + @Override + protected void map(WALKey key, WALEdit value, Context context) + throws IOException, InterruptedException { + if (key == null) { + if (LOG.isTraceEnabled()) LOG.trace("map: received null WALKey, skipping"); + return; + } + if (value == null) { + if (LOG.isTraceEnabled()) + LOG.trace("map: received null WALEdit for table={}, skipping", safeTable(key)); + return; + } + + TableName tname = key.getTableName(); + + // table filtering + if (!(tables.isEmpty() || tables.containsKey(tname))) { + if (LOG.isTraceEnabled()) { + LOG.trace("map: skipping table={} because it is not in configured table list", tname); + } + return; + } + + // Extract relative store-file paths referenced by this WALEdit. + // Delegates parsing to BulkLoadProcessor so parsing logic is centralized. + List relativePaths = BulkLoadProcessor.processBulkLoadFiles(key, value); + if (relativePaths.isEmpty()) return; + + // Determine WAL input path for this split (used to compute date/prefix for full path) + Path walInputPath; + try { + walInputPath = + new Path(((WALInputFormat.WALSplit) context.getInputSplit()).getLogFileName()); + } catch (ClassCastException cce) { + String splitClass = + (context.getInputSplit() == null) ? "null" : context.getInputSplit().getClass().getName(); + LOG.warn( + "map: unexpected InputSplit type (not WALSplit) - cannot determine WAL input path; context.getInputSplit() class={}", + splitClass, cce); + throw new IOException("Unexpected InputSplit type: expected WALSplit but got " + splitClass, + cce); + } + + if (LOG.isTraceEnabled()) { + LOG.trace("map: walInputPath={} table={} relativePathsCount={}", walInputPath, tname, + relativePaths.size()); + } + + // Build full path for each relative path and emit it. + for (Path rel : relativePaths) { + Path full = BackupFileSystemManager.resolveBulkLoadFullPath(walInputPath, rel); + out.set(full.toString()); + context.write(out, NullWritable.get()); + context.getCounter("BulkCollector", "StoreFilesEmitted").increment(1); + } + } + + @Override + protected void setup(Context context) throws IOException { + String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY); + String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY); + if (tableMap == null) { + tableMap = tablesToUse; + } + if (tablesToUse == null) { + // user requested all tables; tables map remains empty to indicate "all" + return; + } + + if (tablesToUse.length != tableMap.length) { + throw new IOException("Incorrect table mapping specified."); + } + + int i = 0; + for (String table : tablesToUse) { + TableName from = TableName.valueOf(table); + TableName to = TableName.valueOf(tableMap[i++]); + tables.put(from, to); + if (LOG.isDebugEnabled()) { + LOG.debug("setup: configuring mapping {} -> {}", from, to); + } + } + } + + private String safeTable(WALKey key) { + try { + return key == null ? "" : key.getTableName().toString(); + } catch (Exception e) { + return ""; + } + } + } + + /** + * Reducer that deduplicates full-path keys emitted by the mappers. It writes each unique key + * exactly once. Reduce input: (Text fullPathString, Iterable) Reduce output: (Text + * fullPathString, NullWritable) + */ + public static class DedupReducer extends Reducer { + @Override + protected void reduce(Text key, Iterable values, Context ctx) + throws IOException, InterruptedException { + // Write the unique path once. + ctx.write(key, NullWritable.get()); + } + } + + /** + * Create and configure a Job instance for bulk-file collection. + * @param args CLI args expected to be: inputDirs bulkFilesOut [tables] [tableMap] + * @throws IOException on misconfiguration + */ + public Job createSubmittableJob(String[] args) throws IOException { + Configuration conf = getConf(); + + setupTime(conf, WALInputFormat.START_TIME_KEY); + setupTime(conf, WALInputFormat.END_TIME_KEY); + + if (args == null || args.length < 2) { + throw new IOException( + "Usage: [ []]"); + } + + String inputDirs = args[0]; + String bulkFilesOut = args[1]; + + // tables are optional (args[2]) + String[] tables = (args.length == 2) ? new String[] {} : args[2].split(","); + String[] tableMap; + if (args.length > 3) { + tableMap = args[3].split(","); + if (tableMap.length != tables.length) { + throw new IOException("The same number of tables and mapping must be provided."); + } + } else { + // if no mapping is specified, map each table to itself + tableMap = tables; + } + + LOG.info("createSubmittableJob: inputDirs='{}' bulkFilesOut='{}' tables={} tableMap={}", + inputDirs, bulkFilesOut, String.join(",", tables), String.join(",", tableMap)); + + conf.setStrings(TABLES_KEY, tables); + conf.setStrings(TABLE_MAP_KEY, tableMap); + conf.set(FileInputFormat.INPUT_DIR, inputDirs); + + // create and return the actual Job configured for bulk-file discovery + return BulkLoadCollectorJob.createSubmittableJob(conf, inputDirs, bulkFilesOut); + } + + /** + * Low-level job wiring. Creates the Job instance and sets input, mapper, reducer and output. + * @param conf configuration used for the job + * @param inputDirs WAL input directories (comma-separated) + * @param bulkFilesOut output directory to write discovered full-paths + * @throws IOException on invalid args + */ + private static Job createSubmittableJob(Configuration conf, String inputDirs, String bulkFilesOut) + throws IOException { + if (bulkFilesOut == null || bulkFilesOut.isEmpty()) { + throw new IOException("bulkFilesOut (output dir) must be provided."); + } + if (inputDirs == null || inputDirs.isEmpty()) { + throw new IOException("inputDirs (WAL input dir) must be provided."); + } + + Job job = Job.getInstance(conf, NAME + "_" + EnvironmentEdgeManager.currentTime()); + job.setJarByClass(BulkLoadCollectorJob.class); + + // Input: use same WALInputFormat used by WALPlayer so we parse WALs consistently + job.setInputFormatClass(WALInputFormat.class); + FileInputFormat.setInputDirRecursive(job, true); + FileInputFormat.setInputPaths(job, inputDirs); + + // Mapper: extract and emit full bulk-load file paths (Text, NullWritable) + job.setMapperClass(BulkLoadCollectorMapper.class); + job.setMapOutputKeyClass(Text.class); + job.setMapOutputValueClass(NullWritable.class); + + // Reducer: deduplicate the full-path keys + job.setReducerClass(DedupReducer.class); + // default to a single reducer (single deduped file); callers can set mapreduce.job.reduces + int reducers = conf.getInt("mapreduce.job.reduces", Integer.parseInt(DEFAULT_REDUCERS)); + job.setNumReduceTasks(reducers); + + // Output: write plain text lines (one path per line) + job.setOutputFormatClass(TextOutputFormat.class); + FileOutputFormat.setOutputPath(job, new Path(bulkFilesOut)); + + LOG.info("createSubmittableJob: created job name='{}' reducers={}", job.getJobName(), reducers); + + String codecCls = WALCellCodec.getWALCellCodecClass(conf).getName(); + try { + TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), + Class.forName(codecCls)); + } catch (Exception e) { + throw new IOException("Cannot determine wal codec class " + codecCls, e); + } + return job; + } + + /** + * Parse a time option. Supports the user-friendly ISO-like format + * {@code yyyy-MM-dd'T'HH:mm:ss.SS} or milliseconds since epoch. If the option is not present, + * this method is a no-op. + * @param conf configuration containing option + * @param option key to read (e.g. WALInputFormat.START_TIME_KEY) + * @throws IOException on parse failure + */ + private void setupTime(Configuration conf, String option) throws IOException { + String val = conf.get(option); + if (val == null) { + return; + } + long ms; + try { + // first try to parse in user-friendly form + ms = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SS").parse(val).getTime(); + } catch (ParseException pe) { + try { + // then see if a number (milliseconds) was specified + ms = Long.parseLong(val); + } catch (NumberFormatException nfe) { + throw new IOException( + option + " must be specified either in the form 2001-02-20T16:35:06.99 " + + "or as number of milliseconds"); + } + } + conf.setLong(option, ms); + } + + /** + * CLI entry point. + * @param args job arguments (see {@link #usage(String)}) + * @throws Exception on job failure + */ + public static void main(String[] args) throws Exception { + int ret = ToolRunner.run(new BulkLoadCollectorJob(HBaseConfiguration.create()), args); + System.exit(ret); + } + + @Override + public int run(String[] args) throws Exception { + if (args.length < 2) { + usage("Wrong number of arguments: " + args.length); + System.exit(-1); + } + + Job job = createSubmittableJob(args); + return job.waitForCompletion(true) ? 0 : 1; + } + + /** + * Print usage/help for the BulkLoadCollectorJob CLI/driver. + *

+ * + *

+   * args layout:
+   *   args[0] = input directory (required)
+   *   args[1] = output directory (required)
+   *   args[2] = tables (comma-separated) (optional)
+   *   args[3] = tableMappings (comma-separated) (optional; must match tables length)
+   * 
+ */ + private void usage(final String errorMsg) { + if (errorMsg != null && !errorMsg.isEmpty()) { + System.err.println("ERROR: " + errorMsg); + } + + System.err.println( + "Usage: " + NAME + " [ []]"); + System.err.println( + " directory of WALs to scan (comma-separated list accepted)"); + System.err.println( + " directory to write discovered store-file paths (output)"); + System.err.println( + " optional comma-separated list of tables to include; if omitted, all tables are processed"); + System.err.println( + " optional comma-separated list of mapped target tables; must match number of tables"); + + System.err.println(); + System.err.println("Time range options (either milliseconds or yyyy-MM-dd'T'HH:mm:ss.SS):"); + System.err.println(" -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]"); + System.err.println(" -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]"); + + System.err.println(); + System.err.println("Configuration alternatives (can be provided via -D):"); + System.err + .println(" -D" + TABLES_KEY + "= (alternative to arg[2])"); + System.err + .println(" -D" + TABLE_MAP_KEY + "= (alternative to arg[3])"); + System.err.println( + " -Dmapreduce.job.reduces= (number of reducers; default 1)"); + System.err.println(); + + System.err.println("Performance hints:"); + System.err.println(" For large inputs consider disabling speculative execution:"); + System.err + .println(" -Dmapreduce.map.speculative=false -Dmapreduce.reduce.speculative=false"); + + System.err.println(); + System.err.println("Example:"); + System.err.println( + " " + NAME + " /wals/input /out/bulkfiles ns:tbl1,ns:tbl2 ns:tbl1_mapped,ns:tbl2_mapped"); + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java index 7a2fce4c418a..4711cba46680 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java @@ -61,11 +61,10 @@ public void run(Path[] dirPaths, TableName[] tableNames, Path restoreRootDir, String dirs = StringUtils.join(dirPaths, ","); if (LOG.isDebugEnabled()) { - LOG.debug("Restore " + (fullBackupRestore ? "full" : "incremental") - + " backup from directory " + dirs + " from hbase tables " - + StringUtils.join(tableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND) - + " to tables " - + StringUtils.join(newTableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND)); + LOG.debug("Restore {} from directory {} from hbase tables {} to tables {}", + fullBackupRestore ? "full backup" : "incremental backup / bulkload files (as part of PITR)", + dirs, StringUtils.join(tableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND), + StringUtils.join(newTableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND)); } for (int i = 0; i < tableNames.length; i++) { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java deleted file mode 100644 index 225d32172766..000000000000 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BackupFileSystemManager.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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; - -/** - * Initializes and organizes backup directories for continuous Write-Ahead Logs (WALs) and - * bulk-loaded files within the specified backup root directory. - */ -@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 final Path walsDir; - private final 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); - this.walsDir = createDirectory(WALS_DIR); - this.bulkLoadFilesDir = createDirectory(BULKLOAD_FILES_DIR); - } - - private Path createDirectory(String dirName) throws IOException { - Path dirPath = new Path(backupRootDir, dirName); - backupFs.mkdirs(dirPath); - LOG.info("{} Initialized directory: {}", Utils.logPeerId(peerId), dirPath); - return dirPath; - } - - public Path getWalsDir() { - return walsDir; - } - - public Path getBulkLoadFilesDir() { - return bulkLoadFilesDir; - } - - public FileSystem getBackupFs() { - return backupFs; - } -} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java index 19624d04c23d..19cd2733af7b 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/ContinuousBackupReplicationEndpoint.java @@ -38,7 +38,9 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; +import org.apache.hadoop.hbase.backup.util.BackupFileSystemManager; import org.apache.hadoop.hbase.backup.util.BackupUtils; +import org.apache.hadoop.hbase.backup.util.BulkLoadProcessor; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupFileSystemManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupFileSystemManager.java new file mode 100644 index 000000000000..a616eb69e47f --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupFileSystemManager.java @@ -0,0 +1,154 @@ +/* + * 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.util; + +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.hadoop.hbase.backup.replication.Utils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Initializes and organizes backup directories for continuous Write-Ahead Logs (WALs) and + * bulk-loaded files within the specified backup root directory. + */ +@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 final Path walsDir; + private final 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); + this.walsDir = createDirectory(WALS_DIR); + this.bulkLoadFilesDir = createDirectory(BULKLOAD_FILES_DIR); + } + + private Path createDirectory(String dirName) throws IOException { + Path dirPath = new Path(backupRootDir, dirName); + backupFs.mkdirs(dirPath); + LOG.info("{} Initialized directory: {}", Utils.logPeerId(peerId), dirPath); + return dirPath; + } + + public Path getWalsDir() { + return walsDir; + } + + public Path getBulkLoadFilesDir() { + return bulkLoadFilesDir; + } + + public FileSystem getBackupFs() { + return backupFs; + } + + public static final class WalPathInfo { + private final Path prefixBeforeWALs; + private final String dateSegment; + + public WalPathInfo(Path prefixBeforeWALs, String dateSegment) { + this.prefixBeforeWALs = prefixBeforeWALs; + this.dateSegment = dateSegment; + } + + public Path getPrefixBeforeWALs() { + return prefixBeforeWALs; + } + + public String getDateSegment() { + return dateSegment; + } + } + + /** + * Validate the walPath has the expected structure: .../WALs// and return + * WalPathInfo(prefixBeforeWALs, dateSegment). + * @throws IOException if the path is not in expected format + */ + public static WalPathInfo extractWalPathInfo(Path walPath) throws IOException { + if (walPath == null) { + throw new IllegalArgumentException("walPath must not be null"); + } + + Path dateDir = walPath.getParent(); // .../WALs/ + if (dateDir == null) { + throw new IOException("Invalid WAL path: missing date directory. Path: " + walPath); + } + + Path walsDir = dateDir.getParent(); // .../WALs + if (walsDir == null) { + throw new IOException("Invalid WAL path: missing WALs directory. Path: " + walPath); + } + + String walsDirName = walsDir.getName(); + if (!WALS_DIR.equals(walsDirName)) { + throw new IOException("Invalid WAL path: expected '" + WALS_DIR + "' segment but found '" + + walsDirName + "'. Path: " + walPath); + } + + String dateSegment = dateDir.getName(); + if (dateSegment == null || dateSegment.isEmpty()) { + throw new IOException("Invalid WAL path: date segment is empty. Path: " + walPath); + } + + Path prefixBeforeWALs = walsDir.getParent(); // might be null if path is like "/WALs/..." + return new WalPathInfo(prefixBeforeWALs, dateSegment); + } + + /** + * Resolve the full bulk-load file path corresponding to a relative bulk-load path referenced from + * a WAL file path. For a WAL path like: /some/prefix/.../WALs/23-08-2025/some-wal-file and a + * relative bulk path like: namespace/table/region/family/file, this returns: + * /some/prefix/.../bulk-load-files/23-08-2025/namespace/table/region/family/file + * @param walPath the Path to the WAL file (must contain the {@link #WALS_DIR} segment + * followed by date) + * @param relativeBulkPath the relative bulk-load file Path + * @return resolved full Path for the bulk-load file + * @throws IOException if the WAL path does not contain the expected segments + */ + public static Path resolveBulkLoadFullPath(Path walPath, Path relativeBulkPath) + throws IOException { + WalPathInfo info = extractWalPathInfo(walPath); + + Path prefixBeforeWALs = info.getPrefixBeforeWALs(); + String dateSegment = info.getDateSegment(); + + Path full; // Build final path: + // /bulk-load-files// + if (prefixBeforeWALs == null || prefixBeforeWALs.toString().isEmpty()) { + full = new Path(BULKLOAD_FILES_DIR, new Path(dateSegment, relativeBulkPath)); + } else { + full = new Path(new Path(prefixBeforeWALs, BULKLOAD_FILES_DIR), + new Path(dateSegment, relativeBulkPath)); + } + return full; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BulkFilesCollector.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BulkFilesCollector.java new file mode 100644 index 000000000000..718a662abb7b --- /dev/null +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BulkFilesCollector.java @@ -0,0 +1,226 @@ +/* + * 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.util; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.mapreduce.BulkLoadCollectorJob; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.util.Tool; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility to run BulkLoadCollectorJob over a comma-separated list of WAL directories and return a + * deduplicated list of discovered bulk-load file paths. + */ +@InterfaceAudience.Private +public final class BulkFilesCollector { + + private static final Logger LOG = LoggerFactory.getLogger(BulkFilesCollector.class); + + private BulkFilesCollector() { + /* static only */ } + + /** + * Convenience overload: collector will create and configure BulkLoadCollectorJob internally. + * @param conf cluster/configuration used to initialize job and access FS + * @param walDirsCsv comma-separated WAL directories + * @param restoreRootDir parent path under which temporary output dir will be created + * @param sourceTable source table name (for args/logging) + * @param targetTable target table name (for args/logging) + * @param startTime start time (ms) to set in the job config (WALInputFormat.START_TIME_KEY) + * @param endTime end time (ms) to set in the job config (WALInputFormat.END_TIME_KEY) + * @return deduplicated list of Paths discovered by the collector + * @throws IOException on IO or job failure + */ + public static List collectFromWalDirs(Configuration conf, String walDirsCsv, + Path restoreRootDir, TableName sourceTable, TableName targetTable, long startTime, long endTime) + throws IOException { + + // prepare job Tool + Configuration jobConf = new Configuration(conf); + if (startTime > 0) jobConf.setLong(WALInputFormat.START_TIME_KEY, startTime); + if (endTime > 0) jobConf.setLong(WALInputFormat.END_TIME_KEY, endTime); + + // ignore empty WAL files by default to make collection robust + jobConf.setBoolean(WALPlayer.IGNORE_EMPTY_FILES, true); + + BulkLoadCollectorJob bulkCollector = new BulkLoadCollectorJob(); + bulkCollector.setConf(jobConf); + + return collectFromWalDirs(conf, walDirsCsv, restoreRootDir, sourceTable, targetTable, + bulkCollector); + } + + /** + * Primary implementation: runs the provided Tool (BulkLoadCollectorJob) with args " + * " and returns deduped list of Paths. + */ + public static List collectFromWalDirs(Configuration conf, String walDirsCsv, + Path restoreRootDir, TableName sourceTable, TableName targetTable, Tool bulkCollector) + throws IOException { + + if (walDirsCsv == null || walDirsCsv.trim().isEmpty()) { + throw new IOException( + "walDirsCsv must be a non-empty comma-separated list of WAL directories"); + } + + List walDirs = + Arrays.stream(walDirsCsv.split(",")).map(String::trim).filter(s -> !s.isEmpty()).toList(); + + if (walDirs.isEmpty()) { + throw new IOException("walDirsCsv did not contain any entries: '" + walDirsCsv + "'"); + } + + List existing = new ArrayList<>(); + for (String d : walDirs) { + Path p = new Path(d); + try { + FileSystem fsForPath = p.getFileSystem(conf); + if (fsForPath.exists(p)) { + existing.add(d); + } else { + LOG.debug("WAL dir does not exist: {}", d); + } + } catch (IOException e) { + // If getting FS or checking existence fails, treat as missing but log the cause. + LOG.warn("Error checking WAL dir {}: {}", d, e.toString()); + } + } + + // If any of the provided walDirs are missing, fail with an informative message. + List missing = new ArrayList<>(walDirs); + missing.removeAll(existing); + + if (!missing.isEmpty()) { + throw new IOException( + "Some of the provided WAL paths do not exist: " + String.join(", ", missing)); + } + + // Create unique temporary output dir under restoreRootDir, e.g. + // /_wal_collect_ + final String unique = String.format("_wal_collect_%s%d", sourceTable.getQualifierAsString(), + EnvironmentEdgeManager.currentTime()); + final Path bulkFilesOut = new Path(restoreRootDir, unique); + + FileSystem fs = bulkFilesOut.getFileSystem(conf); + + try { + // If bulkFilesOut exists for some reason, delete it. + if (fs.exists(bulkFilesOut)) { + LOG.info("Temporary bulkload file collect output directory {} already exists - deleting.", + bulkFilesOut); + fs.delete(bulkFilesOut, true); + } + + final String[] args = new String[] { walDirsCsv, bulkFilesOut.toString(), + sourceTable.getNameAsString(), targetTable.getNameAsString() }; + + LOG.info("Running bulk collector Tool with args: {}", (Object) args); + + int exitCode; + try { + exitCode = bulkCollector.run(args); + } catch (Exception e) { + LOG.error("Error during BulkLoadCollectorJob for {}: {}", sourceTable, e.getMessage(), e); + throw new IOException("Exception during BulkLoadCollectorJob collect", e); + } + + if (exitCode != 0) { + throw new IOException("Bulk collector Tool returned non-zero exit code: " + exitCode); + } + + LOG.info("BulkLoadCollectorJob collect completed successfully for {}", sourceTable); + + // read and dedupe + List results = readBulkFilesListFromOutput(fs, bulkFilesOut); + LOG.info("BulkFilesCollector: discovered {} unique bulk-load files", results.size()); + return results; + } finally { + // best-effort cleanup + try { + if (fs.exists(bulkFilesOut)) { + boolean deleted = fs.delete(bulkFilesOut, true); + if (!deleted) { + LOG.warn("Could not delete temporary bulkFilesOut directory {}", bulkFilesOut); + } else { + LOG.debug("Deleted temporary bulkFilesOut directory {}", bulkFilesOut); + } + } + } catch (IOException ioe) { + LOG.warn("Exception while deleting temporary bulkload file collect output dir {}: {}", + bulkFilesOut, ioe.getMessage(), ioe); + } + } + } + + // reads all non-hidden files under bulkFilesOut, collects lines in insertion order, returns Paths + private static List readBulkFilesListFromOutput(FileSystem fs, Path bulkFilesOut) + throws IOException { + if (!fs.exists(bulkFilesOut)) { + LOG.warn("BulkFilesCollector: bulkFilesOut directory does not exist: {}", bulkFilesOut); + return new ArrayList<>(); + } + + RemoteIterator it = fs.listFiles(bulkFilesOut, true); + Set dedupe = new LinkedHashSet<>(); + + while (it.hasNext()) { + LocatedFileStatus status = it.next(); + Path p = status.getPath(); + String name = p.getName(); + // skip hidden/system files like _SUCCESS or _logs + if (name.startsWith("_") || name.startsWith(".")) continue; + + try (FSDataInputStream in = fs.open(p); + BufferedReader br = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { + String line; + while ((line = br.readLine()) != null) { + line = line.trim(); + if (line.isEmpty()) continue; + dedupe.add(line); + } + } + } + + List result = new ArrayList<>(dedupe.size()); + for (String s : dedupe) + result.add(new Path(s)); + + LOG.info("Collected {} unique bulk-load store files.", result.size()); + return result; + } +} diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BulkLoadProcessor.java similarity index 63% rename from hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java rename to hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BulkLoadProcessor.java index 6e1271313bcd..4ab8bfb104e4 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/replication/BulkLoadProcessor.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BulkLoadProcessor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.backup.replication; +package org.apache.hadoop.hbase.backup.util; import java.io.IOException; import java.util.ArrayList; @@ -26,19 +26,16 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKey; 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. + * Processes bulk load files from Write-Ahead Log (WAL) entries. *

- * 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. - *

- * The class is designed for scenarios where replicable bulk load operations need to be parsed and - * their file paths need to be determined programmatically. + * Used by backup/restore and replication flows to discover HFiles referenced by bulk-load WALEdits. + * Returned {@link Path}s are constructed from the namespace/table/region/family/file components. *

*/ @InterfaceAudience.Private @@ -46,20 +43,41 @@ public final class BulkLoadProcessor { private BulkLoadProcessor() { } + /** + * Extract bulk-load file {@link Path}s from a list of {@link WAL.Entry}. + * @param walEntries list of WAL entries. + * @return list of Paths in discovery order; empty list if none + * @throws IOException if descriptor parsing fails + */ public static List processBulkLoadFiles(List walEntries) throws IOException { List bulkLoadFilePaths = new ArrayList<>(); for (WAL.Entry entry : walEntries) { - WALEdit edit = entry.getEdit(); - for (Cell cell : edit.getCells()) { - if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) { - TableName tableName = entry.getKey().getTableName(); - String namespace = tableName.getNamespaceAsString(); - String table = tableName.getQualifierAsString(); - bulkLoadFilePaths.addAll(processBulkLoadDescriptor(cell, namespace, table)); - } + bulkLoadFilePaths.addAll(processBulkLoadFiles(entry.getKey(), entry.getEdit())); + } + return bulkLoadFilePaths; + } + + /** + * Extract bulk-load file {@link Path}s from a single WAL entry. + * @param key WALKey containing table information; if null returns empty list + * @param edit WALEdit to scan; if null returns empty list + * @return list of Paths referenced by bulk-load descriptor(s) in this edit; may be empty or + * contain duplicates + * @throws IOException if descriptor parsing fails + */ + public static List processBulkLoadFiles(WALKey key, WALEdit edit) throws IOException { + List bulkLoadFilePaths = new ArrayList<>(); + + for (Cell cell : edit.getCells()) { + if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) { + TableName tableName = key.getTableName(); + String namespace = tableName.getNamespaceAsString(); + String table = tableName.getQualifierAsString(); + bulkLoadFilePaths.addAll(processBulkLoadDescriptor(cell, namespace, table)); } } + return bulkLoadFilePaths; } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java index ae26cf960501..24f5237866db 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/PITRTestUtil.java @@ -28,9 +28,17 @@ import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.tool.BulkLoadHFiles; +import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,4 +112,32 @@ public static int getRowCount(HBaseTestingUtil testUtil, TableName tableName) th return HBaseTestingUtil.countRows(table); } } + + public static void generateHFiles(Path outputDir, Configuration conf, String cfName) + throws IOException { + String hFileName = "MyHFile"; + int numRows = 1000; + + FileSystem fs = FileSystem.get(conf); + outputDir = outputDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()); + + byte[] from = Bytes.toBytes(cfName + "begin"); + byte[] to = Bytes.toBytes(cfName + "end"); + + Path familyDir = new Path(outputDir, cfName); + HFileTestUtil.createHFile(conf, fs, new Path(familyDir, hFileName), Bytes.toBytes(cfName), + Bytes.toBytes("qualifier"), from, to, numRows); + } + + public static void bulkLoadHFiles(TableName tableName, Path inputDir, Connection conn, + Configuration conf) throws IOException { + conf.setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true); + + try (Table table = conn.getTable(tableName)) { + BulkLoadHFiles loader = new BulkLoadHFilesTool(conf); + loader.bulkLoad(table.getName(), inputDir); + } finally { + conf.setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false); + } + } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java index 8bd2fe4cc78c..c6c6f5e9799e 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithCleanup.java @@ -19,9 +19,9 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.BULKLOAD_FILES_DIR; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; -import org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager; +import org.apache.hadoop.hbase.backup.util.BackupFileSystemManager; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.ToolRunner; diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java index a1ce9c97a687..54667752f01b 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestPointInTimeRestore.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.ToolRunner; import org.junit.AfterClass; @@ -36,6 +37,21 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +/** + * Integration-style tests for Point-in-Time Restore (PITR). + *

+ * These tests exercise the full backup / continuous backup / restore flow: - create backups at + * multiple historical points in time (via {@code BackupDriver}) - exercise WAL-based + * replication/continuous backup - validate Point-in-Time Restore behavior (successful restores, + * failure cases) + *

+ *

+ * NOTE: Some tests also create HFiles and perform HBase bulk-loads (HFile -> table) so the restore + * flow is validated when bulk-loaded storefiles are present in WALs. This ensures the + * BulkLoadCollector/BulkFilesCollector logic (discovering bulk-loaded store files referenced from + * WAL bulk-load descriptors) is exercised by the test suite. + *

+ */ @Category(LargeTests.class) public class TestPointInTimeRestore extends TestBackupBase { @ClassRule @@ -67,8 +83,8 @@ private static void setUpBackups() throws Exception { // Simulate a backup taken 20 days ago EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - 20 * ONE_DAY_IN_MILLISECONDS); - PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Insert initial data into - // table1 + // Insert initial data into table1 + PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Perform a full backup for table1 with continuous backup enabled String[] args = @@ -80,6 +96,12 @@ private static void setUpBackups() throws Exception { EnvironmentEdgeManager .injectEdge(() -> System.currentTimeMillis() - 15 * ONE_DAY_IN_MILLISECONDS); PITRTestUtil.loadRandomData(TEST_UTIL, table1, famName, 1000); // Add more data to table1 + + Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoadByFamily"); + PITRTestUtil.generateHFiles(dir, TEST_UTIL.getConfiguration(), Bytes.toString(famName)); + PITRTestUtil.bulkLoadHFiles(table1, dir, TEST_UTIL.getConnection(), + TEST_UTIL.getConfiguration()); + PITRTestUtil.loadRandomData(TEST_UTIL, table2, famName, 500); // Insert data into table2 PITRTestUtil.waitForReplication(); // Ensure replication is complete diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java index 15ab2b2bdbe1..30ee495df55b 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/impl/TestBackupCommands.java @@ -20,9 +20,9 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.logDirectoryStructure; import static org.apache.hadoop.hbase.backup.TestBackupDeleteWithCleanup.setupBackupFolders; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.BULKLOAD_FILES_DIR; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJob.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJob.java new file mode 100644 index 000000000000..20295d7e4ea3 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJob.java @@ -0,0 +1,220 @@ +/* + * 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.mapreduce; + +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Arrays; +import java.util.Collections; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BulkLoadProcessor; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +/** + * Unit tests for BulkLoadCollectorJob (mapper, reducer and job creation/validation). + */ +@Category({ MapReduceTests.class, LargeTests.class }) +public class TestBulkLoadCollectorJob { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBulkLoadCollectorJob.class); + + private Configuration conf; + + @Before + public void setUp() { + // fresh configuration for each test + conf = HBaseConfiguration.create(); + } + + @After + public void tearDown() { + // nothing for now + } + + /** + * Ensures {@link BulkLoadCollectorJob#createSubmittableJob(String[])} correctly configures + * input/output paths and parses time options into the job configuration. + */ + @Test + public void testCreateSubmittableJobValid() throws Exception { + // set a start time option to make sure setupTime runs and applies it + String dateStr = "2001-02-20T16:35:06.99"; + conf.set(WALInputFormat.START_TIME_KEY, dateStr); + + BulkLoadCollectorJob jobDriver = new BulkLoadCollectorJob(conf); + String inputDirs = new Path("file:/wals/input").toString(); + String outDir = new Path("file:/out/bulk").toString(); + Job job = jobDriver.createSubmittableJob(new String[] { inputDirs, outDir }); + + // Input path set + Path[] inPaths = FileInputFormat.getInputPaths(job); + assertEquals(1, inPaths.length); + assertEquals(inputDirs, inPaths[0].toString()); + + // Output path set + Path out = FileOutputFormat.getOutputPath(job); + assertEquals(new Path(outDir), out); + + // Ensure the conf had START_TIME_KEY parsed to a long (setupTime executed) + long parsed = conf.getLong(WALInputFormat.START_TIME_KEY, -1L); + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SS"); + long expected = sdf.parse(dateStr).getTime(); + assertEquals(expected, parsed); + } + + /** + * Verifies that {@link BulkLoadCollectorJob#createSubmittableJob(String[])} throws an IOException + * when called with insufficient or null arguments. + */ + @Test(expected = IOException.class) + public void testCreateSubmittableJob_throwsForInsufficientArgs() throws Exception { + BulkLoadCollectorJob jobDriver = new BulkLoadCollectorJob(conf); + // this call must throw IOException for the test to pass + jobDriver.createSubmittableJob(new String[] { "file:/only/one/arg" }); + } + + @Test(expected = IOException.class) + public void testCreateSubmittableJob_throwsForNullArgs() throws Exception { + BulkLoadCollectorJob jobDriver = new BulkLoadCollectorJob(conf); + // this call must throw IOException for the test to pass + jobDriver.createSubmittableJob(null); + } + + /** + * Verifies that {@link BulkLoadCollectorJob.BulkLoadCollectorMapper} ignores WAL entries whose + * table is not present in the configured tables map. + */ + @Test + public void testMapperIgnoresWhenTableNotInMap() throws Exception { + // Prepare mapper and a mocked MapReduce context + BulkLoadCollectorJob.BulkLoadCollectorMapper mapper = + new BulkLoadCollectorJob.BulkLoadCollectorMapper(); + @SuppressWarnings("unchecked") + Mapper.Context ctx = mock(Mapper.Context.class); + + // Build a Configuration that only allows a single table: ns:allowed + // Note: TABLES_KEY / TABLE_MAP_KEY are the same constants used by the mapper.setup(...) + Configuration cfgForTest = new Configuration(conf); + cfgForTest.setStrings(WALPlayer.TABLES_KEY, "ns:allowed"); + cfgForTest.setStrings(WALPlayer.TABLE_MAP_KEY, "ns:allowed"); // maps to itself + + // Have the mocked context return our test configuration when mapper.setup() runs + when(ctx.getConfiguration()).thenReturn(cfgForTest); + mapper.setup(ctx); + + // Create a WALKey for a table that is NOT in the allowed map (ns:other) + WALKey keyForOtherTable = mock(WALKey.class); + when(keyForOtherTable.getTableName()).thenReturn(TableName.valueOf("ns:other")); + WALEdit walEdit = mock(WALEdit.class); + + // Static-mock BulkLoadProcessor to ensure it would not be relied on: + // even if invoked unexpectedly, it returns a non-empty list, but we will assert no writes + // occurred. + try (MockedStatic proc = Mockito.mockStatic(BulkLoadProcessor.class)) { + proc.when(() -> BulkLoadProcessor.processBulkLoadFiles(any(), any())) + .thenReturn(Collections.singletonList(new Path("x"))); + + // Invoke mapper - because the table is not allowed, mapper should do nothing + mapper.map(keyForOtherTable, walEdit, ctx); + + // Assert: mapper did not write any output to the context + verify(ctx, never()).write(any(Text.class), any(NullWritable.class)); + } + } + + /** + * Verifies that {@link BulkLoadCollectorJob.BulkLoadCollectorMapper} safely handles null inputs. + *

+ * The mapper should ignore WAL entries when either the WAL key or the WALEdit value is null, and + * must not emit any output in those cases. + *

+ * @throws Exception on test failure + */ + @Test + public void testMapperHandlesNullKeyOrValue() throws Exception { + BulkLoadCollectorJob.BulkLoadCollectorMapper mapper = + new BulkLoadCollectorJob.BulkLoadCollectorMapper(); + @SuppressWarnings("unchecked") + Mapper.Context ctx = mock(Mapper.Context.class); + when(ctx.getConfiguration()).thenReturn(conf); + mapper.setup(ctx); + + // null key + mapper.map(null, mock(WALEdit.class), ctx); + // null value + mapper.map(mock(WALKey.class), null, ctx); + + // ensure no writes + verify(ctx, never()).write(any(Text.class), any(NullWritable.class)); + } + + /** + * Verifies that {@link BulkLoadCollectorJob.DedupReducer} writes each unique key exactly once. + */ + @Test + public void testDedupReducerWritesOnce() throws Exception { + BulkLoadCollectorJob.DedupReducer reducer = new BulkLoadCollectorJob.DedupReducer(); + @SuppressWarnings("unchecked") + Reducer.Context ctx = mock(Reducer.Context.class); + + Text key = new Text("/some/path"); + + // Simulate three duplicate values for the same key; reducer should still write the key once. + Iterable vals = + Arrays.asList(NullWritable.get(), NullWritable.get(), NullWritable.get()); + + reducer.reduce(key, vals, ctx); + + // verify exactly once write with the same key + verify(ctx, times(1)).write(eq(key), eq(NullWritable.get())); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJobIntegration.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJobIntegration.java new file mode 100644 index 000000000000..b72f931732a4 --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/mapreduce/TestBulkLoadCollectorJobIntegration.java @@ -0,0 +1,252 @@ +/* + * 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.mapreduce; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.NavigableMap; +import java.util.TreeMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellBuilderFactory; +import org.apache.hadoop.hbase.CellBuilderType; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BulkLoadProcessor; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; +import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.FSHLogProvider; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.apache.hadoop.util.ToolRunner; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; + +/** + * Integration-like unit test for BulkLoadCollectorJob. + *

+ * - Creates a WAL with a BULK_LOAD descriptor (ProtobufLogWriter). + *

+ * - Runs BulkLoadCollectorJob. + *

+ * - Verifies the job emits the expected store-file path. + */ +@Category({ MapReduceTests.class, LargeTests.class }) +public class TestBulkLoadCollectorJobIntegration { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBulkLoadCollectorJobIntegration.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestBulkLoadCollectorJobIntegration.class); + private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static Configuration conf; + private static FileSystem fs; + private static Path hbaseDir; + static final TableName tableName = TableName.valueOf(getName()); + static final RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); + private static final byte[] family = Bytes.toBytes("column"); + private static Path logDir; + protected MultiVersionConcurrencyControl mvcc; + protected static NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); + + private static String getName() { + return "TestBulkLoadCollectorJobIntegration"; + } + + @Before + public void setUp() throws Exception { + if (hbaseDir != null && fs != null) fs.delete(hbaseDir, true); + mvcc = new MultiVersionConcurrencyControl(); + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + conf = TEST_UTIL.getConfiguration(); + conf.setInt("dfs.blocksize", 1024 * 1024); + conf.setInt("dfs.replication", 1); + + // Start a mini DFS cluster + TEST_UTIL.startMiniDFSCluster(3); + + conf = TEST_UTIL.getConfiguration(); + fs = TEST_UTIL.getDFSCluster().getFileSystem(); + + hbaseDir = TEST_UTIL.createRootDir(); + + // Use a deterministic test WAL directory under the test filesystem + logDir = new Path(TEST_UTIL.getDataTestDirOnTestFS(), "WALs/23-11-2024"); + fs.mkdirs(logDir); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + if (fs != null && hbaseDir != null) fs.delete(hbaseDir, true); + TEST_UTIL.shutdownMiniDFSCluster(); + } + + /** + * Test that BulkLoadCollectorJob discovers and emits store-file paths from WAL files created + * using WALFactory (no RegionServer needed). + */ + @Test + public void testBulkLoadCollectorEmitsStoreFilesFromWAL() throws Exception { + // Create WAL entry with BULK_LOAD descriptor + final String storeFileName = "storefile-abc.hfile"; + WAL.Entry entry = + createBulkLoadWalEntry(info.getEncodedName(), Bytes.toString(family), storeFileName); + + // Verify the processor would extract relative paths + List relativePaths = + BulkLoadProcessor.processBulkLoadFiles(entry.getKey(), entry.getEdit()); + LOG.debug("BulkLoadProcessor returned {} relative path(s): {}", relativePaths.size(), + relativePaths); + assertEquals("Expected exactly one relative path from BulkLoadProcessor", 1, + relativePaths.size()); + + // Build WAL file path and write WAL using ProtobufLogWriter into logDir + String walFileName = "wal-" + EnvironmentEdgeManager.currentTime(); + Path walFilePath = new Path(logDir, walFileName); + fs.mkdirs(logDir); + + FSHLogProvider.Writer writer = null; + try { + writer = new ProtobufLogWriter(); + long blockSize = WALUtil.getWALBlockSize(conf, fs, walFilePath); + writer.init(fs, walFilePath, conf, true, blockSize, + StreamSlowMonitor.create(conf, walFileName)); + writer.append(entry); + writer.sync(true); + writer.close(); + } catch (Exception e) { + throw new IOException("Failed to write WAL via ProtobufLogWriter", e); + } finally { + try { + if (writer != null) writer.close(); + } catch (Exception ignore) { + } + } + + // Assert WAL file exists and has content + boolean exists = fs.exists(walFilePath); + long len = exists ? fs.getFileStatus(walFilePath).getLen() : -1L; + assertTrue("WAL file should exist at " + walFilePath, exists); + assertTrue("WAL file should have non-zero length, actual=" + len, len > 0); + + // Run the MR job + Path walInputDir = logDir; + Path outDir = new Path("/tmp/test-bulk-files-output-" + System.currentTimeMillis()); + + int res = ToolRunner.run(TEST_UTIL.getConfiguration(), + new BulkLoadCollectorJob(TEST_UTIL.getConfiguration()), + new String[] { walInputDir.toString(), outDir.toString() }); + assertEquals("BulkLoadCollectorJob should exit with code 0", 0, res); + + // Inspect job output + FileSystem dfs = TEST_UTIL.getDFSCluster().getFileSystem(); + assertTrue("Output directory should exist", dfs.exists(outDir)); + + List partFiles = Arrays.stream(dfs.listStatus(outDir)).map(FileStatus::getPath) + .filter(p -> p.getName().startsWith("part-")).toList(); + + assertFalse("Expect at least one part file in output", partFiles.isEmpty()); + + // Read all lines (collect while stream is open) + List lines = partFiles.stream().flatMap(p -> { + try (FSDataInputStream in = dfs.open(p); + BufferedReader r = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { + List fileLines = r.lines().toList(); + return fileLines.stream(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }).toList(); + + assertFalse("Job should have emitted at least one storefile path", lines.isEmpty()); + + boolean found = lines.stream().anyMatch(l -> l.contains(storeFileName)); + assertTrue( + "Expected emitted path to contain store file name: " + storeFileName + " ; got: " + lines, + found); + + // cleanup + dfs.delete(outDir, true); + } + + private WAL.Entry createBulkLoadWalEntry(String regionName, String family, String... storeFiles) { + + WALProtos.StoreDescriptor.Builder storeDescBuilder = + WALProtos.StoreDescriptor.newBuilder().setFamilyName(ByteString.copyFromUtf8(family)) + .setStoreHomeDir(family).addAllStoreFile(Arrays.asList(storeFiles)); + + WALProtos.BulkLoadDescriptor.Builder bulkDescBuilder = WALProtos.BulkLoadDescriptor.newBuilder() + .setReplicate(true).setEncodedRegionName(ByteString.copyFromUtf8(regionName)) + .setTableName(ProtobufUtil.toProtoTableName(tableName)).setBulkloadSeqNum(1000) + .addStores(storeDescBuilder); + + byte[] valueBytes = bulkDescBuilder.build().toByteArray(); + + WALEdit edit = new WALEdit(); + Cell cell = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setType(Cell.Type.Put) + .setRow(new byte[] { 1 }).setFamily(WALEdit.METAFAMILY).setQualifier(WALEdit.BULK_LOAD) + .setValue(valueBytes).build(); + edit.add(cell); + + long ts = EnvironmentEdgeManager.currentTime(); + WALKeyImpl key = getWalKeyImpl(ts, scopes); + return new WAL.Entry(key, edit); + } + + protected WALKeyImpl getWalKeyImpl(final long time, NavigableMap scopes) { + return new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java index cc9200882e3d..78991a463da1 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestContinuousBackupReplicationEndpoint.java @@ -19,8 +19,6 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_BULKLOAD_ENABLE_KEY; import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.BULKLOAD_FILES_DIR; -import static org.apache.hadoop.hbase.backup.replication.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_MAX_WAL_SIZE; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_BACKUP_ROOT_DIR; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.CONF_PEER_UUID; @@ -29,6 +27,8 @@ import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.WAL_FILE_PREFIX; import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.copyWithCleanup; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.BULKLOAD_FILES_DIR; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.backup.util.BackupFileSystemManager; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/util/TestBackupFileSystemManager.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/util/TestBackupFileSystemManager.java new file mode 100644 index 000000000000..2cca13bf19ca --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/util/TestBackupFileSystemManager.java @@ -0,0 +1,164 @@ +/* + * 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.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.File; +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.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; + +/** + * Unit tests for {@link BackupFileSystemManager}. + */ +@Category(SmallTests.class) +public class TestBackupFileSystemManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBackupFileSystemManager.class); + + @Rule + public TemporaryFolder tmp = new TemporaryFolder(); + + /** + * extractWalPathInfo: happy path where WALs dir has a prefix. e.g. + * /some/prefix/WALs/2025-09-14/some-wal + */ + @Test + public void testExtractWalPathInfo_withPrefix() throws Exception { + Path walPath = + new Path("/some/prefix/" + BackupFileSystemManager.WALS_DIR + "/2025-09-14/wal-000"); + BackupFileSystemManager.WalPathInfo info = BackupFileSystemManager.extractWalPathInfo(walPath); + + assertNotNull("WalPathInfo should not be null", info); + // prefixBeforeWALs should be "/some/prefix" + assertEquals("/some/prefix", info.getPrefixBeforeWALs().toString()); + assertEquals("2025-09-14", info.getDateSegment()); + } + + /** + * extractWalPathInfo: case where WALs is at root (leading slash). e.g. /WALs/2025-09-14/some-wal + * Expect prefixBeforeWALs to be "/" (root) or non-null; resolution should still work. + */ + @Test + public void testExtractWalPathInfo_rootWALs() throws Exception { + Path walPath = new Path("/" + BackupFileSystemManager.WALS_DIR + "/2025-09-14/wal-123"); + BackupFileSystemManager.WalPathInfo info = BackupFileSystemManager.extractWalPathInfo(walPath); + + assertNotNull(info); + // parent of "/WALs" in Hadoop Path is "/" (root). Ensure date segment parsed. + assertEquals("2025-09-14", info.getDateSegment()); + assertNotNull("prefixBeforeWALs should not be null for root-style path", + info.getPrefixBeforeWALs()); + // prefix might be "/" (expected), be tolerant: assert it ends with "/" or equals "/" + assertTrue(info.getPrefixBeforeWALs().toString().equals("/") + || !info.getPrefixBeforeWALs().toString().isEmpty()); + } + + /** + * extractWalPathInfo: null input should throw IllegalArgumentException. + */ + @Test(expected = IllegalArgumentException.class) + public void testExtractWalPathInfo_nullPath() throws Exception { + BackupFileSystemManager.extractWalPathInfo(null); + } + + /** + * extractWalPathInfo: missing date directory -> should throw IOException. Example: path that has + * no parent for the wal file. + */ + @Test(expected = IOException.class) + public void testExtractWalPathInfo_missingDateDir() throws Exception { + // A single segment path (no parents) e.g. "walfile" + Path walPath = new Path("walfile"); + BackupFileSystemManager.extractWalPathInfo(walPath); + } + + /** + * extractWalPathInfo: WALs segment name mismatch -> should throw IOException. e.g. + * /prefix/NOT_WALs/2025/wal + */ + @Test(expected = IOException.class) + public void testExtractWalPathInfo_wrongWALsegment() throws Exception { + Path walPath = new Path("/prefix/NOT_WALS/2025/wal"); + BackupFileSystemManager.extractWalPathInfo(walPath); + } + + /** + * resolveBulkLoadFullPath: normal case with prefix. + */ + @Test + public void testResolveBulkLoadFullPath_withPrefix() throws Exception { + Path walPath = + new Path("/some/prefix/" + BackupFileSystemManager.WALS_DIR + "/2025-08-30/wal-1"); + Path relative = new Path("namespace/table/region/family/file1"); + Path full = BackupFileSystemManager.resolveBulkLoadFullPath(walPath, relative); + + // expected: /some/prefix/bulk-load-files/2025-08-30/namespace/table/region/family/file1 + String expected = "/some/prefix/" + BackupFileSystemManager.BULKLOAD_FILES_DIR + + "/2025-08-30/namespace/table/region/family/file1"; + assertEquals(expected, full.toString()); + } + + /** + * resolveBulkLoadFullPath: when WALs is under root (prefix is root) - ensure path resolved under + * /bulk-load-files//... + */ + @Test + public void testResolveBulkLoadFullPath_rootWALs() throws Exception { + Path walPath = new Path("/" + BackupFileSystemManager.WALS_DIR + "/2025-08-30/wal-2"); + Path relative = new Path("ns/tbl/r/f"); + Path full = BackupFileSystemManager.resolveBulkLoadFullPath(walPath, relative); + + String expected = "/" + BackupFileSystemManager.BULKLOAD_FILES_DIR + "/2025-08-30/ns/tbl/r/f"; + assertEquals(expected, full.toString()); + } + + /** + * Integration-y test: constructor should create directories under the provided backup root. Uses + * a temporary folder (local fs). + */ + @Test + public void testConstructorCreatesDirectories() throws Exception { + File root = tmp.newFolder("backupRoot"); + String rootPath = root.getAbsolutePath(); + + Configuration conf = HBaseConfiguration.create(); + BackupFileSystemManager mgr = new BackupFileSystemManager("peer-1", conf, rootPath); + + FileSystem fs = mgr.getBackupFs(); + Path wals = mgr.getWalsDir(); + Path bulk = mgr.getBulkLoadFilesDir(); + + assertTrue("WALs dir should exist", fs.exists(wals)); + assertTrue("bulk-load-files dir should exist", fs.exists(bulk)); + } +} diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestBulkLoadProcessor.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/util/TestBulkLoadProcessor.java similarity index 69% rename from hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestBulkLoadProcessor.java rename to hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/util/TestBulkLoadProcessor.java index 9837f9e926d2..1d3b8ab09eaa 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/replication/TestBulkLoadProcessor.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/util/TestBulkLoadProcessor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.backup.replication; +package org.apache.hadoop.hbase.backup.util; import static org.apache.hadoop.hbase.wal.WALEdit.METAFAMILY; import static org.junit.Assert.assertEquals; @@ -119,6 +119,18 @@ public void testProcessBulkLoadFiles_validEntry() throws IOException { assertTrue(paths.get(1).toString().contains("ns/tbl/region123/cf1/file2")); } + @Test + public void testProcessBulkLoadFiles_validEntry_singleEntryApi() throws IOException { + WAL.Entry entry = createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "region123", true, + "cf1", "file1", "file2"); + + List paths = BulkLoadProcessor.processBulkLoadFiles(entry.getKey(), entry.getEdit()); + + assertEquals(2, paths.size()); + assertTrue(paths.get(0).toString().contains("ns/tbl/region123/cf1/file1")); + assertTrue(paths.get(1).toString().contains("ns/tbl/region123/cf1/file2")); + } + /** * Verifies that a non-replicable bulk load entry is ignored. */ @@ -132,6 +144,16 @@ public void testProcessBulkLoadFiles_nonReplicableSkipped() throws IOException { assertTrue(paths.isEmpty()); } + @Test + public void testProcessBulkLoadFiles_nonReplicableSkipped_singleEntryApi() throws IOException { + WAL.Entry entry = + createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "region123", false, "cf1", "file1"); + + List paths = BulkLoadProcessor.processBulkLoadFiles(entry.getKey(), entry.getEdit()); + + assertTrue(paths.isEmpty()); + } + /** * Verifies that entries without the BULK_LOAD qualifier are ignored. */ @@ -146,6 +168,16 @@ public void testProcessBulkLoadFiles_noBulkLoadQualifier() throws IOException { assertTrue(paths.isEmpty()); } + @Test + public void testProcessBulkLoadFiles_noBulkLoadQualifier_singleEntryApi() throws IOException { + WALEdit edit = new WALEdit(); + WALKeyImpl key = new WALKeyImpl(new byte[] {}, TableName.valueOf("ns", "tbl"), 0L, 0L, null); + + List paths = BulkLoadProcessor.processBulkLoadFiles(key, edit); + + assertTrue(paths.isEmpty()); + } + /** * Verifies that multiple WAL entries with different column families produce the correct set of * file paths. @@ -163,4 +195,38 @@ public void testProcessBulkLoadFiles_multipleFamilies() throws IOException { assertTrue(paths.stream().anyMatch(p -> p.toString().contains("cf1/file1"))); assertTrue(paths.stream().anyMatch(p -> p.toString().contains("cf2/fileA"))); } + + @Test + public void testProcessBulkLoadFiles_multipleFamilies_singleEntryApi() throws IOException { + WAL.Entry entry = + createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "regionXYZ", true, "cf1", "file1"); + WAL.Entry entry2 = + createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "regionXYZ", true, "cf2", "fileA"); + + List paths1 = BulkLoadProcessor.processBulkLoadFiles(entry.getKey(), entry.getEdit()); + List paths2 = BulkLoadProcessor.processBulkLoadFiles(entry2.getKey(), entry2.getEdit()); + + // combine to mimic processing multiple entries + paths1.addAll(paths2); + + assertEquals(2, paths1.size()); + assertTrue(paths1.stream().anyMatch(p -> p.toString().contains("cf1/file1"))); + assertTrue(paths1.stream().anyMatch(p -> p.toString().contains("cf2/fileA"))); + } + + /** + * Sanity check: list-based API should still work and return the same results as invoking the + * single-entry API for the same entry (ensures delegation/backwards compatibility). + */ + @Test + public void testProcessBulkLoadFiles_listApi_delegatesToSingle() throws IOException { + WAL.Entry entry = + createBulkLoadWalEntry(TableName.valueOf("ns", "tbl"), "region123", true, "cf1", "file1"); + + List single = BulkLoadProcessor.processBulkLoadFiles(entry.getKey(), entry.getEdit()); + List listApi = BulkLoadProcessor.processBulkLoadFiles(Collections.singletonList(entry)); + + assertEquals(single.size(), listApi.size()); + assertTrue(listApi.get(0).toString().contains("ns/tbl/region123/cf1/file1")); + } } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java index b5c1d39a550c..b2bda03c44ac 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java @@ -66,7 +66,7 @@ public class WALInputFormat extends InputFormat { /** * {@link InputSplit} for {@link WAL} files. Each split represent exactly one log file. */ - static class WALSplit extends InputSplit implements Writable { + public static class WALSplit extends InputSplit implements Writable { private String logFileName; private long fileSize; private long startTime; From 12e129276b4f840cde55677c61854fb9c6dbc0ff Mon Sep 17 00:00:00 2001 From: asolomon Date: Tue, 28 Oct 2025 01:55:24 +0700 Subject: [PATCH 23/24] HBASE-29656 Scan WALs to identify bulkload operations for incremental backup (#7400) * Scan WALs to identify bulkload operations for incremental backup * Update unit test * Info log * Minor test fix * Address review comments * Spotless apply * Addressed review comment * spotless * Remove log * Retrigger CI --------- Co-authored-by: Ankit Solomon --- .../hadoop/hbase/backup/BackupObserver.java | 10 +- .../impl/AbstractPitrRestoreHandler.java | 72 +----- .../impl/IncrementalTableBackupClient.java | 214 ++++++++---------- .../mapreduce/BulkLoadCollectorJob.java | 2 +- .../hadoop/hbase/backup/util/BackupUtils.java | 73 ++++++ .../hadoop/hbase/backup/TestBackupBase.java | 4 +- .../TestIncrementalBackupWithContinuous.java | 8 +- 7 files changed, 182 insertions(+), 201 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java index 392e27710911..c506d6dc6aed 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java @@ -96,12 +96,18 @@ private void registerBulkLoad(ObserverContext fullyBackedUpTables = tbl.getTablesIncludedInBackups(); + Map continuousBackupTableSet = tbl.getContinuousBackupTableSet(); - if (fullyBackedUpTables.contains(tableName)) { + // Tables in continuousBackupTableSet do not rely on BackupSystemTable but rather + // scan on WAL backup directory to identify bulkload operation HBASE-29656 + if ( + fullyBackedUpTables.contains(tableName) && !continuousBackupTableSet.containsKey(tableName) + ) { tbl.registerBulkLoad(tableName, region.getEncodedNameAsBytes(), cfToHFilePaths); } else { if (LOG.isTraceEnabled()) { - LOG.trace("Table {} has not gone through full backup - skipping.", tableName); + LOG.trace("Table {} has either not gone through full backup or is " + + "part of continuousBackupTableSet - skipping", tableName); } } } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java index ce6c4d4dc683..3f31255d60f6 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/AbstractPitrRestoreHandler.java @@ -20,25 +20,16 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; -import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; -import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import static org.apache.hadoop.hbase.mapreduce.WALPlayer.IGNORE_EMPTY_FILES; import java.io.IOException; -import java.text.ParseException; -import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; @@ -47,7 +38,6 @@ import org.apache.hadoop.hbase.backup.RestoreJob; import org.apache.hadoop.hbase.backup.RestoreRequest; import org.apache.hadoop.hbase.backup.util.BackupUtils; -import org.apache.hadoop.hbase.backup.util.BulkFilesCollector; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.mapreduce.WALInputFormat; import org.apache.hadoop.hbase.mapreduce.WALPlayer; @@ -342,8 +332,8 @@ private void reBulkloadFiles(TableName sourceTable, TableName targetTable, long RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf); - List bulkloadFiles = - collectBulkFiles(sourceTable, targetTable, startTime, endTime, new Path(restoreRootDir)); + List bulkloadFiles = BackupUtils.collectBulkFiles(conn, sourceTable, targetTable, + startTime, endTime, new Path(restoreRootDir), new ArrayList()); if (bulkloadFiles.isEmpty()) { LOG.info("No bulk-load files found for {} in time range {}-{}. Skipping bulkload restore.", @@ -380,7 +370,7 @@ private void replayWal(TableName sourceTable, TableName targetTable, long startT sourceTable, targetTable, startTime, endTime, walDirPath); List validDirs = - getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); + BackupUtils.getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); if (validDirs.isEmpty()) { LOG.warn("No valid WAL directories found for range {} - {}. Skipping WAL replay.", startTime, endTime); @@ -390,62 +380,6 @@ private void replayWal(TableName sourceTable, TableName targetTable, long startT executeWalReplay(validDirs, sourceTable, targetTable, startTime, endTime); } - private List collectBulkFiles(TableName sourceTable, TableName targetTable, long startTime, - long endTime, Path restoreRootDir) throws IOException { - - String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); - Path walDirPath = new Path(walBackupDir); - LOG.info( - "Starting WAL bulk-file collection for source: {}, target: {}, time range: {} - {}, WAL backup dir: {}, restore root: {}", - sourceTable, targetTable, startTime, endTime, walDirPath, restoreRootDir); - - List validDirs = - getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); - if (validDirs.isEmpty()) { - LOG.warn("No valid WAL directories found for range {} - {}. Skipping bulk-file collection.", - startTime, endTime); - return Collections.emptyList(); - } - - String walDirsCsv = String.join(",", validDirs); - - return BulkFilesCollector.collectFromWalDirs(HBaseConfiguration.create(conn.getConfiguration()), - walDirsCsv, restoreRootDir, sourceTable, targetTable, startTime, endTime); - } - - /** - * Fetches valid WAL directories based on the given time range. - */ - private List getValidWalDirs(Configuration conf, Path walBackupDir, long startTime, - long endTime) throws IOException { - FileSystem backupFs = FileSystem.get(walBackupDir.toUri(), conf); - FileStatus[] dayDirs = backupFs.listStatus(new Path(walBackupDir, WALS_DIR)); - - List validDirs = new ArrayList<>(); - SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); - - for (FileStatus dayDir : dayDirs) { - if (!dayDir.isDirectory()) { - continue; // Skip files, only process directories - } - - String dirName = dayDir.getPath().getName(); - try { - Date dirDate = dateFormat.parse(dirName); - long dirStartTime = dirDate.getTime(); // Start of that day (00:00:00) - long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) - - // Check if this day's WAL files overlap with the required time range - if (dirEndTime >= startTime && dirStartTime <= endTime) { - validDirs.add(dayDir.getPath().toString()); - } - } catch (ParseException e) { - LOG.warn("Skipping invalid directory name: {}", dirName, e); - } - } - return validDirs; - } - /** * Executes WAL replay using WALPlayer. */ diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index a78f6c929c45..0815d7c5ea38 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -19,27 +19,19 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.JOB_NAME_CONF_KEY; -import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; -import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.BULKLOAD_FILES_DIR; -import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; -import static org.apache.hadoop.hbase.backup.util.BackupUtils.DATE_FORMAT; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; -import java.text.ParseException; -import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; -import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TimeZone; +import java.util.stream.Collectors; import org.apache.commons.io.FilenameUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -87,6 +79,7 @@ @InterfaceAudience.Private public class IncrementalTableBackupClient extends TableBackupClient { private static final Logger LOG = LoggerFactory.getLogger(IncrementalTableBackupClient.class); + private static final String BULKLOAD_COLLECTOR_OUTPUT = "bulkload-collector-output"; protected IncrementalTableBackupClient() { } @@ -138,89 +131,89 @@ protected static int getIndex(TableName tbl, List sTableList) { * the backup is marked as complete. * @param tablesToBackup list of tables to be backed up */ - protected List handleBulkLoad(List tablesToBackup) throws IOException { + protected List handleBulkLoad(List tablesToBackup, + Map> tablesToWALFileList, Map tablesToPrevBackupTs) + throws IOException { Map toBulkload = new HashMap<>(); - List bulkLoads; - if (backupInfo.isContinuousBackupEnabled()) { - bulkLoads = - backupManager.readBulkloadRows(tablesToBackup, backupInfo.getIncrCommittedWalTs()); - } else { - bulkLoads = backupManager.readBulkloadRows(tablesToBackup); - } + List bulkLoads = new ArrayList<>(); + FileSystem tgtFs; try { tgtFs = FileSystem.get(new URI(backupInfo.getBackupRootDir()), conf); } catch (URISyntaxException use) { throw new IOException("Unable to get FileSystem", use); } + Path rootdir = CommonFSUtils.getRootDir(conf); Path tgtRoot = new Path(new Path(backupInfo.getBackupRootDir()), backupId); - for (BulkLoad bulkLoad : bulkLoads) { - TableName srcTable = bulkLoad.getTableName(); - MergeSplitBulkloadInfo bulkloadInfo = - toBulkload.computeIfAbsent(srcTable, MergeSplitBulkloadInfo::new); - String regionName = bulkLoad.getRegion(); - String fam = bulkLoad.getColumnFamily(); - String filename = FilenameUtils.getName(bulkLoad.getHfilePath()); + if (!backupInfo.isContinuousBackupEnabled()) { + bulkLoads = backupManager.readBulkloadRows(tablesToBackup); + for (BulkLoad bulkLoad : bulkLoads) { + TableName srcTable = bulkLoad.getTableName(); + if (!tablesToBackup.contains(srcTable)) { + LOG.debug("Skipping {} since it is not in tablesToBackup", srcTable); + continue; + } + + MergeSplitBulkloadInfo bulkloadInfo = + toBulkload.computeIfAbsent(srcTable, MergeSplitBulkloadInfo::new); + String regionName = bulkLoad.getRegion(); + String fam = bulkLoad.getColumnFamily(); + String filename = FilenameUtils.getName(bulkLoad.getHfilePath()); + Path tblDir = CommonFSUtils.getTableDir(rootdir, srcTable); + Path p = new Path(tblDir, regionName + Path.SEPARATOR + fam + Path.SEPARATOR + filename); + String srcTableQualifier = srcTable.getQualifierAsString(); + String srcTableNs = srcTable.getNamespaceAsString(); + Path tgtFam = new Path(tgtRoot, srcTableNs + Path.SEPARATOR + srcTableQualifier + + Path.SEPARATOR + regionName + Path.SEPARATOR + fam); + if (!tgtFs.mkdirs(tgtFam)) { + throw new IOException("couldn't create " + tgtFam); + } - if (!tablesToBackup.contains(srcTable)) { - LOG.debug("Skipping {} since it is not in tablesToBackup", srcTable); - continue; - } - Path tblDir = CommonFSUtils.getTableDir(rootdir, srcTable); - Path p = new Path(tblDir, regionName + Path.SEPARATOR + fam + Path.SEPARATOR + filename); - - // For continuous backup: bulkload files are copied from backup directory defined by - // CONF_CONTINUOUS_BACKUP_WAL_DIR instead of source cluster. - String backupRootDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); - if (backupInfo.isContinuousBackupEnabled() && !Strings.isNullOrEmpty(backupRootDir)) { - String dayDirectoryName = BackupUtils.formatToDateString(bulkLoad.getTimestamp()); - Path bulkLoadBackupPath = - new Path(backupRootDir, BULKLOAD_FILES_DIR + Path.SEPARATOR + dayDirectoryName); - Path bulkLoadDir = new Path(bulkLoadBackupPath, - srcTable.getNamespaceAsString() + Path.SEPARATOR + srcTable.getNameAsString()); - FileSystem backupFs = FileSystem.get(bulkLoadDir.toUri(), conf); - Path fullBulkLoadBackupPath = - new Path(bulkLoadDir, regionName + Path.SEPARATOR + fam + Path.SEPARATOR + filename); - if (backupFs.exists(fullBulkLoadBackupPath)) { - LOG.debug("Backup bulkload file found {}", fullBulkLoadBackupPath); - p = fullBulkLoadBackupPath; - } else { - LOG.warn("Backup bulkload file not found {}", fullBulkLoadBackupPath); + Path tgt = new Path(tgtFam, filename); + Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam); + Path archive = new Path(archiveDir, filename); + + if (fs.exists(p)) { + if (LOG.isTraceEnabled()) { + LOG.trace("found bulk hfile {} in {} for {}", bulkLoad.getHfilePath(), p.getParent(), + srcTableQualifier); + LOG.trace("copying {} to {}", p, tgt); + } + bulkloadInfo.addActiveFile(p.toString()); + } else if (fs.exists(archive)) { + LOG.debug("copying archive {} to {}", archive, tgt); + bulkloadInfo.addArchiveFiles(archive.toString()); } } - String srcTableQualifier = srcTable.getQualifierAsString(); - String srcTableNs = srcTable.getNamespaceAsString(); - Path tgtFam = new Path(tgtRoot, srcTableNs + Path.SEPARATOR + srcTableQualifier - + Path.SEPARATOR + regionName + Path.SEPARATOR + fam); - if (!tgtFs.mkdirs(tgtFam)) { - throw new IOException("couldn't create " + tgtFam); + for (MergeSplitBulkloadInfo bulkloadInfo : toBulkload.values()) { + mergeSplitAndCopyBulkloadedHFiles(bulkloadInfo.getActiveFiles(), + bulkloadInfo.getArchiveFiles(), bulkloadInfo.getSrcTable(), tgtFs); } - Path tgt = new Path(tgtFam, filename); + } else { + // Continuous incremental backup: run BulkLoadCollectorJob over backed-up WALs + Path collectorOutput = new Path(getBulkOutputDir(), BULKLOAD_COLLECTOR_OUTPUT); + for (TableName table : tablesToBackup) { + long startTs = tablesToPrevBackupTs.getOrDefault(table, 0L); + long endTs = backupInfo.getIncrCommittedWalTs(); + List walDirs = tablesToWALFileList.getOrDefault(table, new ArrayList()); - Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam); - Path archive = new Path(archiveDir, filename); + List bulkloadPaths = BackupUtils.collectBulkFiles(conn, table, table, startTs, endTs, + collectorOutput, walDirs); - if (fs.exists(p)) { - if (LOG.isTraceEnabled()) { - LOG.trace("found bulk hfile {} in {} for {}", bulkLoad.getHfilePath(), p.getParent(), - srcTableQualifier); - LOG.trace("copying {} to {}", p, tgt); + List bulkLoadFiles = + bulkloadPaths.stream().map(Path::toString).collect(Collectors.toList()); + + if (bulkLoadFiles.isEmpty()) { + LOG.info("No bulk-load files found for table {}", table); + continue; } - bulkloadInfo.addActiveFile(p.toString()); - } else if (fs.exists(archive)) { - LOG.debug("copying archive {} to {}", archive, tgt); - bulkloadInfo.addArchiveFiles(archive.toString()); - } - } - for (MergeSplitBulkloadInfo bulkloadInfo : toBulkload.values()) { - mergeSplitAndCopyBulkloadedHFiles(bulkloadInfo.getActiveFiles(), - bulkloadInfo.getArchiveFiles(), bulkloadInfo.getSrcTable(), tgtFs); + mergeSplitAndCopyBulkloadedHFiles(bulkLoadFiles, table, tgtFs); + } } - return bulkLoads; } @@ -324,11 +317,20 @@ public void updateFileLists(List activeFiles, List archiveFiles) */ @Override public void execute() throws IOException, ColumnFamilyMismatchException { + // tablesToWALFileList and tablesToPrevBackupTs are needed for "continuous" Incremental backup + Map> tablesToWALFileList = new HashMap<>(); + Map tablesToPrevBackupTs = new HashMap<>(); try { Map tablesToFullBackupIds = getFullBackupIds(); verifyCfCompatibility(backupInfo.getTables(), tablesToFullBackupIds); // case PREPARE_INCREMENTAL: + if (backupInfo.isContinuousBackupEnabled()) { + // committedWALsTs is needed only for Incremental backups with continuous backup + // since these do not depend on log roll ts + long committedWALsTs = BackupUtils.getReplicationCheckpoint(conn); + backupInfo.setIncrCommittedWalTs(committedWALsTs); + } beginBackup(backupManager, backupInfo); backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL); // Non-continuous Backup incremental backup is controlled by 'incremental backup table set' @@ -357,7 +359,7 @@ public void execute() throws IOException, ColumnFamilyMismatchException { BackupUtils.copyTableRegionInfo(conn, backupInfo, conf); setupRegionLocator(); // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT - convertWALsToHFiles(); + convertWALsToHFiles(tablesToWALFileList, tablesToPrevBackupTs); incrementalCopyHFiles(new String[] { getBulkOutputDir().toString() }, backupInfo.getBackupRootDir()); } catch (Exception e) { @@ -389,7 +391,8 @@ public void execute() throws IOException, ColumnFamilyMismatchException { backupManager.writeBackupStartCode(newStartCode); } - List bulkLoads = handleBulkLoad(backupInfo.getTableNames()); + List bulkLoads = + handleBulkLoad(backupInfo.getTableNames(), tablesToWALFileList, tablesToPrevBackupTs); // backup complete completeBackup(conn, backupInfo, BackupType.INCREMENTAL, conf); @@ -447,10 +450,19 @@ protected void deleteBulkLoadDirectory() throws IOException { } } - protected void convertWALsToHFiles() throws IOException { + protected void convertWALsToHFiles(Map> tablesToWALFileList, + Map tablesToPrevBackupTs) throws IOException { long previousBackupTs = 0L; + long currentBackupTs = 0L; if (backupInfo.isContinuousBackupEnabled()) { + String walBackupDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + if (Strings.isNullOrEmpty(walBackupDir)) { + throw new IOException( + "Incremental backup requires the WAL backup directory " + CONF_CONTINUOUS_BACKUP_WAL_DIR); + } + Path walBackupPath = new Path(walBackupDir); Set tableSet = backupInfo.getTables(); + currentBackupTs = backupInfo.getIncrCommittedWalTs(); List backupInfos = backupManager.getBackupHistory(true); for (TableName table : tableSet) { for (BackupInfo backup : backupInfos) { @@ -464,7 +476,10 @@ protected void convertWALsToHFiles() throws IOException { } else { previousBackupTs = backup.getIncrCommittedWalTs(); } - walBackupFileList = getBackupLogs(previousBackupTs); + walBackupFileList = + BackupUtils.getValidWalDirs(conf, walBackupPath, previousBackupTs, currentBackupTs); + tablesToWALFileList.put(table, walBackupFileList); + tablesToPrevBackupTs.put(table, previousBackupTs); walToHFiles(walBackupFileList, Arrays.asList(table.getNameAsString()), previousBackupTs); break; @@ -491,47 +506,6 @@ protected void convertWALsToHFiles() throws IOException { } } - private List getBackupLogs(long startTs) throws IOException { - // get log files from backup dir - String walBackupDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); - if (Strings.isNullOrEmpty(walBackupDir)) { - throw new IOException( - "Incremental backup requires the WAL backup directory " + CONF_CONTINUOUS_BACKUP_WAL_DIR); - } - List resultLogFiles = new ArrayList<>(); - Path walBackupPath = new Path(walBackupDir); - FileSystem backupFs = FileSystem.get(walBackupPath.toUri(), conf); - FileStatus[] dayDirs = backupFs.listStatus(new Path(walBackupDir, WALS_DIR)); - SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); - dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); - - for (FileStatus dayDir : dayDirs) { - if (!dayDir.isDirectory()) { - continue; // Skip files, only process directories - } - - String dirName = dayDir.getPath().getName(); - try { - Date dirDate = dateFormat.parse(dirName); - long dirStartTime = dirDate.getTime(); // Start of that day (00:00:00) - long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) - - if (dirEndTime >= startTs) { - Path dirPath = dayDir.getPath(); - FileStatus[] logs = backupFs.listStatus(dirPath); - for (FileStatus log : logs) { - String filepath = log.getPath().toString(); - LOG.debug("Found WAL file: {}", filepath); - resultLogFiles.add(filepath); - } - } - } catch (ParseException e) { - LOG.warn("Skipping invalid directory name: " + dirName, e); - } - } - return resultLogFiles; - } - protected boolean tableExists(TableName table, Connection conn) throws IOException { try (Admin admin = conn.getAdmin()) { return admin.tableExists(table); @@ -557,11 +531,7 @@ protected void walToHFiles(List dirPaths, List tableList, long p conf.setBoolean(HFileOutputFormat2.DISK_BASED_SORTING_ENABLED_KEY, true); if (backupInfo.isContinuousBackupEnabled()) { conf.set(WALInputFormat.START_TIME_KEY, Long.toString(previousBackupTs)); - // committedWALsTs is needed only for Incremental backups with continuous backup - // since these do not depend on log roll ts - long committedWALsTs = BackupUtils.getReplicationCheckpoint(conn); - backupInfo.setIncrCommittedWalTs(committedWALsTs); - conf.set(WALInputFormat.END_TIME_KEY, Long.toString(committedWALsTs)); + conf.set(WALInputFormat.END_TIME_KEY, Long.toString(backupInfo.getIncrCommittedWalTs())); } String[] playerArgs = { dirs, StringUtils.join(tableList, ",") }; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java index b752c7f78e01..cf19d2622216 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/BulkLoadCollectorJob.java @@ -75,7 +75,7 @@ public class BulkLoadCollectorJob extends Configured implements Tool { public BulkLoadCollectorJob() { } - protected BulkLoadCollectorJob(final Configuration c) { + public BulkLoadCollectorJob(final Configuration c) { super(c); } diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java index f76fa7353219..11e22efac97a 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java @@ -17,13 +17,17 @@ */ package org.apache.hadoop.hbase.backup.util; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONF_CONTINUOUS_BACKUP_WAL_DIR; import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; +import static org.apache.hadoop.hbase.backup.replication.ContinuousBackupReplicationEndpoint.ONE_DAY_IN_MILLISECONDS; +import static org.apache.hadoop.hbase.backup.util.BackupFileSystemManager.WALS_DIR; import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_DEFAULT; import static org.apache.hadoop.hbase.replication.regionserver.ReplicationMarkerChore.REPLICATION_MARKER_ENABLED_KEY; import java.io.FileNotFoundException; import java.io.IOException; import java.net.URLDecoder; +import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Collections; @@ -46,6 +50,7 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; @@ -79,6 +84,7 @@ import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Splitter; +import org.apache.hbase.thirdparty.com.google.common.base.Strings; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Iterators; @@ -996,4 +1002,71 @@ public static String formatToDateString(long dayInMillis) { dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")); return dateFormat.format(new Date(dayInMillis)); } + + /** + * Fetches bulkload filepaths based on the given time range from backup WAL directory. + */ + public static List collectBulkFiles(Connection conn, TableName sourceTable, + TableName targetTable, long startTime, long endTime, Path restoreRootDir, List walDirs) + throws IOException { + + if (walDirs.isEmpty()) { + String walBackupDir = conn.getConfiguration().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + if (Strings.isNullOrEmpty(walBackupDir)) { + throw new IOException( + "WAL backup directory is not configured " + CONF_CONTINUOUS_BACKUP_WAL_DIR); + } + Path walDirPath = new Path(walBackupDir); + walDirs = + BackupUtils.getValidWalDirs(conn.getConfiguration(), walDirPath, startTime, endTime); + } + + if (walDirs.isEmpty()) { + LOG.warn("No valid WAL directories found for range {} - {}. Skipping bulk-file collection.", + startTime, endTime); + return Collections.emptyList(); + } + + LOG.info( + "Starting WAL bulk-file collection for source: {}, target: {}, time range: {} - {}, WAL " + + "backup dir: {}, restore root: {}", + sourceTable, targetTable, startTime, endTime, walDirs, restoreRootDir); + String walDirsCsv = String.join(",", walDirs); + + return BulkFilesCollector.collectFromWalDirs(HBaseConfiguration.create(conn.getConfiguration()), + walDirsCsv, restoreRootDir, sourceTable, targetTable, startTime, endTime); + } + + /** + * Fetches valid WAL directories based on the given time range. + */ + public static List getValidWalDirs(Configuration conf, Path walBackupDir, long startTime, + long endTime) throws IOException { + FileSystem backupFs = FileSystem.get(walBackupDir.toUri(), conf); + FileStatus[] dayDirs = backupFs.listStatus(new Path(walBackupDir, WALS_DIR)); + + List validDirs = new ArrayList<>(); + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + + for (FileStatus dayDir : dayDirs) { + if (!dayDir.isDirectory()) { + continue; // Skip files, only process directories + } + + String dirName = dayDir.getPath().getName(); + try { + Date dirDate = dateFormat.parse(dirName); + long dirStartTime = dirDate.getTime(); // Start of that day (00:00:00) + long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) + + // Check if this day's WAL files overlap with the required time range + if (dirEndTime >= startTime && dirStartTime <= endTime) { + validDirs.add(dayDir.getPath().toString()); + } + } catch (ParseException e) { + LOG.warn("Skipping invalid directory name: {}", dirName, e); + } + } + return validDirs; + } } diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index aae88c1bb9a2..9fae7f2120d3 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -173,7 +173,7 @@ public void execute() throws IOException { // copy out the table and region info files for each table BackupUtils.copyTableRegionInfo(conn, backupInfo, conf); // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT - convertWALsToHFiles(); + convertWALsToHFiles(new HashMap<>(), new HashMap<>()); incrementalCopyHFiles(new String[] { getBulkOutputDir().toString() }, backupInfo.getBackupRootDir()); failStageIf(Stage.stage_2); @@ -198,7 +198,7 @@ public void execute() throws IOException { BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap)); backupManager.writeBackupStartCode(newStartCode); - handleBulkLoad(backupInfo.getTableNames()); + handleBulkLoad(backupInfo.getTableNames(), new HashMap<>(), new HashMap<>()); failStageIf(Stage.stage_4); // backup complete diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java index 54f3842f463b..72867da95f17 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java @@ -163,20 +163,18 @@ public void testIncrementalBackupCopyingBulkloadTillIncrCommittedWalTs() throws performBulkLoad("bulkPreIncr", methodName, tableName1); expectedRowCount += ROWS_IN_BULK_LOAD; assertEquals(expectedRowCount, TEST_UTIL.countRows(tableName1)); - assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); + assertTrue(systemTable.readBulkloadRows(List.of(tableName1)).isEmpty()); loadTable(TEST_UTIL.getConnection().getTable(tableName1)); Thread.sleep(15000); performBulkLoad("bulkPostIncr", methodName, tableName1); - assertEquals(2, systemTable.readBulkloadRows(List.of(tableName1)).size()); + assertTrue(systemTable.readBulkloadRows(List.of(tableName1)).isEmpty()); // Incremental backup String backup2 = backupTables(BackupType.INCREMENTAL, List.of(tableName1), BACKUP_ROOT_DIR, true); assertTrue(checkSucceeded(backup2)); - - // bulkPostIncr Bulkload entry should not be deleted post incremental backup - assertEquals(1, systemTable.readBulkloadRows(List.of(tableName1)).size()); + assertTrue(systemTable.readBulkloadRows(List.of(tableName1)).isEmpty()); TEST_UTIL.truncateTable(tableName1); // Restore incremental backup From 6aa212f5abe3b1556740c3e7e80a4761d21a1c15 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Thu, 6 Nov 2025 14:51:46 -0600 Subject: [PATCH 24/24] HBASE-28957. Build + spotless fix --- .../apache/hadoop/hbase/backup/impl/FullTableBackupClient.java | 1 + .../hadoop/hbase/backup/impl/IncrementalTableBackupClient.java | 2 +- .../java/org/apache/hadoop/hbase/backup/TestBackupBase.java | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java index afba233c9a51..63e26fdc245c 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java index 0815d7c5ea38..b3027f09a4ac 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java @@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.mapreduce.WALInputFormat; import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2; +import org.apache.hadoop.hbase.mapreduce.WALInputFormat; import org.apache.hadoop.hbase.mapreduce.WALPlayer; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotManifest; diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java index 9fae7f2120d3..aeabc8698cd8 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map;