From 2104620ac09e54d0db32078706664a0b1e18efb8 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Tue, 18 Feb 2025 23:15:34 +0530 Subject: [PATCH 01/12] 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 094fa4aaa786..d6d59a39b527 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); @@ -861,4 +864,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 70a6d73c6202..f53d9acc24f0 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 @@ -415,7 +415,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 fcee6f7e680984279d04d508cda6ce433a677689 Mon Sep 17 00:00:00 2001 From: vinayak hegde Date: Tue, 4 Mar 2025 22:27:44 +0530 Subject: [PATCH 02/12] 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 | 255 +++++++++++---- .../hbase/backup/impl/TableBackupClient.java | 2 +- .../hbase/backup/TestContinuousBackup.java | 302 ++++++++++++++++++ .../src/main/protobuf/Backup.proto | 1 + 12 files changed, 713 insertions(+), 69 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 30a5674eb021..5d35c8bc3fa1 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 @@ -96,6 +96,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 = @@ -122,6 +127,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 41dc300abfaf..a836ac1e09e6 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 @@ -193,8 +193,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"); } @@ -232,6 +232,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 2ec6c6adbd4f..cfe2e5b80a86 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 @@ -168,6 +168,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:"; @@ -1025,6 +1026,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 @@ -1046,6 +1078,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 @@ -1374,6 +1434,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 @@ -1388,6 +1460,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 c4017e8c1a1a..d71c6ce6b4d2 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,25 @@ */ 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.HashMap; +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; @@ -38,7 +47,13 @@ import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager; 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; @@ -62,9 +77,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 @@ -132,67 +147,14 @@ 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 ..."); - - Map props = new HashMap<>(); - props.put("backupRoot", backupInfo.getBackupRootDir()); - admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, - LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); - - 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); - - // backup complete completeBackup(conn, backupInfo, BackupType.FULL, conf); } catch (Exception e) { failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", @@ -201,6 +163,179 @@ 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(admin); + 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(Admin admin) 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 ..."); + Map props = new HashMap<>(); + props.put("backupRoot", backupInfo.getBackupRootDir()); + admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, + LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props); + + 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 430676cbc0f4daad3b29b50a6e57adf5be82dac0 Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Wed, 12 Mar 2025 18:29:55 +0530 Subject: [PATCH 03/12] HBASE-28990 Modify Incremental Backup for Continuous Backup --- .../hbase/backup/impl/BackupAdminImpl.java | 6 +- .../hbase/backup/impl/BackupManager.java | 11 ++ .../backup/impl/FullTableBackupClient.java | 7 +- .../backup/impl/IncrementalBackupManager.java | 54 ++++++ .../impl/IncrementalTableBackupClient.java | 4 +- .../ContinuousBackupReplicationEndpoint.java | 10 +- .../hadoop/hbase/backup/TestBackupBase.java | 21 +++ .../hbase/backup/TestContinuousBackup.java | 15 +- .../TestIncrementalBackupWithBulkLoad.java | 55 ++++++- .../TestIncrementalBackupWithContinuous.java | 155 ++++++++++++++++++ 10 files changed, 308 insertions(+), 30 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/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java index 1e745c69cdad..686ebe1e9ce9 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 @@ -519,7 +519,11 @@ public String backupTables(BackupRequest request) throws IOException { if (type == BackupType.INCREMENTAL) { Set incrTableSet; try (BackupSystemTable table = new BackupSystemTable(conn)) { - incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); + if (request.isContinuousBackupEnabled()) { + incrTableSet = table.getContinuousBackupTableSet().keySet(); + } else { + incrTableSet = table.getIncrementalBackupTableSet(targetRootDir); + } } if (incrTableSet.isEmpty()) { 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 a836ac1e09e6..2aa8c4b10056 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 @@ -436,4 +436,15 @@ public void addContinuousBackupTableSet(Set tables, long startTimesta throws IOException { systemTable.addContinuousBackupTableSet(tables, startTimestamp); } + + /** + * 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 { + return systemTable.getContinuousBackupTableSet(); + } } 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 d71c6ce6b4d2..334de1160026 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 @@ -148,6 +148,8 @@ protected void snapshotCopy(BackupInfo backupInfo) throws IOException { public void execute() throws IOException { try (Admin admin = conn.getAdmin()) { beginBackup(backupManager, backupInfo); + initializeBackupStartCode(backupManager); + performLogRoll(admin); if (backupInfo.isContinuousBackupEnabled()) { handleContinuousBackup(admin); @@ -155,6 +157,7 @@ public void execute() throws IOException { handleNonContinuousBackup(admin); } + updateBackupMetadata(); completeBackup(conn, backupInfo, BackupType.FULL, conf); } catch (Exception e) { failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", @@ -185,16 +188,12 @@ private void handleContinuousBackup(Admin admin) throws IOException { } private void handleNonContinuousBackup(Admin admin) throws IOException { - initializeBackupStartCode(backupManager); - performLogRoll(admin); 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 { diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index c92c0747e83c..34a95cda394d 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -17,8 +17,16 @@ */ 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.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.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -131,6 +139,10 @@ private List getLogFilesForNewBackup(Map olderTimestamps, LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps + "\n newestTimestamps: " + newestTimestamps); + if (backupInfo.isContinuousBackupEnabled()) { + return getBackupLogs(conf, savedStartCode); + } + Path walRootDir = CommonFSUtils.getWALRootDir(conf); Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME); Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME); @@ -251,6 +263,48 @@ private List getLogFilesForNewBackup(Map olderTimestamps, return resultLogFiles; } + private List getBackupLogs(Configuration conf, String savedStartCode) throws IOException { + // get log files from backup dir + String walBackupDir = getConf().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + if (walBackupDir == null || walBackupDir.isEmpty()) { + 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); + + 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 the day + // (23:59:59) + + if (dirEndTime >= Long.parseLong(savedStartCode)) { + Path dirPath = dayDir.getPath(); + FileStatus[] logs = backupFs.listStatus(dirPath); + ; + for (FileStatus log : logs) { + String filepath = log.getPath().toString(); + LOG.debug("currentLogFile: " + filepath); + resultLogFiles.add(filepath); + } + } + } catch (ParseException e) { + LOG.warn("Skipping invalid directory name: " + dirName, e); + } + } + return resultLogFiles; + } + static class NewestLogFilter implements PathFilter { private Long lastBackupTS = 0L; 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 4aa076caeb14..db3b17dcfc61 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 @@ -384,6 +384,9 @@ protected void convertWALsToHFiles() throws IOException { List incrBackupFileList = backupInfo.getIncrBackupFileList(); // Get list of tables in incremental backup set Set tableSet = backupManager.getIncrementalBackupTableSet(); + if (backupInfo.isContinuousBackupEnabled()) { + tableSet = backupManager.getContinuousBackupTableSet().keySet(); + } // filter missing files out (they have been copied by previous backups) incrBackupFileList = filterMissingFiles(incrBackupFileList); List tableList = new ArrayList(); @@ -396,7 +399,6 @@ protected void convertWALsToHFiles() throws IOException { } } walToHFiles(incrBackupFileList, tableList); - } protected boolean tableExists(TableName table, Connection conn) throws IOException { 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..e2ed33da06bc 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 @@ -19,8 +19,9 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.text.SimpleDateFormat; -import java.util.Date; +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; import java.util.List; import java.util.Map; import java.util.UUID; @@ -282,8 +283,9 @@ 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)); + DateTimeFormatter formatter = + DateTimeFormatter.ofPattern(DATE_FORMAT).withZone(ZoneId.systemDefault()); + String dayDirectoryName = formatter.format(Instant.ofEpochMilli(dayInMillis)); FileSystem fs = backupFileSystemManager.getBackupFs(); Path walsDir = backupFileSystemManager.getWalsDir(); 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 b9a76347440e..abfb9fac4011 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,8 @@ */ package org.apache.hadoop.hbase.backup; +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.CONTINUOUS_BACKUP_REPLICATION_PEER; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -406,6 +408,15 @@ 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 { Connection conn = null; @@ -546,4 +557,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/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/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java index a182144a8abd..92295490f979 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java @@ -17,11 +17,15 @@ */ package org.apache.hadoop.hbase.backup; +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.junit.Assert.assertTrue; +import java.io.IOException; import java.util.List; import java.util.Map; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; @@ -35,7 +39,10 @@ import org.apache.hadoop.hbase.tool.TestBulkLoadHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,7 +56,7 @@ * Incremental backup t1 */ @Category(LargeTests.class) -public class TestIncrementalBackupWithBulkLoad extends TestBackupBase { +public class TestIncrementalBackupWithBulkLoad extends TestIncrementalBackupWithContinuous { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -57,9 +64,29 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase { private static final Logger LOG = LoggerFactory.getLogger(TestIncrementalBackupDeleteTable.class); + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL = new HBaseTestingUtil(); + conf1 = TEST_UTIL.getConfiguration(); + autoRestoreOnFailure = true; + useSecondCluster = false; + conf1.setInt(CONF_STAGED_WAL_FLUSH_INTERVAL, 1); + conf1.setInt(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, 1); + setUpHelper(); + } + + @Before + public void beforeTest() throws IOException { + super.beforeTest(); + } + + @After + public void afterTest() throws IOException { + super.afterTest(); + } + // implement all test cases in 1 test since incremental backup/restore has dependencies - @Test - public void TestIncBackupDeleteTable() throws Exception { + public void TestIncBackupDeleteTable(boolean withContinuousBackup) throws Exception { String testName = "TestIncBackupDeleteTable"; // #1 - create full backup for all tables LOG.info("create full backup image for all tables"); @@ -69,7 +96,8 @@ public void TestIncBackupDeleteTable() throws Exception { Admin admin = conn.getAdmin(); BackupAdminImpl client = new BackupAdminImpl(conn); - BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + BackupRequest request = + createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR, false, withContinuousBackup); String backupIdFull = client.backupTables(request); assertTrue(checkSucceeded(backupIdFull)); @@ -96,7 +124,8 @@ public void TestIncBackupDeleteTable() throws Exception { // #3 - incremental backup for table1 tables = Lists.newArrayList(table1); - request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR, false, + withContinuousBackup); String backupIdIncMultiple = client.backupTables(request); assertTrue(checkSucceeded(backupIdIncMultiple)); // #4 bulk load again @@ -109,7 +138,8 @@ public void TestIncBackupDeleteTable() throws Exception { // #5 - incremental backup for table1 tables = Lists.newArrayList(table1); - request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR, false, + withContinuousBackup); String backupIdIncMultiple1 = client.backupTables(request); assertTrue(checkSucceeded(backupIdIncMultiple1)); // Delete all data in table1 @@ -123,7 +153,8 @@ public void TestIncBackupDeleteTable() throws Exception { Table hTable = conn.getTable(table1); Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2 + actual + actual1); - request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); + request = + createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR, false, withContinuousBackup); backupIdFull = client.backupTables(request); try (final BackupSystemTable table = new BackupSystemTable(conn)) { @@ -138,4 +169,14 @@ public void TestIncBackupDeleteTable() throws Exception { admin.close(); conn.close(); } + + @Test + public void TestIncBackupDeleteTable() throws Exception { + TestIncBackupDeleteTable(false); + } + + @Test + public void TestIncBackupDeleteTableWithContinuousBackup() throws Exception { + TestIncBackupDeleteTable(true); + } } 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..8f851af89c6d --- /dev/null +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithContinuous.java @@ -0,0 +1,155 @@ +/* + * 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.*; +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.junit.Assert.*; + +import java.io.IOException; +import java.util.*; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +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.*; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.util.ToolRunner; +import org.junit.After; +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.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"); + String backupWalDirName = "TestContinuousBackupWalDir"; + + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL = new HBaseTestingUtil(); + conf1 = TEST_UTIL.getConfiguration(); + autoRestoreOnFailure = true; + useSecondCluster = false; + conf1.setInt(CONF_STAGED_WAL_FLUSH_INTERVAL, 1); + conf1.setInt(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, 1); + setUpHelper(); + } + + @Before + public void beforeTest() throws IOException { + super.beforeTest(); + } + + @After + public void afterTest() throws IOException { + super.afterTest(); + } + + @Test + public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception { + LOG.info("Testing incremental backup with continuous backup"); + 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("ANKIT now run incremental backup"); + 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); + + // 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); + } + } + + 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)); + } +} From ba54badd6b1d0b7978f0d43afbbcb789f6b501da Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Fri, 21 Mar 2025 15:50:03 +0530 Subject: [PATCH 04/12] Minor log fix --- .../hbase/backup/TestIncrementalBackupWithContinuous.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8f851af89c6d..5b708e95456e 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 @@ -119,7 +119,7 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception // Thread.sleep(5000); // Run incremental backup - LOG.info("ANKIT now 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); From 8e1d0331de3698713527bd6184be748533cd3984 Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Tue, 22 Apr 2025 00:52:58 +0530 Subject: [PATCH 05/12] Incorporated commit WAL timestamp --- .../hadoop/hbase/backup/BackupInfo.java | 13 ++ .../hbase/backup/impl/BackupAdminImpl.java | 53 ++++--- .../backup/impl/IncrementalBackupManager.java | 54 ------- .../impl/IncrementalTableBackupClient.java | 143 ++++++++++++++---- .../hbase/backup/impl/TableBackupClient.java | 3 + .../ContinuousBackupReplicationEndpoint.java | 2 - .../hadoop/hbase/backup/util/BackupUtils.java | 4 + .../hbase/backup/TestContinuousBackup.java | 9 -- .../TestIncrementalBackupWithBulkLoad.java | 1 - 9 files changed, 164 insertions(+), 118 deletions(-) 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..e6a578691c0c 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; } 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 686ebe1e9ce9..b3a8e6bca60e 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 @@ -517,32 +517,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)) { - if (request.isContinuousBackupEnabled()) { - incrTableSet = table.getContinuousBackupTableSet().keySet(); - } else { + 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) { @@ -569,7 +584,7 @@ public String backupTables(BackupRequest request) throws IOException { } } if (nonExistingTableList != null) { - if (type == BackupType.INCREMENTAL) { + 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/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index 34a95cda394d..c92c0747e83c 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -17,16 +17,8 @@ */ 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.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.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -139,10 +131,6 @@ private List getLogFilesForNewBackup(Map olderTimestamps, LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps + "\n newestTimestamps: " + newestTimestamps); - if (backupInfo.isContinuousBackupEnabled()) { - return getBackupLogs(conf, savedStartCode); - } - Path walRootDir = CommonFSUtils.getWALRootDir(conf); Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME); Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME); @@ -263,48 +251,6 @@ private List getLogFilesForNewBackup(Map olderTimestamps, return resultLogFiles; } - private List getBackupLogs(Configuration conf, String savedStartCode) throws IOException { - // get log files from backup dir - String walBackupDir = getConf().get(CONF_CONTINUOUS_BACKUP_WAL_DIR); - if (walBackupDir == null || walBackupDir.isEmpty()) { - 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); - - 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 the day - // (23:59:59) - - if (dirEndTime >= Long.parseLong(savedStartCode)) { - Path dirPath = dayDir.getPath(); - FileStatus[] logs = backupFs.listStatus(dirPath); - ; - for (FileStatus log : logs) { - String filepath = log.getPath().toString(); - LOG.debug("currentLogFile: " + filepath); - resultLogFiles.add(filepath); - } - } - } catch (ParseException e) { - LOG.warn("Skipping invalid directory name: " + dirName, e); - } - } - return resultLogFiles; - } - static class NewestLogFilter implements PathFilter { private Long lastBackupTS = 0L; 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 bf9be881daa7..1cdbd9ac280d 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,27 @@ */ 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 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; @@ -262,9 +271,13 @@ 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(); + if (backupInfo.isContinuousBackupEnabled()) { + LOG.debug("For incremental backup, current table set is " + backupInfo.getTables()); + } else { + LOG.debug("For incremental backup, current table set is " + + backupManager.getIncrementalBackupTableSet()); + newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap(); + } } catch (Exception e) { // fail the overall backup and return failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ", @@ -291,21 +304,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()); @@ -362,25 +378,86 @@ 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(); if (backupInfo.isContinuousBackupEnabled()) { - tableSet = backupManager.getContinuousBackupTableSet().keySet(); + 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) { + walBackupFileList = getBackupLogs(backup.getStartTs()); + } else { + walBackupFileList = getBackupLogs(backup.getIncrCommittedWalTs()); + } + walToHFiles(walBackupFileList, Arrays.asList(table.getNameAsString())); + 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); } - // 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"); + } + + private List getBackupLogs(long startTs) throws IOException { + // get log files from backup dir + String walBackupDir = conf.get(CONF_CONTINUOUS_BACKUP_WAL_DIR); + if (walBackupDir == null || walBackupDir.isEmpty()) { + 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); + + 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 the 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("currentLogFile: " + filepath); + resultLogFiles.add(filepath); + } + } + } catch (ParseException e) { + LOG.warn("Skipping invalid directory name: " + dirName, e); } } - walToHFiles(incrBackupFileList, tableList); + return resultLogFiles; } protected boolean tableExists(TableName table, Connection conn) throws IOException { 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 9e31ca409ada..18e95a3fb20f 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 @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; +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.util.CommonFSUtils; @@ -112,7 +113,9 @@ protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo) backupManager.setBackupInfo(backupInfo); // set the start timestamp of the overall backup long startTs = EnvironmentEdgeManager.currentTime(); + long committedWALsTs = BackupUtils.getReplicationCheckpoint(conn); backupInfo.setStartTs(startTs); + backupInfo.setIncrCommittedWalTs(committedWALsTs); // set overall backup status: ongoing backupInfo.setState(BackupState.RUNNING); backupInfo.setPhase(BackupPhase.REQUEST); 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 fa552a9d0734..e2ed33da06bc 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 @@ -22,8 +22,6 @@ import java.time.Instant; import java.time.ZoneId; import java.time.format.DateTimeFormatter; -import java.text.SimpleDateFormat; -import java.util.Date; 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/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java index 15159ed73e46..a40f65e7b844 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 @@ -770,4 +770,8 @@ public static String findMostRecentBackupId(String[] backupIds) { return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp; } + public static long getReplicationCheckpoint(Connection conn) throws IOException { + // TODO this will be fixed in PR https://github.com/apache/hbase/pull/6717 + return System.currentTimeMillis(); + } } 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 f9b67170a086..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 @@ -288,13 +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/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java index 995fbcefdf77..9ea82995d34f 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.tool.TestBulkLoadHFiles; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; import org.junit.After; import org.junit.Assert; import org.junit.Before; From 7b26cb03cbbb37f97e2b38c1f2028ab06e9f9e34 Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Mon, 28 Apr 2025 13:12:12 +0530 Subject: [PATCH 06/12] Addressed review comments --- .../hbase/backup/impl/BackupAdminImpl.java | 4 +++ .../hbase/backup/impl/BackupManager.java | 11 -------- .../backup/impl/FullTableBackupClient.java | 3 --- .../impl/IncrementalTableBackupClient.java | 27 ++++++++++++++----- .../ContinuousBackupReplicationEndpoint.java | 10 +++---- 5 files changed, 29 insertions(+), 26 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 b3a8e6bca60e..2c98940f0c4f 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 @@ -584,6 +584,10 @@ public String backupTables(BackupRequest request) throws IOException { } } if (nonExistingTableList != null) { + // 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 time + // HBASE-14038 if (type == BackupType.INCREMENTAL && !request.isContinuousBackupEnabled()) { // Update incremental backup set tableList = excludeNonExistingTables(tableList, nonExistingTableList); 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 14c3d5e9056c..91563a29b5e4 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 @@ -434,15 +434,4 @@ public void addContinuousBackupTableSet(Set tables, long startTimesta throws IOException { systemTable.addContinuousBackupTableSet(tables, startTimestamp); } - - /** - * 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 { - return systemTable.getContinuousBackupTableSet(); - } } 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 369033e68e8b..d71c6ce6b4d2 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 @@ -148,8 +148,6 @@ protected void snapshotCopy(BackupInfo backupInfo) throws IOException { public void execute() throws IOException { try (Admin admin = conn.getAdmin()) { beginBackup(backupManager, backupInfo); - initializeBackupStartCode(backupManager); - performLogRoll(admin); if (backupInfo.isContinuousBackupEnabled()) { handleContinuousBackup(admin); @@ -157,7 +155,6 @@ public void execute() throws IOException { handleNonContinuousBackup(admin); } - updateBackupMetadata(); completeBackup(conn, backupInfo, BackupType.FULL, conf); } catch (Exception e) { failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", 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 1cdbd9ac280d..11447b0f1a3b 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 @@ -35,6 +35,7 @@ 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; @@ -57,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.WALPlayer; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotManifest; @@ -271,6 +273,11 @@ public void execute() throws IOException, ColumnFamilyMismatchException { // case PREPARE_INCREMENTAL: beginBackup(backupManager, backupInfo); backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL); + // 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 time + // HBASE-14038 + // Continuous-incremental backup backs up user provided table list/set if (backupInfo.isContinuousBackupEnabled()) { LOG.debug("For incremental backup, current table set is " + backupInfo.getTables()); } else { @@ -378,6 +385,7 @@ protected void deleteBulkLoadDirectory() throws IOException { } protected void convertWALsToHFiles() throws IOException { + long previousBackupTs = 0L; if (backupInfo.isContinuousBackupEnabled()) { Set tableSet = backupInfo.getTables(); List backupInfos = backupManager.getBackupHistory(true); @@ -389,11 +397,13 @@ protected void convertWALsToHFiles() throws IOException { backup.getBackupId(), table.getNameAsString()); List walBackupFileList; if (backup.getType() == BackupType.FULL) { - walBackupFileList = getBackupLogs(backup.getStartTs()); + previousBackupTs = backup.getStartTs(); } else { - walBackupFileList = getBackupLogs(backup.getIncrCommittedWalTs()); + previousBackupTs = backup.getIncrCommittedWalTs(); } - walToHFiles(walBackupFileList, Arrays.asList(table.getNameAsString())); + walBackupFileList = getBackupLogs(previousBackupTs); + walToHFiles(walBackupFileList, Arrays.asList(table.getNameAsString()), + previousBackupTs); break; } } @@ -414,7 +424,7 @@ protected void convertWALsToHFiles() throws IOException { LOG.warn("Table " + table + " does not exists. Skipping in WAL converter"); } } - walToHFiles(incrBackupFileList, tableList); + walToHFiles(incrBackupFileList, tableList, previousBackupTs); } } @@ -430,6 +440,7 @@ private List getBackupLogs(long startTs) throws IOException { 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()) { @@ -446,7 +457,6 @@ private List getBackupLogs(long startTs) throws IOException { if (dirEndTime >= startTs) { Path dirPath = dayDir.getPath(); FileStatus[] logs = backupFs.listStatus(dirPath); - ; for (FileStatus log : logs) { String filepath = log.getPath().toString(); LOG.debug("currentLogFile: " + filepath); @@ -466,7 +476,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 @@ -480,6 +491,10 @@ 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); + if (backupInfo.isContinuousBackupEnabled()) { + conf.set(WALInputFormat.START_TIME_KEY, Long.toString(previousBackupTs)); + conf.set(WALInputFormat.END_TIME_KEY, Long.toString(backupInfo.getIncrCommittedWalTs())); + } String[] playerArgs = { dirs, StringUtils.join(tableList, ",") }; try { 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 e2ed33da06bc..c973af8102e7 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 @@ -19,9 +19,8 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.time.Instant; -import java.time.ZoneId; -import java.time.format.DateTimeFormatter; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.UUID; @@ -283,9 +282,8 @@ private void backupWalEntries(long day, List walEntries) throws IOExc private FSHLogProvider.Writer createWalWriter(long dayInMillis) { // Convert dayInMillis to "yyyy-MM-dd" format - DateTimeFormatter formatter = - DateTimeFormatter.ofPattern(DATE_FORMAT).withZone(ZoneId.systemDefault()); - String dayDirectoryName = formatter.format(Instant.ofEpochMilli(dayInMillis)); + SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT); + String dayDirectoryName = dateFormat.format(new Date(dayInMillis)); FileSystem fs = backupFileSystemManager.getBackupFs(); Path walsDir = backupFileSystemManager.getWalsDir(); From 39a4f8ce6587df70e4429112b277f867aadc8b66 Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Mon, 12 May 2025 10:57:39 +0530 Subject: [PATCH 07/12] UT fixed --- .../backup/impl/FullTableBackupClient.java | 8 ++ .../hadoop/hbase/backup/TestBackupBase.java | 9 +- .../TestIncrementalBackupWithBulkLoad.java | 55 ++---------- .../TestIncrementalBackupWithContinuous.java | 90 +++++++++++++++++++ 4 files changed, 113 insertions(+), 49 deletions(-) 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 d71c6ce6b4d2..6a7cf9f6ba2c 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 @@ -149,12 +149,20 @@ public void execute() throws IOException { try (Admin admin = conn.getAdmin()) { beginBackup(backupManager, backupInfo); + // 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); + if (backupInfo.isContinuousBackupEnabled()) { handleContinuousBackup(admin); } else { handleNonContinuousBackup(admin); } + backupManager + .deleteBulkLoadedRows(bulkLoadsToDelete.stream().map(BulkLoad::getRowKey).toList()); + completeBackup(conn, backupInfo, BackupType.FULL, conf); } catch (Exception e) { failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ", 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 abfb9fac4011..a3324b96255a 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 @@ -419,13 +419,20 @@ protected BackupRequest createBackupRequest(BackupType type, List tab 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) { diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java index 9ea82995d34f..d24ec160d0cb 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java @@ -17,14 +17,10 @@ */ package org.apache.hadoop.hbase.backup; -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.junit.Assert.assertTrue; -import java.io.IOException; import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupSystemTable; @@ -38,10 +34,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.tool.TestBulkLoadHFiles; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,7 +48,7 @@ * Incremental backup t1 */ @Category(LargeTests.class) -public class TestIncrementalBackupWithBulkLoad extends TestIncrementalBackupWithContinuous { +public class TestIncrementalBackupWithBulkLoad extends TestBackupBase { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -63,29 +56,9 @@ public class TestIncrementalBackupWithBulkLoad extends TestIncrementalBackupWith private static final Logger LOG = LoggerFactory.getLogger(TestIncrementalBackupDeleteTable.class); - @BeforeClass - public static void setUp() throws Exception { - TEST_UTIL = new HBaseTestingUtil(); - conf1 = TEST_UTIL.getConfiguration(); - autoRestoreOnFailure = true; - useSecondCluster = false; - conf1.setInt(CONF_STAGED_WAL_FLUSH_INTERVAL, 1); - conf1.setInt(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, 1); - setUpHelper(); - } - - @Before - public void beforeTest() throws IOException { - super.beforeTest(); - } - - @After - public void afterTest() throws IOException { - super.afterTest(); - } - // implement all test cases in 1 test since incremental backup/restore has dependencies - public void TestIncBackupDeleteTable(boolean withContinuousBackup) throws Exception { + @Test + public void TestIncBackupDeleteTable() throws Exception { String testName = "TestIncBackupDeleteTable"; // #1 - create full backup for all tables LOG.info("create full backup image for all tables"); @@ -95,8 +68,7 @@ public void TestIncBackupDeleteTable(boolean withContinuousBackup) throws Except Admin admin = conn.getAdmin(); BackupAdminImpl client = new BackupAdminImpl(conn); - BackupRequest request = - createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR, false, withContinuousBackup); + BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); String backupIdFull = client.backupTables(request); assertTrue(checkSucceeded(backupIdFull)); @@ -123,8 +95,7 @@ public void TestIncBackupDeleteTable(boolean withContinuousBackup) throws Except // #3 - incremental backup for table1 tables = Lists.newArrayList(table1); - request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR, false, - withContinuousBackup); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); String backupIdIncMultiple = client.backupTables(request); assertTrue(checkSucceeded(backupIdIncMultiple)); // #4 bulk load again @@ -137,8 +108,7 @@ public void TestIncBackupDeleteTable(boolean withContinuousBackup) throws Except // #5 - incremental backup for table1 tables = Lists.newArrayList(table1); - request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR, false, - withContinuousBackup); + request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); String backupIdIncMultiple1 = client.backupTables(request); assertTrue(checkSucceeded(backupIdIncMultiple1)); // Delete all data in table1 @@ -152,8 +122,7 @@ public void TestIncBackupDeleteTable(boolean withContinuousBackup) throws Except Table hTable = conn.getTable(table1); Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2 + actual + actual1); - request = - createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR, false, withContinuousBackup); + request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR); backupIdFull = client.backupTables(request); try (final BackupSystemTable table = new BackupSystemTable(conn)) { @@ -166,14 +135,4 @@ public void TestIncBackupDeleteTable(boolean withContinuousBackup) throws Except admin.close(); conn.close(); } - - @Test - public void TestIncBackupDeleteTable() throws Exception { - TestIncBackupDeleteTable(false); - } - - @Test - public void TestIncBackupDeleteTableWithContinuousBackup() throws Exception { - TestIncBackupDeleteTable(true); - } } 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 5b708e95456e..57f692af837e 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 @@ -23,7 +23,10 @@ import static org.junit.Assert.*; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.*; +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.HBaseTestingUtil; @@ -31,10 +34,13 @@ 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.*; 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.HFileTestUtil; import org.apache.hadoop.util.ToolRunner; import org.junit.After; import org.junit.Before; @@ -61,6 +67,7 @@ public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { private final byte[] FAMILY = Bytes.toBytes("family"); private final byte[] COLUMN = Bytes.toBytes("col"); String backupWalDirName = "TestContinuousBackupWalDir"; + private static final int ROWS_IN_BULK_LOAD = 100; @BeforeClass public static void setUp() throws Exception { @@ -146,10 +153,93 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception } } + @Test + public void testContinuousBackupWithIncrementalBackupAndBulkloadSuccess() throws Exception { + 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)); + // expectedRowCount += ROWS_IN_BULK_LOAD; + 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()); + } + } + 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); + + Map result = + BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(table1, baseDirectory); + assertFalse(result.isEmpty()); + } + + 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 da0703f01ab56f20bd4c533576a5df5382c3b6db Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Sat, 31 May 2025 16:20:38 +0530 Subject: [PATCH 08/12] Addressed review comments --- .../hadoop/hbase/backup/BackupInfo.java | 1 + .../hbase/backup/impl/BackupAdminImpl.java | 5 +- .../hbase/backup/impl/TableBackupClient.java | 8 ++- .../hadoop/hbase/backup/util/BackupUtils.java | 5 -- .../hbase/backup/TestBackupDescribe.java | 1 + .../hbase/backup/TestIncrementalBackup.java | 7 +++ .../TestIncrementalBackupWithContinuous.java | 52 ++++++++++++++----- 7 files changed, 56 insertions(+), 23 deletions(-) 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 e6a578691c0c..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 @@ -562,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 2c98940f0c4f..ea1cd5d61ece 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 @@ -584,10 +584,11 @@ public String backupTables(BackupRequest request) throws IOException { } } if (nonExistingTableList != null) { - // Non-continuous Backup incremental backup is controlled by 'incremental backup table set' + // 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 time - // HBASE-14038 + // 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); 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 18e95a3fb20f..fcb78ef8e2de 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 @@ -113,9 +113,13 @@ protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo) backupManager.setBackupInfo(backupInfo); // set the start timestamp of the overall backup long startTs = EnvironmentEdgeManager.currentTime(); - long committedWALsTs = BackupUtils.getReplicationCheckpoint(conn); backupInfo.setStartTs(startTs); - backupInfo.setIncrCommittedWalTs(committedWALsTs); + 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); + } // set overall backup status: ongoing backupInfo.setState(BackupState.RUNNING); backupInfo.setPhase(BackupPhase.REQUEST); 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 a40f65e7b844..85eeb1c0d189 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 @@ -769,9 +769,4 @@ public static String findMostRecentBackupId(String[] backupIds) { } return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp; } - - public static long getReplicationCheckpoint(Connection conn) throws IOException { - // TODO this will be fixed in PR https://github.com/apache/hbase/pull/6717 - return System.currentTimeMillis(); - } } 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/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java index 7f6182c6e543..134c9d6cbc54 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java @@ -159,6 +159,7 @@ public void TestIncBackupRestore() throws Exception { validateRootPathCanBeOverridden(BACKUP_ROOT_DIR, backupIdFull); assertTrue(checkSucceeded(backupIdFull)); + /* // #2 - insert some data to table Table t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS); LOG.debug("writing " + ADD_ROWS + " rows to " + table1); @@ -285,6 +286,7 @@ public void TestIncBackupRestore() throws Exception { Assert.assertEquals(NB_ROWS_IN_BATCH + 5, HBaseTestingUtil.countRows(hTable)); hTable.close(); admin.close(); + */ } } @@ -323,9 +325,12 @@ public void TestIncBackupRestoreWithOriginalSplits() throws Exception { try (Admin admin = conn.getAdmin()) { List currentRegions = TEST_UTIL.getHBaseCluster().getRegions(table1); + LOG.info("TODO Regions count {}", currentRegions.size()); for (HRegion region : currentRegions) { + LOG.info("TODO Split starting {}", region.getRegionInfo().getRegionNameAsString()); byte[] name = region.getRegionInfo().getEncodedNameAsBytes(); admin.splitRegionAsync(name).get(); + LOG.info("TODO Split done {}", region.getRegionInfo().getRegionNameAsString()); } TEST_UTIL.waitTableAvailable(table1); @@ -336,6 +341,7 @@ public void TestIncBackupRestoreWithOriginalSplits() throws Exception { request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); String incrementalBackupId = backupAdmin.backupTables(request); assertTrue(checkSucceeded(incrementalBackupId)); +/* preRestoreBackupFiles = getBackupFiles(); backupAdmin.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, incrementalBackupId, false, fromTables, toTables, true, true)); @@ -380,6 +386,7 @@ public void TestIncBackupRestoreWithOriginalSplits() throws Exception { int rowsActual = TEST_UTIL.countRows(table1_restore); Assert.assertEquals(rowsExpected, rowsActual); + */ } } 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 57f692af837e..5d8bdd54cb85 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,14 +17,21 @@ */ package org.apache.hadoop.hbase.backup; -import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.*; 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.junit.Assert.*; +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.*; +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; @@ -36,10 +43,15 @@ 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.*; + +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.After; @@ -80,16 +92,6 @@ public static void setUp() throws Exception { setUpHelper(); } - @Before - public void beforeTest() throws IOException { - super.beforeTest(); - } - - @After - public void afterTest() throws IOException { - super.afterTest(); - } - @Test public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception { LOG.info("Testing incremental backup with continuous backup"); @@ -229,11 +231,33 @@ private void performBulkLoad(String keyPrefix, String testDir) throws IOExceptio 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) { + //LOG.debug(Objects.toString(fstat.getPath())); + 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++) { From 940f6c0a97aaa6f1a4f45867edf2f90a7b31a07c Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Sun, 1 Jun 2025 00:29:42 +0530 Subject: [PATCH 09/12] Addressed review comments --- .../hadoop/hbase/backup/impl/BackupAdminImpl.java | 6 +++--- .../backup/impl/IncrementalTableBackupClient.java | 15 ++++++++------- .../TestIncrementalBackupWithContinuous.java | 6 ------ 3 files changed, 11 insertions(+), 16 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 4501d80300b3..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 @@ -873,7 +873,7 @@ public String backupTables(BackupRequest request) throws IOException { 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"; + + "Perform Continuous backup on " + extraTables + " first, then retry the command"; throw new IOException(msg); } } else { @@ -894,7 +894,7 @@ public String backupTables(BackupRequest request) throws IOException { 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"; + + "Perform full backup on " + extraTables + " first, then retry the command"; throw new IOException(msg); } } @@ -928,7 +928,7 @@ public String backupTables(BackupRequest request) throws IOException { if (nonExistingTableList != null) { // 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 time + // 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()) { 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 11447b0f1a3b..f60aeab8b8f3 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 @@ -36,6 +36,7 @@ import java.util.Map; import java.util.Set; import java.util.TimeZone; +import com.google.common.base.Strings; import org.apache.commons.io.FilenameUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileStatus; @@ -275,16 +276,17 @@ public void execute() throws IOException, ColumnFamilyMismatchException { backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL); // 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 time + // 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()) { - LOG.debug("For incremental backup, current table set is " + backupInfo.getTables()); + currentTableSet = backupInfo.getTables(); } else { - LOG.debug("For incremental backup, current table set is " - + backupManager.getIncrementalBackupTableSet()); + 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 : ", @@ -431,7 +433,7 @@ 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 (walBackupDir == null || walBackupDir.isEmpty()) { + if (Strings.isNullOrEmpty(walBackupDir)) { throw new IOException( "Incremental backup requires the WAL backup directory " + CONF_CONTINUOUS_BACKUP_WAL_DIR); } @@ -451,8 +453,7 @@ private List getBackupLogs(long startTs) throws IOException { 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 the day - // (23:59:59) + long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; //End time of day (23:59:59) if (dirEndTime >= startTs) { Path dirPath = dayDir.getPath(); 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 5d8bdd54cb85..3b0fb35dacde 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 @@ -54,8 +54,6 @@ 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.BeforeClass; import org.junit.ClassRule; import org.junit.Test; @@ -78,7 +76,6 @@ public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { private byte[] ROW = Bytes.toBytes("row1"); private final byte[] FAMILY = Bytes.toBytes("family"); private final byte[] COLUMN = Bytes.toBytes("col"); - String backupWalDirName = "TestContinuousBackupWalDir"; private static final int ROWS_IN_BULK_LOAD = 100; @BeforeClass @@ -125,7 +122,6 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception 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"); @@ -175,7 +171,6 @@ public void testContinuousBackupWithIncrementalBackupAndBulkloadSuccess() throws assertTrue(checkSucceeded(backup1)); loadTable(TEST_UTIL.getConnection().getTable(table1)); - // expectedRowCount += ROWS_IN_BULK_LOAD; assertEquals(expectedRowCount, TEST_UTIL.countRows(table1)); performBulkLoad("bulk2", methodName); expectedRowCount += ROWS_IN_BULK_LOAD; @@ -244,7 +239,6 @@ private static Set listFiles(final FileSystem fs, final Path root, final FileStatus[] list = CommonFSUtils.listStatus(fs, dir); if (list != null) { for (FileStatus fstat : list) { - //LOG.debug(Objects.toString(fstat.getPath())); if (fstat.isDirectory()) { LOG.info("Found directory {}", Objects.toString(fstat.getPath())); files.addAll(listFiles(fs, root, fstat.getPath())); From 945774c06567d71d5215432b0dab43532f1e2f05 Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Sun, 1 Jun 2025 00:33:35 +0530 Subject: [PATCH 10/12] Spotless --- .../hbase/backup/impl/IncrementalTableBackupClient.java | 4 ++-- .../apache/hadoop/hbase/backup/TestIncrementalBackup.java | 7 ------- .../hbase/backup/TestIncrementalBackupWithContinuous.java | 2 -- 3 files changed, 2 insertions(+), 11 deletions(-) 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 f60aeab8b8f3..24404d4a42e9 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 @@ -23,6 +23,7 @@ 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 com.google.common.base.Strings; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -36,7 +37,6 @@ import java.util.Map; import java.util.Set; import java.util.TimeZone; -import com.google.common.base.Strings; import org.apache.commons.io.FilenameUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileStatus; @@ -453,7 +453,7 @@ private List getBackupLogs(long startTs) throws IOException { 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) + long dirEndTime = dirStartTime + ONE_DAY_IN_MILLISECONDS - 1; // End time of day (23:59:59) if (dirEndTime >= startTs) { Path dirPath = dayDir.getPath(); diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java index 134c9d6cbc54..7f6182c6e543 100644 --- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java +++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java @@ -159,7 +159,6 @@ public void TestIncBackupRestore() throws Exception { validateRootPathCanBeOverridden(BACKUP_ROOT_DIR, backupIdFull); assertTrue(checkSucceeded(backupIdFull)); - /* // #2 - insert some data to table Table t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS); LOG.debug("writing " + ADD_ROWS + " rows to " + table1); @@ -286,7 +285,6 @@ public void TestIncBackupRestore() throws Exception { Assert.assertEquals(NB_ROWS_IN_BATCH + 5, HBaseTestingUtil.countRows(hTable)); hTable.close(); admin.close(); - */ } } @@ -325,12 +323,9 @@ public void TestIncBackupRestoreWithOriginalSplits() throws Exception { try (Admin admin = conn.getAdmin()) { List currentRegions = TEST_UTIL.getHBaseCluster().getRegions(table1); - LOG.info("TODO Regions count {}", currentRegions.size()); for (HRegion region : currentRegions) { - LOG.info("TODO Split starting {}", region.getRegionInfo().getRegionNameAsString()); byte[] name = region.getRegionInfo().getEncodedNameAsBytes(); admin.splitRegionAsync(name).get(); - LOG.info("TODO Split done {}", region.getRegionInfo().getRegionNameAsString()); } TEST_UTIL.waitTableAvailable(table1); @@ -341,7 +336,6 @@ public void TestIncBackupRestoreWithOriginalSplits() throws Exception { request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR); String incrementalBackupId = backupAdmin.backupTables(request); assertTrue(checkSucceeded(incrementalBackupId)); -/* preRestoreBackupFiles = getBackupFiles(); backupAdmin.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, incrementalBackupId, false, fromTables, toTables, true, true)); @@ -386,7 +380,6 @@ public void TestIncBackupRestoreWithOriginalSplits() throws Exception { int rowsActual = TEST_UTIL.countRows(table1_restore); Assert.assertEquals(rowsExpected, rowsActual); - */ } } 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 3b0fb35dacde..aceb89d4f73c 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 @@ -30,7 +30,6 @@ 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; @@ -43,7 +42,6 @@ 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; From eb6f9f45b400a5f791473f7e2667d5753faf3a06 Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Wed, 4 Jun 2025 21:57:11 +0530 Subject: [PATCH 11/12] Minor fix --- .../impl/IncrementalTableBackupClient.java | 10 ++++++--- .../hbase/backup/impl/TableBackupClient.java | 7 ------- .../TestIncrementalBackupWithContinuous.java | 21 +++++++------------ 3 files changed, 15 insertions(+), 23 deletions(-) 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 24404d4a42e9..de97694f22fb 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 @@ -23,7 +23,6 @@ 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 com.google.common.base.Strings; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -72,6 +71,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.generated.SnapshotProtos; @@ -460,7 +460,7 @@ private List getBackupLogs(long startTs) throws IOException { FileStatus[] logs = backupFs.listStatus(dirPath); for (FileStatus log : logs) { String filepath = log.getPath().toString(); - LOG.debug("currentLogFile: " + filepath); + LOG.debug("Found WAL file: {}", filepath); resultLogFiles.add(filepath); } } @@ -494,7 +494,11 @@ protected void walToHFiles(List dirPaths, List tableList, long p conf.set(JOB_NAME_CONF_KEY, jobname); if (backupInfo.isContinuousBackupEnabled()) { conf.set(WALInputFormat.START_TIME_KEY, Long.toString(previousBackupTs)); - conf.set(WALInputFormat.END_TIME_KEY, Long.toString(backupInfo.getIncrCommittedWalTs())); + // 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, ",") }; 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 fcb78ef8e2de..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 @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage; -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.util.CommonFSUtils; @@ -114,12 +113,6 @@ protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo) // set the start timestamp of the overall backup long startTs = EnvironmentEdgeManager.currentTime(); backupInfo.setStartTs(startTs); - 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); - } // set overall backup status: ongoing backupInfo.setState(BackupState.RUNNING); backupInfo.setPhase(BackupPhase.REQUEST); 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 aceb89d4f73c..c4c7bfa2af01 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,8 +17,8 @@ */ package org.apache.hadoop.hbase.backup; -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.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; @@ -76,20 +76,10 @@ public class TestIncrementalBackupWithContinuous extends TestContinuousBackup { private final byte[] COLUMN = Bytes.toBytes("col"); private static final int ROWS_IN_BULK_LOAD = 100; - @BeforeClass - public static void setUp() throws Exception { - TEST_UTIL = new HBaseTestingUtil(); - conf1 = TEST_UTIL.getConfiguration(); - autoRestoreOnFailure = true; - useSecondCluster = false; - conf1.setInt(CONF_STAGED_WAL_FLUSH_INTERVAL, 1); - conf1.setInt(CONF_STAGED_WAL_FLUSH_INITIAL_DELAY, 1); - setUpHelper(); - } - @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); @@ -120,6 +110,7 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception 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"); @@ -127,6 +118,7 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception 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(); @@ -146,11 +138,13 @@ public void testContinuousBackupWithIncrementalBackupSuccess() throws Exception 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. @@ -205,6 +199,7 @@ public void testContinuousBackupWithIncrementalBackupAndBulkloadSuccess() throws 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); } } From 64fd7b82065725eaa59bb23cdcc510bc5fa677a7 Mon Sep 17 00:00:00 2001 From: Ankit Solomon Date: Wed, 4 Jun 2025 22:03:14 +0530 Subject: [PATCH 12/12] Spotless fix --- .../hbase/backup/TestIncrementalBackupWithContinuous.java | 2 -- 1 file changed, 2 deletions(-) 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 c4c7bfa2af01..79d1df645b95 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 @@ -35,7 +35,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl; import org.apache.hadoop.hbase.backup.impl.BackupManifest; @@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.util.ToolRunner; -import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category;