4242import org .apache .hadoop .hbase .replication .ReplicationResult ;
4343import org .apache .hadoop .hbase .replication .regionserver .ReplicationSourceInterface ;
4444import org .apache .hadoop .hbase .util .CommonFSUtils ;
45+ import org .apache .hadoop .hbase .util .EnvironmentEdgeManager ;
4546import org .apache .hadoop .hbase .wal .FSHLogProvider ;
4647import org .apache .hadoop .hbase .wal .WAL ;
4748import org .apache .yetus .audience .InterfaceAudience ;
4849import org .slf4j .Logger ;
4950import org .slf4j .LoggerFactory ;
5051
52+ /**
53+ * ContinuousBackupReplicationEndpoint is responsible for replicating WAL entries to a backup
54+ * storage. It organizes WAL entries by day and periodically flushes the data, ensuring that WAL
55+ * files do not exceed the configured size. The class includes mechanisms for handling the WAL
56+ * files, performing bulk load backups, and ensuring that the replication process is safe.
57+ */
5158@ InterfaceAudience .Private
5259public class ContinuousBackupReplicationEndpoint extends BaseReplicationEndpoint {
5360 private static final Logger LOG =
@@ -75,9 +82,9 @@ public class ContinuousBackupReplicationEndpoint extends BaseReplicationEndpoint
7582 private String peerId ;
7683 private ScheduledExecutorService flushExecutor ;
7784
78- private static final long ONE_DAY_IN_MILLISECONDS = TimeUnit .DAYS .toMillis (1 );
85+ public static final long ONE_DAY_IN_MILLISECONDS = TimeUnit .DAYS .toMillis (1 );
7986 public static final String WAL_FILE_PREFIX = "wal_file." ;
80- private static final String DATE_FORMAT = "yyyy-MM-dd" ;
87+ public static final String DATE_FORMAT = "yyyy-MM-dd" ;
8188
8289 @ Override
8390 public void init (Context context ) throws IOException {
@@ -139,6 +146,8 @@ private void flushAndBackupSafely() {
139146 LOG .info ("{} Periodic WAL flush triggered" , Utils .logPeerId (peerId ));
140147 flushWriters ();
141148 replicationSource .persistOffsets ();
149+ LOG .info ("{} Periodic WAL flush and offset persistence completed successfully" ,
150+ Utils .logPeerId (peerId ));
142151 } catch (IOException e ) {
143152 LOG .error ("{} Error during WAL flush: {}" , Utils .logPeerId (peerId ), e .getMessage (), e );
144153 } finally {
@@ -150,7 +159,13 @@ private void flushWriters() throws IOException {
150159 for (Map .Entry <Long , FSHLogProvider .Writer > entry : walWriters .entrySet ()) {
151160 FSHLogProvider .Writer writer = entry .getValue ();
152161 if (writer != null ) {
153- writer .close ();
162+ try {
163+ writer .close ();
164+ } catch (IOException e ) {
165+ LOG .error ("{} Failed to close WAL writer for day: {}. Error: {}" , Utils .logPeerId (peerId ),
166+ entry .getKey (), e .getMessage ());
167+ throw e ;
168+ }
154169 }
155170 }
156171 walWriters .clear ();
@@ -250,14 +265,17 @@ private FSHLogProvider.Writer createWalWriter(long dayInMillis) {
250265 fs .mkdirs (dayDir );
251266
252267 // Generate a unique WAL file name
253- String walFileName = WAL_FILE_PREFIX + dayInMillis + "." + UUID .randomUUID ();
268+ long currentTime = EnvironmentEdgeManager .getDelegate ().currentTime ();
269+ String walFileName = WAL_FILE_PREFIX + currentTime + "." + UUID .randomUUID ();
254270 Path walFilePath = new Path (dayDir , walFileName );
255271
256272 // Initialize the WAL writer
257273 FSHLogProvider .Writer writer =
258274 ObjectStoreProtobufWalWriter .class .getDeclaredConstructor ().newInstance ();
259275 writer .init (fs , walFilePath , conf , true , WALUtil .getWALBlockSize (conf , fs , walFilePath ),
260276 StreamSlowMonitor .create (conf , walFileName ));
277+
278+ LOG .debug ("{} Created WAL writer for day: {}" , Utils .logPeerId (peerId ), dayDirectoryName );
261279 return writer ;
262280 } catch (Exception e ) {
263281 throw new UncheckedIOException (
@@ -282,12 +300,15 @@ protected void doStop() {
282300
283301 private void close () {
284302 shutdownFlushExecutor ();
303+ lock .lock ();
285304 try {
286305 flushWriters ();
287306 replicationSource .persistOffsets ();
288307 } catch (IOException e ) {
289308 LOG .error ("{} Failed to Flush Open Wal Writers: {}" , Utils .logPeerId (peerId ), e .getMessage (),
290309 e );
310+ } finally {
311+ lock .unlock ();
291312 }
292313 }
293314
@@ -302,13 +323,14 @@ private void uploadBulkLoadFiles(List<Path> bulkLoadFiles) throws IOException {
302323 LOG .info ("{} Bulk load file {} successfully backed up to {}" , Utils .logPeerId (peerId ), file ,
303324 destPath );
304325 } catch (IOException e ) {
305- LOG .error ("{} Failed to back up bulk load file: {}" , Utils .logPeerId (peerId ), file , e );
326+ LOG .error ("{} Failed to back up bulk load file {}: {}" , Utils .logPeerId (peerId ), file ,
327+ e .getMessage (), e );
306328 throw e ;
307329 }
308330 }
309331 }
310332
311- public Path getBulkLoadFileStagingPath (Path relativePathFromNamespace ) throws IOException {
333+ private Path getBulkLoadFileStagingPath (Path relativePathFromNamespace ) throws IOException {
312334 FileSystem rootFs = CommonFSUtils .getRootDirFileSystem (conf );
313335 Path rootDir = CommonFSUtils .getRootDir (conf );
314336 Path baseNSDir = new Path (HConstants .BASE_NAMESPACE_DIR );
@@ -344,6 +366,8 @@ private void shutdownFlushExecutor() {
344366 !flushExecutor .awaitTermination (EXECUTOR_TERMINATION_TIMEOUT_SECONDS , TimeUnit .SECONDS )
345367 ) {
346368 flushExecutor .shutdownNow ();
369+ LOG .warn ("{} Flush executor did not terminate within timeout, forcing shutdown." ,
370+ Utils .logPeerId (peerId ));
347371 }
348372 } catch (InterruptedException e ) {
349373 Thread .currentThread ().interrupt ();
0 commit comments