diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java index 416e9e8dc9b..99a66bd2d7e 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java @@ -43,6 +43,7 @@ import org.apache.phoenix.protobuf.ProtobufUtil; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.replication.reader.ReplicationLogReplayService; import org.apache.phoenix.util.ClientUtil; import org.apache.phoenix.util.SchemaUtil; import org.apache.phoenix.util.ServerUtil; @@ -69,10 +70,14 @@ public void start(CoprocessorEnvironment env) throws IOException { MetricsPhoenixCoprocessorSourceFactory.getInstance().getMetadataCachingSource(); initUncoveredIndexThreadPool(this.conf); this.zkUrl = getLocalZkUrl(conf); + // Start replication log replay + ReplicationLogReplayService.getInstance(conf).start(); } @Override public void stop(CoprocessorEnvironment env) throws IOException { + // Stop replication log replay + ReplicationLogReplayService.getInstance(conf).stop(); RegionServerCoprocessor.super.stop(env); if (uncoveredIndexThreadPool != null) { uncoveredIndexThreadPool diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscovery.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscovery.java new file mode 100644 index 00000000000..1123c205642 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscovery.java @@ -0,0 +1,476 @@ +/* + * 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.phoenix.replication; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogDiscovery; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + +/** + * Abstract base class for discovering and processing replication log files in a round-by-round + * manner. This class provides the core framework for: - Discovering replication log files from + * configured directories (new files and in-progress files) - Processing files in time-based rounds + * with configurable duration and buffer periods - Tracking progress via lastRoundProcessed to + * enable resumable processing - Scheduling periodic replay operations via a configurable executor + * service Round-based Processing: - Files are organized into replication rounds based on timestamps + * - Each round represents a time window (e.g., 1 minute) of replication activity - Processing waits + * for round completion + buffer time before processing to ensure all files are available Subclasses + * must implement: - processFile(Path): Defines how individual replication log files are processed - + * createMetricsSource(): Provides metrics tracking for monitoring - Configuration methods: Thread + * counts, intervals, probabilities, etc. File Processing Flow: 1. Discover new files for the + * current round 2. Mark files as in-progress (move to in-progress directory) 3. Process each file + * via abstract processFile() method 4. Mark successfully processed files as completed (delete from + * in-progress) 5. Update lastRoundProcessed to track progress + */ +public abstract class ReplicationLogDiscovery { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogDiscovery.class); + + /** + * Default number of threads in the executor pool for processing replication logs + */ + private static final int DEFAULT_EXECUTOR_THREAD_COUNT = 1; + + /** + * Default thread name format for executor threads + */ + private static final String DEFAULT_EXECUTOR_THREAD_NAME_FORMAT = "ReplicationLogDiscovery-%d"; + + /** + * Default interval in seconds between replay operations + */ + private static final long DEFAULT_REPLAY_INTERVAL_SECONDS = 10; + + /** + * Default timeout in seconds for graceful shutdown of the executor service + */ + private static final long DEFAULT_SHUTDOWN_TIMEOUT_SECONDS = 30; + + /** + * Default probability (in percentage) for processing files from in-progress directory + */ + private static final double DEFAULT_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY = 5.0; + + /** + * Default buffer percentage for waiting time between processing rounds + */ + private static final double DEFAULT_WAITING_BUFFER_PERCENTAGE = 15.0; + + protected final Configuration conf; + protected final String haGroupName; + protected final ReplicationLogTracker replicationLogTracker; + protected ScheduledExecutorService scheduler; + protected volatile boolean isRunning = false; + protected ReplicationRound lastRoundProcessed; + protected MetricsReplicationLogDiscovery metrics; + protected long roundTimeMills; + protected long bufferMillis; + + public ReplicationLogDiscovery(final ReplicationLogTracker replicationLogTracker) { + this.replicationLogTracker = replicationLogTracker; + this.haGroupName = replicationLogTracker.getHaGroupName(); + this.conf = replicationLogTracker.getConf(); + this.roundTimeMills = replicationLogTracker.getReplicationShardDirectoryManager() + .getReplicationRoundDurationSeconds() * 1000L; + this.bufferMillis = (long) (roundTimeMills * getWaitingBufferPercentage() / 100.0); + } + + public void init() throws IOException { + initializeLastRoundProcessed(); + this.metrics = createMetricsSource(); + } + + public void close() { + if (this.metrics != null) { + this.metrics.close(); + } + } + + /** + * Starts the replication log discovery service by initializing the scheduler and scheduling + * periodic replay operations. Creates a thread pool with configured thread count and schedules + * replay tasks at fixed intervals. + * @throws IOException if there's an error during initialization + */ + public void start() throws IOException { + synchronized (this) { + if (isRunning) { + LOG.warn("ReplicationLogDiscovery is already running for group: {}", haGroupName); + return; + } + // Initialize and schedule the executors + scheduler = Executors.newScheduledThreadPool(getExecutorThreadCount(), + new ThreadFactoryBuilder().setNameFormat(getExecutorThreadNameFormat()).build()); + scheduler.scheduleAtFixedRate(() -> { + try { + replay(); + } catch (Exception e) { + LOG.error("Error during replay", e); + } + }, 0, getReplayIntervalSeconds(), TimeUnit.SECONDS); + + isRunning = true; + LOG.info("ReplicationLogDiscovery started for group: {}", haGroupName); + } + } + + /** + * Stops the replication log discovery service by shutting down the scheduler gracefully. Waits + * for the configured shutdown timeout before forcing shutdown if necessary. + * @throws IOException if there's an error during shutdown + */ + public void stop() throws IOException { + ScheduledExecutorService schedulerToShutdown; + + synchronized (this) { + if (!isRunning) { + LOG.warn("ReplicationLogDiscovery is not running for group: {}", haGroupName); + return; + } + + isRunning = false; + schedulerToShutdown = scheduler; + } + + if (schedulerToShutdown != null && !schedulerToShutdown.isShutdown()) { + schedulerToShutdown.shutdown(); + try { + if (!schedulerToShutdown.awaitTermination(getShutdownTimeoutSeconds(), TimeUnit.SECONDS)) { + schedulerToShutdown.shutdownNow(); + } + } catch (InterruptedException e) { + schedulerToShutdown.shutdownNow(); + Thread.currentThread().interrupt(); + } + } + + LOG.info("ReplicationLogDiscovery stopped for group: {}", haGroupName); + } + + /** + * Executes a replay operation for the next set of replication rounds. This method continuously + * retrieves and processes rounds using getNextRoundToProcess() until: - No more rounds are ready + * to process (not enough time has elapsed), or - An error occurs during processing (will retry in + * next scheduled run) For each round: 1. Calls processRound() to handle new files and optionally + * in-progress files 2. Updates lastRoundProcessed to mark progress 3. Retrieves the next round to + * process + * @throws IOException if there's an error during replay processing + */ + public void replay() throws IOException { + Optional optionalNextRound = getNextRoundToProcess(); + while (optionalNextRound.isPresent()) { + ReplicationRound replicationRound = optionalNextRound.get(); + try { + processRound(replicationRound); + } catch (IOException e) { + LOG.error("Failed processing replication round {}. Will retry in next " + "scheduled run.", + replicationRound, e); + break; // stop this run, retry later + } + setLastRoundProcessed(replicationRound); + optionalNextRound = getNextRoundToProcess(); + } + } + + /** + * Returns the next replication round to process based on lastRoundProcessed. Ensures sufficient + * time (round duration + buffer) has elapsed before returning the next round. + * @return Optional containing the next round to process, or empty if not enough time has passed + */ + protected Optional getNextRoundToProcess() { + long lastRoundEndTimestamp = getLastRoundProcessed().getEndTime(); + long currentTime = EnvironmentEdgeManager.currentTime(); + if (currentTime - lastRoundEndTimestamp < roundTimeMills + bufferMillis) { + // nothing more to process + return Optional.empty(); + } + return Optional + .of(new ReplicationRound(lastRoundEndTimestamp, lastRoundEndTimestamp + roundTimeMills)); + } + + /** + * Processes a single replication round by handling new files and optionally in-progress files. + * Always processes new files for the round, and conditionally processes in-progress files based + * on probability. + * @param replicationRound - The replication round to process + * @throws IOException if there's an error during round processing + */ + protected void processRound(ReplicationRound replicationRound) throws IOException { + LOG.info("Starting to process round: {}", replicationRound); + // Increment the number of rounds processed + getMetrics().incrementNumRoundsProcessed(); + + // Process new files for the round + processNewFilesForRound(replicationRound); + if (shouldProcessInProgressDirectory()) { + // Conditionally process the in progress files for the round + processInProgressDirectory(); + } + LOG.info("Finished processing round: {}", replicationRound); + } + + /** + * Determines whether to process in-progress directory files based on configured probability. Uses + * random number generation to decide if in-progress files should be processed in this cycle. + * @return true if in-progress directory should be processed, false otherwise + */ + protected boolean shouldProcessInProgressDirectory() { + return ThreadLocalRandom.current().nextDouble(100.0) + < getInProgressDirectoryProcessProbability(); + } + + /** + * Processes all new files for a specific replication round. Continuously processes files until no + * new files remain for the round. + * @param replicationRound - The replication round for which to process new files + * @throws IOException if there's an error during file processing + */ + protected void processNewFilesForRound(ReplicationRound replicationRound) throws IOException { + LOG.info("Starting new files processing for round: {}", replicationRound); + long startTime = EnvironmentEdgeManager.currentTime(); + List files = replicationLogTracker.getNewFilesForRound(replicationRound); + LOG.info("Number of new files for round {} is {}", replicationRound, files.size()); + while (!files.isEmpty()) { + processOneRandomFile(files); + files = replicationLogTracker.getNewFilesForRound(replicationRound); + } + long duration = EnvironmentEdgeManager.currentTime() - startTime; + LOG.info("Finished new files processing for round: {} in {}ms", replicationRound, duration); + getMetrics().updateTimeToProcessNewFiles(duration); + } + + /** + * Processes all files (older than 1 round time) in the in-progress directory. Continuously + * processes files until no in-progress files remain. + * @throws IOException if there's an error during file processing + */ + protected void processInProgressDirectory() throws IOException { + // Increase the count for number of times in progress directory is processed + getMetrics().incrementNumInProgressDirectoryProcessed(); + LOG.info("Starting {} directory processing", + replicationLogTracker.getInProgressLogSubDirectoryName()); + long startTime = EnvironmentEdgeManager.currentTime(); + long oldestTimestampToProcess = + replicationLogTracker.getReplicationShardDirectoryManager().getNearestRoundStartTimestamp( + EnvironmentEdgeManager.currentTime()) - getReplayIntervalSeconds() * 1000L; + List files = replicationLogTracker.getOlderInProgressFiles(oldestTimestampToProcess); + LOG.info("Number of {} files with oldestTimestampToProcess {} is {}", + replicationLogTracker.getInProgressLogSubDirectoryName(), oldestTimestampToProcess, + files.size()); + while (!files.isEmpty()) { + processOneRandomFile(files); + files = replicationLogTracker.getOlderInProgressFiles(oldestTimestampToProcess); + } + long duration = EnvironmentEdgeManager.currentTime() - startTime; + LOG.info("Finished in-progress files processing in {}ms", duration); + getMetrics().updateTimeToProcessInProgressFiles(duration); + } + + /** + * Processes a single random file from the provided list. Marks the file as in-progress, processes + * it, and marks it as completed or failed. + * @param files - List of files from which to select and process one randomly + */ + private void processOneRandomFile(final List files) throws IOException { + // Pick a random file and process it + Path file = files.get(ThreadLocalRandom.current().nextInt(files.size())); + Optional optionalInProgressFilePath = Optional.empty(); + try { + optionalInProgressFilePath = replicationLogTracker.markInProgress(file); + if (optionalInProgressFilePath.isPresent()) { + processFile(optionalInProgressFilePath.get()); + replicationLogTracker.markCompleted(optionalInProgressFilePath.get()); + } + } catch (IOException exception) { + LOG.error("Failed to process the file {}", file, exception); + optionalInProgressFilePath.ifPresent(replicationLogTracker::markFailed); + // Not throwing this exception because next time another random file will be retried. + // If it's persistent failure for in_progress directory, + // cluster state should to be DEGRADED_STANDBY_FOR_READER. + } + } + + /** + * Handles the processing of a single file. + * @param path - The file to be processed + * @throws IOException if there's an error during file processing + */ + protected abstract void processFile(Path path) throws IOException; + + /** Creates a new metrics source for monitoring operations. */ + protected abstract MetricsReplicationLogDiscovery createMetricsSource(); + + /** + * Initializes lastRoundProcessed based on minimum timestamp from 1. In-progress files (highest + * priority) - indicates partially processed rounds 2. New files (medium priority) - indicates + * unprocessed rounds waiting to be replayed 3. Current time (fallback) - used when no files + * exist, starts from current time The minimum timestamp is converted to a replication round using + * getReplicationRoundFromEndTime(), which rounds down to the nearest round boundary to ensure we + * start from a complete round. + * @throws IOException if there's an error reading file timestamps + */ + protected void initializeLastRoundProcessed() throws IOException { + Optional minTimestampFromInProgressFiles = getMinTimestampFromInProgressFiles(); + if (minTimestampFromInProgressFiles.isPresent()) { + LOG.info("Initializing lastRoundProcessed from {} files with minimum " + "timestamp as {}", + replicationLogTracker.getInProgressLogSubDirectoryName(), + minTimestampFromInProgressFiles.get()); + this.lastRoundProcessed = replicationLogTracker.getReplicationShardDirectoryManager() + .getReplicationRoundFromEndTime(minTimestampFromInProgressFiles.get()); + } else { + Optional minTimestampFromNewFiles = getMinTimestampFromNewFiles(); + if (minTimestampFromNewFiles.isPresent()) { + LOG.info("Initializing lastRoundProcessed from {} files with minimum timestamp " + "as {}", + replicationLogTracker.getInSubDirectoryName(), minTimestampFromNewFiles.get()); + this.lastRoundProcessed = replicationLogTracker.getReplicationShardDirectoryManager() + .getReplicationRoundFromEndTime(minTimestampFromNewFiles.get()); + } else { + long currentTime = EnvironmentEdgeManager.currentTime(); + LOG.info("Initializing lastRoundProcessed from current time {}", currentTime); + this.lastRoundProcessed = replicationLogTracker.getReplicationShardDirectoryManager() + .getReplicationRoundFromEndTime(EnvironmentEdgeManager.currentTime()); + } + } + } + + /** + * Get minimum timestamp from in progress files. If no in progress files, return empty. + * @return minimum timestamp from in progress files. + */ + protected Optional getMinTimestampFromInProgressFiles() throws IOException { + List inProgressFiles = replicationLogTracker.getInProgressFiles(); + if (inProgressFiles.isEmpty()) { + return Optional.empty(); + } + return Optional.of(getMinTimestampFromFiles(inProgressFiles)); + } + + /** + * Get minimum timestamp from new files. If no new files, return empty. + * @return minimum timestamp from new files. + */ + protected Optional getMinTimestampFromNewFiles() throws IOException { + List newFiles = replicationLogTracker.getNewFiles(); + if (newFiles.isEmpty()) { + return Optional.empty(); + } + return Optional.of(getMinTimestampFromFiles(newFiles)); + } + + private long getMinTimestampFromFiles(List files) { + long minTimestamp = org.apache.hadoop.hbase.util.EnvironmentEdgeManager.currentTime(); + for (Path file : files) { + minTimestamp = Math.min(minTimestamp, replicationLogTracker.getFileTimestamp(file)); + } + return minTimestamp; + } + + /** + * Returns the executor thread count. Subclasses can override this method to provide custom name + * format. + * @return the executor thread count (default: 1). + */ + public int getExecutorThreadCount() { + return DEFAULT_EXECUTOR_THREAD_COUNT; + } + + /** + * Returns the executor thread name format. Subclasses can override this method to provide custom + * name format. + * @return the executor thread name format (default: ReplicationLogDiscovery-%d). + */ + public String getExecutorThreadNameFormat() { + return DEFAULT_EXECUTOR_THREAD_NAME_FORMAT; + } + + /** + * Returns the replay interval in seconds. Subclasses can override this method to provide custom + * intervals. + * @return The replay interval in seconds (default: 10 seconds). + */ + public long getReplayIntervalSeconds() { + return DEFAULT_REPLAY_INTERVAL_SECONDS; + } + + /** + * Returns the shutdown timeout in seconds. Subclasses can override this method to provide custom + * timeout values. + * @return The shutdown timeout in seconds (default: 30 seconds). + */ + public long getShutdownTimeoutSeconds() { + return DEFAULT_SHUTDOWN_TIMEOUT_SECONDS; + } + + /** + * Returns the probability (in percentage) for processing files from in-progress directory. + * Subclasses can override this method to provide custom probabilities. + * @return The probability (default 5.0%) + */ + public double getInProgressDirectoryProcessProbability() { + return DEFAULT_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY; + } + + /** + * Returns the buffer percentage for calculating buffer time. Subclasses can override this method + * to provide custom buffer percentages. + * @return The buffer percentage (default 15.0%) + */ + public double getWaitingBufferPercentage() { + return DEFAULT_WAITING_BUFFER_PERCENTAGE; + } + + public ReplicationLogTracker getReplicationLogFileTracker() { + return this.replicationLogTracker; + } + + public Configuration getConf() { + return this.conf; + } + + public String getHaGroupName() { + return this.haGroupName; + } + + public boolean isRunning() { + return isRunning; + } + + public ReplicationRound getLastRoundProcessed() { + return lastRoundProcessed; + } + + public void setLastRoundProcessed(final ReplicationRound replicationRound) { + this.lastRoundProcessed = replicationRound; + } + + public MetricsReplicationLogDiscovery getMetrics() { + return this.metrics; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroup.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroup.java index 589b6b72a34..880e74a74ea 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroup.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroup.java @@ -57,9 +57,9 @@ public class ReplicationLogGroup { "phoenix.replication.log.standby.hdfs.url"; public static final String REPLICATION_FALLBACK_HDFS_URL_KEY = "phoenix.replication.log.fallback.hdfs.url"; - public static final String REPLICATION_NUM_SHARDS_KEY = "phoenix.replication.log.shards"; - public static final int DEFAULT_REPLICATION_NUM_SHARDS = 1000; - public static final int MAX_REPLICATION_NUM_SHARDS = 100000; + public static final String REPLICATION_LOG_ROTATION_TIME_MS_KEY = + "phoenix.replication.log.rotation.time.ms"; + public static final long DEFAULT_REPLICATION_LOG_ROTATION_TIME_MS = 60 * 1000L; public static final String REPLICATION_LOG_ROTATION_SIZE_BYTES_KEY = "phoenix.replication.log.rotation.size.bytes"; public static final long DEFAULT_REPLICATION_LOG_ROTATION_SIZE_BYTES = 256 * 1024 * 1024L; @@ -85,8 +85,7 @@ public class ReplicationLogGroup { "phoenix.replication.log.retry.delay.ms"; public static final long DEFAULT_REPLICATION_LOG_RETRY_DELAY_MS = 100L; - public static final String SHARD_DIR_FORMAT = "%05d"; - public static final String FILE_NAME_FORMAT = "%d-%s.plog"; + public static final String FILE_NAME_FORMAT = "%d_%s.plog"; /** Cache of ReplicationLogGroup instances by HA Group ID */ protected static final ConcurrentHashMap INSTANCES = diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroupWriter.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroupWriter.java index 385e3c3c0d0..875295f0577 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroupWriter.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroupWriter.java @@ -126,6 +126,7 @@ public abstract class ReplicationLogGroupWriter { protected Disruptor disruptor; protected RingBuffer ringBuffer; protected volatile boolean closed = false; + protected ReplicationShardDirectoryManager replicationShardDirectoryManager; /** The reason for requesting a log rotation. */ protected enum RotationReason { @@ -175,6 +176,7 @@ protected ReplicationLogGroupWriter(ReplicationLogGroup logGroup) { /** Initialize the writer. */ public void init() throws IOException { initializeFileSystems(); + initializeReplicationShardDirectoryManager(); // Start time based rotation. lastRotationTime.set(EnvironmentEdgeManager.currentTimeMillis()); startRotationExecutor(); @@ -246,6 +248,12 @@ public void sync() throws IOException { /** Initialize file systems needed by this writer implementation. */ protected abstract void initializeFileSystems() throws IOException; + /** + * Initialize the {@link ReplicationShardDirectoryManager} to manage file to shard directory + * mapping + */ + protected abstract void initializeReplicationShardDirectoryManager(); + /** * Create a new log writer for rotation. */ diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogTracker.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogTracker.java new file mode 100644 index 00000000000..de12f81fee7 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogTracker.java @@ -0,0 +1,487 @@ +/* + * 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.phoenix.replication; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTracker; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is responsible to track and managing replication log files across different states. Handles + * file lifecycle management including new files, in-progress files, and completed files. + */ +public class ReplicationLogTracker { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogTracker.class); + + /** + * Configuration key for number of retries when deleting files + */ + private static final String FILE_DELETE_RETRIES_KEY = "phoenix.replication.file.delete.retries"; + + /** + * Default number of retries for file deletion operations + */ + private static final int DEFAULT_FILE_DELETE_RETRIES = 3; + + /** + * Configuration key for delay between file deletion retry attempts + */ + private static final String FILE_DELETE_RETRY_DELAY_MS_KEY = + "phoenix.replication.file.delete.retry.delay.ms"; + + /** + * Default delay in milliseconds between file deletion retry attempts + */ + private static final long DEFAULT_FILE_DELETE_RETRY_DELAY_MS = 1000L; + + private final FileSystem fileSystem; + private Path inProgressDirPath; + private final ReplicationShardDirectoryManager replicationShardDirectoryManager; + protected final Configuration conf; + protected final String haGroupName; + protected MetricsReplicationLogTracker metrics; + + public ReplicationLogTracker(final Configuration conf, final String haGroupName, + final FileSystem fileSystem, + final ReplicationShardDirectoryManager replicationShardDirectoryManager, + final MetricsReplicationLogTracker metrics) { + this.conf = conf; + this.haGroupName = haGroupName; + this.fileSystem = fileSystem; + this.replicationShardDirectoryManager = replicationShardDirectoryManager; + this.metrics = metrics; + } + + protected MetricsReplicationLogTracker getMetricsSource() { + return this.metrics; + } + + protected String getInSubDirectoryName() { + return getReplicationShardDirectoryManager().getRootDirectoryPath().getName(); + } + + protected String getInProgressLogSubDirectoryName() { + return getInSubDirectoryName() + "_progress"; + } + + /** + * Initializes the file tracker by setting up new files directory and in-progress directory. + * Creates the in-progress directory (rootURI//[in/out]_progress) if it doesn't exist. + */ + public void init() throws IOException { + this.inProgressDirPath = + new Path(getReplicationShardDirectoryManager().getRootDirectoryPath().getParent(), + getInProgressLogSubDirectoryName()); + createDirectoryIfNotExists(inProgressDirPath); + } + + public void close() { + if (this.metrics != null) { + this.metrics.close(); + } + } + + /** + * Retrieves new replication log files that belong to a specific replication round. It skips the + * invalid files (if any) + * @param replicationRound - The replication round for which to retrieve files + * @return List of valid log file paths that belong to the specified replication round + * @throws IOException if there's an error accessing the file system + */ + protected List getNewFilesForRound(ReplicationRound replicationRound) throws IOException { + Path roundDirectory = replicationShardDirectoryManager.getShardDirectory(replicationRound); + LOG.info("Getting new files for round {} from shard {}", replicationRound, roundDirectory); + if (!fileSystem.exists(roundDirectory)) { + return Collections.emptyList(); + } + + // List the files in roundDirectory + FileStatus[] fileStatuses = fileSystem.listStatus(roundDirectory); + LOG.info("Number of new files found {}", fileStatuses.length); + List filesInRound = new ArrayList<>(); + + // Filter the files belonging to current round + for (FileStatus status : fileStatuses) { + if (status.isFile()) { + if (!isValidLogFile(status.getPath())) { + LOG.warn("Invalid log file found at {}", status.getPath()); + continue; // Skip invalid files + } + try { + long fileTimestamp = getFileTimestamp(status.getPath()); + if ( + fileTimestamp >= replicationRound.getStartTime() + && fileTimestamp <= replicationRound.getEndTime() + ) { + filesInRound.add(status.getPath()); + } + } catch (NumberFormatException exception) { + // Should we throw an exception here instead? + LOG.warn("Failed to extract timestamp from {}. Ignoring the file.", status.getPath()); + } + } + } + return filesInRound; + } + + /** + * Retrieves all valid log files currently in the in-progress directory. + * @return List of valid log file paths in the in-progress directory, empty list if directory + * doesn't exist + * @throws IOException if there's an error accessing the file system + */ + public List getInProgressFiles() throws IOException { + if (!fileSystem.exists(getInProgressDirPath())) { + return Collections.emptyList(); + } + + FileStatus[] fileStatuses = fileSystem.listStatus(getInProgressDirPath()); + List inProgressFiles = new ArrayList<>(); + + for (FileStatus status : fileStatuses) { + if (status.isFile() && isValidLogFile(status.getPath())) { + inProgressFiles.add(status.getPath()); + } + } + + return inProgressFiles; + } + + /** + * Retrieves all valid log files in the in-progress directory that are older than the specified + * timestamp. + * @param timestampThreshold - The timestamp threshold in milliseconds. Files with timestamps less + * than this value will be returned. + * @return List of valid log file paths in the in-progress directory that are older than the + * threshold, empty list if directory doesn't exist or no files match + * @throws IOException if there's an error accessing the file system + */ + public List getOlderInProgressFiles(long timestampThreshold) throws IOException { + if (!fileSystem.exists(getInProgressDirPath())) { + return Collections.emptyList(); + } + + FileStatus[] fileStatuses = fileSystem.listStatus(getInProgressDirPath()); + List olderInProgressFiles = new ArrayList<>(); + + for (FileStatus status : fileStatuses) { + if (status.isFile() && isValidLogFile(status.getPath())) { + try { + long fileTimestamp = getFileTimestamp(status.getPath()); + if (fileTimestamp < timestampThreshold) { + olderInProgressFiles.add(status.getPath()); + } + } catch (NumberFormatException e) { + LOG.warn("Failed to extract timestamp from file {}, skipping", + status.getPath().getName()); + } + } + } + + LOG.debug("Found {} in-progress files older than timestamp {}", olderInProgressFiles.size(), + timestampThreshold); + return olderInProgressFiles; + } + + /** + * Retrieves all valid log files from all shard directories. + * @return List of all valid log file paths from all shard directories + * @throws IOException if there's an error accessing the file system + */ + public List getNewFiles() throws IOException { + List shardPaths = replicationShardDirectoryManager.getAllShardPaths(); + List newFiles = new ArrayList<>(); + for (Path shardPath : shardPaths) { + if (fileSystem.exists(shardPath)) { + FileStatus[] fileStatuses = fileSystem.listStatus(shardPath); + for (FileStatus fileStatus : fileStatuses) { + if (fileStatus.isFile() && isValidLogFile(fileStatus.getPath())) { + newFiles.add(fileStatus.getPath()); + } + } + } + } + return newFiles; + } + + /** + * Marks a file as completed by deleting it from the file system. Uses retry logic with + * configurable retry count and delay. During retry attempts, it fetches the file with same prefix + * (instead of re-using the same file) because it would likely be re-named by some other process + * @param file - The file path to mark as completed + * @return true if file was successfully deleted, false otherwise + */ + protected boolean markCompleted(final Path file) { + long startTime = EnvironmentEdgeManager.currentTimeMillis(); + // Increment the metrics count + getMetrics().incrementMarkFileCompletedRequestCount(); + + int maxRetries = conf.getInt(FILE_DELETE_RETRIES_KEY, DEFAULT_FILE_DELETE_RETRIES); + long retryDelayMs = + conf.getLong(FILE_DELETE_RETRY_DELAY_MS_KEY, DEFAULT_FILE_DELETE_RETRY_DELAY_MS); + + Path fileToDelete = file; + final String filePrefix = getFilePrefix(fileToDelete); + + for (int attempt = 0; attempt <= maxRetries; attempt++) { + try { + if (fileSystem.delete(fileToDelete, false)) { + LOG.info("Successfully deleted completed file: {}", fileToDelete); + long endTime = EnvironmentEdgeManager.currentTimeMillis(); + getMetrics().updateMarkFileCompletedTime(endTime - startTime); + return true; + } else { + LOG.warn("Failed to delete file (attempt {}): {}", attempt + 1, fileToDelete); + } + } catch (IOException e) { + LOG.warn("IOException while deleting file (attempt {}): {}", attempt + 1, fileToDelete, e); + } + + // Increment the deletion failure count + getMetrics().incrementMarkFileCompletedRequestFailedCount(); + + // If deletion failed and it's not the last attempt, sleep first then try to find + // matching in-progress file + if (attempt < maxRetries) { + // Sleep before next retry + try { + Thread.sleep(retryDelayMs); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.warn("Interrupted while waiting to retry file deletion: {}", file); + return false; + } + + try { + // List in-progress files and find matching file with same + // _ prefix + List inProgressFiles = getInProgressFiles(); + List matchingFiles = inProgressFiles.stream() + .filter(path -> getFilePrefix(path).equals(filePrefix)).collect(Collectors.toList()); + // Assert only single file exists with that prefix + if (matchingFiles.size() == 1) { + Path matchingFile = matchingFiles.get(0); + LOG.info("Found matching in-progress file: {} for original file: {}", matchingFile, + file); + // Update fileToDelete to the matching file for subsequent retries + fileToDelete = matchingFile; + } else if (matchingFiles.size() > 1) { + LOG.warn("Multiple matching in-progress files found for prefix {}: {}", filePrefix, + matchingFiles.size()); + long endTime = EnvironmentEdgeManager.currentTimeMillis(); + getMetrics().updateMarkFileCompletedTime(endTime - startTime); + return false; + } else { + LOG.warn("No matching in-progress file found for prefix: {}. File must " + "have " + + "been deleted by some other process.", filePrefix); + long endTime = EnvironmentEdgeManager.currentTimeMillis(); + getMetrics().updateMarkFileCompletedTime(endTime - startTime); + return true; + } + } catch (IOException e) { + LOG.warn( + "IOException while searching for matching in-progress file " + "(attempt {}): " + "{}", + attempt + 1, file, e); + } + } + } + + long endTime = EnvironmentEdgeManager.currentTimeMillis(); + getMetrics().updateMarkFileCompletedTime(endTime - startTime); + + LOG.error("Failed to delete file after {} attempts: {}", maxRetries + 1, fileToDelete); + return false; + } + + /** + * Marks a file as in-progress by renaming it with a UUID and moving to in-progress directory. If + * file is already in in-progress directory, only updates the UUID. + * @param file - The file path to mark as in progress + * @return Optional value of renamed path if file rename was successful, else Optional.empty() + */ + protected Optional markInProgress(final Path file) { + long startTime = EnvironmentEdgeManager.currentTimeMillis(); + try { + + final String fileName = file.getName(); + final String newFileName; + final Path targetDirectory; + + // Check if file is already in in-progress directory + if (file.getParent().toUri().getPath().equals(getInProgressDirPath().toString())) { + // File is already in in-progress directory, replace UUID with a new one + // keep the directory same as in progress + String[] parts = fileName.split("_"); + // Remove the last part (UUID) and add new UUID + StringBuilder newNameBuilder = new StringBuilder(); + for (int i = 0; i < parts.length - 1; i++) { + if (i > 0) { + newNameBuilder.append("_"); + } + newNameBuilder.append(parts[i]); + } + String extension = fileName.substring(fileName.lastIndexOf(".")); + newNameBuilder.append("_").append(UUID.randomUUID()).append(extension); + newFileName = newNameBuilder.toString(); + targetDirectory = file.getParent(); + } else { + // File is not in in-progress directory, add UUID and move to IN_PROGRESS directory + String baseName = fileName.substring(0, fileName.lastIndexOf(".")); + String extension = fileName.substring(fileName.lastIndexOf(".")); + newFileName = baseName + "_" + UUID.randomUUID() + extension; + targetDirectory = getInProgressDirPath(); + } + + Path newPath = new Path(targetDirectory, newFileName); + if (fileSystem.rename(file, newPath)) { + LOG.debug("Successfully marked file as in progress: {} -> {}", file.getName(), newFileName); + return Optional.of(newPath); + } else { + LOG.warn("Failed to rename file for in-progress marking: {}", file); + return Optional.empty(); + } + } catch (IOException e) { + LOG.error("IOException while marking file as in progress: {}", file, e); + return Optional.empty(); + } finally { + // Update the metrics + getMetrics().incrementMarkFileInProgressRequestCount(); + long endTime = EnvironmentEdgeManager.currentTimeMillis(); + getMetrics().updateMarkFileInProgressTime(endTime - startTime); + } + } + + /** + * Validates if a file is a valid log file by checking if it ends with ".plog" extension. + * @param file - The file path to validate. + * @return true if file format is correct, else false + */ + protected boolean isValidLogFile(Path file) { + final String fileName = file.getName(); + return fileName.endsWith(".plog"); + } + + /** + * Extracts the timestamp from a log file name. Assumes timestamp is the first part of the + * filename separated by underscore. + * @param file - The file path to extract timestamp from. return the timestamp from the file name + */ + public long getFileTimestamp(Path file) throws NumberFormatException { + String[] parts = file.getName().split("_"); + return Long.parseLong(parts[0]); + } + + /** + * Extracts the UUID from a log file name. Assumes UUID is the last part of the filename before + * the extension. + * @param file - The file path to extract UUID from. + * @return Optional of UUID if file was in progress, else Optional.empty() + */ + protected Optional getFileUUID(Path file) throws NumberFormatException { + String[] parts = file.getName().split("_"); + if (parts.length < 3) { + return Optional.empty(); + } + return Optional.of(parts[parts.length - 1].split("\\.")[0]); + } + + /** + * Extracts everything except the UUID (last part) from a file path. For example, from + * "1704153600000_rs1_12345678-1234-1234-1234-123456789abc.plog" This method will return + * "1704153600000_rs1" + * @param file - The file path to extract prefix from. + */ + protected String getFilePrefix(Path file) { + String fileName = file.getName(); + String[] parts = fileName.split("_"); + if (parts.length < 3) { + return fileName.split("\\.")[0]; // Return full filename if no underscore found + } + + // Return everything except the last part (UUID) + StringBuilder prefix = new StringBuilder(); + for (int i = 0; i < parts.length - 1; i++) { + if (i > 0) { + prefix.append("_"); + } + prefix.append(parts[i]); + } + + return prefix.toString(); + } + + /** + * No op implementation for marking a file as failed as its expected to be picked by by other + * process from in-progress directory + * @param file - The file which needs to be marked as failed + * @return - true if marked as failed, false otherwise + */ + public boolean markFailed(final Path file) { + getMetrics().incrementMarkFileFailedRequestCount(); + return true; + } + + public FileSystem getFileSystem() { + return this.fileSystem; + } + + public ReplicationShardDirectoryManager getReplicationShardDirectoryManager() { + return this.replicationShardDirectoryManager; + } + + protected String getHaGroupName() { + return this.haGroupName; + } + + protected Configuration getConf() { + return this.conf; + } + + protected Path getInProgressDirPath() { + return this.inProgressDirPath; + } + + protected MetricsReplicationLogTracker getMetrics() { + return this.metrics; + } + + /** + * Creates a directory if it doesn't exist. + */ + private void createDirectoryIfNotExists(Path directoryPath) throws IOException { + if (!fileSystem.exists(directoryPath)) { + LOG.info("Creating directory {}", directoryPath); + if (!fileSystem.mkdirs(directoryPath)) { + throw new IOException("Failed to create directory: " + directoryPath); + } + } + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationRound.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationRound.java new file mode 100644 index 00000000000..fe5d38143b4 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationRound.java @@ -0,0 +1,69 @@ +/* + * 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.phoenix.replication; + +import java.util.Objects; + +import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions; + +/** + * Represents a replication round with defined start and end timestamp. Used for grouping + * replication log files into time-based processing windows and managing file distribution across + * shard directories. + */ +public class ReplicationRound { + + private final long startTime; + private final long endTime; + + public ReplicationRound(long startTime, long endTime) { + Preconditions.checkArgument(startTime < endTime); + this.startTime = startTime; + this.endTime = endTime; + } + + public long getStartTime() { + return startTime; + } + + public long getEndTime() { + return endTime; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReplicationRound that = (ReplicationRound) o; + return startTime == that.startTime && endTime == that.endTime; + } + + @Override + public int hashCode() { + return Objects.hash(startTime, endTime); + } + + @Override + public String toString() { + return "ReplicationRound{" + "startTime=" + startTime + ", endTime=" + endTime + '}'; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationShardDirectoryManager.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationShardDirectoryManager.java new file mode 100644 index 00000000000..76c2afcb193 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationShardDirectoryManager.java @@ -0,0 +1,199 @@ +/* + * 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.phoenix.replication; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +/** + * Manages shard-based directory structure for Phoenix replication log files. This class manages + * mapping between replication log files and different shard directories based on timestamp. The + * root directory could be IN (on standby cluster) or OUT(on active cluster) and it manages shard + * interaction within given root directory. + *

+ * Directory Structure: + *

+ * + *
+ * /phoenix/replication//in/shard/
+ * ├── 000/  (files from 00:00:00-00:01:00)
+ * ├── 001/  (files from 00:01:00-00:02:00)
+ * ├── 002/  (files from 00:02:00-00:03:00)
+ * └── ...   (continues for numShards directories)
+ * 
+ */ +public class ReplicationShardDirectoryManager { + + /** + * The number of shards (subfolders) to maintain in the "IN" / "OUT" directory. + */ + public static final String REPLICATION_NUM_SHARDS_KEY = "phoenix.replication.log.shards"; + + /** + * Default number of shard directories. Assuming 400 workers on standby writing replication log + * files every 1 min, and a lag of 2 days, number of files would be 400 * 2 * 24 * 60 = 1152000 + * files. Each shard will have (1152000 / 128) = 9000 files which is very well manageable for + * single HDFS directory + */ + public static final int DEFAULT_REPLICATION_NUM_SHARDS = 128; + + /** + * Format string for shard directory names. Uses 3-digit zero-padded format (e.g., "000", "001", + * "002"). + */ + public static final String SHARD_DIR_FORMAT = "%03d"; + + /** + * Configuration key for the duration of each replication round in seconds. + */ + public static final String PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY = + "phoenix.replication.round.duration.seconds"; + + /** + * Default duration of each replication round in seconds. Files with timestamps within the same + * 60-second window will be placed in the same shard directory. This provides a good balance + * between file distribution and processing efficiency. + */ + public static final int DEFAULT_REPLICATION_ROUND_DURATION_SECONDS = 60; + + private static final String REPLICATION_SHARD_SUB_DIRECTORY_NAME = "shard"; + + private final int numShards; + + private final int replicationRoundDurationSeconds; + + private final Path shardDirectoryPath; + + private final Path rootDirectoryPath; + + public ReplicationShardDirectoryManager(final Configuration conf, final Path rootPath) { + this.rootDirectoryPath = rootPath; + this.shardDirectoryPath = + new Path(rootPath.toUri().getPath(), REPLICATION_SHARD_SUB_DIRECTORY_NAME); + this.numShards = conf.getInt(REPLICATION_NUM_SHARDS_KEY, DEFAULT_REPLICATION_NUM_SHARDS); + this.replicationRoundDurationSeconds = conf.getInt( + PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, DEFAULT_REPLICATION_ROUND_DURATION_SECONDS); + } + + /** + * Returns the shard directory to which file with given timestamp belongs to based on round time + * period + * @param fileTimestamp The timestamp in milliseconds since epoch + * @return The shard directory path for the given timestamp + */ + public Path getShardDirectory(long fileTimestamp) { + // 1. Figure out how many seconds have passed from start of the day for this file + // Convert timestamp to seconds since epoch + long secondsSinceEpoch = fileTimestamp / 1000L; + + // Calculate seconds since start of the day (00:00:00) + // Get the number of seconds since the start of the current day + long secondsSinceStartOfDay = secondsSinceEpoch % TimeUnit.DAYS.toSeconds(1); + + // 2. Calculate which shard this timestamp belongs to + // Each shard represents a time range: 0 to roundTimeSeconds = shard 0, + // roundTimeSeconds to 2*roundTimeSeconds = shard 1, etc. + int shardIndex = (int) (secondsSinceStartOfDay / replicationRoundDurationSeconds); + + // Apply modulo to ensure shard index doesn't exceed numShards + shardIndex = shardIndex % numShards; + + // Create the shard directory path with formatted shard number + String shardDirName = String.format(SHARD_DIR_FORMAT, shardIndex); + return new Path(shardDirectoryPath, shardDirName); + } + + /** + * Returns the shard directory to which file with given replication round belongs to. + * @param replicationRound The replication round for which to get the shard directory + * @return The shard directory path for the given replication round + */ + public Path getShardDirectory(ReplicationRound replicationRound) { + return getShardDirectory(replicationRound.getStartTime()); + } + + /** + * Returns a ReplicationRound object based on the given round start time, calculating the end time + * as start time + round duration. + * @param roundStartTime - start time of the given round. + * @return The round to which input roundStartTime belongs to + */ + public ReplicationRound getReplicationRoundFromStartTime(long roundStartTime) { + long validRoundStartTime = getNearestRoundStartTimestamp(roundStartTime); + long validRoundEndTime = validRoundStartTime + replicationRoundDurationSeconds * 1000L; + return new ReplicationRound(validRoundStartTime, validRoundEndTime); + } + + /** + * Returns a ReplicationRound object based on the given round end time, calculating the start time + * as end time - round duration. + * @param roundEndTime - end time of the given round. + * @return The round to which input roundEndTime belongs to + */ + public ReplicationRound getReplicationRoundFromEndTime(long roundEndTime) { + long validRoundEndTime = getNearestRoundStartTimestamp(roundEndTime); + long validRoundStartTime = validRoundEndTime - replicationRoundDurationSeconds * 1000L; + return new ReplicationRound(validRoundStartTime, validRoundEndTime); + } + + /** + * Returns a list of all shard directory paths, formatted with 3-digit zero-padded shard numbers. + */ + public List getAllShardPaths() { + List shardPaths = new ArrayList<>(); + for (int i = 0; i < numShards; i++) { + String shardDirName = String.format(SHARD_DIR_FORMAT, i); + Path shardPath = new Path(shardDirectoryPath, shardDirName); + shardPaths.add(shardPath); + } + return shardPaths; + } + + /** + * Returns the nearest replication round start timestamp for the given timestamp. + * @param timestamp The timestamp in milliseconds since epoch + * @return The nearest replication round start timestamp + */ + protected long getNearestRoundStartTimestamp(long timestamp) { + // Convert round time from seconds to milliseconds + long roundTimeMs = replicationRoundDurationSeconds * 1000L; + + // Calculate the nearest round start timestamp + // This rounds down to the nearest multiple of round time + return (timestamp / roundTimeMs) * roundTimeMs; + } + + public int getReplicationRoundDurationSeconds() { + return this.replicationRoundDurationSeconds; + } + + public Path getShardDirectoryPath() { + return this.shardDirectoryPath; + } + + public Path getRootDirectoryPath() { + return this.rootDirectoryPath; + } + + public int getNumShards() { + return this.numShards; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StandbyLogGroupWriter.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StandbyLogGroupWriter.java index 5c750886089..26ef908d1ba 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StandbyLogGroupWriter.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StandbyLogGroupWriter.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.phoenix.replication.log.LogFileWriter; import org.apache.phoenix.replication.log.LogFileWriterContext; +import org.apache.phoenix.replication.reader.ReplicationLogReplay; import org.apache.phoenix.util.EnvironmentEdgeManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,7 +44,7 @@ public class StandbyLogGroupWriter extends ReplicationLogGroupWriter { private FileSystem standbyFs; private URI standbyUrl; - protected int numShards; + private Path haGroupLogFilesPath; protected final ConcurrentHashMap shardMap = new ConcurrentHashMap<>(); /** @@ -51,18 +52,11 @@ public class StandbyLogGroupWriter extends ReplicationLogGroupWriter { */ public StandbyLogGroupWriter(ReplicationLogGroup logGroup) { super(logGroup); - Configuration conf = logGroup.getConfiguration(); - this.numShards = conf.getInt(ReplicationLogGroup.REPLICATION_NUM_SHARDS_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_NUM_SHARDS); LOG.debug("Created StandbyLogGroupWriter for HA Group: {}", logGroup.getHaGroupName()); } @Override protected void initializeFileSystems() throws IOException { - if (numShards > ReplicationLogGroup.MAX_REPLICATION_NUM_SHARDS) { - throw new IllegalArgumentException(ReplicationLogGroup.REPLICATION_NUM_SHARDS_KEY + " is " - + numShards + ", but the limit is " + ReplicationLogGroup.MAX_REPLICATION_NUM_SHARDS); - } Configuration conf = logGroup.getConfiguration(); String standbyUrlString = conf.get(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY); if (standbyUrlString == null || standbyUrlString.trim().isEmpty()) { @@ -78,32 +72,29 @@ protected void initializeFileSystems() throws IOException { } } + @Override + protected void initializeReplicationShardDirectoryManager() { + this.haGroupLogFilesPath = new Path(new Path(standbyUrl.getPath(), logGroup.getHaGroupName()), + ReplicationLogReplay.IN_DIRECTORY_NAME); + this.replicationShardDirectoryManager = + new ReplicationShardDirectoryManager(logGroup.getConfiguration(), haGroupLogFilesPath); + } + /** - * Creates a new log file path in a sharded directory structure based on server name and - * timestamp. The resulting path structure is - * - *
-   * [url]/[haGroupId]/[shard]/[timestamp]-[servername].plog
-   * 
+ * Creates a new log file path in a sharded directory structure using + * {@link ReplicationShardDirectoryManager}. Directory Structure: + * [root_path]/[ha_group_name]/in/shard/[shard_directory]/[file_name] */ - protected Path makeWriterPath(FileSystem fs, URI url) throws IOException { - Path haGroupPath = new Path(url.getPath(), logGroup.getHaGroupName()); + protected Path makeWriterPath(FileSystem fs) throws IOException { long timestamp = EnvironmentEdgeManager.currentTimeMillis(); - // To have all logs for a given regionserver appear in the same shard, hash only the - // serverName. However we expect some regionservers will have significantly more load than - // others so we instead distribute the logs over all of the shards randomly for a more even - // overall distribution by also hashing the timestamp. - int shard = - Math.floorMod(logGroup.getServerName().hashCode() ^ Long.hashCode(timestamp), numShards); - Path shardPath = - new Path(haGroupPath, String.format(ReplicationLogGroup.SHARD_DIR_FORMAT, shard)); + Path shardPath = replicationShardDirectoryManager.getShardDirectory(timestamp); // Ensure the shard directory exists. We track which shard directories we have probed or // created to avoid a round trip to the namenode for repeats. IOException[] exception = new IOException[1]; shardMap.computeIfAbsent(shardPath, p -> { try { if (!fs.exists(p)) { - fs.mkdirs(haGroupPath); // This probably exists, but just in case. + fs.mkdirs(haGroupLogFilesPath); // This probably exists, but just in case. if (!fs.mkdirs(shardPath)) { throw new IOException("Could not create path: " + p); } @@ -125,7 +116,7 @@ protected Path makeWriterPath(FileSystem fs, URI url) throws IOException { /** Creates and initializes a new LogFileWriter. */ protected LogFileWriter createNewWriter() throws IOException { - Path filePath = makeWriterPath(standbyFs, standbyUrl); + Path filePath = makeWriterPath(standbyFs); LogFileWriterContext writerContext = new LogFileWriterContext(logGroup.getConfiguration()) .setFileSystem(standbyFs).setFilePath(filePath).setCompression(compression); LogFileWriter newWriter = new LogFileWriter(); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardLogGroupWriter.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardLogGroupWriter.java index c491ff82d91..304ebe491bc 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardLogGroupWriter.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardLogGroupWriter.java @@ -66,6 +66,11 @@ protected void initializeFileSystems() throws IOException { // TODO } + @Override + protected void initializeReplicationShardDirectoryManager() { + // TODO + } + @Override protected LogFileWriter createNewWriter() throws IOException { // TODO diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscovery.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscovery.java new file mode 100644 index 00000000000..73371bb55b7 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscovery.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.phoenix.replication.metrics; + +import org.apache.hadoop.hbase.metrics.BaseSource; + +public interface MetricsReplicationLogDiscovery extends BaseSource { + + String METRICS_NAME = "MetricsReplicationLogDiscovery"; + String METRICS_CONTEXT = "phoenix"; + String METRICS_DESCRIPTION = "Metrics about Replication Log File Discovery for an HA Group"; + String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + String NUM_ROUNDS_PROCESSED = "numRoundsProcessed"; + String NUM_ROUNDS_PROCESSED_DESC = "Number of rounds processed during replication log discovery"; + String NUM_IN_PROGRESS_DIRECTORY_PROCESSED = "numInProgressDirectoryProcessed"; + String NUM_IN_PROGRESS_DIRECTORY_PROCESSED_DESC = + "Number of times in progress directory is processed during replication log discovery"; + String TIME_TO_PROCESS_NEW_FILES = "timeToProcessNewFilesMs"; + String TIME_TO_PROCESS_NEW_FILES_DESC = + "Histogram of time taken to process new files in milliseconds"; + String TIME_TO_PROCESS_IN_PROGRESS_FILES = "timeToProcessInProgressFilesMs"; + String TIME_TO_PROCESS_IN_PROGRESS_FILES_DESC = + "Histogram of time taken to process in progress files in milliseconds"; + + /** + * Increments the counter for rounds processed. This counter tracks the number of rounds processed + * during replication log discovery. + */ + void incrementNumRoundsProcessed(); + + /** + * Increments the counter for in progress directory processing. This counter tracks the number of + * times the in progress directory is processed during replication log discovery. + */ + void incrementNumInProgressDirectoryProcessed(); + + /** + * Update the time taken to process new files in milliseconds. + * @param timeMs Time taken in milliseconds + */ + void updateTimeToProcessNewFiles(long timeMs); + + /** + * Update the time taken to process in progress files in milliseconds. + * @param timeMs Time taken in milliseconds + */ + void updateTimeToProcessInProgressFiles(long timeMs); + + /** + * Unregister this metrics source. + */ + void close(); + + // Get current values for testing + ReplicationLogDiscoveryMetricValues getCurrentMetricValues(); + +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryImpl.java new file mode 100644 index 00000000000..35183d92b31 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryImpl.java @@ -0,0 +1,85 @@ +/* + * 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.phoenix.replication.metrics; + +import org.apache.hadoop.hbase.metrics.BaseSourceImpl; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MutableFastCounter; +import org.apache.hadoop.metrics2.lib.MutableHistogram; + +/** Implementation of metrics source for ReplicationLogDiscovery operations. */ +public class MetricsReplicationLogDiscoveryImpl extends BaseSourceImpl + implements MetricsReplicationLogDiscovery { + + protected String groupMetricsContext; + protected final MutableFastCounter numRoundsProcessed; + protected final MutableFastCounter numInProgressDirectoryProcessed; + protected final MutableHistogram timeToProcessNewFiles; + protected final MutableHistogram timeToProcessInProgressFiles; + + public MetricsReplicationLogDiscoveryImpl(String metricsName, String metricsDescription, + String metricsContext, String metricsJmxContext) { + super(metricsName, metricsDescription, metricsContext, metricsJmxContext); + numRoundsProcessed = + getMetricsRegistry().newCounter(NUM_ROUNDS_PROCESSED, NUM_ROUNDS_PROCESSED_DESC, 0L); + numInProgressDirectoryProcessed = getMetricsRegistry().newCounter( + NUM_IN_PROGRESS_DIRECTORY_PROCESSED, NUM_IN_PROGRESS_DIRECTORY_PROCESSED_DESC, 0L); + timeToProcessNewFiles = + getMetricsRegistry().newHistogram(TIME_TO_PROCESS_NEW_FILES, TIME_TO_PROCESS_NEW_FILES_DESC); + timeToProcessInProgressFiles = getMetricsRegistry() + .newHistogram(TIME_TO_PROCESS_IN_PROGRESS_FILES, TIME_TO_PROCESS_IN_PROGRESS_FILES_DESC); + } + + @Override + public void incrementNumRoundsProcessed() { + numRoundsProcessed.incr(); + } + + @Override + public void incrementNumInProgressDirectoryProcessed() { + numInProgressDirectoryProcessed.incr(); + } + + @Override + public void updateTimeToProcessNewFiles(long timeMs) { + timeToProcessNewFiles.add(timeMs); + } + + @Override + public void updateTimeToProcessInProgressFiles(long timeMs) { + timeToProcessInProgressFiles.add(timeMs); + } + + @Override + public void close() { + // Unregister this metrics source + DefaultMetricsSystem.instance().unregisterSource(groupMetricsContext); + } + + @Override + public ReplicationLogDiscoveryMetricValues getCurrentMetricValues() { + return new ReplicationLogDiscoveryMetricValues(numRoundsProcessed.value(), + numInProgressDirectoryProcessed.value(), timeToProcessNewFiles.getMax(), + timeToProcessInProgressFiles.getMax()); + } + + @Override + public String getMetricsContext() { + return groupMetricsContext; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryReplayImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryReplayImpl.java new file mode 100644 index 00000000000..c7dcf4f6ae9 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryReplayImpl.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.phoenix.replication.metrics; + +/** Implementation of metrics source for ReplicationLogDiscoveryReplay operations. */ +public class MetricsReplicationLogDiscoveryReplayImpl extends MetricsReplicationLogDiscoveryImpl { + + private static final String METRICS_NAME = "ReplicationLogDiscoveryReplay"; + private static final String METRICS_DESCRIPTION = + "Metrics about Replication Replay Log Discovery for an HA Group"; + private static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + public MetricsReplicationLogDiscoveryReplayImpl(final String haGroupName) { + super(MetricsReplicationLogDiscoveryReplayImpl.METRICS_NAME, + MetricsReplicationLogDiscoveryReplayImpl.METRICS_DESCRIPTION, + MetricsReplicationLogDiscoveryImpl.METRICS_CONTEXT, + MetricsReplicationLogDiscoveryReplayImpl.METRICS_JMX_CONTEXT + ",haGroup=" + haGroupName); + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTracker.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTracker.java new file mode 100644 index 00000000000..dcc2af2a5db --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTracker.java @@ -0,0 +1,99 @@ +/* + * 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.phoenix.replication.metrics; + +import org.apache.hadoop.hbase.metrics.BaseSource; + +public interface MetricsReplicationLogTracker extends BaseSource { + + String METRICS_NAME = "MetricsReplicationLogTracker"; + String METRICS_CONTEXT = "phoenix"; + String METRICS_DESCRIPTION = "Metrics about Replication Log File Tracker for an HA Group"; + String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + String MARK_FILE_IN_PROGRESS_REQUEST_COUNT = "markFileInProgressRequestCount"; + String MARK_FILE_IN_PROGRESS_REQUEST_COUNT_DESC = + "Number of requests made to mark file in progress"; + String MARK_FILE_COMPLETED_REQUEST_COUNT = "markFileCompletedRequestCount"; + String MARK_FILE_COMPLETED_REQUEST_COUNT_DESC = "Number of requests made to mark file completed"; + String MARK_FILE_FAILED_REQUEST_COUNT = "markFileFailedRequestCount"; + String MARK_FILE_FAILED_REQUEST_COUNT_DESC = "Number of requests made to mark file failed"; + String MARK_FILE_COMPLETED_REQUEST_FAILED_COUNT = "markFileCompletedRequestFailedCount"; + String MARK_FILE_COMPLETED_REQUEST_FAILED_COUNT_DESC = + "Number of requests made to mark file completed failed"; + String MARK_FILE_IN_PROGRESS_TIME = "markFileInProgressTimeMs"; + String MARK_FILE_IN_PROGRESS_TIME_DESC = + "Histogram of time taken for marking a file in progress in milliseconds"; + String MARK_FILE_COMPLETED_TIME = "markFileCompletedTimeMs"; + String MARK_FILE_COMPLETED_TIME_DESC = + "Histogram of time taken for marking a file completed in milliseconds"; + String MARK_FILE_FAILED_TIME = "markFileFailedTimeMs"; + String MARK_FILE_FAILED_TIME_DESC = + "Histogram of time taken for marking a file failed in milliseconds"; + + /** + * Increments the counter for mark file in progress requests. This counter tracks the number of + * requests made to mark files in progress. + */ + void incrementMarkFileInProgressRequestCount(); + + /** + * Increments the counter for mark file completed requests. This counter tracks the number of + * requests made to mark files completed. + */ + void incrementMarkFileCompletedRequestCount(); + + /** + * Increments the counter for mark file failed requests. This counter tracks the number of + * requests made to mark files failed. + */ + void incrementMarkFileFailedRequestCount(); + + /** + * Increments the counter for mark file completed request failures. This counter tracks the number + * of requests made to mark files completed that failed. + */ + void incrementMarkFileCompletedRequestFailedCount(); + + /** + * Update the time taken for marking a file in progress in milliseconds. + * @param timeMs Time taken in milliseconds + */ + void updateMarkFileInProgressTime(long timeMs); + + /** + * Update the time taken for marking a file completed in milliseconds. + * @param timeMs Time taken in milliseconds + */ + void updateMarkFileCompletedTime(long timeMs); + + /** + * Update the time taken for marking a file failed in milliseconds. + * @param timeMs Time taken in milliseconds + */ + void updateMarkFileFailedTime(long timeMs); + + /** + * Unregister this metrics source. + */ + void close(); + + // Get current values for testing + ReplicationLogTrackerMetricValues getCurrentMetricValues(); + +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerImpl.java new file mode 100644 index 00000000000..b938a504d49 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerImpl.java @@ -0,0 +1,110 @@ +/* + * 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.phoenix.replication.metrics; + +import org.apache.hadoop.hbase.metrics.BaseSourceImpl; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MutableFastCounter; +import org.apache.hadoop.metrics2.lib.MutableHistogram; + +/** Implementation of metrics source for ReplicationLogTracker operations. */ +public class MetricsReplicationLogTrackerImpl extends BaseSourceImpl + implements MetricsReplicationLogTracker { + + protected String groupMetricsContext; + protected final MutableFastCounter markFileInProgressRequestCount; + protected final MutableFastCounter markFileCompletedRequestCount; + protected final MutableFastCounter markFileFailedRequestCount; + protected final MutableFastCounter markFileCompletedRequestFailedCount; + protected final MutableHistogram markFileInProgressTime; + protected final MutableHistogram markFileCompletedTime; + protected final MutableHistogram markFileFailedTime; + + public MetricsReplicationLogTrackerImpl(String metricsName, String metricsDescription, + String metricsContext, String metricsJmxContext) { + super(metricsName, metricsDescription, metricsContext, metricsJmxContext); + markFileInProgressRequestCount = getMetricsRegistry().newCounter( + MARK_FILE_IN_PROGRESS_REQUEST_COUNT, MARK_FILE_IN_PROGRESS_REQUEST_COUNT_DESC, 0L); + markFileCompletedRequestCount = getMetricsRegistry() + .newCounter(MARK_FILE_COMPLETED_REQUEST_COUNT, MARK_FILE_COMPLETED_REQUEST_COUNT_DESC, 0L); + markFileFailedRequestCount = getMetricsRegistry().newCounter(MARK_FILE_FAILED_REQUEST_COUNT, + MARK_FILE_FAILED_REQUEST_COUNT_DESC, 0L); + markFileCompletedRequestFailedCount = getMetricsRegistry().newCounter( + MARK_FILE_COMPLETED_REQUEST_FAILED_COUNT, MARK_FILE_COMPLETED_REQUEST_FAILED_COUNT_DESC, 0L); + markFileInProgressTime = getMetricsRegistry().newHistogram(MARK_FILE_IN_PROGRESS_TIME, + MARK_FILE_IN_PROGRESS_TIME_DESC); + markFileCompletedTime = + getMetricsRegistry().newHistogram(MARK_FILE_COMPLETED_TIME, MARK_FILE_COMPLETED_TIME_DESC); + markFileFailedTime = + getMetricsRegistry().newHistogram(MARK_FILE_FAILED_TIME, MARK_FILE_FAILED_TIME_DESC); + } + + @Override + public void incrementMarkFileInProgressRequestCount() { + markFileInProgressRequestCount.incr(); + } + + @Override + public void incrementMarkFileCompletedRequestCount() { + markFileCompletedRequestCount.incr(); + } + + @Override + public void incrementMarkFileFailedRequestCount() { + markFileFailedRequestCount.incr(); + } + + @Override + public void incrementMarkFileCompletedRequestFailedCount() { + markFileCompletedRequestFailedCount.incr(); + } + + @Override + public void updateMarkFileInProgressTime(long timeMs) { + markFileInProgressTime.add(timeMs); + } + + @Override + public void updateMarkFileCompletedTime(long timeMs) { + markFileCompletedTime.add(timeMs); + } + + @Override + public void updateMarkFileFailedTime(long timeMs) { + markFileFailedTime.add(timeMs); + } + + @Override + public void close() { + // Unregister this metrics source + DefaultMetricsSystem.instance().unregisterSource(groupMetricsContext); + } + + @Override + public ReplicationLogTrackerMetricValues getCurrentMetricValues() { + return new ReplicationLogTrackerMetricValues(markFileInProgressRequestCount.value(), + markFileCompletedRequestCount.value(), markFileFailedRequestCount.value(), + markFileCompletedRequestFailedCount.value(), markFileInProgressTime.getMax(), + markFileCompletedTime.getMax(), markFileFailedTime.getMax()); + } + + @Override + public String getMetricsContext() { + return groupMetricsContext; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerReplayImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerReplayImpl.java new file mode 100644 index 00000000000..1fad372c5c6 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerReplayImpl.java @@ -0,0 +1,36 @@ +/* + * 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.phoenix.replication.metrics; + +public class MetricsReplicationLogTrackerReplayImpl extends MetricsReplicationLogTrackerImpl { + + private static final String METRICS_NAME = "ReplicationLogReplayFileTracker"; + private static final String METRICS_DESCRIPTION = + "Metrics about Replication Log Replay File Tracker for an HA Group"; + private static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + public MetricsReplicationLogTrackerReplayImpl(final String haGroupName) { + super(MetricsReplicationLogTrackerReplayImpl.METRICS_NAME, + MetricsReplicationLogTrackerReplayImpl.METRICS_DESCRIPTION, + MetricsReplicationLogTracker.METRICS_CONTEXT, + MetricsReplicationLogTrackerReplayImpl.METRICS_JMX_CONTEXT + ",haGroup=" + haGroupName); + super.groupMetricsContext = + MetricsReplicationLogTrackerReplayImpl.METRICS_JMX_CONTEXT + ",haGroup=" + haGroupName; + } + +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/ReplicationLogDiscoveryMetricValues.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/ReplicationLogDiscoveryMetricValues.java new file mode 100644 index 00000000000..dd380a36080 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/ReplicationLogDiscoveryMetricValues.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.phoenix.replication.metrics; + +/** + * Class to hold the values of all metrics tracked by the ReplicationLogDiscovery metrics source. + */ +public class ReplicationLogDiscoveryMetricValues { + + private final long numRoundsProcessed; + private final long numInProgressDirectoryProcessed; + private final long timeToProcessNewFilesMs; + private final long timeToProcessInProgressFilesMs; + + public ReplicationLogDiscoveryMetricValues(long numRoundsProcessed, + long numInProgressDirectoryProcessed, long timeToProcessNewFilesMs, + long timeToProcessInProgressFilesMs) { + this.numRoundsProcessed = numRoundsProcessed; + this.numInProgressDirectoryProcessed = numInProgressDirectoryProcessed; + this.timeToProcessNewFilesMs = timeToProcessNewFilesMs; + this.timeToProcessInProgressFilesMs = timeToProcessInProgressFilesMs; + } + + public long getNumRoundsProcessed() { + return numRoundsProcessed; + } + + public long getNumInProgressDirectoryProcessed() { + return numInProgressDirectoryProcessed; + } + + public long getTimeToProcessNewFilesMs() { + return timeToProcessNewFilesMs; + } + + public long getTimeToProcessInProgressFilesMs() { + return timeToProcessInProgressFilesMs; + } + +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/ReplicationLogTrackerMetricValues.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/ReplicationLogTrackerMetricValues.java new file mode 100644 index 00000000000..76678db000f --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/ReplicationLogTrackerMetricValues.java @@ -0,0 +1,74 @@ +/* + * 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.phoenix.replication.metrics; + +/** + * Class to hold the values of all metrics tracked by the ReplicationLogTracker metrics source. + */ +public class ReplicationLogTrackerMetricValues { + + private final long markFileInProgressRequestCount; + private final long markFileCompletedRequestCount; + private final long markFileFailedRequestCount; + private final long markFileCompletedRequestFailedCount; + private final long markFileInProgressTimeMs; + private final long markFileCompletedTimeMs; + private final long markFileFailedTimeMs; + + public ReplicationLogTrackerMetricValues(long markFileInProgressRequestCount, + long markFileCompletedRequestCount, long markFileFailedRequestCount, + long markFileCompletedRequestFailedCount, long markFileInProgressTimeMs, + long markFileCompletedTimeMs, long markFileFailedTimeMs) { + this.markFileInProgressRequestCount = markFileInProgressRequestCount; + this.markFileCompletedRequestCount = markFileCompletedRequestCount; + this.markFileFailedRequestCount = markFileFailedRequestCount; + this.markFileCompletedRequestFailedCount = markFileCompletedRequestFailedCount; + this.markFileInProgressTimeMs = markFileInProgressTimeMs; + this.markFileCompletedTimeMs = markFileCompletedTimeMs; + this.markFileFailedTimeMs = markFileFailedTimeMs; + } + + public long getMarkFileInProgressRequestCount() { + return markFileInProgressRequestCount; + } + + public long getMarkFileCompletedRequestCount() { + return markFileCompletedRequestCount; + } + + public long getMarkFileFailedRequestCount() { + return markFileFailedRequestCount; + } + + public long getMarkFileCompletedRequestFailedCount() { + return markFileCompletedRequestFailedCount; + } + + public long getMarkFileInProgressTimeMs() { + return markFileInProgressTimeMs; + } + + public long getMarkFileCompletedTimeMs() { + return markFileCompletedTimeMs; + } + + public long getMarkFileFailedTimeMs() { + return markFileFailedTimeMs; + } + +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogDiscoveryReplay.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogDiscoveryReplay.java new file mode 100644 index 00000000000..f1292519a55 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogDiscoveryReplay.java @@ -0,0 +1,462 @@ +/* + * 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.phoenix.replication.reader; + +import java.io.IOException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.phoenix.exception.InvalidClusterRoleTransitionException; +import org.apache.phoenix.jdbc.ClusterType; +import org.apache.phoenix.jdbc.HAGroupStateListener; +import org.apache.phoenix.jdbc.HAGroupStoreManager; +import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.replication.ReplicationLogDiscovery; +import org.apache.phoenix.replication.ReplicationLogTracker; +import org.apache.phoenix.replication.ReplicationRound; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogDiscovery; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogDiscoveryReplayImpl; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * State-aware implementation of ReplicationLogDiscovery for HA replication replay on standby + * clusters. This class extends the base ReplicationLogDiscovery with support for three replication + * states: - SYNC: Normal synchronized processing where both lastRoundProcessed and lastRoundInSync + * advance together - DEGRADED: Degraded mode where lastRoundProcessed advances but lastRoundInSync + * is preserved - SYNCED_RECOVERY: Recovery mode that rewinds to lastRoundInSync and re-processes + * from that point Key features: - Uses getFirstRoundToProcess() to start replay from + * lastRoundInSync (not just from lastRoundProcessed) - Dynamically responds to HA state changes via + * listeners during replay execution - Maintains separate tracking of lastRoundProcessed and + * lastRoundInSync for recovery scenarios - Integrates with HAGroupStoreManager for cluster state + * coordination + */ +public class ReplicationLogDiscoveryReplay extends ReplicationLogDiscovery { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogDiscoveryReplay.class); + + public static final String EXECUTOR_THREAD_NAME_FORMAT = + "Phoenix-ReplicationLogDiscoveryReplay-%d"; + + /** + * Configuration key for replay interval in seconds + */ + public static final String REPLICATION_REPLAY_INTERVAL_SECONDS_KEY = + "phoenix.replication.replay.interval.seconds"; + + /** + * Configuration key for shutdown timeout in seconds + */ + public static final String REPLICATION_REPLAY_SHUTDOWN_TIMEOUT_SECONDS_KEY = + "phoenix.replication.replay.executor.shutdown.timeout.seconds"; + + /** + * Configuration key for executor thread count + */ + public static final String REPLICATION_REPLAY_EXECUTOR_THREAD_COUNT_KEY = + "phoenix.replication.replay.executor.thread.count"; + + /** + * Configuration key for in-progress directory processing probability + */ + public static final String REPLICATION_REPLAY_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY_KEY = + "phoenix.replication.replay.in.progress.directory.processing.probability"; + + /** + * Configuration key for waiting buffer percentage + */ + public static final String REPLICATION_REPLAY_WAITING_BUFFER_PERCENTAGE_KEY = + "phoenix.replication.replay.waiting.buffer.percentage"; + + /** + * Default replay interval in seconds. Controls how frequently the replay process runs. + */ + public static final long DEFAULT_REPLAY_INTERVAL_SECONDS = 60; + + /** + * Default shutdown timeout in seconds. Maximum time to wait for executor service to shutdown + * gracefully. + */ + public static final long DEFAULT_SHUTDOWN_TIMEOUT_SECONDS = 30; + + /** + * Default number of executor threads for processing replication log files. + */ + public static final int DEFAULT_EXECUTOR_THREAD_COUNT = 1; + + /** + * Default probability (in percentage) for processing in-progress directory during each replay + * cycle. + */ + public static final double DEFAULT_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY = 5.0; + + /** + * Default waiting buffer percentage. Buffer time is calculated as this percentage of round time. + */ + public static final double DEFAULT_WAITING_BUFFER_PERCENTAGE = 15.0; + + private ReplicationRound lastRoundInSync; + + // AtomicReference ensures listener updates are visible to replay thread + private final AtomicReference replicationReplayState = + new AtomicReference<>(ReplicationReplayState.NOT_INITIALIZED); + + private final AtomicBoolean failoverPending = new AtomicBoolean(false); + + public ReplicationLogDiscoveryReplay( + final ReplicationLogTracker replicationLogReplayFileTracker) { + super(replicationLogReplayFileTracker); + } + + @Override + public void init() throws IOException { + HAGroupStateListener degradedListener = + (groupName, fromState, toState, modifiedTime, clusterType, lastSyncStateTimeInMs) -> { + if ( + clusterType == ClusterType.LOCAL + && HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY.equals(toState) + ) { + replicationReplayState.set(ReplicationReplayState.DEGRADED); + LOG.info("Cluster degraded detected for {}. replicationReplayState={}", haGroupName, + ReplicationReplayState.DEGRADED); + } + }; + + HAGroupStateListener recoveryListener = + (groupName, fromState, toState, modifiedTime, clusterType, lastSyncStateTimeInMs) -> { + if ( + clusterType == ClusterType.LOCAL + && HAGroupStoreRecord.HAGroupState.STANDBY.equals(toState) + ) { + replicationReplayState.set(ReplicationReplayState.SYNCED_RECOVERY); + LOG.info("Cluster recovered detected for {}. replicationReplayState={}", haGroupName, + getReplicationReplayState()); + } + }; + + HAGroupStateListener triggerFailoverListner = + (groupName, fromState, toState, modifiedTime, clusterType, lastSyncStateTimeInMs) -> { + if ( + clusterType == ClusterType.LOCAL + && HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE.equals(toState) + ) { + failoverPending.set(true); + LOG.info( + "Failover trigger detected for {}. replicationReplayState={}. " + + "Setting failover pending to {}", + haGroupName, getReplicationReplayState(), failoverPending.get()); + } + }; + + HAGroupStateListener abortFailoverListner = + (groupName, fromState, toState, modifiedTime, clusterType, lastSyncStateTimeInMs) -> { + if ( + clusterType == ClusterType.LOCAL + && HAGroupStoreRecord.HAGroupState.ABORT_TO_STANDBY.equals(toState) + ) { + failoverPending.set(false); + LOG.info( + "Failover abort detected for {}. replicationReplayState={}. " + + "Setting failover pending to {}", + haGroupName, getReplicationReplayState(), failoverPending.get()); + } + }; + + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(conf); + + // Subscribe degraded states + haGroupStoreManager.subscribeToTargetState(haGroupName, + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, ClusterType.LOCAL, degradedListener); + + // Subscribe recovery/healthy states + haGroupStoreManager.subscribeToTargetState(haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY, + ClusterType.LOCAL, recoveryListener); + + // Subscribe to trigger failover state + haGroupStoreManager.subscribeToTargetState(haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE, ClusterType.LOCAL, triggerFailoverListner); + + // Subscribe to abort failover state + haGroupStoreManager.subscribeToTargetState(haGroupName, + HAGroupStoreRecord.HAGroupState.ABORT_TO_STANDBY, ClusterType.LOCAL, abortFailoverListner); + + super.init(); + } + + @Override + protected void processFile(Path path) throws IOException { + LOG.info("Starting to process file {}", path); + ReplicationLogProcessor.get(getConf(), getHaGroupName()) + .processLogFile(getReplicationLogFileTracker().getFileSystem(), path); + } + + /** + * Initializes lastRoundProcessed and lastRoundInSync based on HA group state. For DEGRADED states + * (DEGRADED_STANDBY, DEGRADED_STANDBY_FOR_WRITER): - Sets replicationReplayState to DEGRADED - + * Initializes lastRoundProcessed from minimum of: in-progress files, new files, or current time - + * Initializes lastRoundInSync from minimum of: lastSyncStateTimeInMs (from HA Store) or minimum + * timestamp from IN and IN PROGRESS files - This ensures lastRoundInSync represents the last + * known good sync point before degradation For SYNC states (STANDBY): - Sets + * replicationReplayState to SYNC - Calls parent's initializeLastRoundProcessed() to initialize + * lastRoundProcessed - Sets lastRoundInSync equal to lastRoundProcessed (both are in sync) + * @throws IOException if there's an error reading HA group state or file timestamps + */ + @Override + protected void initializeLastRoundProcessed() throws IOException { + HAGroupStoreRecord haGroupStoreRecord = getHAGroupRecord(); + if ( + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY.equals(haGroupStoreRecord.getHAGroupState()) + ) { + replicationReplayState.compareAndSet(ReplicationReplayState.NOT_INITIALIZED, + ReplicationReplayState.DEGRADED); + long minimumTimestampFromFiles = EnvironmentEdgeManager.currentTime(); + Optional minTimestampFromInProgressFiles = getMinTimestampFromInProgressFiles(); + Optional minTimestampFromNewFiles = getMinTimestampFromNewFiles(); + if (minTimestampFromInProgressFiles.isPresent()) { + LOG.info("Found minimum timestamp from IN PROGRESS files as {}", + minTimestampFromInProgressFiles.get()); + minimumTimestampFromFiles = + Math.min(minimumTimestampFromFiles, minTimestampFromInProgressFiles.get()); + } + if (minTimestampFromNewFiles.isPresent()) { + LOG.info("Found minimum timestamp from IN files as {}", minTimestampFromNewFiles.get()); + minimumTimestampFromFiles = + Math.min(minimumTimestampFromFiles, minTimestampFromNewFiles.get()); + } + this.lastRoundProcessed = replicationLogTracker.getReplicationShardDirectoryManager() + .getReplicationRoundFromEndTime(minimumTimestampFromFiles); + this.lastRoundInSync = + replicationLogTracker.getReplicationShardDirectoryManager().getReplicationRoundFromEndTime( + Math.min(haGroupStoreRecord.getLastSyncStateTimeInMs(), minimumTimestampFromFiles)); + } else { + replicationReplayState.compareAndSet(ReplicationReplayState.NOT_INITIALIZED, + ReplicationReplayState.SYNC); + super.initializeLastRoundProcessed(); + this.lastRoundInSync = + new ReplicationRound(lastRoundProcessed.getStartTime(), lastRoundProcessed.getEndTime()); + } + LOG.info( + "Initialized last round processed as {}, last round in sync as {} and " + + "replication replay state as {}", + lastRoundProcessed, lastRoundInSync, replicationReplayState); + + // Update the failoverPending variable during initialization + if ( + HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE.equals(haGroupStoreRecord.getHAGroupState()) + ) { + failoverPending.compareAndSet(false, true); + } + } + + /** + * Executes a replay operation with state-aware processing for HA replication scenarios. This + * method extends the base replay() by handling three replication states: 1. SYNC: Normal + * processing - Updates both lastRoundProcessed and lastRoundInSync - Both pointers advance + * together, indicating cluster is fully synchronized 2. DEGRADED: Degraded mode processing - + * Updates only lastRoundProcessed (advances in memory) - Does NOT update lastRoundInSync + * (preserves last known good sync point) - Allows processing to continue during degradation + * without losing sync reference 3. SYNCED_RECOVERY: Recovery mode - Rewinds lastRoundProcessed + * back to lastRoundInSync - Transitions to SYNC state - Re-processes rounds from last known good + * sync point to ensure data consistency The first round is retrieved using + * getFirstRoundToProcess() (starts from lastRoundInSync), subsequent rounds use + * getNextRoundToProcess() (starts from lastRoundProcessed). State transitions can occur + * dynamically via HA group listeners during replay execution. + * @throws IOException if there's an error during replay processing + */ + @Override + public void replay() throws IOException { + LOG.info("Starting replay with lastRoundProcessed={}, lastRoundInSync={}", lastRoundProcessed, + lastRoundInSync); + Optional optionalNextRound = getFirstRoundToProcess(); + while (optionalNextRound.isPresent()) { + ReplicationRound replicationRound = optionalNextRound.get(); + try { + processRound(replicationRound); + } catch (IOException e) { + LOG.error("Failed processing replication round {}. Will retry in next " + "scheduled run.", + replicationRound, e); + break; // stop this run, retry later + } + + // Always read the latest listener state + ReplicationReplayState currentState = replicationReplayState.get(); + + switch (currentState) { + case SYNCED_RECOVERY: + // Rewind to last in-sync round + LOG.info("SYNCED_RECOVERY detected, rewinding to lastRoundInSync={}", lastRoundInSync); + setLastRoundProcessed(lastRoundInSync); + // Only reset to NORMAL if state hasn't been flipped to DEGRADED + replicationReplayState.compareAndSet(ReplicationReplayState.SYNCED_RECOVERY, + ReplicationReplayState.SYNC); + break; + + case SYNC: + // Normal processing, update last round processed and in-sync + setLastRoundProcessed(replicationRound); + setLastRoundInSync(replicationRound); + LOG.info( + "Processed round {} successfully, lastRoundProcessed={}, " + "lastRoundInSync={}", + replicationRound, lastRoundProcessed, lastRoundInSync); + break; + + case DEGRADED: + // Only update last round processed, and NOT last round in sync + setLastRoundProcessed(replicationRound); + LOG.info( + "Processed round {} successfully with cluster in DEGRADED " + + "state, lastRoundProcessed={}, lastRoundInSync={}", + replicationRound, lastRoundProcessed, lastRoundInSync); + break; + + default: + throw new IllegalStateException("Unexpected state: " + currentState); + } + optionalNextRound = getNextRoundToProcess(); + } + + if (!optionalNextRound.isPresent() && shouldTriggerFailover()) { + LOG.info( + "No more rounds to process, lastRoundInSync={}, lastRoundProcessed={}. " + + "Failover is triggered & in progress directory is empty. " + + "Marking cluster state as {}", + lastRoundInSync, lastRoundProcessed, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + try { + triggerFailover(); + LOG.info("Successfully updated the cluster state"); + failoverPending.set(false); + } catch (InvalidClusterRoleTransitionException exception) { + LOG.warn("Failed to update the cluster state.", exception); + failoverPending.set(false); + } + } + } + + /** + * Returns the first replication round to process based on lastRoundInSync. Unlike + * getNextRoundToProcess() which uses lastRoundProcessed, this method uses lastRoundInSync to + * ensure replay starts from the last known synchronized point. This is critical for recovery + * scenarios where lastRoundProcessed may be ahead of lastRoundInSync. + * @return Optional containing the first round to process, or empty if not enough time has passed + */ + private Optional getFirstRoundToProcess() { + long lastRoundEndTimestamp = getLastRoundInSync().getEndTime(); + long currentTime = EnvironmentEdgeManager.currentTime(); + if (currentTime - lastRoundEndTimestamp < roundTimeMills + bufferMillis) { + // nothing more to process + return Optional.empty(); + } + return Optional + .of(new ReplicationRound(lastRoundEndTimestamp, lastRoundEndTimestamp + roundTimeMills)); + } + + @Override + protected MetricsReplicationLogDiscovery createMetricsSource() { + return new MetricsReplicationLogDiscoveryReplayImpl(haGroupName); + } + + @Override + public String getExecutorThreadNameFormat() { + return EXECUTOR_THREAD_NAME_FORMAT; + } + + @Override + public long getReplayIntervalSeconds() { + return getConf().getLong(REPLICATION_REPLAY_INTERVAL_SECONDS_KEY, + DEFAULT_REPLAY_INTERVAL_SECONDS); + } + + @Override + public long getShutdownTimeoutSeconds() { + return getConf().getLong(REPLICATION_REPLAY_SHUTDOWN_TIMEOUT_SECONDS_KEY, + DEFAULT_SHUTDOWN_TIMEOUT_SECONDS); + } + + @Override + public int getExecutorThreadCount() { + return getConf().getInt(REPLICATION_REPLAY_EXECUTOR_THREAD_COUNT_KEY, + DEFAULT_EXECUTOR_THREAD_COUNT); + } + + @Override + public double getInProgressDirectoryProcessProbability() { + return getConf().getDouble(REPLICATION_REPLAY_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY_KEY, + DEFAULT_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY); + } + + @Override + public double getWaitingBufferPercentage() { + return getConf().getDouble(REPLICATION_REPLAY_WAITING_BUFFER_PERCENTAGE_KEY, + DEFAULT_WAITING_BUFFER_PERCENTAGE); + } + + protected ReplicationRound getLastRoundInSync() { + return lastRoundInSync; + } + + protected ReplicationReplayState getReplicationReplayState() { + return replicationReplayState.get(); + } + + protected void setLastRoundInSync(ReplicationRound lastRoundInSync) { + this.lastRoundInSync = lastRoundInSync; + } + + protected void setReplicationReplayState(ReplicationReplayState replicationReplayState) { + this.replicationReplayState.set(replicationReplayState); + } + + protected void setFailoverPending(boolean failoverPending) { + this.failoverPending.set(failoverPending); + } + + protected boolean getFailoverPending() { + return this.failoverPending.get(); + } + + protected HAGroupStoreRecord getHAGroupRecord() throws IOException { + Optional optionalHAGroupStateRecord = + HAGroupStoreManager.getInstance(conf).getHAGroupStoreRecord(haGroupName); + if (!optionalHAGroupStateRecord.isPresent()) { + throw new IOException("HAGroupStoreRecord not found for HA Group: " + haGroupName); + } + return optionalHAGroupStateRecord.get(); + } + + protected boolean shouldTriggerFailover() throws IOException { + return failoverPending.get() && lastRoundInSync.equals(lastRoundProcessed) + && replicationLogTracker.getInProgressFiles().isEmpty(); + // TODO: Check for in files of lastRoundProcessed, lastRoundProcessed + roundTime as well - + // because there can be new files for upcoming round + } + + protected void triggerFailover() throws IOException, InvalidClusterRoleTransitionException { + // TODO: Update cluster state to ACTIVE_IN_SYNC within try block + // (once API is supported in HA Store) + // Throw any exception back to caller. + } + + public enum ReplicationReplayState { + NOT_INITIALIZED, // not initialized yet + SYNC, // fully in sync / standby + DEGRADED, // degraded for writer + SYNCED_RECOVERY // came back from degraded → standby, needs rewind + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogReplay.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogReplay.java new file mode 100644 index 00000000000..89c05ce63c3 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogReplay.java @@ -0,0 +1,165 @@ +/* + * 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.phoenix.replication.reader; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.phoenix.replication.ReplicationLogTracker; +import org.apache.phoenix.replication.ReplicationShardDirectoryManager; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTrackerReplayImpl; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages replication replay operations for a specific HA group. Provides singleton instances per + * group name and orchestrates the initialization of file system, file tracker, state tracker, and + * log discovery components. It also handles starting and stopping replay operations through the log + * discovery service. + */ +public class ReplicationLogReplay { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogReplay.class); + + /** + * The path on the HDFS where log files are to be read. + */ + public static final String REPLICATION_LOG_REPLAY_HDFS_URL_KEY = + "phoenix.replication.log.replay.hdfs.url"; + + public static final String IN_DIRECTORY_NAME = "in"; + /** + * Singleton instances per group name + */ + private static final ConcurrentHashMap INSTANCES = + new ConcurrentHashMap<>(); + + private final Configuration conf; + private final String haGroupName; + private FileSystem fileSystem; + private URI rootURI; + private ReplicationLogDiscoveryReplay replicationLogDiscoveryReplay; + + protected ReplicationLogReplay(final Configuration conf, final String haGroupName) { + this.conf = conf; + this.haGroupName = haGroupName; + } + + /** + * Gets or creates a singleton instance of ReplicationLogReplay for the specified group name. + * @param conf The configuration + * @param haGroupName The HA group name + * @return The singleton instance for the group + */ + public static ReplicationLogReplay get(final Configuration conf, final String haGroupName) { + return INSTANCES.computeIfAbsent(haGroupName, groupName -> { + try { + ReplicationLogReplay instance = new ReplicationLogReplay(conf, groupName); + instance.init(); + return instance; + } catch (IOException e) { + LOG.error("Failed to initialize ReplicationLogReplay for group: " + groupName, e); + throw new RuntimeException("Failed to initialize ReplicationLogReplay", e); + } + }); + } + + /** + * Delegate the start replay task to the {@link ReplicationLogDiscoveryReplay} + * @throws IOException - in case the start operation fails + */ + public void startReplay() throws IOException { + replicationLogDiscoveryReplay.start(); + } + + /** + * Delegate the stop replay task to the {@link ReplicationLogDiscoveryReplay} + * @throws IOException - in case the stop operation fails + */ + public void stopReplay() throws IOException { + replicationLogDiscoveryReplay.stop(); + } + + /** + * Initializes the replication replay components including file system, file tracker, state + * tracker, and log discovery service. Sets up the complete replay components for the HA group. + * @throws IOException if there's an error during initialization + */ + protected void init() throws IOException { + initializeFileSystem(); + Path newFilesDirectory = + new Path(new Path(rootURI.getPath(), haGroupName), ReplicationLogReplay.IN_DIRECTORY_NAME); + ReplicationShardDirectoryManager replicationShardDirectoryManager = + new ReplicationShardDirectoryManager(conf, newFilesDirectory); + ReplicationLogTracker replicationLogReplayFileTracker = + new ReplicationLogTracker(conf, haGroupName, fileSystem, replicationShardDirectoryManager, + new MetricsReplicationLogTrackerReplayImpl(haGroupName)); + replicationLogReplayFileTracker.init(); + this.replicationLogDiscoveryReplay = + new ReplicationLogDiscoveryReplay(replicationLogReplayFileTracker); + this.replicationLogDiscoveryReplay.init(); + } + + public void close() { + replicationLogDiscoveryReplay.getReplicationLogFileTracker().close(); + replicationLogDiscoveryReplay.close(); + // Remove the instance from cache + INSTANCES.remove(haGroupName); + } + + /** Initializes the filesystem and creates root log directory. */ + private void initializeFileSystem() throws IOException { + String uriString = conf.get(REPLICATION_LOG_REPLAY_HDFS_URL_KEY); + if (uriString == null || uriString.isEmpty()) { + throw new IOException(REPLICATION_LOG_REPLAY_HDFS_URL_KEY + " is not configured"); + } + try { + this.rootURI = new URI(uriString); + this.fileSystem = FileSystem.get(rootURI, conf); + Path haGroupFilesPath = new Path(rootURI.getPath(), haGroupName); + if (!fileSystem.exists(haGroupFilesPath)) { + LOG.info("Creating directory {}", haGroupFilesPath); + if (!fileSystem.mkdirs(haGroupFilesPath)) { + throw new IOException("Failed to create directory: " + uriString); + } + } + } catch (URISyntaxException e) { + throw new IOException(REPLICATION_LOG_REPLAY_HDFS_URL_KEY + " is not valid", e); + } + } + + protected ReplicationLogDiscoveryReplay getReplicationReplayLogDiscovery() { + return this.replicationLogDiscoveryReplay; + } + + protected FileSystem getFileSystem() { + return this.fileSystem; + } + + protected URI getRootURI() { + return this.rootURI; + } + + protected String getHaGroupName() { + return this.haGroupName; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogReplayService.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogReplayService.java new file mode 100644 index 00000000000..14856750508 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogReplayService.java @@ -0,0 +1,215 @@ +/* + * 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.phoenix.replication.reader; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.phoenix.jdbc.HAGroupStoreManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + +/** + * This component is responsible to start/stop replication log replay via + * {@link ReplicationLogReplay} for all the HA groups + */ +public class ReplicationLogReplayService { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogReplayService.class); + + /** + * Configuration key for enabling/disabling replication replay service + */ + public static final String PHOENIX_REPLICATION_REPLAY_ENABLED = + "phoenix.replication.replay.enabled"; + + /** + * Default value for replication replay service enabled flag + */ + public static final boolean DEFAULT_REPLICATION_REPLAY_ENABLED = false; + + /** + * Number of threads in the executor pool for the replication replay service + */ + public static final int REPLICATION_REPLAY_SERVICE_EXECUTOR_THREAD_COUNT = 1; + + /** + * Configuration key for executor thread frequency in seconds + */ + public static final String REPLICATION_REPLAY_SERVICE_EXECUTOR_THREAD_FREQUENCY_SECONDS_KEY = + "phoenix.replication.replay.service.executor.frequency.seconds"; + + /** + * Default frequency in seconds for executor thread execution + */ + public static final int DEFAULT_REPLICATION_REPLAY_SERVICE_EXECUTOR_THREAD_FREQUENCY_SECONDS = 60; + + /** + * Configuration key for executor shutdown timeout in seconds + */ + public static final String REPLICATION_REPLAY_SERVICE_EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS_KEY = + "phoenix.replication.replay.service.executor.shutdown.timeout.seconds"; + + /** + * Default shutdown timeout in seconds for graceful executor shutdown + */ + public static final int DEFAULT_REPLICATION_REPLAY_SERVICE_EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS = 30; + + private static volatile ReplicationLogReplayService instance; + + private final Configuration conf; + private ScheduledExecutorService scheduler; + private volatile boolean isRunning = false; + + private ReplicationLogReplayService(final Configuration conf) { + this.conf = conf; + } + + /** + * Gets the singleton instance of the ReplicationLogReplayService using the lazy initializer + * pattern. Initializes the instance if it hasn't been created yet. + * @param conf Configuration object. + * @return The singleton ReplicationLogManager instance. + * @throws IOException If initialization fails. + */ + public static ReplicationLogReplayService getInstance(Configuration conf) throws IOException { + if (instance == null) { + synchronized (ReplicationLogReplayService.class) { + if (instance == null) { + instance = new ReplicationLogReplayService(conf); + } + } + } + return instance; + } + + /** + * Starts the replication log replay service by initializing the scheduler and scheduling periodic + * replay operations for each HA Group. + * @throws IOException if there's an error during initialization + */ + public void start() throws IOException { + boolean isEnabled = + conf.getBoolean(PHOENIX_REPLICATION_REPLAY_ENABLED, DEFAULT_REPLICATION_REPLAY_ENABLED); + if (!isEnabled) { + LOG.info("Replication replay service is disabled. Skipping start operation."); + return; + } + synchronized (this) { + if (isRunning) { + LOG.debug("ReplicationLogReplayService is already running"); + return; + } + int executorFrequencySeconds = + conf.getInt(REPLICATION_REPLAY_SERVICE_EXECUTOR_THREAD_FREQUENCY_SECONDS_KEY, + DEFAULT_REPLICATION_REPLAY_SERVICE_EXECUTOR_THREAD_FREQUENCY_SECONDS); + // Initialize and schedule the executors + scheduler = Executors.newScheduledThreadPool(REPLICATION_REPLAY_SERVICE_EXECUTOR_THREAD_COUNT, + new ThreadFactoryBuilder().setNameFormat("ReplicationLogReplayService-%d").build()); + scheduler.scheduleAtFixedRate(() -> { + try { + startReplicationReplay(); + } catch (Exception e) { + LOG.error("Error during trigger of start replication replay", e); + } + }, 0, executorFrequencySeconds, TimeUnit.SECONDS); + isRunning = true; + LOG.info("ReplicationLogReplayService started "); + } + } + + /** + * Stops the replication log replay service by shutting down the scheduler gracefully. Waits for + * the configured shutdown timeout before forcing shutdown if necessary. + * @throws IOException if there's an error during shutdown + */ + public void stop() throws IOException { + boolean isEnabled = + conf.getBoolean(PHOENIX_REPLICATION_REPLAY_ENABLED, DEFAULT_REPLICATION_REPLAY_ENABLED); + if (!isEnabled) { + LOG.info("Replication replay service is disabled. Skipping stop operation."); + return; + } + ScheduledExecutorService schedulerToShutdown = null; + synchronized (this) { + if (!isRunning) { + LOG.warn("ReplicationLogReplayService is not running"); + return; + } + isRunning = false; + schedulerToShutdown = scheduler; + } + int executorShutdownTimeoutSeconds = + conf.getInt(REPLICATION_REPLAY_SERVICE_EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS_KEY, + DEFAULT_REPLICATION_REPLAY_SERVICE_EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS); + if (schedulerToShutdown != null && !schedulerToShutdown.isShutdown()) { + schedulerToShutdown.shutdown(); + try { + if ( + !schedulerToShutdown.awaitTermination(executorShutdownTimeoutSeconds, TimeUnit.SECONDS) + ) { + schedulerToShutdown.shutdownNow(); + } + } catch (InterruptedException e) { + schedulerToShutdown.shutdownNow(); + Thread.currentThread().interrupt(); + } + } + try { + stopReplicationReplay(); + } catch (Exception exception) { + LOG.error("Error while stopping the replication replay.", exception); + // TODO: Should there be an exception thrown instead and halt the RS stop? + } + + LOG.info("ReplicationLogReplayService stopped successfully"); + } + + /** + * Start Replication Replay for all the HA groups + */ + protected void startReplicationReplay() throws IOException, SQLException { + List replicationGroups = getReplicationGroups(); + for (String replicationGroup : replicationGroups) { + ReplicationLogReplay.get(conf, replicationGroup).startReplay(); + } + } + + /** + * Stops Replication Replay for all the HA groups + */ + protected void stopReplicationReplay() throws IOException, SQLException { + List replicationGroups = getReplicationGroups(); + for (String replicationGroup : replicationGroups) { + ReplicationLogReplay replicationLogReplay = ReplicationLogReplay.get(conf, replicationGroup); + replicationLogReplay.stopReplay(); + replicationLogReplay.close(); + } + } + + /** Returns the list of HA groups on the cluster */ + protected List getReplicationGroups() throws SQLException { + return HAGroupStoreManager.getInstance(conf).getHAGroupNames(); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/reader/RecoverLeaseFSUtilsTest.java b/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/RecoverLeaseFSUtilsTestIT.java similarity index 96% rename from phoenix-core/src/test/java/org/apache/phoenix/replication/reader/RecoverLeaseFSUtilsTest.java rename to phoenix-core/src/it/java/org/apache/phoenix/replication/reader/RecoverLeaseFSUtilsTestIT.java index 81bbbcd2bc3..2cab4e2f0c1 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/reader/RecoverLeaseFSUtilsTest.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/RecoverLeaseFSUtilsTestIT.java @@ -29,18 +29,21 @@ import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.end2end.ParallelStatsDisabledIT; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; /** * Test our recoverLease loop against mocked up filesystem. */ -public class RecoverLeaseFSUtilsTest extends ParallelStatsDisabledIT { +@Category(NeedsOwnMiniClusterTest.class) +public class RecoverLeaseFSUtilsTestIT extends ParallelStatsDisabledIT { @ClassRule public static TemporaryFolder testFolder = new TemporaryFolder(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/ReplicationLogDiscoveryReplayTestIT.java b/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/ReplicationLogDiscoveryReplayTestIT.java new file mode 100644 index 00000000000..6eb8cd8b3e2 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/ReplicationLogDiscoveryReplayTestIT.java @@ -0,0 +1,1652 @@ +/* + * 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.phoenix.replication.reader; + +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; +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.mockito.Mockito.when; + +import java.io.IOException; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.util.EnvironmentEdge; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.jdbc.ClusterRoleRecord; +import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.jdbc.HighAvailabilityPolicy; +import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.replication.ReplicationLogGroup; +import org.apache.phoenix.replication.ReplicationLogTracker; +import org.apache.phoenix.replication.ReplicationRound; +import org.apache.phoenix.replication.ReplicationShardDirectoryManager; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTracker; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTrackerReplayImpl; +import org.apache.phoenix.util.HAGroupStoreTestUtil; +import org.apache.phoenix.util.ReadOnlyProps; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; +import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; + +@Category(NeedsOwnMiniClusterTest.class) +public class ReplicationLogDiscoveryReplayTestIT extends BaseTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ReplicationLogDiscoveryReplayTestIT.class); + + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + private String zkUrl; + private String peerZkUrl; + private FileSystem localFs; + private URI standbyUri; + private static final String haGroupName = "testGroup"; + private static final MetricsReplicationLogTracker METRICS_REPLICATION_LOG_TRACKER = + new MetricsReplicationLogTrackerReplayImpl(haGroupName); + + @ClassRule + public static TemporaryFolder testFolder = new TemporaryFolder(); + + @Rule + public TestName testName = new TestName(); + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(1); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + CLUSTERS.start(); + } + + @Before + public void setUp() throws Exception { + zkUrl = getLocalZkUrl(config); + peerZkUrl = CLUSTERS.getZkUrl2(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(testName.getMethodName(), zkUrl, + peerZkUrl, CLUSTERS.getMasterAddress1(), CLUSTERS.getMasterAddress2(), + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + localFs = FileSystem.getLocal(config); + standbyUri = testFolder.getRoot().toURI(); + config.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + } + + @After + public void tearDown() throws IOException { + localFs.delete(new Path(testFolder.getRoot().toURI()), true); + } + + /** + * Tests that the executor thread name format is correctly configured. + */ + @Test + public void testGetExecutorThreadNameFormat() throws IOException { + // Create ReplicationLogDiscoveryReplay instance + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + ReplicationLogDiscoveryReplay discovery = new ReplicationLogDiscoveryReplay(fileTracker); + + // Test that it returns the expected constant value + String result = discovery.getExecutorThreadNameFormat(); + assertEquals("Should return the expected thread name format", + "Phoenix-ReplicationLogDiscoveryReplay-%d", result); + } + + /** + * Tests the replay interval configuration with default and custom values. + */ + @Test + public void testGetReplayIntervalSeconds() throws IOException { + // Create ReplicationLogDiscoveryReplay instance + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + ReplicationLogDiscoveryReplay discovery = new ReplicationLogDiscoveryReplay(fileTracker); + + // Test default value when no custom config is set + long defaultResult = discovery.getReplayIntervalSeconds(); + assertEquals("Should return default value when no custom config is set", + ReplicationLogDiscoveryReplay.DEFAULT_REPLAY_INTERVAL_SECONDS, defaultResult); + + // Test custom value when config is set + config.setLong(ReplicationLogDiscoveryReplay.REPLICATION_REPLAY_INTERVAL_SECONDS_KEY, 120L); + long customResult = discovery.getReplayIntervalSeconds(); + assertEquals("Should return custom value when config is set", 120L, customResult); + } + + /** + * Tests the shutdown timeout configuration with default and custom values. + */ + @Test + public void testGetShutdownTimeoutSeconds() throws IOException { + // Create ReplicationLogDiscoveryReplay instance + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + ReplicationLogDiscoveryReplay discovery = new ReplicationLogDiscoveryReplay(fileTracker); + + // Test default value when no custom config is set + long defaultResult = discovery.getShutdownTimeoutSeconds(); + assertEquals("Should return default value when no custom config is set", + ReplicationLogDiscoveryReplay.DEFAULT_SHUTDOWN_TIMEOUT_SECONDS, defaultResult); + + // Test custom value when config is set + config.setLong(ReplicationLogDiscoveryReplay.REPLICATION_REPLAY_SHUTDOWN_TIMEOUT_SECONDS_KEY, + 45L); + long customResult = discovery.getShutdownTimeoutSeconds(); + assertEquals("Should return custom value when config is set", 45L, customResult); + } + + /** + * Tests the executor thread count configuration with default and custom values. + */ + @Test + public void testGetExecutorThreadCount() throws IOException { + // Create ReplicationLogDiscoveryReplay instance + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + ReplicationLogDiscoveryReplay discovery = new ReplicationLogDiscoveryReplay(fileTracker); + + // Test default value when no custom config is set + int defaultResult = discovery.getExecutorThreadCount(); + assertEquals("Should return default value when no custom config is set", + ReplicationLogDiscoveryReplay.DEFAULT_EXECUTOR_THREAD_COUNT, defaultResult); + + // Test custom value when config is set + config.setInt(ReplicationLogDiscoveryReplay.REPLICATION_REPLAY_EXECUTOR_THREAD_COUNT_KEY, 3); + int customResult = discovery.getExecutorThreadCount(); + assertEquals("Should return custom value when config is set", 3, customResult); + } + + /** + * Tests the in-progress directory processing probability configuration. + */ + @Test + public void testGetInProgressDirectoryProcessProbability() throws IOException { + // Create ReplicationLogDiscoveryReplay instance + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + ReplicationLogDiscoveryReplay discovery = new ReplicationLogDiscoveryReplay(fileTracker); + + // Test default value when no custom config is set + double defaultResult = discovery.getInProgressDirectoryProcessProbability(); + assertEquals("Should return default value when no custom config is set", + ReplicationLogDiscoveryReplay.DEFAULT_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY, + defaultResult, 0.001); + + // Test custom value when config is set + config.setDouble( + ReplicationLogDiscoveryReplay.REPLICATION_REPLAY_IN_PROGRESS_DIRECTORY_PROCESSING_PROBABILITY_KEY, + 10.5); + double customResult = discovery.getInProgressDirectoryProcessProbability(); + assertEquals("Should return custom value when config is set", 10.5, customResult, 0.001); + } + + /** + * Tests the waiting buffer percentage configuration with default and custom values. + */ + @Test + public void testGetWaitingBufferPercentage() throws IOException { + // Create ReplicationLogDiscoveryReplay instance + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + ReplicationLogDiscoveryReplay discovery = new ReplicationLogDiscoveryReplay(fileTracker); + + // Test default value when no custom config is set + double defaultResult = discovery.getWaitingBufferPercentage(); + assertEquals("Should return default value when no custom config is set", + ReplicationLogDiscoveryReplay.DEFAULT_WAITING_BUFFER_PERCENTAGE, defaultResult, 0.001); + + // Test custom value when config is set + config.setDouble(ReplicationLogDiscoveryReplay.REPLICATION_REPLAY_WAITING_BUFFER_PERCENTAGE_KEY, + 20.0); + double customResult = discovery.getWaitingBufferPercentage(); + assertEquals("Should return custom value when config is set", 20.0, customResult, 0.001); + + // Clear the custom config + config.unset(ReplicationLogDiscoveryReplay.REPLICATION_REPLAY_WAITING_BUFFER_PERCENTAGE_KEY); + } + + /** + * Tests initialization in DEGRADED state with both in-progress and new files present. Validates + * that lastRoundProcessed uses minimum timestamp and lastRoundInSync is preserved. + */ + @Test + public void testInitializeLastRoundProcessed_DegradedStateWithInProgressAndNewFiles() + throws IOException { + long currentTime = 1704153600000L; // 2024-01-02 00:00:00 + long inProgressFileTimestamp = 1704153420000L; // Earlier timestamp (00:57:00) - 3 min before + // current + long newFileTimestamp = 1704153540000L; // Middle timestamp (00:59:00) - 1 min before current + long lastSyncStateTime = 1704153480000L; // Between in-progress and new file (00:58:00) + long roundTimeMills = 60000L; // 1 minute + + // Expected: lastRoundProcessed uses minimum timestamp (in-progress file) + long expectedEndTime = + (inProgressFileTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync uses minimum of lastSyncStateTime and file timestamps + long expectedSyncEndTime = + (inProgressFileTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundInSync = + new ReplicationRound(expectedSyncEndTime - roundTimeMills, expectedSyncEndTime); + + testInitializeLastRoundProcessedHelper(currentTime, lastSyncStateTime, newFileTimestamp, + inProgressFileTimestamp, HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, + expectedLastRoundProcessed, expectedLastRoundInSync, + ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED, false); + } + + @Test + public void testInitializeLastRoundProcessed_DegradedStateWithBothFilesNewFileIsMin() + throws IOException { + long currentTime = 1704153600000L; // 2024-01-02 00:00:00 + long newFileTimestamp = 1704153420000L; // Earlier timestamp (00:57:00) - 3 min before current + long inProgressFileTimestamp = 1704153540000L; // Later timestamp (00:59:00) - 1 min before + // current + long lastSyncStateTime = 1704153480000L; // Between new and in-progress file (00:58:00) + long roundTimeMills = 60000L; // 1 minute + + // Expected: lastRoundProcessed uses minimum timestamp (new file) + long expectedEndTime = + (newFileTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync uses minimum of lastSyncStateTime and file timestamps + long expectedSyncEndTime = + (newFileTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundInSync = + new ReplicationRound(expectedSyncEndTime - roundTimeMills, expectedSyncEndTime); + + testInitializeLastRoundProcessedHelper(currentTime, lastSyncStateTime, newFileTimestamp, + inProgressFileTimestamp, HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, + expectedLastRoundProcessed, expectedLastRoundInSync, + ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED, false); + } + + @Test + public void testInitializeLastRoundProcessed_DegradedStateWithLastSyncStateAsMin() + throws IOException { + long newFileTimestamp = 1704240060000L; + long lastSyncStateTime = 1704240030000L; + long currentTime = 1704240900000L; + long roundTimeMills = 60000L; // 1 minute + + // Expected: lastRoundProcessed uses minimum of new files and current time + long expectedEndTime = + (newFileTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync uses minimum of lastSyncStateTime and file timestamps + long expectedSyncEndTime = + (lastSyncStateTime / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundInSync = + new ReplicationRound(expectedSyncEndTime - roundTimeMills, expectedSyncEndTime); + + testInitializeLastRoundProcessedHelper(currentTime, lastSyncStateTime, newFileTimestamp, null, // no + // in-progress + // file + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, expectedLastRoundProcessed, + expectedLastRoundInSync, ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED, + false); + } + + @Test + public void testInitializeLastRoundProcessed_DegradedStateWithNoFiles() throws IOException { + long currentTime = 1704326400000L; + long lastSyncStateTime = 1704326300000L; + long roundTimeMills = 60000L; // 1 minute + + // Expected: lastRoundProcessed uses current time + long expectedEndTime = + (currentTime / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync uses minimum of lastSyncStateTime and current time + long expectedSyncEndTime = + (lastSyncStateTime / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundInSync = + new ReplicationRound(expectedSyncEndTime - roundTimeMills, expectedSyncEndTime); + + testInitializeLastRoundProcessedHelper(currentTime, lastSyncStateTime, null, // no new file + null, // no in-progress file + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, expectedLastRoundProcessed, + expectedLastRoundInSync, ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED, + false); + } + + @Test + public void testInitializeLastRoundProcessed_SyncStateWithInProgressFiles() throws IOException { + long currentTime = 1704412800000L; + long inProgressTimestamp = 1704412680000L; // 2 min before current time + long roundTimeMills = 60000L; // 1 minute + + // Expected: uses in-progress file timestamp + long expectedEndTime = + (inProgressTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync equals lastRoundProcessed in SYNC state + ReplicationRound expectedLastRoundInSync = expectedLastRoundProcessed; + + testInitializeLastRoundProcessedHelper(currentTime, null, // SYNC state - lastSyncStateTime not + // used + null, // no new file + inProgressTimestamp, HAGroupStoreRecord.HAGroupState.STANDBY, expectedLastRoundProcessed, + expectedLastRoundInSync, ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, false); + } + + @Test + public void testInitializeLastRoundProcessed_SyncStateWithNewFiles() throws IOException { + long currentTime = 1704499200000L; + long newFileTimestamp = 1704499080000L; // 2 min before current time + long roundTimeMills = 60000L; // 1 minute + + // Expected: uses new file timestamp + long expectedEndTime = + (newFileTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync equals lastRoundProcessed in SYNC state + ReplicationRound expectedLastRoundInSync = expectedLastRoundProcessed; + + testInitializeLastRoundProcessedHelper(currentTime, null, // SYNC state - lastSyncStateTime not + // used + newFileTimestamp, null, // no in-progress file + HAGroupStoreRecord.HAGroupState.STANDBY, expectedLastRoundProcessed, expectedLastRoundInSync, + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, false); + } + + @Test + public void testInitializeLastRoundProcessed_SyncStateWithBothFiles() throws IOException { + long currentTime = 1704499200000L; + long inProgressTimestamp = 1704499020000L; // Earlier timestamp - 3 min before current + long newFileTimestamp = 1704499140000L; // Later timestamp - 1 min before current + long roundTimeMills = 60000L; // 1 minute + + // Expected: uses minimum timestamp (in-progress file) + long expectedEndTime = + (inProgressTimestamp / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync equals lastRoundProcessed in SYNC state + ReplicationRound expectedLastRoundInSync = expectedLastRoundProcessed; + + testInitializeLastRoundProcessedHelper(currentTime, null, // SYNC state - lastSyncStateTime not + // used + newFileTimestamp, inProgressTimestamp, HAGroupStoreRecord.HAGroupState.STANDBY, + expectedLastRoundProcessed, expectedLastRoundInSync, + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, false); + } + + @Test + public void testInitializeLastRoundProcessed_SyncStateWithNoFiles() throws IOException { + long currentTime = 1704585600000L; + long roundTimeMills = 60000L; // 1 minute + + // Expected: uses current time when no files exist + long expectedEndTime = + (currentTime / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(expectedEndTime - roundTimeMills, expectedEndTime); + + // Expected: lastRoundInSync equals lastRoundProcessed in SYNC state + ReplicationRound expectedLastRoundInSync = expectedLastRoundProcessed; + + testInitializeLastRoundProcessedHelper(currentTime, null, // SYNC state - lastSyncStateTime not + // used + null, // no new file + null, // no in-progress file + HAGroupStoreRecord.HAGroupState.STANDBY, expectedLastRoundProcessed, expectedLastRoundInSync, + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, false); + } + + /** + * Tests initializeLastRoundProcessed in STANDBY_TO_ACTIVE state. Validates that failoverPending + * is set to true when HA group state is STANDBY_TO_ACTIVE. + */ + @Test + public void testInitializeLastRoundProcessed_StandbyToActiveState() throws IOException { + long currentTime = 1704153600000L; // 2024-01-02 00:00:00 + + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + fileTracker.init(); + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + + ReplicationRound expectedLastRoundProcessed = + new ReplicationRound(currentTime - roundTimeMills, currentTime); + ReplicationRound expectedLastRoundInSync = + new ReplicationRound(currentTime - roundTimeMills, currentTime); + + testInitializeLastRoundProcessedHelper(currentTime, null, null, null, + HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE, expectedLastRoundProcessed, + expectedLastRoundInSync, ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, true); + } + + /** + * Helper method to test initializeLastRoundProcessed with various file and state configurations. + * Handles file creation, state setup, and validation of lastRoundProcessed and lastRoundInSync. + * @param currentTime Current time for the test + * @param lastSyncStateTime Last sync state time for HAGroupStoreRecord (use null for + * SYNC state) + * @param newFileTimestamp Timestamp for new file (use null to skip creating new file) + * @param inProgressFileTimestamp Timestamp for in-progress file (use null to skip creating + * in-progress file) + * @param haGroupState HAGroupState for the test + * @param expectedLastRoundProcessed Expected lastRoundProcessed after initialization + * @param expectedLastRoundInSync Expected lastRoundInSync after initialization + * @param expectedReplayState Expected ReplicationReplayState after initialization + * @param expectedFailoverPending Expected failoverPending value after initialization + */ + private void testInitializeLastRoundProcessedHelper(long currentTime, Long lastSyncStateTime, + Long newFileTimestamp, Long inProgressFileTimestamp, + HAGroupStoreRecord.HAGroupState haGroupState, ReplicationRound expectedLastRoundProcessed, + ReplicationRound expectedLastRoundInSync, + ReplicationLogDiscoveryReplay.ReplicationReplayState expectedReplayState, + boolean expectedFailoverPending) throws IOException { + + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + fileTracker.init(); + + try { + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + + // Create in-progress file if timestamp provided + if (inProgressFileTimestamp != null) { + Path inProgressDir = fileTracker.getInProgressDirPath(); + localFs.mkdirs(inProgressDir); + Path inProgressFile = new Path(inProgressDir, inProgressFileTimestamp + "_rs-1_uuid.plog"); + localFs.create(inProgressFile, true).close(); + } + + // Create new file if timestamp provided + if (newFileTimestamp != null) { + ReplicationRound newFileRound = + new ReplicationRound(newFileTimestamp - roundTimeMills, newFileTimestamp); + Path shardPath = fileTracker.getReplicationShardDirectoryManager() + .getShardDirectory(newFileRound.getStartTime()); + localFs.mkdirs(shardPath); + Path newFile = new Path(shardPath, newFileTimestamp + "_rs-1.plog"); + localFs.create(newFile, true).close(); + } + + // Create HAGroupStoreRecord + long recordTime = lastSyncStateTime != null ? lastSyncStateTime : currentTime; + HAGroupStoreRecord mockRecord = new HAGroupStoreRecord( + HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, haGroupState, recordTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + discovery.initializeLastRoundProcessed(); + + // Verify lastRoundProcessed + ReplicationRound lastRoundProcessed = discovery.getLastRoundProcessed(); + assertNotNull("Last round processed should not be null", lastRoundProcessed); + assertEquals("Last round processed should match expected", expectedLastRoundProcessed, + lastRoundProcessed); + + // Verify lastRoundInSync + ReplicationRound lastRoundInSync = discovery.getLastRoundInSync(); + assertNotNull("Last round in sync should not be null", lastRoundInSync); + assertEquals("Last round in sync should match expected", expectedLastRoundInSync, + lastRoundInSync); + + // Verify state + assertEquals("Replication replay state should match expected", expectedReplayState, + discovery.getReplicationReplayState()); + + // Verify failoverPending + assertEquals("Failover pending should match expected", expectedFailoverPending, + discovery.getFailoverPending()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests replay in SYNC state processing multiple rounds. Validates that both lastRoundProcessed + * and lastRoundInSync advance together. + */ + @Test + public void testReplay_SyncState_ProcessMultipleRounds() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704153600000L; // 2024-01-02 00:00:00 + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + long bufferMillis = (long) (roundTimeMills * 0.15); + long totalWaitTime = roundTimeMills + bufferMillis; + + // Create HAGroupStoreRecord for SYNC state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to allow processing 3 rounds + long currentTime = initialEndTime + (3 * totalWaitTime); + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + ReplicationRound initialRound = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + discovery.setLastRoundProcessed(initialRound); + discovery.setLastRoundInSync(initialRound); + discovery + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC); + + discovery.replay(); + + // Verify processRound was called 3 times + assertEquals("processRound should be called 3 times", 3, + discovery.getProcessRoundCallCount()); + + // Verify the rounds passed to processRound + List processedRounds = discovery.getProcessedRounds(); + ReplicationRound expectedRound1 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round should match expected", expectedRound1, processedRounds.get(0)); + + ReplicationRound expectedRound2 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Second round should match expected", expectedRound2, processedRounds.get(1)); + + ReplicationRound expectedRound3 = new ReplicationRound( + initialEndTime + (2 * roundTimeMills), initialEndTime + (3 * roundTimeMills)); + assertEquals("Third round should match expected", expectedRound3, processedRounds.get(2)); + + // Verify lastRoundProcessed was updated to 3rd round + ReplicationRound lastRoundProcessed = discovery.getLastRoundProcessed(); + assertNotNull("Last round processed should not be null", lastRoundProcessed); + assertEquals("Last round processed should be updated to 3rd round", expectedRound3, + lastRoundProcessed); + + // Verify lastRoundInSync was also updated in SYNC state + ReplicationRound lastRoundInSync = discovery.getLastRoundInSync(); + assertNotNull("Last round in sync should not be null", lastRoundInSync); + assertEquals( + "Last round in sync should be updated to match last round processed in SYNC state", + expectedRound3, lastRoundInSync); + + // Verify state remains SYNC + assertEquals("State should remain SYNC", + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, + discovery.getReplicationReplayState()); + + // Verify triggerFailover was not called + assertEquals("triggerFailover should not be called", 0, + discovery.getTriggerFailoverCallCount()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests replay in DEGRADED state processing multiple rounds. Validates that lastRoundProcessed + * advances but lastRoundInSync is preserved. + */ + @Test + public void testReplay_DegradedState_MultipleRounds() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704240000000L; + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + long bufferMillis = (long) (roundTimeMills * 0.15); + long totalWaitTime = roundTimeMills + bufferMillis; + + // Create HAGroupStoreRecord for DEGRADED state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to allow processing 3 rounds + long currentTime = initialEndTime + (3 * totalWaitTime); + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + discovery.setLastRoundProcessed( + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime)); + ReplicationRound lastRoundInSyncBeforeReplay = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + discovery.setLastRoundInSync(lastRoundInSyncBeforeReplay); + discovery + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED); + + discovery.replay(); + + // Verify processRound was called 3 times + assertEquals("processRound should be called 3 times", 3, + discovery.getProcessRoundCallCount()); + + // Verify the rounds passed to processRound + List processedRounds = discovery.getProcessedRounds(); + ReplicationRound expectedRound1 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round should match expected", expectedRound1, processedRounds.get(0)); + + ReplicationRound expectedRound2 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Second round should match expected", expectedRound2, processedRounds.get(1)); + + ReplicationRound expectedRound3 = new ReplicationRound( + initialEndTime + (2 * roundTimeMills), initialEndTime + (3 * roundTimeMills)); + assertEquals("Third round should match expected", expectedRound3, processedRounds.get(2)); + + // Verify lastRoundProcessed was updated + ReplicationRound lastRoundAfterReplay = discovery.getLastRoundProcessed(); + assertEquals("Last round processed should be updated to 3rd round in DEGRADED state", + expectedRound3, lastRoundAfterReplay); + + // Verify lastRoundInSync was NOT updated (preserved in DEGRADED state) + ReplicationRound lastRoundInSyncAfterReplay = discovery.getLastRoundInSync(); + assertEquals("Last round in sync should NOT be updated in DEGRADED state", + lastRoundInSyncBeforeReplay, lastRoundInSyncAfterReplay); + + // Verify state remains DEGRADED + assertEquals("State should remain DEGRADED", + ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED, + discovery.getReplicationReplayState()); + + // Verify triggerFailover was not called + assertEquals("triggerFailover should not be called", 0, + discovery.getTriggerFailoverCallCount()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests replay in SYNCED_RECOVERY state with rewind to lastRoundInSync. Validates that processing + * rewinds and re-processes from lastRoundInSync. + */ + @Test + public void testReplay_SyncedRecoveryState_RewindToLastInSync() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704326400000L; + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + long bufferMillis = (long) (roundTimeMills * 0.15); + long totalWaitTime = roundTimeMills + bufferMillis; + + // Create HAGroupStoreRecord for STANDBY state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to allow processing multiple rounds + long currentTime = initialEndTime + (5 * totalWaitTime); + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + + // Set initial state: lastRoundProcessed is ahead, lastRoundInSync is behind + ReplicationRound lastInSyncRound = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + ReplicationRound currentRound = new ReplicationRound(initialEndTime + (2 * roundTimeMills), + initialEndTime + (3 * roundTimeMills)); + + discovery.setLastRoundProcessed(currentRound); + discovery.setLastRoundInSync(lastInSyncRound); + discovery.setReplicationReplayState( + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNCED_RECOVERY); + + discovery.replay(); + + // Verify processRound was called 6 times + // Flow: 1 round in SYNCED_RECOVERY (triggers rewind), then 5 rounds in SYNC + // getFirstRoundToProcess() uses lastRoundInSync.endTime = initialEndTime + // After processing first round in SYNCED_RECOVERY, it rewinds to lastRoundInSync + // Then continues processing from initialEndTime again (re-processing first round) + assertEquals("processRound should be called 6 times", 6, + discovery.getProcessRoundCallCount()); + + // Verify the first round - starts from lastRoundInSync.endTime = initialEndTime (via + // getFirstRoundToProcess) + List processedRounds = discovery.getProcessedRounds(); + ReplicationRound expectedFirstRound = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round (SYNCED_RECOVERY) should start from lastRoundInSync.endTime", + expectedFirstRound, processedRounds.get(0)); + + // After SYNCED_RECOVERY rewind, processing restarts from lastRoundInSync.endTime = + // initialEndTime + // This means round 1 is processed again, then rounds 2-5 + ReplicationRound expectedSecondRound = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("Second round (first in SYNC, re-processing) should match expected", + expectedSecondRound, processedRounds.get(1)); + + ReplicationRound expectedThirdRound = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Third round should match expected", expectedThirdRound, + processedRounds.get(2)); + + ReplicationRound expectedFourthRound = new ReplicationRound( + initialEndTime + (2 * roundTimeMills), initialEndTime + (3 * roundTimeMills)); + assertEquals("Fourth round should match expected", expectedFourthRound, + processedRounds.get(3)); + + ReplicationRound expectedFifthRound = new ReplicationRound( + initialEndTime + (3 * roundTimeMills), initialEndTime + (4 * roundTimeMills)); + assertEquals("Fifth round should match expected", expectedFifthRound, + processedRounds.get(4)); + + ReplicationRound expectedSixthRound = new ReplicationRound( + initialEndTime + (4 * roundTimeMills), initialEndTime + (5 * roundTimeMills)); + assertEquals("Sixth round should match expected", expectedSixthRound, + processedRounds.get(5)); + + // Verify lastRoundProcessed was updated + ReplicationRound lastRoundProcessed = discovery.getLastRoundProcessed(); + assertNotNull("Last round processed should not be null", lastRoundProcessed); + assertEquals("Last round processed should be updated to 5th round", expectedSixthRound, + lastRoundProcessed); + + // Verify lastRoundInSync was also updated in SYNC state (after rewind and transition) + ReplicationRound lastRoundInSync = discovery.getLastRoundInSync(); + assertNotNull("Last round in sync should not be null", lastRoundInSync); + assertEquals( + "Last round in sync should be updated to match last round processed after SYNC transition", + expectedSixthRound, lastRoundInSync); + + // Verify state transitioned to SYNC + assertEquals("State should transition to SYNC", + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, + discovery.getReplicationReplayState()); + + // Verify triggerFailover was not called + assertEquals("triggerFailover should not be called", 0, + discovery.getTriggerFailoverCallCount()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests state transition from SYNC to DEGRADED during replay processing. Validates that + * lastRoundInSync is preserved at the last SYNC round. + */ + @Test + public void testReplay_StateTransition_SyncToDegradedDuringProcessing() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704412800000L; + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + long bufferMillis = (long) (roundTimeMills * 0.15); + long totalWaitTime = roundTimeMills + bufferMillis; + + // Create HAGroupStoreRecord for STANDBY state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to allow processing 5 rounds + long currentTime = initialEndTime + (5 * totalWaitTime); + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + ReplicationRound initialRound = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + discovery.setLastRoundProcessed(initialRound); + discovery.setLastRoundInSync(initialRound); + discovery + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC); + + // Simulate listener changing state to DEGRADED after 2 rounds + discovery.setStateChangeAfterRounds(2, + ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED); + + discovery.replay(); + + // Verify processRound was called 5 times (2 in SYNC, 3 in DEGRADED) + assertEquals("processRound should be called 5 times", 5, + discovery.getProcessRoundCallCount()); + + // Verify the rounds passed to processRound + List processedRounds = discovery.getProcessedRounds(); + + // First 2 rounds in SYNC mode + ReplicationRound expectedRound1 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round should match expected", expectedRound1, processedRounds.get(0)); + + ReplicationRound expectedRound2 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Second round should match expected", expectedRound2, processedRounds.get(1)); + + // Remaining 3 rounds in DEGRADED mode + ReplicationRound expectedRound3 = new ReplicationRound( + initialEndTime + (2 * roundTimeMills), initialEndTime + (3 * roundTimeMills)); + assertEquals("Third round should match expected", expectedRound3, processedRounds.get(2)); + + ReplicationRound expectedRound4 = new ReplicationRound( + initialEndTime + (3 * roundTimeMills), initialEndTime + (4 * roundTimeMills)); + assertEquals("Fourth round should match expected", expectedRound4, processedRounds.get(3)); + + ReplicationRound expectedRound5 = new ReplicationRound( + initialEndTime + (4 * roundTimeMills), initialEndTime + (5 * roundTimeMills)); + assertEquals("Fifth round should match expected", expectedRound5, processedRounds.get(4)); + + // Verify lastRoundProcessed was updated to 5th round + ReplicationRound lastRoundAfterReplay = discovery.getLastRoundProcessed(); + assertEquals("Last round processed should be updated to 5th round", expectedRound5, + lastRoundAfterReplay); + + // Verify lastRoundInSync was updated only for first round (SYNC), then preserved + // State changed to DEGRADED AFTER processing round 2, but BEFORE updating lastRoundInSync + // for round 2 + // So lastRoundInSync should remain at round 1 (the last round fully completed in SYNC + // state) + ReplicationRound lastRoundInSync = discovery.getLastRoundInSync(); + assertNotNull("Last round in sync should not be null", lastRoundInSync); + assertEquals( + "Last round in sync should be preserved at round 1 (last fully completed SYNC round)", + expectedRound1, lastRoundInSync); + + // Verify state is now DEGRADED + assertEquals("State should be DEGRADED", + ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED, + discovery.getReplicationReplayState()); + + // Verify triggerFailover was not called + assertEquals("triggerFailover should not be called", 0, + discovery.getTriggerFailoverCallCount()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests state transition from DEGRADED to SYNCED_RECOVERY and then to SYNC. Validates rewind + * behavior and lastRoundInSync update after SYNC transition. + */ + @Test + public void testReplay_StateTransition_DegradedToSyncedRecovery() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704499200000L; + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + long bufferMillis = (long) (roundTimeMills * 0.15); + + // Create HAGroupStoreRecord for DEGRADED state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to allow processing 5 rounds + long currentTime = initialEndTime + (5 * roundTimeMills) + bufferMillis; + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + + ReplicationRound lastInSyncRound = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + discovery.setLastRoundProcessed(new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills))); + discovery.setLastRoundInSync(lastInSyncRound); + discovery + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED); + + // Simulate listener changing state to SYNCED_RECOVERY after 2 rounds + discovery.setStateChangeAfterRounds(2, + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNCED_RECOVERY); + + discovery.replay(); + + // Verify processRound was called: 2 in DEGRADED, then 5 in SYNC + // Total: 2 + 5 = 7 calls + // First call uses getFirstRoundToProcess() which starts from lastRoundInSync.endTime = + // initialEndTime + // After state change to SYNCED_RECOVERY, it rewinds and continues from + // lastRoundInSync.endTime + assertEquals("processRound should be called 7 times", 7, + discovery.getProcessRoundCallCount()); + + // Verify the rounds passed to processRound + List processedRounds = discovery.getProcessedRounds(); + + // First 2 rounds in DEGRADED mode (starting from lastRoundInSync.endTime = initialEndTime) + // getFirstRoundToProcess() uses lastRoundInSync.endTime, not lastRoundProcessed.endTime + ReplicationRound expectedRound1 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round should start from lastRoundInSync.endTime", expectedRound1, + processedRounds.get(0)); + + ReplicationRound expectedRound2 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Second round should match expected", expectedRound2, processedRounds.get(1)); + + // After 2 rounds, state changes to SYNCED_RECOVERY and rewinds to lastRoundInSync + // Processing restarts from lastRoundInSync.endTime = initialEndTime (re-processing rounds + // 1-2 and continuing) + ReplicationRound expectedRound3 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("Third round (after rewind) should restart from lastRoundInSync.endTime", + expectedRound3, processedRounds.get(2)); + + ReplicationRound expectedRound4 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Fourth round should match expected", expectedRound4, processedRounds.get(3)); + + ReplicationRound expectedRound5 = new ReplicationRound( + initialEndTime + (2 * roundTimeMills), initialEndTime + (3 * roundTimeMills)); + assertEquals("Fifth round should match expected", expectedRound5, processedRounds.get(4)); + + ReplicationRound expectedRound6 = new ReplicationRound( + initialEndTime + (3 * roundTimeMills), initialEndTime + (4 * roundTimeMills)); + assertEquals("Sixth round should match expected", expectedRound6, processedRounds.get(5)); + + ReplicationRound expectedRound7 = new ReplicationRound( + initialEndTime + (4 * roundTimeMills), initialEndTime + (5 * roundTimeMills)); + assertEquals("Seventh round should match expected", expectedRound7, processedRounds.get(6)); + + // Verify lastRoundProcessed was updated to 7th round + ReplicationRound lastRoundAfterReplay = discovery.getLastRoundProcessed(); + assertEquals("Last round processed should be updated to 7th round", expectedRound7, + lastRoundAfterReplay); + + // Verify lastRoundInSync was preserved during DEGRADED, then updated during SYNC + // After transition to SYNC (from SYNCED_RECOVERY), lastRoundInSync should match + // lastRoundProcessed + ReplicationRound lastRoundInSync = discovery.getLastRoundInSync(); + assertNotNull("Last round in sync should not be null", lastRoundInSync); + assertEquals( + "Last round in sync should be updated to match last round processed after SYNC transition", + expectedRound7, lastRoundInSync); + + // Verify state transitioned to SYNC (from SYNCED_RECOVERY) + assertEquals("State should be SYNC after SYNCED_RECOVERY", + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, + discovery.getReplicationReplayState()); + + // Verify triggerFailover was not called + assertEquals("triggerFailover should not be called", 0, + discovery.getTriggerFailoverCallCount()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests state transition from SYNC to DEGRADED and back through SYNCED_RECOVERY to SYNC. + * Validates lastRoundInSync preservation during DEGRADED, rewind in SYNCED_RECOVERY, and update + * in SYNC. + */ + @Test + public void testReplay_StateTransition_SyncToDegradedAndBackToSync() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704672000000L; + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + long bufferMillis = (long) (roundTimeMills * 0.15); + + // Create HAGroupStoreRecord for STANDBY state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to allow processing enough rounds (including rewind) + long currentTime = initialEndTime + (10 * roundTimeMills) + bufferMillis; + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + ReplicationRound initialRound = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + discovery.setLastRoundProcessed(initialRound); + discovery.setLastRoundInSync(initialRound); + discovery + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC); + + // Simulate state transitions: + // - After 2 rounds: SYNC -> DEGRADED + // - After 5 rounds: DEGRADED -> SYNCED_RECOVERY (triggers rewind to lastRoundInSync) + TestableReplicationLogDiscoveryReplay discoveryWithTransitions = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord) { + private int roundCount = 0; + + @Override + protected void processRound(ReplicationRound replicationRound) throws IOException { + super.processRound(replicationRound); + roundCount++; + + // Transition to DEGRADED after 2 rounds + if (roundCount == 2) { + setReplicationReplayState( + ReplicationLogDiscoveryReplay.ReplicationReplayState.DEGRADED); + } + // Transition to SYNCED_RECOVERY after 5 rounds (will trigger rewind) + else if (roundCount == 5) { + setReplicationReplayState( + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNCED_RECOVERY); + } + } + }; + + discoveryWithTransitions.setLastRoundProcessed(initialRound); + discoveryWithTransitions.setLastRoundInSync(initialRound); + discoveryWithTransitions + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC); + + discoveryWithTransitions.replay(); + + // Verify processRound was called exactly 15 times: + // - 2 rounds in SYNC (rounds 1-2) + // - 3 rounds in DEGRADED (rounds 3-5) + // - 1 round in SYNCED_RECOVERY (round 6, triggers rewind to lastRoundInSync) + // - After rewind, continues from lastRoundInSync.endTime = firstRound + // - 8 more rounds in SYNC (rounds 7-14, from initialEndTime to initialEndTime + + // 8*roundTimeMills) + int totalRoundsProcessed = discoveryWithTransitions.getProcessRoundCallCount(); + assertEquals("processRound should be called exactly 14 times", 14, totalRoundsProcessed); + + // Verify the rounds passed to processRound + List processedRounds = discoveryWithTransitions.getProcessedRounds(); + + // Rounds 1-2: SYNC mode (starting from initialEndTime) + for (int i = 0; i < 2; i++) { + long startTime = initialEndTime + (i * roundTimeMills); + long endTime = initialEndTime + ((i + 1) * roundTimeMills); + ReplicationRound expectedRound = new ReplicationRound(startTime, endTime); + assertEquals("Round " + (i + 1) + " (SYNC) should match expected", expectedRound, + processedRounds.get(i)); + } + + // Rounds 3-5: DEGRADED mode (continuing from round 2) + for (int i = 2; i < 5; i++) { + long startTime = initialEndTime + (i * roundTimeMills); + long endTime = initialEndTime + ((i + 1) * roundTimeMills); + ReplicationRound expectedRound = new ReplicationRound(startTime, endTime); + assertEquals("Round " + (i + 1) + " (DEGRADED) should match expected", expectedRound, + processedRounds.get(i)); + } + + // Round 6-14: SYNC mode after SYNCED_RECOVERY rewind + // After rewind, starts from lastRoundInSync.endTime = initialEndTime + roundTimeMills + for (int i = 5; i < 14; i++) { + // Offset by 1 because rewind goes back to lastRoundInSync.endTime + long startTime = initialEndTime + ((i - 4) * roundTimeMills); + long endTime = initialEndTime + ((i - 3) * roundTimeMills); + ReplicationRound expectedRound = new ReplicationRound(startTime, endTime); + assertEquals("Round " + (i + 1) + " (SYNC after rewind) should match expected", + expectedRound, processedRounds.get(i)); + } + + // Verify lastRoundProcessed was updated to the last processed round (round 14) + ReplicationRound lastRoundAfterReplay = discoveryWithTransitions.getLastRoundProcessed(); + assertNotNull("Last round processed should not be null", lastRoundAfterReplay); + ReplicationRound expectedLastRound = new ReplicationRound( + initialEndTime + (9 * roundTimeMills), initialEndTime + (10 * roundTimeMills)); + assertEquals("Last round processed should be the 14th round", expectedLastRound, + lastRoundAfterReplay); + + // Verify lastRoundInSync behavior: + // - Updated for round 1 (SYNC) + // - State changed to DEGRADED after round 2, so lastRoundInSync stays at round 1 + // - Preserved during rounds 3-5 (DEGRADED) + // - SYNCED_RECOVERY triggers rewind to lastRoundInSync + // - After transition to SYNC, lastRoundInSync updates for rounds 6-14 + // - Final lastRoundInSync should match lastRoundProcessed (round 14) + ReplicationRound lastRoundInSync = discoveryWithTransitions.getLastRoundInSync(); + assertNotNull("Last round in sync should not be null", lastRoundInSync); + assertEquals("Last round in sync should match last round processed after returning to SYNC", + expectedLastRound, lastRoundInSync); + + // Verify state transitioned to SYNC (from SYNCED_RECOVERY) + assertEquals("State should be SYNC after SYNCED_RECOVERY", + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, + discoveryWithTransitions.getReplicationReplayState()); + + // Verify triggerFailover was not called + assertEquals("triggerFailover should not be called", 0, + discoveryWithTransitions.getTriggerFailoverCallCount()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests replay when no rounds are ready to process. Validates that lastRoundProcessed and + * lastRoundInSync remain unchanged. + */ + @Test + public void testReplay_NoRoundsToProcess() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704585600000L; + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + + // Create HAGroupStoreRecord for SYNC state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to NOT allow processing any rounds (not enough time has passed) + long currentTime = initialEndTime + 1000L; // Only 1 second after + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + ReplicationRound initialRound = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + discovery.setLastRoundProcessed(initialRound); + discovery.setLastRoundInSync(initialRound); + discovery + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC); + + ReplicationRound lastRoundBeforeReplay = discovery.getLastRoundProcessed(); + ReplicationRound lastRoundInSyncBeforeReplay = discovery.getLastRoundInSync(); + + discovery.replay(); + + // Verify processRound was not called + assertEquals("processRound should not be called when no rounds to process", 0, + discovery.getProcessRoundCallCount()); + + // Verify lastRoundProcessed was not changed + ReplicationRound lastRoundAfterReplay = discovery.getLastRoundProcessed(); + assertEquals("Last round processed should not change when no rounds to process", + lastRoundBeforeReplay, lastRoundAfterReplay); + + // Verify lastRoundInSync was not changed + ReplicationRound lastRoundInSyncAfterReplay = discovery.getLastRoundInSync(); + assertEquals("Last round in sync should not change when no rounds to process", + lastRoundInSyncBeforeReplay, lastRoundInSyncAfterReplay); + + // Verify state remains SYNC + assertEquals("State should remain SYNC", + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, + discovery.getReplicationReplayState()); + + // Verify triggerFailover was not called + assertEquals("triggerFailover should not be called", 0, + discovery.getTriggerFailoverCallCount()); + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + /** + * Tests replay method when failoverPending becomes true during processing and triggers failover + * after all rounds. Validates that triggerFailover is called exactly once when all conditions are + * met. + */ + @Test + public void testReplay_TriggerFailoverAfterProcessing() throws IOException { + TestableReplicationLogTracker fileTracker = + createReplicationLogTracker(config, haGroupName, localFs, standbyUri); + + try { + long initialEndTime = 1704153600000L; // 2024-01-02 00:00:00 + long roundTimeMills = + fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + * 1000L; + long bufferMillis = (long) (roundTimeMills * 0.15); + long totalWaitTime = roundTimeMills + bufferMillis; + + // Create HAGroupStoreRecord for STANDBY state + HAGroupStoreRecord mockRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + // Set current time to allow processing 3 rounds + long currentTime = initialEndTime + (3 * totalWaitTime); + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(fileTracker, mockRecord); + ReplicationRound initialRound = + new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime); + discovery.setLastRoundProcessed(initialRound); + discovery.setLastRoundInSync(initialRound); + discovery + .setReplicationReplayState(ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC); + + // Set up state change: after processing 2 rounds, set failoverPending to true + discovery.setStateChangeAfterRounds(2, + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC); + discovery.setFailoverPendingAfterRounds(2, true); + + discovery.replay(); + + // Verify processRound was called 3 times + assertEquals("processRound should be called 3 times", 3, + discovery.getProcessRoundCallCount()); + + // Verify the rounds passed to processRound + List processedRounds = discovery.getProcessedRounds(); + ReplicationRound expectedRound1 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round should match expected", expectedRound1, processedRounds.get(0)); + + ReplicationRound expectedRound2 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Second round should match expected", expectedRound2, processedRounds.get(1)); + + ReplicationRound expectedRound3 = new ReplicationRound( + initialEndTime + (2 * roundTimeMills), initialEndTime + (3 * roundTimeMills)); + assertEquals("Third round should match expected", expectedRound3, processedRounds.get(2)); + + // Verify lastRoundProcessed was updated to 3rd round + ReplicationRound lastRoundProcessed = discovery.getLastRoundProcessed(); + assertNotNull("Last round processed should not be null", lastRoundProcessed); + assertEquals("Last round processed should be updated to 3rd round", expectedRound3, + lastRoundProcessed); + + // Verify lastRoundInSync was also updated in SYNC state + ReplicationRound lastRoundInSync = discovery.getLastRoundInSync(); + assertNotNull("Last round in sync should not be null", lastRoundInSync); + assertEquals( + "Last round in sync should be updated to match last round processed in SYNC state", + expectedRound3, lastRoundInSync); + + // Verify state remains SYNC + assertEquals("State should remain SYNC", + ReplicationLogDiscoveryReplay.ReplicationReplayState.SYNC, + discovery.getReplicationReplayState()); + + // Verify triggerFailover was called exactly once + assertEquals("triggerFailover should be called exactly once", 1, + discovery.getTriggerFailoverCallCount()); + + // Verify failoverPending is set to false after failover is triggered + assertFalse("failoverPending should be set to false after failover is triggered", + discovery.getFailoverPending()); + + // TODO: Ensure cluster state is updated to ACTIVE_IN_SYNC once failover is triggered. + } finally { + EnvironmentEdgeManager.reset(); + } + } finally { + fileTracker.close(); + } + } + + private TestableReplicationLogTracker createReplicationLogTracker(final Configuration config, + final String haGroupName, final FileSystem fileSystem, final URI rootURI) throws IOException { + Path newFilesDirectory = + new Path(new Path(rootURI.getPath(), haGroupName), ReplicationLogReplay.IN_DIRECTORY_NAME); + ReplicationShardDirectoryManager replicationShardDirectoryManager = + new ReplicationShardDirectoryManager(config, newFilesDirectory); + TestableReplicationLogTracker testableReplicationLogTracker = + new TestableReplicationLogTracker(config, haGroupName, fileSystem, + replicationShardDirectoryManager, METRICS_REPLICATION_LOG_TRACKER); + testableReplicationLogTracker.init(); + return testableReplicationLogTracker; + } + + /** + * Testable implementation of ReplicationLogTracker for unit testing. Exposes protected methods to + * allow test access. + */ + private static class TestableReplicationLogTracker extends ReplicationLogTracker { + public TestableReplicationLogTracker(Configuration config, String haGroupName, + FileSystem fileSystem, ReplicationShardDirectoryManager replicationShardDirectoryManager, + MetricsReplicationLogTracker metrics) { + super(config, haGroupName, fileSystem, replicationShardDirectoryManager, metrics); + } + + public Path getInProgressDirPath() { + return super.getInProgressDirPath(); + } + } + + /** + * Tests the shouldTriggerFailover method with various combinations of failoverPending, + * lastRoundInSync, lastRoundProcessed and in-progress files state. + */ + @Test + public void testShouldTriggerFailover() throws IOException { + // Set up current time for consistent testing + long currentTime = 1704153660000L; // 00:01:00 + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + // Initialize haGroupStoreRecord + final ReplicationLogTracker tracker = + Mockito.spy(createReplicationLogTracker(config, haGroupName, localFs, standbyUri)); + long initialEndTime = currentTime + - tracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() * 1000L; + HAGroupStoreRecord haGroupStoreRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.STANDBY, initialEndTime, + HighAvailabilityPolicy.FAILOVER.toString(), peerZkUrl, zkUrl, peerZkUrl, 0L); + + try { + // Create test rounds + ReplicationRound testRound = new ReplicationRound(1704153600000L, 1704153660000L); + ReplicationRound differentRound = new ReplicationRound(1704153540000L, 1704153600000L); + + // Test Case 1: All conditions true - should return true + { + when(tracker.getInProgressFiles()).thenReturn(Collections.emptyList()); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(testRound); + discovery.setFailoverPending(true); + + assertTrue("Should trigger failover when all conditions are met", + discovery.shouldTriggerFailover()); + } + + // Test Case 2: failoverPending is false - should return false + { + when(tracker.getInProgressFiles()).thenReturn(Collections.emptyList()); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(testRound); + discovery.setFailoverPending(false); + + assertFalse("Should not trigger failover when failoverPending is false", + discovery.shouldTriggerFailover()); + } + + // Test Case 3: lastRoundInSync not equals lastRoundProcessed - should return false + { + when(tracker.getInProgressFiles()).thenReturn(Collections.emptyList()); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(differentRound); + discovery.setFailoverPending(true); + + assertFalse("Should not trigger failover when lastRoundInSync != lastRoundProcessed", + discovery.shouldTriggerFailover()); + } + + // Test Case 4: in-progress files not empty - should return false + { + when(tracker.getInProgressFiles()) + .thenReturn(Collections.singletonList(new Path("test.plog"))); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(testRound); + discovery.setFailoverPending(true); + + assertFalse("Should not trigger failover when in-progress files are not empty", + discovery.shouldTriggerFailover()); + } + + // Test Case 5: failoverPending false AND lastRoundInSync != lastRoundProcessed - should + // return false + { + when(tracker.getInProgressFiles()).thenReturn(Collections.emptyList()); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(differentRound); + discovery.setFailoverPending(false); + + assertFalse("Should not trigger failover when failoverPending is false and rounds differ", + discovery.shouldTriggerFailover()); + } + + // Test Case 6: failoverPending false AND in-progress files not empty - should return false + { + when(tracker.getInProgressFiles()) + .thenReturn(Collections.singletonList(new Path("test.plog"))); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(testRound); + discovery.setFailoverPending(false); + + assertFalse("Should not trigger failover when failoverPending is false and files exist", + discovery.shouldTriggerFailover()); + } + + // Test Case 7: lastRoundInSync != lastRoundProcessed AND in-progress files not empty - should + // return false + { + when(tracker.getInProgressFiles()) + .thenReturn(Collections.singletonList(new Path("test.plog"))); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(differentRound); + discovery.setFailoverPending(true); + + assertFalse("Should not trigger failover when rounds differ and files exist", + discovery.shouldTriggerFailover()); + } + + // Test Case 8: All conditions false - should return false + { + when(tracker.getInProgressFiles()) + .thenReturn(Collections.singletonList(new Path("test.plog"))); + TestableReplicationLogDiscoveryReplay discovery = + new TestableReplicationLogDiscoveryReplay(tracker, haGroupStoreRecord); + discovery.setLastRoundInSync(testRound); + discovery.setLastRoundProcessed(differentRound); + discovery.setFailoverPending(false); + + assertFalse("Should not trigger failover when all conditions are false", + discovery.shouldTriggerFailover()); + } + + } finally { + EnvironmentEdgeManager.reset(); + } + } + + /** + * Testable implementation of ReplicationLogDiscoveryReplay for unit testing. Provides dependency + * injection for HAGroupStoreRecord, tracks processed rounds, and supports simulating state + * changes during replay. + */ + private static class TestableReplicationLogDiscoveryReplay extends ReplicationLogDiscoveryReplay { + private final HAGroupStoreRecord haGroupStoreRecord; + private int roundsProcessed = 0; + private int stateChangeAfterRounds = -1; + private ReplicationReplayState newStateAfterRounds = null; + private int failoverPendingChangeAfterRounds = -1; + private boolean failoverPendingValueAfterRounds = false; + private final List processedRounds = new java.util.ArrayList<>(); + + public TestableReplicationLogDiscoveryReplay( + ReplicationLogTracker replicationLogReplayFileTracker, + HAGroupStoreRecord haGroupStoreRecord) { + super(replicationLogReplayFileTracker); + this.haGroupStoreRecord = haGroupStoreRecord; + } + + @Override + protected HAGroupStoreRecord getHAGroupRecord() { + return haGroupStoreRecord; + } + + @Override + protected void processRound(ReplicationRound replicationRound) throws IOException { + LOG.info("Processing Round: {}", replicationRound); + // Track processed rounds + processedRounds.add(replicationRound); + + // Simulate state change by listener after certain number of rounds + roundsProcessed++; + if ( + stateChangeAfterRounds > 0 && roundsProcessed == stateChangeAfterRounds + && newStateAfterRounds != null + ) { + LOG.info("Rounds Processed: {}, newStateAfterRounds: {}", roundsProcessed, + newStateAfterRounds); + setReplicationReplayState(newStateAfterRounds); + } + + // Simulate failover pending change by listener after certain number of rounds + if ( + failoverPendingChangeAfterRounds > 0 && roundsProcessed == failoverPendingChangeAfterRounds + ) { + LOG.info("Rounds Processed: {}, setting failoverPending to: {}", roundsProcessed, + failoverPendingValueAfterRounds); + setFailoverPending(failoverPendingValueAfterRounds); + } + // Don't actually process files in tests + } + + public void setStateChangeAfterRounds(int afterRounds, ReplicationReplayState newState) { + this.stateChangeAfterRounds = afterRounds; + this.newStateAfterRounds = newState; + } + + public void setFailoverPendingAfterRounds(int afterRounds, boolean failoverPendingValue) { + this.failoverPendingChangeAfterRounds = afterRounds; + this.failoverPendingValueAfterRounds = failoverPendingValue; + } + + private int triggerFailoverCallCount = 0; + + @Override + protected void triggerFailover() { + // Track calls to triggerFailover for validation + triggerFailoverCallCount++; + + // Simulate the real behavior: set failoverPending to false after triggering failover + setFailoverPending(false); + } + + public int getTriggerFailoverCallCount() { + return triggerFailoverCallCount; + } + + public int getProcessRoundCallCount() { + return processedRounds.size(); + } + + public List getProcessedRounds() { + return new java.util.ArrayList<>(processedRounds); + } + } +} diff --git a/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/ReplicationLogReplayTestIT.java b/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/ReplicationLogReplayTestIT.java new file mode 100644 index 00000000000..415ab9edd20 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/replication/reader/ReplicationLogReplayTestIT.java @@ -0,0 +1,208 @@ +/* + * 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.phoenix.replication.reader; + +import static org.apache.phoenix.jdbc.HAGroupStoreClient.ZK_CONSISTENT_HA_GROUP_RECORD_NAMESPACE; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.toPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.net.URI; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.jdbc.ClusterRoleRecord; +import org.apache.phoenix.jdbc.HAGroupStoreClient; +import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility; +import org.apache.phoenix.jdbc.PhoenixHAAdmin; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.util.HAGroupStoreTestUtil; +import org.apache.phoenix.util.ReadOnlyProps; +import org.junit.*; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; + +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; + +@Category(NeedsOwnMiniClusterTest.class) +public class ReplicationLogReplayTestIT extends BaseTest { + + @ClassRule + public static TemporaryFolder testFolder = new TemporaryFolder(); + + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + private String zkUrl; + private String peerZkUrl;; + private FileSystem localFs; + private URI rootURI; + private PhoenixHAAdmin haAdmin; + private PhoenixHAAdmin peerHaAdmin; + + @Rule + public TestName testName = new TestName(); + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(1); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + CLUSTERS.start(); + } + + @Before + public void setUp() throws Exception { + zkUrl = getLocalZkUrl(config); + peerZkUrl = CLUSTERS.getZkUrl2(); + localFs = FileSystem.getLocal(config); + standbyUri = testFolder.getRoot().toURI(); + rootURI = new URI(standbyUri.toString()); + zkUrl = getLocalZkUrl(config); + peerZkUrl = CLUSTERS.getZkUrl2(); + peerHaAdmin = new PhoenixHAAdmin(peerZkUrl, config, ZK_CONSISTENT_HA_GROUP_RECORD_NAMESPACE); + cleanupHAGroupState(); + + // Set the required configuration for ReplicationLogReplay + config.set(ReplicationLogReplay.REPLICATION_LOG_REPLAY_HDFS_URL_KEY, standbyUri.toString()); + } + + @After + public void tearDown() throws IOException { + localFs.delete(new Path(testFolder.getRoot().toURI()), true); + } + + @Test + public void testInit() throws IOException, SQLException { + final String haGroupName = "testGroup"; + + // Create TestableReplicationReplay instance + ReplicationLogReplay replicationLogReplay = new ReplicationLogReplay(config, haGroupName); + + // Insert a HAGroupStoreRecord into the system table + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, zkUrl, peerZkUrl, + CLUSTERS.getMasterAddress1(), CLUSTERS.getMasterAddress2(), + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + // Call init method + replicationLogReplay.init(); + + // 1. Ensure filesystem and rootURI are initialized correctly + assertNotNull("FileSystem should be initialized", replicationLogReplay.getFileSystem()); + assertNotNull("RootURI should be initialized", replicationLogReplay.getRootURI()); + assertEquals("RootURI should match the configured URI", rootURI, + replicationLogReplay.getRootURI()); + + // 2. Ensure expected haGroupFilesPath is created + Path expectedHaGroupFilesPath = new Path(rootURI.getPath(), haGroupName); + assertTrue("HA group files path should be created", + replicationLogReplay.getFileSystem().exists(expectedHaGroupFilesPath)); + + // 3. Ensure replicationReplayLogDiscovery is initialized correctly + assertNotNull("ReplicationLogDiscoveryReplay should be initialized", + replicationLogReplay.getReplicationReplayLogDiscovery()); + } + + @Test + public void testReplicationReplayInstanceCaching() throws SQLException { + final String haGroupName1 = "testHAGroup_1"; + final String haGroupName2 = "testHAGroup_2"; + + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName1, zkUrl, peerZkUrl, + CLUSTERS.getMasterAddress1(), CLUSTERS.getMasterAddress2(), + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName2, zkUrl, peerZkUrl, + CLUSTERS.getMasterAddress1(), CLUSTERS.getMasterAddress2(), + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + // Get instances for the first HA group + ReplicationLogReplay group1Instance1 = ReplicationLogReplay.get(config, haGroupName1); + ReplicationLogReplay group1Instance2 = ReplicationLogReplay.get(config, haGroupName1); + + // Verify same instance is returned for same haGroupName + assertNotNull("ReplicationLogReplay should not be null", group1Instance1); + assertNotNull("ReplicationLogReplay should not be null", group1Instance2); + assertSame("Same instance should be returned for same haGroup", group1Instance1, + group1Instance2); + + // Get instance for a different HA group + ReplicationLogReplay group2Instance1 = ReplicationLogReplay.get(config, haGroupName2); + assertNotNull("ReplicationLogReplay should not be null", group2Instance1); + assertNotSame("Different instance should be returned for different haGroup", group2Instance1, + group1Instance1); + + // Verify multiple calls still return cached instances + ReplicationLogReplay group1Instance3 = ReplicationLogReplay.get(config, haGroupName1); + ReplicationLogReplay group2Instance2 = ReplicationLogReplay.get(config, haGroupName2); + assertSame("Cached instance should be returned", group1Instance3, group1Instance1); + assertSame("Cached instance should be returned", group2Instance2, group2Instance1); + } + + @Test + public void testReplicationReplayCacheRemovalOnClose() throws SQLException { + final String haGroupName = "testHAGroup"; + + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, zkUrl, peerZkUrl, + CLUSTERS.getMasterAddress1(), CLUSTERS.getMasterAddress2(), + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + // Get initial instance + ReplicationLogReplay group1Instance1 = ReplicationLogReplay.get(config, haGroupName); + assertNotNull("ReplicationLogReplay should not be null", group1Instance1); + + // Verify cached instance is returned + ReplicationLogReplay group1Instance2 = ReplicationLogReplay.get(config, haGroupName); + assertSame("Same instance should be returned before close", group1Instance2, group1Instance1); + + // Close the replay instance + group1Instance1.close(); + + // Get instance after close - should be a new instance + ReplicationLogReplay group1Instance3 = ReplicationLogReplay.get(config, haGroupName); + assertNotNull("ReplicationLogReplay should not be null after close", group1Instance3); + assertNotSame("New instance should be created after close", group1Instance1, group1Instance3); + assertEquals("HA Group ID should match", haGroupName, group1Instance3.getHaGroupName()); + + // Clean up + group1Instance3.close(); + } + + private void cleanupHAGroupState() throws SQLException { + // Clean up existing HAGroupStoreRecords + try { + List haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + for (String haGroupName : haGroupNames) { + haAdmin.getCurator().delete().quietly().forPath(toPath(haGroupName)); + peerHaAdmin.getCurator().delete().quietly().forPath(toPath(haGroupName)); + } + + } catch (Exception e) { + // Ignore cleanup errors + } + // Remove any existing entries in the system table + HAGroupStoreTestUtil.deleteAllHAGroupRecordsInSystemTable(zkUrl); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryTest.java new file mode 100644 index 00000000000..75fd7ef2e42 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryTest.java @@ -0,0 +1,2185 @@ +/* + * 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.phoenix.replication; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.*; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.util.EnvironmentEdge; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogDiscovery; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogDiscoveryReplayImpl; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTracker; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTrackerReplayImpl; +import org.apache.phoenix.replication.reader.ReplicationLogReplay; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions; + +public class ReplicationLogDiscoveryTest { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogDiscoveryTest.class); + + @ClassRule + public static TemporaryFolder testFolder = new TemporaryFolder(); + + private TestableReplicationLogDiscovery discovery; + private TestableReplicationLogTracker fileTracker; + private Configuration conf; + private FileSystem localFs; + private Path testFolderPath; + private static final String haGroupName = "testGroup"; + private static final MetricsReplicationLogTracker metricsLogTracker = + new MetricsReplicationLogTrackerReplayImpl(haGroupName); + private static final MetricsReplicationLogDiscovery metricsLogDiscovery = + new MetricsReplicationLogDiscoveryReplayImpl(haGroupName); + + @Before + public void setUp() throws IOException { + conf = HBaseConfiguration.create(); + localFs = FileSystem.getLocal(conf); + URI rootURI = new Path(testFolder.getRoot().toString()).toUri(); + testFolderPath = new Path(testFolder.getRoot().getAbsolutePath()); + Path newFilesDirectory = + new Path(new Path(rootURI.getPath(), haGroupName), ReplicationLogReplay.IN_DIRECTORY_NAME); + ReplicationShardDirectoryManager replicationShardDirectoryManager = + new ReplicationShardDirectoryManager(conf, newFilesDirectory); + fileTracker = Mockito.spy(new TestableReplicationLogTracker(conf, haGroupName, localFs, + replicationShardDirectoryManager)); + fileTracker.init(); + + discovery = Mockito.spy(new TestableReplicationLogDiscovery(fileTracker)); + Mockito.doReturn(metricsLogDiscovery).when(discovery).getMetrics(); + } + + @After + public void tearDown() throws IOException { + if (discovery != null) { + discovery.stop(); + discovery.close(); + } + if (fileTracker != null) { + fileTracker.close(); + } + localFs.delete(new Path(testFolder.getRoot().toURI()), true); + } + + /** + * Tests the start and stop lifecycle of ReplicationLogDiscovery. Validates scheduler + * initialization, thread naming, and proper cleanup. + */ + @Test + public void testStartAndStop() throws IOException { + // 1. Validate that it's not running initially + assertFalse("Discovery should not be running initially", discovery.isRunning()); + + // 2. Validate that scheduler is set to null initially + assertNull("Scheduler should be null initially", discovery.getScheduler()); + + // 3. Call the start method + discovery.start(); + + // 4. Ensure isRunning is set to true + assertTrue("Discovery should be running after start", discovery.isRunning()); + + // 5. Ensure scheduler is started with correct parameters + assertNotNull("Scheduler should not be null after start", discovery.getScheduler()); + assertFalse("Scheduler should not be shutdown after start", + discovery.getScheduler().isShutdown()); + + // Verify thread name format + String threadName = discovery.getExecutorThreadNameFormat(); + assertTrue("Thread name should contain ReplicationLogDiscovery", + threadName.contains("ReplicationLogDiscovery")); + + // Verify replay interval + long replayInterval = discovery.getReplayIntervalSeconds(); + assertEquals("Replay interval should be 10 seconds", 10L, replayInterval); + + // 6. Ensure starting again does not create a new scheduler (and also should not throw any + // exception) + ScheduledExecutorService originalScheduler = discovery.getScheduler(); + discovery.start(); // Should not create new scheduler + ScheduledExecutorService sameScheduler = discovery.getScheduler(); + assertSame("Should reuse the same scheduler instance", originalScheduler, sameScheduler); + assertTrue("Discovery should still be running", discovery.isRunning()); + + // 7. Call stop + discovery.stop(); + + // 8. Ensure scheduler is stopped + assertTrue("Scheduler should be shutdown after stop", discovery.getScheduler().isShutdown()); + + // 9. Ensure isRunning is false + assertFalse("Discovery should not be running after stop", discovery.isRunning()); + } + + /** + * Tests processRound with in-progress directory processing enabled. Validates that both new files + * and in-progress files are processed correctly. + */ + @Test + public void testProcessRoundWithInProgressDirectoryProcessing() throws IOException { + // 1. Create new files with start of the day round (00:00:00) + ReplicationRound replicationRound = new ReplicationRound(1704153600000L, 1704153660000L); // 00:00:00 + // - + // 00:01:00 + List newFilesForRound = createNewFilesForRound(replicationRound, 3); + + // 2. Create file for shard count min round (which should also go to same shard) + int shardCount = fileTracker.getReplicationShardDirectoryManager().getAllShardPaths().size(); + ReplicationRound differentRoundSameShard = new ReplicationRound( + 1704153600000L + (shardCount * 60 * 1000L), 1704153660000L + (shardCount * 60 * 1000L)); + List differentRoundSameShardFiles = createNewFilesForRound(differentRoundSameShard, 2); + + // 3. Create files for (00:01:00) and (00:02:00) start time of the rounds + ReplicationRound round0100 = new ReplicationRound(1704153660000L, 1704153720000L); // 00:01:00 - + // 00:02:00 + List round0100NewFiles = createNewFilesForRound(round0100, 2); + ReplicationRound round0200 = new ReplicationRound(1704153720000L, 1704153780000L); // 00:02:00 - + // 00:03:00 + List round0200NewFiles = createNewFilesForRound(round0200, 2); + + // 4. Create 2 in progress files for (00:00:04) timestamp + long timestamp0004 = 1704153600000L + (4 * 1000L); // 00:00:04 + List inProgressFiles0004 = createInProgressFiles(timestamp0004, 2); + + // 5. Create 2 in progress files for (00:01:02) timestamp + long timestamp0102 = 1704153660000L + (2 * 1000L); // 00:01:02 + List inProgressFiles0102 = createInProgressFiles(timestamp0102, 2); + + // 6. Mock shouldProcessInProgressDirectory to return true + discovery.setMockShouldProcessInProgressDirectory(true); + + // Process the start of day round + discovery.processRound(replicationRound); + + // 7. Ensure current round new files (3) are processed and in progress (4) are processed (Total + // 7) + List processedFiles = discovery.getProcessedFiles(); + assertEquals("Invalid number of files processed", 7, processedFiles.size()); + + // Create set of expected files that should be processed (by prefix, since UUIDs are added + // during markInProgress) + Set expectedProcessedFilePrefixes = new HashSet<>(); + for (Path file : newFilesForRound) { + // Extract prefix before the file extension (for new files before markInProgress) + // After markInProgress, files will have format: {timestamp}_{rs-n}_{UUID}.plog + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf(".")); + expectedProcessedFilePrefixes.add(prefix); + } + for (Path file : inProgressFiles0004) { + // For in-progress files, they already have format: {timestamp}_{rs-n}_{UUID}.plog + // Extract prefix before the UUID (everything before the last underscore) + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf("_")); + expectedProcessedFilePrefixes.add(prefix); + } + for (Path file : inProgressFiles0102) { + // For in-progress files, they already have format: {timestamp}_{rs-n}_{UUID}.plog + // Extract prefix before the UUID (everything before the last underscore) + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf("_")); + expectedProcessedFilePrefixes.add(prefix); + } + + // Create set of actually processed file paths (extract prefixes) + // Files after markInProgress will have format: {timestamp}_{rs-n}_{UUID}.plog + Set actualProcessedFilePrefixes = new HashSet<>(); + for (Path file : processedFiles) { + String fileName = file.getName(); + // Extract prefix before UUID and extension (everything before the last underscore before + // .plog) + // Remove the extension first + String withoutExtension = fileName.substring(0, fileName.lastIndexOf(".")); + // Then get everything before the last underscore (which is the UUID) + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + actualProcessedFilePrefixes.add(prefix); + } + + // Validate that sets are equal + assertEquals("Expected and actual processed files should match", expectedProcessedFilePrefixes, + actualProcessedFilePrefixes); + + // Verify that markInProgress was called 7 times (3 new files + 4 in-progress files) + // markInProgress is called before each file is processed + Mockito.verify(fileTracker, Mockito.times(7)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + // For new files + for (Path expectedFile : newFilesForRound) { + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress( + Mockito.argThat(path -> path.getName().startsWith(expectedFile.getName().split("\\.")[0]))); + } + // For in-progress files + for (Path expectedFile : inProgressFiles0004) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress(Mockito.argThat(path -> path + .getName().substring(0, path.getName().lastIndexOf("_")).equals(expectedPrefix))); + } + for (Path expectedFile : inProgressFiles0102) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress(Mockito.argThat(path -> path + .getName().substring(0, path.getName().lastIndexOf("_")).equals(expectedPrefix))); + } + + // Verify that markCompleted was called 7 times (once for each successfully processed file) + Mockito.verify(fileTracker, Mockito.times(7)).markCompleted(Mockito.any(Path.class)); + + // Verify that markCompleted was called for each expected file with correct paths + // For new files (they will have format: {timestamp}_{rs-n}_{UUID}.plog in in-progress dir) + for (Path expectedFile : newFilesForRound) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + // For in-progress files (they will have updated UUIDs, but same prefix) + for (Path expectedFile : inProgressFiles0004) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + for (Path expectedFile : inProgressFiles0102) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that shouldProcessInProgressDirectory was called once + Mockito.verify(discovery, Mockito.times(1)).shouldProcessInProgressDirectory(); + + // Validate that files from other rounds were NOT processed (using prefix comparison) + for (Path unexpectedFile : differentRoundSameShardFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed shard count round file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : round0100NewFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed round 00:01:00 file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : round0200NewFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed round 00:02:00 file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + } + + /** + * Tests processRound with in-progress directory processing disabled. Validates that only new + * files for the current round are processed. + */ + @Test + public void testProcessRoundWithoutInProgressDirectoryProcessing() throws IOException { + // 1. Create new files with start of the day round (00:00:00) + ReplicationRound replicationRound = new ReplicationRound(1704153600000L, 1704153660000L); // 00:00:00 + // - + // 00:01:00 + List newFilesForRound = createNewFilesForRound(replicationRound, 3); + + // 2. Create file for shard count min round (which should also go to same shard) + int shardCount = fileTracker.getReplicationShardDirectoryManager().getAllShardPaths().size(); + ReplicationRound differentRoundSameShard = new ReplicationRound( + 1704153600000L + (shardCount * 60 * 1000L), 1704153660000L + (shardCount * 60 * 1000L)); + List differentRoundSameShardFiles = createNewFilesForRound(differentRoundSameShard, 2); + + // 3. Create files for (00:01:00) and (00:02:00) start time of the rounds + ReplicationRound round0100 = new ReplicationRound(1704153660000L, 1704153720000L); // 00:01:00 - + // 00:02:00 + List round0100NewFiles = createNewFilesForRound(round0100, 2); + ReplicationRound round0200 = new ReplicationRound(1704153720000L, 1704153780000L); // 00:02:00 - + // 00:03:00 + List round0200NewFiles = createNewFilesForRound(round0200, 2); + + // 4. Create 2 in progress files for (00:00:04) timestamp + long timestamp0004 = 1704153600000L + (4 * 1000L); // 00:00:04 + List inProgressFiles0004 = createInProgressFiles(timestamp0004, 2); + + // 5. Create 2 in progress files for (00:01:02) timestamp + long timestamp0102 = 1704153660000L + (2 * 1000L); // 00:01:02 + List inProgressFiles0102 = createInProgressFiles(timestamp0102, 2); + + // 6. Mock shouldProcessInProgressDirectory to return false + discovery.setMockShouldProcessInProgressDirectory(false); + + // Process the start of day round + discovery.processRound(replicationRound); + + // 7. Ensure only current round new files (3) are processed (Total 3, no in-progress files) + List processedFiles = discovery.getProcessedFiles(); + assertEquals("Invalid number of files processed", 3, processedFiles.size()); + + // Create set of expected files that should be processed (by prefix, since UUIDs are added + // during markInProgress) + Set expectedProcessedFilePrefixes = new HashSet<>(); + for (Path file : newFilesForRound) { + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf(".")); + expectedProcessedFilePrefixes.add(prefix); + } + + // Create set of actually processed file paths (extract prefixes) + Set actualProcessedFilePrefixes = new HashSet<>(); + for (Path file : processedFiles) { + String fileName = file.getName(); + String withoutExtension = fileName.substring(0, fileName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + actualProcessedFilePrefixes.add(prefix); + } + + // Validate that sets are equal + assertEquals("Expected and actual processed files should match", expectedProcessedFilePrefixes, + actualProcessedFilePrefixes); + + // Verify that markInProgress was called 3 times (only new files, no in-progress files) + Mockito.verify(fileTracker, Mockito.times(3)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + for (Path expectedFile : newFilesForRound) { + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress( + Mockito.argThat(path -> path.getName().startsWith(expectedFile.getName().split("\\.")[0]))); + } + + // Verify that markCompleted was called 3 times + Mockito.verify(fileTracker, Mockito.times(3)).markCompleted(Mockito.any(Path.class)); + + // Verify that markCompleted was called for each expected file with correct paths + for (Path expectedFile : newFilesForRound) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that shouldProcessInProgressDirectory was called once + Mockito.verify(discovery, Mockito.times(1)).shouldProcessInProgressDirectory(); + + // Validate that files from other rounds were NOT processed (using prefix comparison) + for (Path unexpectedFile : differentRoundSameShardFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed shard count round file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : round0100NewFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed round 00:01:00 file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : round0200NewFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed round 00:02:00 file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + // Validate that in-progress files were NOT processed + for (Path unexpectedFile : inProgressFiles0004) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf("_")); + assertFalse( + "Should NOT have processed in-progress file from 00:00:04: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : inProgressFiles0102) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf("_")); + assertFalse( + "Should NOT have processed in-progress file from 00:01:02: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + } + + /** + * Tests the probability-based in-progress directory processing decision. Validates that the + * actual probability matches the configured probability. + */ + @Test + public void testShouldProcessInProgressDirectory() { + // Test multiple times to verify probability-based behavior + int totalTests = 100000; + int trueCount = 0; + + for (int i = 0; i < totalTests; i++) { + if (discovery.shouldProcessInProgressDirectory()) { + trueCount++; + } + } + + // Calculate the actual probability + double actualProbability = (double) trueCount / totalTests * 100.0; + double expectedProbability = discovery.getInProgressDirectoryProcessProbability(); + + // Verify that the actual probability is close to the expected probability + // Allow for some variance due to randomness (within 2% of expected) + double variance = Math.abs(actualProbability - expectedProbability); + assertTrue( + "Actual probability (" + actualProbability + "%) should be close to expected probability (" + + expectedProbability + "%), variance: " + variance + "%", + variance < 2.0); + + // Verify that we have some true results (probability > 0) + assertTrue("Should have some true results", trueCount > 0); + + // Verify that we don't have too many true results (probability < 100%) + assertTrue("Should not have too many true results", trueCount < totalTests); + + LOG.info("ShouldProcessInProgressDirectory test results:"); + LOG.info("Total tests: " + totalTests); + LOG.info("True count: " + trueCount); + LOG.info("Actual probability: " + actualProbability + "%"); + LOG.info("Expected probability: " + expectedProbability + "%"); + LOG.info("Variance: " + variance + "%"); + } + + /** + * Tests processing of new files for a specific round. Validates that only files belonging to the + * current round are processed. + */ + @Test + public void testProcessNewFilesForRound() throws IOException { + // 1. Create new files with start of the day round (00:00:00) + ReplicationRound replicationRound = new ReplicationRound(1704153600000L, 1704153660000L); // 00:00:00 + // - + // 00:01:00 + List newFilesForRound = createNewFilesForRound(replicationRound, 3); + + // 2. Create file for shard count min round (which should also go to same shard) + int shardCount = fileTracker.getReplicationShardDirectoryManager().getAllShardPaths().size(); + ReplicationRound differentRoundSameShard = new ReplicationRound( + 1704153600000L + (shardCount * 60 * 1000L), 1704153660000L + (shardCount * 60 * 1000L)); + List differentRoundSameShardFiles = createNewFilesForRound(differentRoundSameShard, 2); + + // 3. Create files for (00:01:00) and (00:02:00) start time of the rounds + ReplicationRound round0100 = new ReplicationRound(1704153660000L, 1704153720000L); // 00:01:00 - + // 00:02:00 + List round0100NewFiles = createNewFilesForRound(round0100, 2); + ReplicationRound round0200 = new ReplicationRound(1704153720000L, 1704153780000L); // 00:02:00 - + // 00:03:00 + List round0200NewFiles = createNewFilesForRound(round0200, 2); + + // 4. Create 2 in progress files for (00:00:04) timestamp + long timestamp0004 = 1704153600000L + (4 * 1000L); // 00:00:04 + List inProgressFiles0004 = createInProgressFiles(timestamp0004, 2); + + // 5. Create 2 in progress files for (00:01:02) timestamp + long timestamp0102 = 1704153660000L + (2 * 1000L); // 00:01:02 + List inProgressFiles0102 = createInProgressFiles(timestamp0102, 2); + + // Process new files for the round + discovery.processNewFilesForRound(replicationRound); + + // 7. Ensure only current round new files (3) are processed + List processedFiles = discovery.getProcessedFiles(); + assertEquals("Invalid number of files processed", 3, processedFiles.size()); + + // Create set of expected files that should be processed (by prefix, since UUIDs are added + // during markInProgress) + Set expectedProcessedFilePrefixes = new HashSet<>(); + for (Path file : newFilesForRound) { + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf(".")); + expectedProcessedFilePrefixes.add(prefix); + } + + // Create set of actually processed file paths (extract prefixes) + Set actualProcessedFilePrefixes = new HashSet<>(); + for (Path file : processedFiles) { + String fileName = file.getName(); + String withoutExtension = fileName.substring(0, fileName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + actualProcessedFilePrefixes.add(prefix); + } + + // Validate that sets are equal + assertEquals("Expected and actual processed files should match", expectedProcessedFilePrefixes, + actualProcessedFilePrefixes); + + // Verify that markInProgress was called 3 times + Mockito.verify(fileTracker, Mockito.times(3)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + for (Path expectedFile : newFilesForRound) { + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress( + Mockito.argThat(path -> path.getName().startsWith(expectedFile.getName().split("\\.")[0]))); + } + + // Verify that markCompleted was called 3 times + Mockito.verify(fileTracker, Mockito.times(3)).markCompleted(Mockito.any(Path.class)); + + // Verify that markCompleted was called for each processed file with correct paths + for (Path expectedFile : newFilesForRound) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Validate that files from other rounds were NOT processed (using prefix comparison) + for (Path unexpectedFile : differentRoundSameShardFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed shard count round file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : round0100NewFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed round 00:01:00 file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : round0200NewFiles) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed round 00:02:00 file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + // Validate that in-progress files were NOT processed (processNewFilesForRound only processes + // new files) + for (Path unexpectedFile : inProgressFiles0004) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf("_")); + assertFalse( + "Should NOT have processed in-progress file from 00:00:04: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + for (Path unexpectedFile : inProgressFiles0102) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf("_")); + assertFalse( + "Should NOT have processed in-progress file from 00:01:02: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + } + + /** + * Tests partial failure handling during new file processing. Validates that successful files are + * marked completed while failed files are marked failed. + */ + @Test + public void testProcessNewFilesForRoundWithPartialFailure() throws IOException { + // Create new files with start of the day round (00:00:00) + ReplicationRound replicationRound = new ReplicationRound(1704153600000L, 1704153660000L); // 00:00:00 + // - + // 00:01:00 + List newFilesForRound = createNewFilesForRound(replicationRound, 5); + + // Mock processFile to throw exception for specific files (files 1 and 3) - using prefix + // matching + String file1Prefix = newFilesForRound.get(1).getName().substring(0, + newFilesForRound.get(1).getName().lastIndexOf(".")); + Mockito.doThrow(new IOException("Processing failed for file 1")).when(discovery) + .processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(file1Prefix); + })); + String file3Prefix = newFilesForRound.get(3).getName().substring(0, + newFilesForRound.get(3).getName().lastIndexOf(".")); + Mockito.doThrow(new IOException("Processing failed for file 3")).when(discovery) + .processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(file3Prefix); + })); + + // Process new files for the round + discovery.processNewFilesForRound(replicationRound); + + // Verify that markInProgress was called 5 times (for all files) + Mockito.verify(fileTracker, Mockito.times(5)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + for (Path expectedFile : newFilesForRound) { + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress( + Mockito.argThat(path -> path.getName().startsWith(expectedFile.getName().split("\\.")[0]))); + } + + // Verify that processFile was called for each file in the round + Mockito.verify(discovery, Mockito.times(5)).processFile(Mockito.any(Path.class)); + + // Verify that processFile was called for each specific file (using prefix matching) + for (Path expectedFile : newFilesForRound) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf(".")); + Mockito.verify(discovery, Mockito.times(1)).processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that markCompleted was called for each successfully processed file + Mockito.verify(fileTracker, Mockito.times(3)).markCompleted(Mockito.any(Path.class)); + + // Verify that markCompleted was called for each successfully processed file with correct paths + String expectedPrefix0 = newFilesForRound.get(0).getName().substring(0, + newFilesForRound.get(0).getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix0); + })); + String expectedPrefix2 = newFilesForRound.get(2).getName().substring(0, + newFilesForRound.get(2).getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix2); + })); + String expectedPrefix4 = newFilesForRound.get(4).getName().substring(0, + newFilesForRound.get(4).getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix4); + })); + + // Verify that markCompleted was NOT called for failed files + String unexpectedPrefix1 = newFilesForRound.get(1).getName().substring(0, + newFilesForRound.get(1).getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.never()).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(unexpectedPrefix1); + })); + String unexpectedPrefix3 = newFilesForRound.get(3).getName().substring(0, + newFilesForRound.get(3).getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.never()).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(unexpectedPrefix3); + })); + + // Verify that markFailed was called for failed files + Mockito.verify(fileTracker, Mockito.times(1)).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(unexpectedPrefix1); + })); + Mockito.verify(fileTracker, Mockito.times(1)).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(unexpectedPrefix3); + })); + + // Verify that markFailed was NOT called for successfully processed files + Mockito.verify(fileTracker, Mockito.never()).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix0); + })); + Mockito.verify(fileTracker, Mockito.never()).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix2); + })); + Mockito.verify(fileTracker, Mockito.never()).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix4); + })); + } + + /** + * Tests complete failure handling during new file processing. Validates that all failed files are + * marked as failed with no completed files. + */ + @Test + public void testProcessNewFilesForRoundWithAllFailures() throws IOException { + // Create new files with start of the day round (00:00:00) + ReplicationRound replicationRound = new ReplicationRound(1704153600000L, 1704153660000L); // 00:00:00 + // - + // 00:01:00 + List newFilesForRound = createNewFilesForRound(replicationRound, 5); + + // Mock processFile to throw exception for all files (using prefix matching since files are + // moved to in-progress) + for (Path file : newFilesForRound) { + String filePrefix = file.getName().substring(0, file.getName().lastIndexOf(".")); + Mockito.doThrow(new IOException("Processing failed for file: " + file.getName())) + .when(discovery).processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(filePrefix); + })); + } + + // Process new files for the round + discovery.processNewFilesForRound(replicationRound); + + // Verify that processFile was called for each file in the round + Mockito.verify(discovery, Mockito.times(5)).processFile(Mockito.any(Path.class)); + + // Verify that markInProgress was called 5 times (before processing fails) + Mockito.verify(fileTracker, Mockito.times(5)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + for (Path expectedFile : newFilesForRound) { + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress( + Mockito.argThat(path -> path.getName().startsWith(expectedFile.getName().split("\\.")[0]))); + } + + // Verify that processFile was called for each specific file (using prefix matching) + for (Path expectedFile : newFilesForRound) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf(".")); + Mockito.verify(discovery, Mockito.times(1)).processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that markCompleted was NOT called for any file (all failed) + Mockito.verify(fileTracker, Mockito.never()).markCompleted(Mockito.any(Path.class)); + + // Verify that markFailed was called for all files + Mockito.verify(fileTracker, Mockito.times(5)).markFailed(Mockito.any(Path.class)); + + // Verify that markFailed was called for each specific file with correct paths + for (Path failedFile : newFilesForRound) { + String expectedPrefix = + failedFile.getName().substring(0, failedFile.getName().lastIndexOf(".")); + Mockito.verify(fileTracker, Mockito.times(1)).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + } + + /** + * Tests processing of all files in the in-progress directory. Validates that only in-progress + * files are processed, not new files. + */ + @Test + public void testProcessInProgressDirectory() throws IOException { + // 1. Create in-progress files for different timestamps + long timestamp0004 = 1704153600000L + (4 * 1000L); // 00:00:04 + List inProgressFiles0004 = createInProgressFiles(timestamp0004, 3); + + long timestamp0102 = 1704153660000L + (2 * 1000L); // 00:01:02 + List inProgressFiles0102 = createInProgressFiles(timestamp0102, 2); + + long timestamp0206 = 1704153720000L + (6 * 1000L); // 00:02:06 + List inProgressFiles0206 = createInProgressFiles(timestamp0206, 2); + + // 2. Create some new files to ensure they are NOT processed + ReplicationRound replicationRound = new ReplicationRound(1704153600000L, 1704153660000L); // 00:00:00 + // - + // 00:01:00 + List newFilesForRound = createNewFilesForRound(replicationRound, 3); + + // Process in-progress directory + discovery.processInProgressDirectory(); + + // 3. Ensure all in-progress files (7 total) are processed + List processedFiles = discovery.getProcessedFiles(); + assertEquals("Invalid number of files processed", 7, processedFiles.size()); + + // Create set of expected files that should be processed (by prefix, since UUIDs are updated + // during markInProgress) + Set expectedProcessedFilePrefixes = new HashSet<>(); + for (Path file : inProgressFiles0004) { + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf("_")); + expectedProcessedFilePrefixes.add(prefix); + } + for (Path file : inProgressFiles0102) { + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf("_")); + expectedProcessedFilePrefixes.add(prefix); + } + for (Path file : inProgressFiles0206) { + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf("_")); + expectedProcessedFilePrefixes.add(prefix); + } + + // Create set of actually processed file paths (extract prefixes) + Set actualProcessedFilePrefixes = new HashSet<>(); + for (Path file : processedFiles) { + String fileName = file.getName(); + String withoutExtension = fileName.substring(0, fileName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + actualProcessedFilePrefixes.add(prefix); + } + + // Validate that sets are equal + assertEquals("Expected and actual processed files should match", expectedProcessedFilePrefixes, + actualProcessedFilePrefixes); + + // Verify that markInProgress was called 7 times + Mockito.verify(fileTracker, Mockito.times(7)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + for (Path expectedFile : inProgressFiles0004) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress(Mockito.argThat(path -> { + String pathName = path.getName(); + String prefix = pathName.substring(0, pathName.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + for (Path expectedFile : inProgressFiles0102) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress(Mockito.argThat(path -> { + String pathName = path.getName(); + String prefix = pathName.substring(0, pathName.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + for (Path expectedFile : inProgressFiles0206) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress(Mockito.argThat(path -> { + String pathName = path.getName(); + String prefix = pathName.substring(0, pathName.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that markCompleted was called for each processed file + Mockito.verify(fileTracker, Mockito.times(7)).markCompleted(Mockito.any(Path.class)); + + // Verify that markCompleted was called for each processed file with correct paths + for (Path expectedFile : inProgressFiles0004) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + for (Path expectedFile : inProgressFiles0102) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + for (Path expectedFile : inProgressFiles0206) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Validate that new files were NOT processed (processInProgressDirectory only processes + // in-progress files) + for (Path unexpectedFile : newFilesForRound) { + String unexpectedPrefix = + unexpectedFile.getName().substring(0, unexpectedFile.getName().lastIndexOf(".")); + assertFalse("Should NOT have processed new file: " + unexpectedFile.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + } + + @Test + public void testProcessInProgressDirectoryWithIntermittentFailure() throws IOException { + // Create in-progress files for different timestamps + long timestamp0004 = 1704153600000L + (4 * 1000L); // 00:00:04 + List inProgressFiles0004 = createInProgressFiles(timestamp0004, 3); + + long timestamp0102 = 1704153660000L + (2 * 1000L); // 00:01:02 + List inProgressFiles0102 = createInProgressFiles(timestamp0102, 2); + + // Combine all in-progress files for easier access + List allInProgressFiles = new ArrayList<>(); + allInProgressFiles.addAll(inProgressFiles0004); + allInProgressFiles.addAll(inProgressFiles0102); + + // Mock processFile to throw exception for specific files (files 1 and 3) only on first call, + // succeed on retry + String file1Prefix = allInProgressFiles.get(1).getName().substring(0, + allInProgressFiles.get(1).getName().lastIndexOf("_")); + Mockito.doThrow(new IOException("Processing failed for file 1")).doCallRealMethod() + .when(discovery).processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(file1Prefix); + })); + String file3Prefix = allInProgressFiles.get(3).getName().substring(0, + allInProgressFiles.get(3).getName().lastIndexOf("_")); + Mockito.doThrow(new IOException("Processing failed for file 3")).doCallRealMethod() + .when(discovery).processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(file3Prefix); + })); + + // Process in-progress directory + discovery.processInProgressDirectory(); + + // Verify that markInProgress was called 7 times (5 initially + 2 for retries) + Mockito.verify(fileTracker, Mockito.times(7)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + // Files 1 and 3 are called twice (initial attempt + retry), others once + for (int i = 0; i < allInProgressFiles.size(); i++) { + Path expectedFile = allInProgressFiles.get(i); + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + int expectedTimes = (i == 1 || i == 3) ? 2 : 1; // Files 1 and 3 are retried + Mockito.verify(fileTracker, Mockito.times(expectedTimes)) + .markInProgress(Mockito.argThat(path -> { + String pathName = path.getName(); + String prefix = pathName.substring(0, pathName.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that processFile was called for each file in the directory (i.e. 5 + 2 times for + // failed once that would succeed in next retry) + Mockito.verify(discovery, Mockito.times(7)).processFile(Mockito.any(Path.class)); + + // Verify that processFile was called for each specific file (using prefix matching) + // Files 1 and 3 should be called twice (fail once, succeed on retry), others once + for (int i = 0; i < allInProgressFiles.size(); i++) { + Path expectedFile = allInProgressFiles.get(i); + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + int expectedTimes = (i == 1 || i == 3) ? 2 : 1; // Files 1 and 3 are called twice (fail + + // retry success) + Mockito.verify(discovery, Mockito.times(expectedTimes)).processFile(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that markCompleted was called for each successfully processed file + Mockito.verify(fileTracker, Mockito.times(5)).markCompleted(Mockito.any(Path.class)); + + // Verify that markCompleted was called for 2 intermittent failed processed file + Mockito.verify(fileTracker, Mockito.times(2)).markFailed(Mockito.any(Path.class)); + + // Verify that markFailed was called once ONLY for failed files + String failedPrefix1 = allInProgressFiles.get(1).getName().substring(0, + allInProgressFiles.get(1).getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(failedPrefix1); + })); + String failedPrefix3 = allInProgressFiles.get(3).getName().substring(0, + allInProgressFiles.get(3).getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(failedPrefix3); + })); + + // Verify that markFailed was NOT called for files processed successfully in first iteration + String successPrefix0 = allInProgressFiles.get(0).getName().substring(0, + allInProgressFiles.get(0).getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.never()).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(successPrefix0); + })); + String successPrefix2 = allInProgressFiles.get(2).getName().substring(0, + allInProgressFiles.get(2).getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.never()).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(successPrefix2); + })); + String successPrefix4 = allInProgressFiles.get(4).getName().substring(0, + allInProgressFiles.get(4).getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.never()).markFailed(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(successPrefix4); + })); + + // Verify that markCompleted was called for each successfully processed file with correct paths + for (Path expectedFile : allInProgressFiles) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + } + + /** + * Tests processing of in-progress directory when no files meet the timestamp criteria. Validates + * that no files are processed when all files are too recent. + */ + @Test + public void testProcessInProgressDirectoryWithNoOldFiles() throws IOException { + // Set up current time for consistent testing + long currentTime = 1704153660000L; // 00:01:00 + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + // Create only recent files (all within the threshold) + long recentTimestamp1 = 1704153655000L; // 00:00:55 (5 seconds old) + long recentTimestamp2 = 1704153658000L; // 00:00:58 (2 seconds old) + + List recentFiles1 = createInProgressFiles(recentTimestamp1, 2); + List recentFiles2 = createInProgressFiles(recentTimestamp2, 2); + + // Process in-progress directory + discovery.processInProgressDirectory(); + + // Get processed files + List processedFiles = discovery.getProcessedFiles(); + + // Verify that no files were processed (all files are too recent) + assertEquals("Should not process any files when all files are too recent", 0, + processedFiles.size()); + + } finally { + EnvironmentEdgeManager.reset(); + } + } + + /** + * Tests processing of in-progress directory with timestamp filtering using + * getOlderInProgressFiles. Validates that only files older than the calculated threshold are + * processed, excluding recent files. + */ + @Test + public void testProcessInProgressDirectoryWithTimestampFiltering() throws IOException { + // Set up current time for consistent testing + long currentTime = 1704153660000L; // 00:01:00 + EnvironmentEdge edge = () -> currentTime; + EnvironmentEdgeManager.injectEdge(edge); + + try { + // Create files with various ages + long veryOldTimestamp = 1704153600000L; // 00:00:00 (1 minute old) - should be processed + long oldTimestamp = 1704153630000L; // 00:00:30 (30 seconds old) - should be processed + long recentTimestamp = 1704153655000L; // 00:00:55 (5 seconds old) - should NOT be processed + long veryRecentTimestamp = 1704153658000L; // 00:00:58 (2 seconds old) - should NOT be + // processed + + List veryOldFiles = createInProgressFiles(veryOldTimestamp, 1); + List oldFiles = createInProgressFiles(oldTimestamp, 1); + List recentFiles = createInProgressFiles(recentTimestamp, 1); + List veryRecentFiles = createInProgressFiles(veryRecentTimestamp, 1); + + // Process in-progress directory + discovery.processInProgressDirectory(); + + // Get processed files + List processedFiles = discovery.getProcessedFiles(); + + // Verify that only old files were processed (2 old files, 0 recent files) + assertEquals("Should process only old files based on timestamp filtering", 2, + processedFiles.size()); + + // Create set of expected processed files (by prefix, since UUIDs are updated during + // markInProgress) + Set expectedProcessedFilePrefixes = new HashSet<>(); + for (Path file : veryOldFiles) { + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf("_")); + expectedProcessedFilePrefixes.add(prefix); + } + for (Path file : oldFiles) { + String fileName = file.getName(); + String prefix = fileName.substring(0, fileName.lastIndexOf("_")); + expectedProcessedFilePrefixes.add(prefix); + } + + // Create set of actually processed file paths (extract prefixes) + Set actualProcessedFilePrefixes = new HashSet<>(); + for (Path file : processedFiles) { + String fileName = file.getName(); + String withoutExtension = fileName.substring(0, fileName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + actualProcessedFilePrefixes.add(prefix); + } + + // Validate that only old files were processed + assertEquals("Expected and actual processed files should match (only old files)", + expectedProcessedFilePrefixes, actualProcessedFilePrefixes); + + // Verify that markInProgress was called 2 times (only old files) + Mockito.verify(fileTracker, Mockito.times(2)).markInProgress(Mockito.any(Path.class)); + + // Verify that markInProgress was called for each expected file + for (Path expectedFile : veryOldFiles) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress(Mockito.argThat(path -> { + String pathName = path.getName(); + String prefix = pathName.substring(0, pathName.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + for (Path expectedFile : oldFiles) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markInProgress(Mockito.argThat(path -> { + String pathName = path.getName(); + String prefix = pathName.substring(0, pathName.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that markCompleted was called 2 times + Mockito.verify(fileTracker, Mockito.times(2)).markCompleted(Mockito.any(Path.class)); + + // Verify that markCompleted was called for each expected file with correct paths + for (Path expectedFile : veryOldFiles) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + for (Path expectedFile : oldFiles) { + String expectedPrefix = + expectedFile.getName().substring(0, expectedFile.getName().lastIndexOf("_")); + Mockito.verify(fileTracker, Mockito.times(1)).markCompleted(Mockito.argThat(path -> { + String pathName = path.getName(); + String withoutExtension = pathName.substring(0, pathName.lastIndexOf(".")); + String prefix = withoutExtension.substring(0, withoutExtension.lastIndexOf("_")); + return prefix.equals(expectedPrefix); + })); + } + + // Verify that recent files were NOT processed + for (Path file : recentFiles) { + String unexpectedPrefix = file.getName().substring(0, file.getName().lastIndexOf("_")); + assertFalse( + "Recent files should not be processed due to timestamp filtering: " + file.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + for (Path file : veryRecentFiles) { + String unexpectedPrefix = file.getName().substring(0, file.getName().lastIndexOf("_")); + assertFalse( + "Recent files should not be processed due to timestamp filtering: " + file.getName(), + actualProcessedFilePrefixes.contains(unexpectedPrefix)); + } + + } finally { + EnvironmentEdgeManager.reset(); + } + } + + @Test + public void testGetMinTimestampFromInProgressFilesEmptyList() throws IOException { + // Mock empty list of in-progress files + doReturn(Collections.emptyList()).when(fileTracker).getInProgressFiles(); + + // Call the method + Optional result = discovery.getMinTimestampFromInProgressFiles(); + + // Verify result is empty + assertFalse("Result should be empty for empty file list", result.isPresent()); + + // Verify getInProgressFiles was called once + verify(fileTracker, times(1)).getInProgressFiles(); + } + + @Test + public void testGetMinTimestampFromInProgressFilesSingleFile() throws IOException { + // Create a single file path + Path filePath = new Path("/test/1704153600000_rs1.plog"); + + // Mock single file list + doReturn(Collections.singletonList(filePath)).when(fileTracker).getInProgressFiles(); + // when(fileTracker.getInProgressFiles()).thenReturn(Collections.singletonList(filePath)); + when(fileTracker.getFileTimestamp(filePath)).thenReturn(1704153600000L); + + // Call the method + Optional result = discovery.getMinTimestampFromInProgressFiles(); + + // Verify result contains the timestamp + assertTrue("Result should be present for single file", result.isPresent()); + assertEquals("Should return the timestamp of the single file", Long.valueOf(1704153600000L), + result.get()); + + // Verify method calls + verify(fileTracker, times(1)).getInProgressFiles(); + verify(fileTracker, times(1)).getFileTimestamp(filePath); + } + + @Test + public void testGetMinTimestampFromInProgressFilesMultipleFiles() throws IOException { + // Create multiple file paths with different timestamps + Path file1 = new Path("/test/1704153660000_rs2.plog"); + Path file2 = new Path("/test/1704153600000_rs1.plog"); + Path file3 = new Path("/test/1704153720000_rs3.plog"); + + List files = Arrays.asList(file1, file2, file3); + + // Mock file list and timestamps + doReturn(files).when(fileTracker).getInProgressFiles(); + when(fileTracker.getFileTimestamp(file1)).thenReturn(1704153660000L); + when(fileTracker.getFileTimestamp(file2)).thenReturn(1704153600000L); + when(fileTracker.getFileTimestamp(file3)).thenReturn(1704153720000L); + + // Call the method + Optional result = discovery.getMinTimestampFromInProgressFiles(); + + // Verify result contains the minimum timestamp + assertTrue("Result should be present for multiple files", result.isPresent()); + assertEquals("Should return the minimum timestamp", Long.valueOf(1704153600000L), result.get()); + + // Verify method calls + verify(fileTracker, times(1)).getInProgressFiles(); + verify(fileTracker, times(1)).getFileTimestamp(file1); + verify(fileTracker, times(1)).getFileTimestamp(file2); + verify(fileTracker, times(1)).getFileTimestamp(file3); + } + + @Test + public void testGetMinTimestampFromNewFilesEmptyList() throws IOException { + // Mock empty list of new files + when(fileTracker.getNewFiles()).thenReturn(Collections.emptyList()); + + // Call the method + Optional result = discovery.getMinTimestampFromNewFiles(); + + // Verify result is empty + assertFalse("Result should be empty for empty file list", result.isPresent()); + + // Verify getNewFiles was called once + verify(fileTracker, times(1)).getNewFiles(); + } + + @Test + public void testGetMinTimestampFromNewFilesSingleFile() throws IOException { + // Create a single file path + Path filePath = new Path("/test/1704153600000_rs1.plog"); + + // Mock single file list + when(fileTracker.getNewFiles()).thenReturn(Collections.singletonList(filePath)); + when(fileTracker.getFileTimestamp(filePath)).thenReturn(1704153600000L); + + // Call the method + Optional result = discovery.getMinTimestampFromNewFiles(); + + // Verify result contains the timestamp + assertTrue("Result should be present for single file", result.isPresent()); + assertEquals("Should return the timestamp of the single file", Long.valueOf(1704153600000L), + result.get()); + + // Verify method calls + verify(fileTracker, times(1)).getNewFiles(); + verify(fileTracker, times(1)).getFileTimestamp(filePath); + } + + @Test + public void testGetMinTimestampFromNewFilesMultipleFiles() throws IOException { + // Create multiple file paths with different timestamps + Path file1 = new Path("/test/1704153660000_rs2.plog"); + Path file2 = new Path("/test/1704153600000_rs1.plog"); + Path file3 = new Path("/test/1704153720000_rs3.plog"); + + List files = Arrays.asList(file1, file2, file3); + + // Mock file list and timestamps + when(fileTracker.getNewFiles()).thenReturn(files); + when(fileTracker.getFileTimestamp(file1)).thenReturn(1704153660000L); + when(fileTracker.getFileTimestamp(file2)).thenReturn(1704153600000L); + when(fileTracker.getFileTimestamp(file3)).thenReturn(1704153720000L); + + // Call the method + Optional result = discovery.getMinTimestampFromNewFiles(); + + // Verify result contains the minimum timestamp + assertTrue("Result should be present for multiple files", result.isPresent()); + assertEquals("Should return the minimum timestamp", Long.valueOf(1704153600000L), result.get()); + + // Verify method calls + verify(fileTracker, times(1)).getNewFiles(); + verify(fileTracker, times(1)).getFileTimestamp(file1); + verify(fileTracker, times(1)).getFileTimestamp(file2); + verify(fileTracker, times(1)).getFileTimestamp(file3); + } + + @Test + public void testInitializeLastRoundInSync_MultipleInProgressFiles_NoNewFiles() + throws IOException { + // Create multiple in-progress files with timestamps + Path inProgressFile1 = new Path(testFolderPath, "1704067200000_rs1.plog"); // 2024-01-01 + // 00:00:00 + Path inProgressFile2 = new Path(testFolderPath, "1704067260000_rs2.plog"); // 2024-01-01 + // 00:01:00 + Path inProgressFile3 = new Path(testFolderPath, "1704067320000_rs3.plog"); // 2024-01-01 + // 00:02:00 + + List inProgressFiles = Arrays.asList(inProgressFile1, inProgressFile2, inProgressFile3); + + // Mock the tracker to return in-progress files and empty new files + when(fileTracker.getInProgressFiles()).thenReturn(inProgressFiles); + when(fileTracker.getNewFiles()).thenReturn(Collections.emptyList()); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + // Verify the round uses the minimum timestamp (1704067200000L) + long expectedEndTime = + (1704067200000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use minimum timestamp from in-progress files", expectedStartTime, + result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_MultipleInProgressFiles_OneNewFile() throws IOException { + // Create multiple in-progress files with timestamps + Path inProgressFile1 = new Path(testFolderPath, "1704153600000_rs1.plog"); // 2024-01-02 + // 00:00:00 + Path inProgressFile2 = new Path(testFolderPath, "1704153660000_rs2.plog"); // 2024-01-02 + // 00:01:00 + Path inProgressFile3 = new Path(testFolderPath, "1704153720000_rs3.plog"); // 2024-01-02 + // 00:02:00 + + List inProgressFiles = Arrays.asList(inProgressFile1, inProgressFile2, inProgressFile3); + + // Create one new file with later timestamp + Path newFile = new Path(testFolderPath, "1704153780000_rs4.plog"); // 2024-01-02 00:03:00 + List newFiles = Collections.singletonList(newFile); + + // Mock the tracker to return in-progress files and one new file + when(fileTracker.getInProgressFiles()).thenReturn(inProgressFiles); + when(fileTracker.getNewFiles()).thenReturn(newFiles); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result - should use minimum timestamp from in-progress files + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + long expectedEndTime = + (1704153600000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use minimum timestamp from in-progress files", expectedStartTime, + result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_MultipleInProgressFiles_MultipleNewFiles() throws IOException { + // Create multiple in-progress files with timestamps + Path inProgressFile1 = new Path(testFolderPath, "1704240000000_rs1.plog"); // 2024-01-03 + // 00:00:00 + Path inProgressFile2 = new Path(testFolderPath, "1704240060000_rs2.plog"); // 2024-01-03 + // 00:01:00 + Path inProgressFile3 = new Path(testFolderPath, "1704240120000_rs3.plog"); // 2024-01-03 + // 00:02:00 + + List inProgressFiles = Arrays.asList(inProgressFile1, inProgressFile2, inProgressFile3); + + // Create multiple new files with later timestamps + Path newFile1 = new Path(testFolderPath, "1704240180000_rs4.plog"); // 2024-01-03 00:03:00 + Path newFile2 = new Path(testFolderPath, "1704240240000_rs5.plog"); // 2024-01-03 00:04:00 + List newFiles = Arrays.asList(newFile1, newFile2); + + // Mock the tracker to return in-progress files and multiple new files + when(fileTracker.getInProgressFiles()).thenReturn(inProgressFiles); + when(fileTracker.getNewFiles()).thenReturn(newFiles); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result - should use minimum timestamp from in-progress files + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + long expectedEndTime = + (1704240000000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use minimum timestamp from in-progress files", expectedStartTime, + result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_OneInProgressFile_NoNewFiles() throws IOException { + // Create single in-progress file + Path inProgressFile = new Path(testFolderPath, "1704326400000_rs1.plog"); // 2024-01-04 00:00:00 + List inProgressFiles = Collections.singletonList(inProgressFile); + + // Mock the tracker to return single in-progress file and empty new files + when(fileTracker.getInProgressFiles()).thenReturn(inProgressFiles); + when(fileTracker.getNewFiles()).thenReturn(Collections.emptyList()); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + long expectedEndTime = + (1704326400000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use timestamp from single in-progress file", expectedStartTime, + result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_OneInProgressFile_OneNewFile() throws IOException { + // Create single in-progress file + Path inProgressFile = new Path(testFolderPath, "1704412800000_rs1.plog"); // 2024-01-05 00:00:00 + List inProgressFiles = Collections.singletonList(inProgressFile); + + // Create one new file with later timestamp + Path newFile = new Path(testFolderPath, "1704412860000_rs2.plog"); // 2024-01-05 00:01:00 + List newFiles = Collections.singletonList(newFile); + + // Mock the tracker to return single in-progress file and one new file + when(fileTracker.getInProgressFiles()).thenReturn(inProgressFiles); + when(fileTracker.getNewFiles()).thenReturn(newFiles); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result - should use timestamp from in-progress file + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + long expectedEndTime = + (1704412800000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use timestamp from in-progress file", expectedStartTime, + result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_OneInProgressFile_MultipleNewFiles() throws IOException { + // Create single in-progress file + Path inProgressFile = new Path(testFolderPath, "1704499200000_rs1.plog"); // 2024-01-06 00:00:00 + List inProgressFiles = Collections.singletonList(inProgressFile); + + // Create multiple new files with later timestamps + Path newFile1 = new Path(testFolderPath, "1704499260000_rs2.plog"); // 2024-01-06 00:01:00 + Path newFile2 = new Path(testFolderPath, "1704499320000_rs3.plog"); // 2024-01-06 00:02:00 + List newFiles = Arrays.asList(newFile1, newFile2); + + // Mock the tracker to return single in-progress file and multiple new files + when(fileTracker.getInProgressFiles()).thenReturn(inProgressFiles); + when(fileTracker.getNewFiles()).thenReturn(newFiles); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result - should use timestamp from in-progress file + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + long expectedEndTime = + (1704499200000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use timestamp from in-progress file", expectedStartTime, + result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_NoInProgressFiles_MultipleNewFiles() throws IOException { + // Mock empty in-progress files + when(fileTracker.getInProgressFiles()).thenReturn(Collections.emptyList()); + + // Create multiple new files with timestamps + Path newFile1 = new Path(testFolderPath, "1704585600000_rs1.plog"); // 2024-01-07 00:00:00 + Path newFile2 = new Path(testFolderPath, "1704585660000_rs2.plog"); // 2024-01-07 00:01:00 + Path newFile3 = new Path(testFolderPath, "1704585720000_rs3.plog"); // 2024-01-07 00:02:00 + List newFiles = Arrays.asList(newFile1, newFile2, newFile3); + + // Mock the tracker to return empty in-progress files and multiple new files + when(fileTracker.getNewFiles()).thenReturn(newFiles); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + // Verify the round uses the minimum timestamp from new files (1704585600000L) + long expectedEndTime = + (1704585600000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use minimum timestamp from new files when no in-progress files", + expectedStartTime, result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_NoInProgressFiles_OneNewFile() throws IOException { + // Mock empty in-progress files + when(fileTracker.getInProgressFiles()).thenReturn(Collections.emptyList()); + + // Create single new file + Path newFile = new Path(testFolderPath, "1704672000000_rs1.plog"); // 2024-01-08 00:00:00 + List newFiles = Collections.singletonList(newFile); + + // Mock the tracker to return empty in-progress files and one new file + when(fileTracker.getNewFiles()).thenReturn(newFiles); + + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + long expectedEndTime = + (1704672000000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round down + // to nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use timestamp from single new file when no in-progress files", + expectedStartTime, result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } + + @Test + public void testInit_NoInProgressFiles_NoNewFiles() throws IOException { + // Mock empty in-progress files + when(fileTracker.getInProgressFiles()).thenReturn(Collections.emptyList()); + + // Mock empty new files + when(fileTracker.getNewFiles()).thenReturn(Collections.emptyList()); + + // Set custom time using EnvironmentEdgeManager + EnvironmentEdge customEdge = () -> { + return 1704758400000L; // 2024-01-09 00:00:00 + }; + EnvironmentEdgeManager.injectEdge(customEdge); + + try { + // Call the method + discovery.initializeLastRoundProcessed(); + + // Verify the result + ReplicationRound result = discovery.getLastRoundProcessed(); + assertNotNull("Last successfully processed round should not be null", result); + + // Verify the round uses the custom time + long expectedEndTime = + (1704758400000L / TimeUnit.MINUTES.toMillis(1)) * TimeUnit.MINUTES.toMillis(1); // Round + // down to + // nearest + // 60-second + // boundary + long expectedStartTime = expectedEndTime - TimeUnit.MINUTES.toMillis(1); // Start time is 60 + // seconds less than + // end time + assertEquals("Should use custom time when no files are found", expectedStartTime, + result.getStartTime()); + assertEquals("End time must be rounded down to nearest 60-second boundary", expectedEndTime, + result.getEndTime()); + } finally { + // Reset EnvironmentEdgeManager to default + EnvironmentEdgeManager.reset(); + } + } + + private List createNewFilesForRound(ReplicationRound replicationRound, int fileCount) + throws IOException { + // Create files for multiple rounds in the same shard with each file at gap of 2 seconds + Preconditions.checkArgument(fileCount + <= fileTracker.getReplicationShardDirectoryManager().getReplicationRoundDurationSeconds() + / 2); + ReplicationShardDirectoryManager shardManager = + fileTracker.getReplicationShardDirectoryManager(); + Path shardPath = shardManager.getShardDirectory(replicationRound.getStartTime()); + localFs.mkdirs(shardPath); + List newFiles = new ArrayList<>(); + for (int i = 0; i < fileCount; i++) { + Path file = + new Path(shardPath, replicationRound.getStartTime() + (1234L * i) + "_rs-" + i + ".plog"); + localFs.create(file, true).close(); + newFiles.add(file); + } + return newFiles; + } + + private List createInProgressFiles(long timestamp, int count) throws IOException { + // Create in-progress files + Path inProgressDir = fileTracker.getInProgressDirPath(); + localFs.mkdirs(inProgressDir); + List inProgressFiles = new ArrayList<>(); + for (int i = 0; i < count; i++) { + String uuid = "12345678-1234-1234-1234-123456789abc" + i; + Path inProgressFile = new Path(inProgressDir, timestamp + "_rs-" + i + "_" + uuid + ".plog"); + localFs.create(inProgressFile, true).close(); + inProgressFiles.add(inProgressFile); + } + return inProgressFiles; + } + + @Test + public void testReplay() throws IOException { + long roundTimeMills = discovery.getRoundTimeMills(); + long bufferMillis = discovery.getBufferMillis(); + long totalWaitTime = roundTimeMills + bufferMillis; + + // Test Case 1: No rounds to process (not enough time has passed) + long initialEndTime = 1704153600000L; // 2024-01-02 00:00:00 + discovery + .setLastRoundProcessed(new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime)); + discovery.resetProcessedRounds(); + + EnvironmentEdge edge1 = () -> { + return initialEndTime + 1000L; // Only 1 second after last round + }; + EnvironmentEdgeManager.injectEdge(edge1); + + try { + discovery.replay(); + List processedRounds = discovery.getProcessedRounds(); + assertEquals("Should not process any rounds when not enough time has passed", 0, + processedRounds.size()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 2: Exactly one round to process + discovery + .setLastRoundProcessed(new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime)); + discovery.resetProcessedRounds(); + + EnvironmentEdge edge2 = () -> { + return initialEndTime + totalWaitTime; // Exactly threshold for one round + }; + EnvironmentEdgeManager.injectEdge(edge2); + + try { + discovery.replay(); + List processedRounds = discovery.getProcessedRounds(); + assertEquals("Should process exactly one round", 1, processedRounds.size()); + + ReplicationRound expectedRound = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("Processed round should match expected round", expectedRound, + processedRounds.get(0)); + assertEquals("Last round processed should be updated", expectedRound, + discovery.getLastRoundProcessed()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 3: Multiple rounds to process (3 rounds worth of time) + discovery + .setLastRoundProcessed(new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime)); + discovery.resetProcessedRounds(); + + EnvironmentEdge edge3 = () -> { + return initialEndTime + (3 * totalWaitTime); // 3 rounds worth of time + }; + EnvironmentEdgeManager.injectEdge(edge3); + + try { + discovery.replay(); + List processedRounds = discovery.getProcessedRounds(); + assertEquals("Should process 3 rounds", 3, processedRounds.size()); + + // Verify first round + ReplicationRound expectedRound1 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round should match expected", expectedRound1, processedRounds.get(0)); + + // Verify second round + ReplicationRound expectedRound2 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Second round should match expected", expectedRound2, processedRounds.get(1)); + + // Verify third round + ReplicationRound expectedRound3 = new ReplicationRound(initialEndTime + (2 * roundTimeMills), + initialEndTime + (3 * roundTimeMills)); + assertEquals("Third round should match expected", expectedRound3, processedRounds.get(2)); + + // Verify last round processed was updated to the last round + assertEquals("Last round processed should be updated to third round", expectedRound3, + discovery.getLastRoundProcessed()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 4: Exception during processing (should stop and not update last round) + discovery + .setLastRoundProcessed(new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime)); + discovery.resetProcessedRounds(); + + // Create files for multiple rounds + ReplicationRound round1 = new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + ReplicationRound round2 = + new ReplicationRound(initialEndTime + roundTimeMills, initialEndTime + (2 * roundTimeMills)); + ReplicationRound round3 = new ReplicationRound(initialEndTime + (2 * roundTimeMills), + initialEndTime + (3 * roundTimeMills)); + + createNewFilesForRound(round1, 2); + createNewFilesForRound(round2, 2); + createNewFilesForRound(round3, 2); + + // Mock shouldProcessInProgressDirectory to return false (only process new files) + discovery.setMockShouldProcessInProgressDirectory(false); + + // Mock processRound to throw exception on second round + Mockito.doCallRealMethod().doThrow(new IOException("Simulated failure on round 2")) + .when(discovery).processRound(Mockito.any(ReplicationRound.class)); + + EnvironmentEdge edge4 = () -> { + return initialEndTime + (3 * totalWaitTime); // 3 rounds worth of time + }; + EnvironmentEdgeManager.injectEdge(edge4); + + try { + discovery.replay(); + + // Should have processed only the first round before exception + List processedRounds = discovery.getProcessedRounds(); + assertEquals("Should process only 1 round before exception", 1, processedRounds.size()); + + ReplicationRound expectedRound1 = + new ReplicationRound(initialEndTime, initialEndTime + roundTimeMills); + assertEquals("First round should be processed", expectedRound1, processedRounds.get(0)); + + // Last round processed should be updated to first round (before exception) + assertEquals("Last round processed should be updated to first round only", expectedRound1, + discovery.getLastRoundProcessed()); + } finally { + EnvironmentEdgeManager.reset(); + // Reset the mock + Mockito.doCallRealMethod().when(discovery).processRound(Mockito.any(ReplicationRound.class)); + } + + // Test Case 5: Resume after exception (should continue from last successful round) + discovery.resetProcessedRounds(); + + EnvironmentEdge edge5 = () -> { + return initialEndTime + (3 * totalWaitTime); // Still 3 rounds worth from original + }; + EnvironmentEdgeManager.injectEdge(edge5); + + try { + discovery.replay(); + + // Should process the remaining 2 rounds + List processedRounds = discovery.getProcessedRounds(); + assertEquals("Should process remaining 2 rounds", 2, processedRounds.size()); + + // Verify second round (continuing from where we left off) + ReplicationRound expectedRound2 = new ReplicationRound(initialEndTime + roundTimeMills, + initialEndTime + (2 * roundTimeMills)); + assertEquals("Second round should match expected", expectedRound2, processedRounds.get(0)); + + // Verify third round + ReplicationRound expectedRound3 = new ReplicationRound(initialEndTime + (2 * roundTimeMills), + initialEndTime + (3 * roundTimeMills)); + assertEquals("Third round should match expected", expectedRound3, processedRounds.get(1)); + + // Last round processed should now be updated to third round + assertEquals("Last round processed should be updated to third round", expectedRound3, + discovery.getLastRoundProcessed()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 6: Boundary condition - exactly at buffer threshold + discovery + .setLastRoundProcessed(new ReplicationRound(initialEndTime - roundTimeMills, initialEndTime)); + discovery.resetProcessedRounds(); + + EnvironmentEdge edge6 = () -> { + return initialEndTime + totalWaitTime - 1L; // 1ms before threshold + }; + EnvironmentEdgeManager.injectEdge(edge6); + + try { + discovery.replay(); + List processedRounds = discovery.getProcessedRounds(); + assertEquals("Should not process any rounds when 1ms before threshold", 0, + processedRounds.size()); + } finally { + EnvironmentEdgeManager.reset(); + } + } + + @Test + public void testGetNextRoundToProcess() { + // Setup: roundTimeMills = 60000ms (1 minute), bufferMillis = 15% of 60000 = 9000ms + // Total wait time needed = 69000ms + long roundTimeMills = discovery.getRoundTimeMills(); + long bufferMillis = discovery.getBufferMillis(); + long totalWaitTime = roundTimeMills + bufferMillis; + + // Test Case 1: Not enough time has passed (just started) + long lastRoundEndTimestamp = 1704153600000L; // 2024-01-02 00:00:00 + discovery.setLastRoundProcessed( + new ReplicationRound(lastRoundEndTimestamp - roundTimeMills, lastRoundEndTimestamp)); + + long currentTime1 = lastRoundEndTimestamp + 1000L; // Only 1 second later + + EnvironmentEdge edge1 = () -> currentTime1; + EnvironmentEdgeManager.injectEdge(edge1); + + try { + Optional result1 = discovery.getNextRoundToProcess(); + assertFalse("Should return empty when not enough time has passed (1 second)", + result1.isPresent()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 2: Not enough time has passed (just before threshold) + discovery.setLastRoundProcessed( + new ReplicationRound(lastRoundEndTimestamp - roundTimeMills, lastRoundEndTimestamp)); + + long currentTime2 = lastRoundEndTimestamp + totalWaitTime - 1L; // 1ms before threshold + + EnvironmentEdge edge2 = () -> currentTime2; + EnvironmentEdgeManager.injectEdge(edge2); + + try { + Optional result2 = discovery.getNextRoundToProcess(); + assertFalse("Should return empty when not enough time has passed (1ms before threshold)", + result2.isPresent()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 3: Exactly at threshold - should return next round + discovery.setLastRoundProcessed( + new ReplicationRound(lastRoundEndTimestamp - roundTimeMills, lastRoundEndTimestamp)); + + long currentTime3 = lastRoundEndTimestamp + totalWaitTime; // Exactly at threshold + + EnvironmentEdge edge3 = () -> currentTime3; + EnvironmentEdgeManager.injectEdge(edge3); + + try { + Optional result3 = discovery.getNextRoundToProcess(); + assertTrue("Should return next round when exactly at threshold", result3.isPresent()); + + ReplicationRound expectedRound = + new ReplicationRound(lastRoundEndTimestamp, lastRoundEndTimestamp + roundTimeMills); + assertEquals("Returned round should match expected", expectedRound, result3.get()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 4: Time has passed beyond threshold (1ms after) + discovery.setLastRoundProcessed( + new ReplicationRound(lastRoundEndTimestamp - roundTimeMills, lastRoundEndTimestamp)); + + long currentTime4 = lastRoundEndTimestamp + totalWaitTime + 1L; // 1ms after threshold + + EnvironmentEdge edge4 = () -> currentTime4; + EnvironmentEdgeManager.injectEdge(edge4); + + try { + Optional result4 = discovery.getNextRoundToProcess(); + assertTrue("Should return next round when 1ms after threshold", result4.isPresent()); + + ReplicationRound expectedRound = + new ReplicationRound(lastRoundEndTimestamp, lastRoundEndTimestamp + roundTimeMills); + assertEquals("Returned round should match expected", expectedRound, result4.get()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 5: Much time has passed (multiple rounds worth) + discovery.setLastRoundProcessed( + new ReplicationRound(lastRoundEndTimestamp - roundTimeMills, lastRoundEndTimestamp)); + + long currentTime5 = lastRoundEndTimestamp + (3 * totalWaitTime); // 3 rounds worth of time + + EnvironmentEdge edge5 = () -> currentTime5; + EnvironmentEdgeManager.injectEdge(edge5); + + try { + Optional result5 = discovery.getNextRoundToProcess(); + assertTrue("Should return next round when multiple rounds worth of time has passed", + result5.isPresent()); + + ReplicationRound expectedRound = + new ReplicationRound(lastRoundEndTimestamp, lastRoundEndTimestamp + roundTimeMills); + assertEquals("Returned round should match expected", expectedRound, result5.get()); + } finally { + EnvironmentEdgeManager.reset(); + } + + // Test Case 6: Halfway through buffer time + discovery.setLastRoundProcessed( + new ReplicationRound(lastRoundEndTimestamp - roundTimeMills, lastRoundEndTimestamp)); + + long currentTime8 = lastRoundEndTimestamp + roundTimeMills + (bufferMillis / 2); + + EnvironmentEdge edge8 = () -> currentTime8; + EnvironmentEdgeManager.injectEdge(edge8); + + try { + Optional result8 = discovery.getNextRoundToProcess(); + assertFalse("Should return empty when halfway through buffer time", result8.isPresent()); + } finally { + EnvironmentEdgeManager.reset(); + } + } + + /** + * Testable implementation of ReplicationLogTracker for unit testing. Exposes protected methods + * and provides minimal implementation for testing. + */ + private static class TestableReplicationLogTracker extends ReplicationLogTracker { + public TestableReplicationLogTracker(final Configuration conf, final String haGroupName, + final FileSystem fileSystem, + final ReplicationShardDirectoryManager replicationShardDirectoryManager) { + super(conf, haGroupName, fileSystem, replicationShardDirectoryManager, metricsLogTracker); + } + } + + /** + * Testable implementation of ReplicationLogDiscovery for unit testing. Tracks processed files and + * rounds, and provides access to protected methods. + */ + private static class TestableReplicationLogDiscovery extends ReplicationLogDiscovery { + private final List processedFiles = new ArrayList<>(); + private final List processedRounds = new ArrayList<>(); + + public TestableReplicationLogDiscovery(ReplicationLogTracker fileTracker) { + super(fileTracker); + } + + public long getRoundTimeMills() { + return roundTimeMills; + } + + public long getBufferMillis() { + return bufferMillis; + } + + @Override + protected void processFile(Path path) throws IOException { + // Simulate file processing + processedFiles.add(path); + } + + @Override + protected MetricsReplicationLogDiscovery createMetricsSource() { + return metricsLogDiscovery; + } + + @Override + protected void processRound(ReplicationRound replicationRound) throws IOException { + super.processRound(replicationRound); + // Track processed rounds + processedRounds.add(replicationRound); + } + + public List getProcessedFiles() { + return new ArrayList<>(processedFiles); + } + + public List getProcessedRounds() { + return new ArrayList<>(processedRounds); + } + + public void resetProcessedRounds() { + processedRounds.clear(); + } + + public void setLastRoundProcessed(ReplicationRound round) { + super.setLastRoundProcessed(round); + } + + public ScheduledExecutorService getScheduler() { + return super.scheduler; + } + + private Boolean mockShouldProcessInProgressDirectory = null; + + @Override + protected boolean shouldProcessInProgressDirectory() { + if (mockShouldProcessInProgressDirectory != null) { + return mockShouldProcessInProgressDirectory; + } + return super.shouldProcessInProgressDirectory(); + } + + public void setMockShouldProcessInProgressDirectory(boolean value) { + this.mockShouldProcessInProgressDirectory = value; + } + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogTrackerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogTrackerTest.java new file mode 100644 index 00000000000..8c7c239c671 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogTrackerTest.java @@ -0,0 +1,1395 @@ +/* + * 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.phoenix.replication; + +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.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; + +import java.io.IOException; +import java.net.URI; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTracker; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogTrackerReplayImpl; +import org.apache.phoenix.replication.reader.ReplicationLogReplay; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; + +public class ReplicationLogTrackerTest { + + @ClassRule + public static TemporaryFolder testFolder = new TemporaryFolder(); + + private ReplicationLogTracker tracker; + private Configuration conf; + private FileSystem localFs; + private FileSystem mockFs; + private URI rootURI; + private static final String haGroupName = "testGroup"; + private static final MetricsReplicationLogTracker metrics = + new MetricsReplicationLogTrackerReplayImpl(haGroupName); + + @Before + public void setUp() throws IOException { + conf = HBaseConfiguration.create(); + localFs = Mockito.spy(FileSystem.getLocal(conf)); + mockFs = Mockito.spy(localFs); + rootURI = new Path(testFolder.getRoot().toString()).toUri(); + tracker = createReplicationLogFileTracker(conf, haGroupName, mockFs, rootURI); + } + + @After + public void tearDown() throws IOException { + if (tracker != null) { + tracker.close(); + } + localFs.delete(new Path(testFolder.getRoot().toURI()), true); + } + + @Test + public void testInit() throws IOException { + // Call init method + tracker.init(); + + // Verify in-progress path is set correctly + Path actualInProgressPath = tracker.getInProgressDirPath(); + Path expectedInProgressPath = new Path(new Path(rootURI.getPath(), haGroupName), "in_progress"); + assertNotNull("In-progress path should not be null", actualInProgressPath); + assertEquals("In-progress path should be set correctly", expectedInProgressPath, + actualInProgressPath); + + // Verify in-progress directory was created + assertTrue("In-progress directory should exist", localFs.exists(expectedInProgressPath)); + assertTrue("In-progress directory should be a directory", + localFs.isDirectory(expectedInProgressPath)); + + // Verify ReplicationShardDirectoryManager was created + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + assertNotNull("ReplicationShardDirectoryManager should not be null", shardManager); + + // Verify the shard directory path is correct + Path expectedShardDirectory = + new Path(new Path(new Path(rootURI.getPath(), haGroupName), "in"), "shard"); + assertEquals("Shard directory should be set correctly", expectedShardDirectory, + shardManager.getShardDirectoryPath()); + + // Assert mkdir is called only once with expected parameter + Mockito.verify(mockFs, times(1)).mkdirs(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(expectedInProgressPath); + Mockito.verify(mockFs, times(1)).mkdirs(expectedInProgressPath); + } + + @Test + public void testInitWithDifferentHaGroupName() throws IOException { + // Test with different HA group name + String differentHaGroupName = "differentGroup"; + tracker = createReplicationLogFileTracker(conf, differentHaGroupName, mockFs, rootURI); + + // Call init method + tracker.init(); + + // Verify correct path was created + Path expectedInProgressPath = + new Path(new Path(rootURI.getPath(), differentHaGroupName), "in_progress"); + assertTrue("In-progress directory should exist for different HA group", + localFs.exists(expectedInProgressPath)); + assertTrue("In-progress directory should be a directory", + localFs.isDirectory(expectedInProgressPath)); + + // Verify shard directory uses correct HA group name + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + Path expectedShardDirectory = + new Path(new Path(new Path(rootURI.getPath(), differentHaGroupName), "in"), "shard"); + assertEquals("Shard directory should use correct HA group name", expectedShardDirectory, + shardManager.getShardDirectoryPath()); + + // Assert mkdir is called only once with expected parameter + Mockito.verify(mockFs, times(1)).mkdirs(expectedInProgressPath); + } + + @Test + public void testInitIsIdempotent() throws IOException { + // Create the directory manually first + Path expectedInProgressPath = new Path(new Path(rootURI.getPath(), haGroupName), "in_progress"); + localFs.mkdirs(expectedInProgressPath); + assertTrue("Directory should exist before init", localFs.exists(expectedInProgressPath)); + + // Get initial directory count + Path rootPath = new Path(rootURI.getPath()); + int initialDirCount = countDirectories(localFs, rootPath); + + // Call init method multiple times + tracker.init(); + tracker.close(); + tracker.init(); + tracker.close(); + tracker.init(); + + // Verify directory still exists and is valid + assertTrue("In-progress directory should still exist", localFs.exists(expectedInProgressPath)); + assertTrue("In-progress directory should still be a directory", + localFs.isDirectory(expectedInProgressPath)); + + // Verify no additional directories were created + int finalDirCount = countDirectories(localFs, rootPath); + assertEquals("No additional directories should be created", initialDirCount, finalDirCount); + + // Verify ReplicationShardDirectoryManager is still valid + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + assertNotNull("ReplicationShardDirectoryManager should not be null after multiple init calls", + shardManager); + Path expectedShardDirectory = + new Path(new Path(new Path(rootURI.getPath(), haGroupName), "in"), "shard"); + assertEquals("Shard directory should use correct HA group name", expectedShardDirectory, + shardManager.getShardDirectoryPath()); + + // Assert mkdir is NOT called at all because directory already exist + Mockito.verify(mockFs, times(0)).mkdirs(expectedInProgressPath); + } + + @Test + public void testGetNewFilesForRound() throws IOException { + // Initialize tracker + tracker.init(); + + // Create a replication round (60 seconds duration) + long roundStartTime = 1704153600000L; // 2024-01-02 00:00:00 + long roundEndTime = roundStartTime + TimeUnit.MINUTES.toMillis(1); // 60 seconds later + ReplicationRound targetRound = new ReplicationRound(roundStartTime, roundEndTime); + + // Get the shard directory for this round + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + Path shardDirectory = shardManager.getShardDirectory(targetRound); + + // Create the shard directory + localFs.mkdirs(shardDirectory); + + // Create files in the target round's shard directory + Path fileInTargetRound1 = new Path(shardDirectory, "1704153600000_rs1.plog"); // Start of round + Path fileInTargetRound2 = new Path(shardDirectory, "1704153630000_rs2.plog"); // Middle of round + Path fileInTargetRound3 = new Path(shardDirectory, "1704153659000_rs3.plog"); // End of round + + // Create files in other rounds (same shard) + Path fileInOtherRound1 = new Path(shardDirectory, "1704161280000_rs4.plog"); // future round for + // same shard + Path fileInOtherRound2 = new Path(shardDirectory, "1704161310000_rs5.plog"); // Previous round + + // Create files in other shards + Path fileInOtherShard1 = new Path(shardDirectory.getParent(), "001/1704153600000_rs6.plog"); + Path fileInOtherShard2 = new Path(shardDirectory.getParent(), "002/1704153600000_rs7.plog"); + + // Create all directories and files + localFs.mkdirs(fileInOtherShard1.getParent()); + localFs.mkdirs(fileInOtherShard2.getParent()); + + // Create empty files + localFs.create(fileInTargetRound1, true).close(); + localFs.create(fileInTargetRound2, true).close(); + localFs.create(fileInTargetRound3, true).close(); + localFs.create(fileInOtherRound1, true).close(); + localFs.create(fileInOtherRound2, true).close(); + localFs.create(fileInOtherShard1, true).close(); + localFs.create(fileInOtherShard2, true).close(); + + // Call getNewFilesForRound + List result = tracker.getNewFilesForRound(targetRound); + + // Verify file system operation counts + Mockito.verify(mockFs, times(1)).listStatus(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardDirectory)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(shardDirectory)); + + Set expectedPaths = new HashSet<>(); + expectedPaths.add(fileInTargetRound1.toString()); + expectedPaths.add(fileInTargetRound2.toString()); + expectedPaths.add(fileInTargetRound3.toString()); + + // Create actual set of paths + Set actualPaths = + result.stream().map(path -> path.toUri().getPath()).collect(Collectors.toSet()); + + assertEquals("Should return exactly 3 files from target round", expectedPaths.size(), + actualPaths.size()); + assertEquals("File paths do not match", expectedPaths, actualPaths); + } + + @Test + public void testGetNewFilesForRoundForNoFilesExist() throws IOException { + // Initialize tracker + tracker.init(); + + // Create a replication round + long roundStartTime = 1704153600000L; + long roundEndTime = roundStartTime + TimeUnit.MINUTES.toMillis(1); + ReplicationRound targetRound = new ReplicationRound(roundStartTime, roundEndTime); + + // Get the shard directory for this round + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + Path shardDirectory = shardManager.getShardDirectory(targetRound); + + // Create the shard directory but leave it empty + localFs.mkdirs(shardDirectory); + + // Call getNewFilesForRound + List result = tracker.getNewFilesForRound(targetRound); + + // Verify file system operation counts + Mockito.verify(mockFs, times(1)).listStatus(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardDirectory)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(shardDirectory)); + + // Verify empty list is returned + assertTrue("Should return empty list for empty directory", result.isEmpty()); + } + + @Test + public void testGetNewFilesForRoundForNonExistentShardDirectory() throws IOException { + // Initialize tracker + tracker.init(); + + // Create a replication round + long roundStartTime = 1704153600000L; + long roundEndTime = roundStartTime + TimeUnit.MINUTES.toMillis(1); + ReplicationRound targetRound = new ReplicationRound(roundStartTime, roundEndTime); + + // Get the shard directory for this round + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + Path shardDirectory = shardManager.getShardDirectory(targetRound); + + // Assert that shard directory does not exist + assertFalse("Shard directory should not exist", localFs.exists(shardDirectory)); + + // Call getNewFilesForRound + List result = tracker.getNewFilesForRound(targetRound); + + // Verify file system operation counts + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardDirectory)); + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + + // Verify empty list is returned + assertTrue("Should return empty list for non-existent directory", result.isEmpty()); + } + + @Test + public void testGetNewFilesForRoundWithInvalidFiles() throws IOException { + // Initialize tracker + tracker.init(); + + // Create a replication round + long roundStartTime = 1704153600000L; + long roundEndTime = roundStartTime + TimeUnit.MINUTES.toMillis(1); + ReplicationRound targetRound = new ReplicationRound(roundStartTime, roundEndTime); + + // Get the shard directory for this round + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + Path shardDirectory = shardManager.getShardDirectory(targetRound); + + // Create the shard directory + localFs.mkdirs(shardDirectory); + + // Create valid files + Path validFile1 = new Path(shardDirectory, "1704153600000_rs1.plog"); + Path validFile2 = new Path(shardDirectory, "1704153630000_rs2.plog"); + + // Create invalid files (wrong extension, invalid timestamp format) + Path invalidFile1 = new Path(shardDirectory, "1704153600000_rs1.txt"); + Path invalidFile2 = new Path(shardDirectory, "invalid_timestamp_rs2.plog"); + Path invalidFile3 = new Path(shardDirectory, "1704153600000_rs3.log"); + + // Create all files + localFs.create(validFile1, true).close(); + localFs.create(validFile2, true).close(); + localFs.create(invalidFile1, true).close(); + localFs.create(invalidFile2, true).close(); + localFs.create(invalidFile3, true).close(); + + // Call getNewFilesForRound + List result = tracker.getNewFilesForRound(targetRound); + + // Verify file system operation counts + Mockito.verify(mockFs, times(1)).listStatus(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardDirectory)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(shardDirectory)); + + // Prepare expected set of valid file paths + Set expectedPaths = new HashSet<>(); + expectedPaths.add(validFile1.toString()); + expectedPaths.add(validFile2.toString()); + + // Create actual set of paths + Set actualPaths = + result.stream().map(path -> path.toUri().getPath()).collect(Collectors.toSet()); + + // Verify only valid files are returned + assertEquals("Should return exactly 2 valid files", expectedPaths.size(), actualPaths.size()); + assertEquals("File paths do not match", expectedPaths, actualPaths); + } + + @Test + public void testGetInProgressFiles() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + // Create valid files in in-progress directory + Path validFile1 = new Path(inProgressDir, "1704153600000_rs1.plog"); + Path validFile2 = new Path(inProgressDir, "1704153630000_rs2.plog"); + Path validFile3 = new Path(inProgressDir, "1704153659000_rs3.plog"); + + // Create invalid files (wrong extension, invalid format) + Path invalidFile1 = new Path(inProgressDir, "1704153600000_rs1.txt"); + Path invalidFile3 = new Path(inProgressDir, "1704153600000_rs3.log"); + + // Create all files + localFs.create(validFile1, true).close(); + localFs.create(validFile2, true).close(); + localFs.create(validFile3, true).close(); + localFs.create(invalidFile1, true).close(); + localFs.create(invalidFile3, true).close(); + + // Call getInProgressFiles + List result = tracker.getInProgressFiles(); + + // Verify file system operation counts + Mockito.verify(mockFs, times(1)).listStatus(Mockito.any(Path.class)); + // exists should be called twice for in-progress directory, first during init and second during + // getInProgressFiles + Mockito.verify(mockFs, times(2)).exists(Mockito.eq(inProgressDir)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(inProgressDir)); + + // Prepare expected set of valid file paths + Set expectedPaths = new HashSet<>(); + expectedPaths.add(validFile1.toString()); + expectedPaths.add(validFile2.toString()); + expectedPaths.add(validFile3.toString()); + + // Create actual set of paths + Set actualPaths = + result.stream().map(path -> path.toUri().getPath()).collect(Collectors.toSet()); + + // Verify only valid files are returned + assertEquals("Should return exactly 3 valid files", expectedPaths.size(), actualPaths.size()); + assertEquals("File paths do not match", expectedPaths, actualPaths); + } + + @Test + public void testGetInProgressFilesForEmptyDirectory() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + // Ensure directory exists but is empty + assertTrue("In-progress directory should exist", localFs.exists(inProgressDir)); + assertTrue("In-progress directory should be a directory", localFs.isDirectory(inProgressDir)); + + // Call getInProgressFiles + List result = tracker.getInProgressFiles(); + + // Verify file system operation counts + Mockito.verify(mockFs, times(1)).listStatus(Mockito.any(Path.class)); + // exists should be called twice for in-progress directory, first during init and second during + // getInProgressFiles + Mockito.verify(mockFs, times(2)).exists(Mockito.eq(inProgressDir)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(inProgressDir)); + + // Verify empty list is returned + assertTrue("Should return empty list for empty directory", result.isEmpty()); + } + + @Test + public void testGetInProgressFilesForNonExistentDirectory() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + // Delete the in-progress directory to make it non-existent + localFs.delete(inProgressDir, true); + assertFalse("In-progress directory should not exist", localFs.exists(inProgressDir)); + + // Call getInProgressFiles + List result = tracker.getInProgressFiles(); + + // Verify file system operation counts + // listStatus() should not be called when directory doesn't exist + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + // exists should be called twice for in-progress directory, first during init and second during + // getInProgressFiles + Mockito.verify(mockFs, times(2)).exists(Mockito.eq(inProgressDir)); + + // Verify empty list is returned + assertTrue("Should return empty list for non-existent directory", result.isEmpty()); + } + + @Test + public void testGetNewFiles() throws IOException { + // Initialize tracker + tracker.init(); + + // Get all shard paths + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + List allShardPaths = shardManager.getAllShardPaths(); + + // Create files in multiple shards + Path shard0Path = allShardPaths.get(0); + Path shard1Path = allShardPaths.get(1); + Path shard2Path = allShardPaths.get(2); + + // Create valid files in shard 0 + Path validFile1 = new Path(shard0Path, "1704153600000_rs1.plog"); + Path validFile2 = new Path(shard0Path, "1704153630000_rs2.plog"); + + // Create valid files in shard 1 + Path validFile3 = new Path(shard1Path, "1704153660000_rs3.plog"); + Path validFile4 = new Path(shard1Path, "1704153690000_rs4.plog"); + + // Create valid files in shard 2 + Path validFile5 = new Path(shard2Path, "1704153720000_rs5.plog"); + + // Create invalid files in shards + Path invalidFile1 = new Path(shard0Path, "1704153600000_rs1.txt"); + Path invalidFile3 = new Path(shard2Path, "1704153720000_rs3.log"); + + // Create all directories and files + localFs.mkdirs(shard0Path); + localFs.mkdirs(shard1Path); + localFs.mkdirs(shard2Path); + + // Create valid files + localFs.create(validFile1, true).close(); + localFs.create(validFile2, true).close(); + localFs.create(validFile3, true).close(); + localFs.create(validFile4, true).close(); + localFs.create(validFile5, true).close(); + + // Create invalid files + localFs.create(invalidFile1, true).close(); + localFs.create(invalidFile3, true).close(); + + // Call getNewFiles + List result = tracker.getNewFiles(); + + // Verify file system operation counts + // getNewFiles calls exists() and listStatus() for each shard path + // There are 3 shards for which directory is created, so exists() should be called once for each + // shard and once for in-progress directory during creation + Mockito + .verify(mockFs, times(ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS + 1)) + .exists(Mockito.any(Path.class)); + // listStatus() should be called only for 3 shards + Mockito.verify(mockFs, times(3)).listStatus(Mockito.any(Path.class)); + + for (int i = 0; i < 3; i++) { + Path shardPath = allShardPaths.get(i); + // Verify file system operations for each shard path + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardPath)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(shardPath)); + } + + for (int i = 4; i < ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS; i++) { + Path shardPath = allShardPaths.get(i); + // Verify file system operations for each shard path + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardPath)); + } + + // Prepare expected set of valid file paths + Set expectedPaths = new HashSet<>(); + expectedPaths.add(validFile1.toString()); + expectedPaths.add(validFile2.toString()); + expectedPaths.add(validFile3.toString()); + expectedPaths.add(validFile4.toString()); + expectedPaths.add(validFile5.toString()); + + // Create actual set of paths + Set actualPaths = + result.stream().map(path -> path.toUri().getPath()).collect(Collectors.toSet()); + + // Verify all valid files from all shards are returned + assertEquals("Should return exactly 5 valid files from all shards", expectedPaths.size(), + actualPaths.size()); + assertEquals("File paths do not match", expectedPaths, actualPaths); + } + + @Test + public void testGetNewFilesForEmptyShards() throws IOException { + // Initialize tracker + tracker.init(); + + // Get all shard paths + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + List allShardPaths = shardManager.getAllShardPaths(); + + // Create directories but leave them empty + for (Path shardPath : allShardPaths) { + localFs.mkdirs(shardPath); + } + + // Call getNewFiles + List result = tracker.getNewFiles(); + + // Verify file system operation counts + // getNewFiles calls exists() and listStatus() for each shard path + // All shard directories exist but are empty, so exists() should be called for all shards and + // once for in-progress directory during creation + Mockito + .verify(mockFs, times(ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS + 1)) + .exists(Mockito.any(Path.class)); + // listStatus() should be called for all shards + Mockito.verify(mockFs, times(ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS)) + .listStatus(Mockito.any(Path.class)); + + for (int i = 0; i < ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS; i++) { + Path shardPath = allShardPaths.get(i); + // Verify file system operations for each shard path + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardPath)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(shardPath)); + } + + // Verify empty list is returned + assertTrue("Should return empty list for empty shards", result.isEmpty()); + } + + @Test + public void testGetNewFilesForNonExistentShards() throws IOException { + // Initialize tracker + tracker.init(); + + // Get all shard paths + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + List allShardPaths = shardManager.getAllShardPaths(); + + // Assert that no shard directories exist + for (Path shardPath : allShardPaths) { + assertFalse("Shard directory should not exist: " + shardPath, localFs.exists(shardPath)); + } + + // Call getNewFiles + List result = tracker.getNewFiles(); + + // Verify file system operation counts + // getNewFiles calls exists() for each shard path + // All shard directories don't exist, so exists() should be called for all shards and once for + // in-progress directory during creation + Mockito + .verify(mockFs, times(ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS + 1)) + .exists(Mockito.any(Path.class)); + // listStatus() should not be called when directories don't exist + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + + for (int i = 0; i < ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS; i++) { + Path shardPath = allShardPaths.get(i); + // Verify file system operations for each shard path + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(shardPath)); + // listStatus() should not be called for non-existent directories + } + + // Verify empty list is returned + assertTrue("Should return empty list for non-existent shards", result.isEmpty()); + } + + @Test + public void testMarkInProgressForNewFile() throws IOException { + // Initialize tracker + tracker.init(); + + // Create a file in a shard directory (without UUID) + ReplicationShardDirectoryManager shardManager = tracker.getReplicationShardDirectoryManager(); + List allShardPaths = shardManager.getAllShardPaths(); + Path shardPath = allShardPaths.get(0); + localFs.mkdirs(shardPath); + + // Create original file without UUID + Path originalFile = new Path(shardPath, "1704153600000_rs1.plog"); + localFs.create(originalFile, true).close(); + + // Verify original file exists + assertTrue("Original file should exist", localFs.exists(originalFile)); + + // Call markInProgress + Optional result = tracker.markInProgress(originalFile); + + // Verify file system operation counts + // markInProgress involves moving file from shard directory to in-progress directory + // It should call exists() for only in progress directory (during init), rename() to move file + Mockito.verify(mockFs, times(1)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(tracker.getInProgressDirPath())); + Mockito.verify(mockFs, times(1)).rename(Mockito.any(Path.class), Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).rename(Mockito.eq(originalFile), Mockito.any(Path.class)); + // Ensure no listStatus() is called + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + + // Verify operation was successful + assertTrue("markInProgress should be successful", result.isPresent()); + + // Verify original file no longer exists + assertFalse("Original file should no longer exist", localFs.exists(originalFile)); + + // Verify file was moved to in-progress directory with UUID + Path inProgressDir = tracker.getInProgressDirPath(); + FileStatus[] files = localFs.listStatus(inProgressDir); + assertEquals("Should have exactly one file in in-progress directory", 1, files.length); + + // Verify the new file has UUID format and is in in-progress directory + String newFileName = files[0].getPath().getName(); + assertTrue("New file should have UUID suffix", newFileName.matches( + "1704153600000_rs1_[a-f0-9]{8}-[a-f0-9]{4}-[a-f0-9]{4}-[a-f0-9]{4}-[a-f0-9]{12}\\.plog")); + + // Assert that renamed file is in in-progress directory + Path renamedFile = files[0].getPath(); + assertTrue("Renamed file should be in in-progress directory", + renamedFile.getParent().toUri().getPath().equals(tracker.getInProgressDirPath().toString())); + + // Assert that renamed file has same prefix as original file + String originalFileName = originalFile.getName(); + String originalPrefix = originalFileName.substring(0, originalFileName.lastIndexOf('.')); + assertTrue("Renamed file should have same prefix as original file", + newFileName.startsWith(originalPrefix + "_")); + } + + @Test + public void testMarkInProgressForAlreadyInProgressFile() throws IOException { + // Initialize tracker + tracker.init(); + + // Create a file in in-progress directory with existing UUID + Path inProgressDir = tracker.getInProgressDirPath(); + String existingUUID = "12345678-1234-1234-1234-123456789abc"; + Path originalFile = new Path(inProgressDir, "1704153600000_rs1_" + existingUUID + ".plog"); + localFs.create(originalFile, true).close(); + + // Verify original file exists + assertTrue("Original file should exist", localFs.exists(originalFile)); + + // Call markInProgress + Optional result = tracker.markInProgress(originalFile); + + // Verify file system operation counts + // markInProgress involves re-naming file int the in-progress directory + // It should call exists() for only in progress directory (during init), rename() to move file + Mockito.verify(mockFs, times(1)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(tracker.getInProgressDirPath())); + Mockito.verify(mockFs, times(1)).rename(Mockito.any(Path.class), Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).rename(Mockito.eq(originalFile), Mockito.any(Path.class)); + // Ensure no listStatus() is called + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + + // Verify operation was successful + assertTrue("markInProgress should be successful", result.isPresent()); + + // Verify original file no longer exists + assertFalse("Original file should no longer exist", localFs.exists(originalFile)); + + // Verify new file exists in same directory with new UUID + FileStatus[] files = localFs.listStatus(inProgressDir); + assertEquals("Should have exactly one file in in-progress directory", 1, files.length); + + // Verify the new file has different UUID + String newFileName = files[0].getPath().getName(); + assertTrue("New file should have UUID suffix", newFileName.matches( + "1704153600000_rs1_[a-f0-9]{8}-[a-f0-9]{4}-[a-f0-9]{4}-[a-f0-9]{4}-[a-f0-9]{12}\\.plog")); + assertFalse("New file should have different UUID", newFileName.contains(existingUUID)); + + // Assert that renamed file has same prefix as original file + String originalFileName = originalFile.getName(); + String originalPrefix = originalFileName.substring(0, originalFileName.lastIndexOf('_')); + assertTrue("Renamed file should have same prefix as original file", + newFileName.startsWith(originalPrefix + "_")); + } + + @Test + public void testMarkInProgressForNonExistentFile() throws IOException { + // Initialize tracker + tracker.init(); + + // Create a non-existent file path + Path nonExistentFile = new Path(tracker.getInProgressDirPath(), "non_existent_file.plog"); + + // Verify file doesn't exist + assertFalse("File should not exist", localFs.exists(nonExistentFile)); + + // Call markInProgress + Optional result = tracker.markInProgress(nonExistentFile); + + // Verify file system operation counts + Mockito.verify(mockFs, times(1)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(tracker.getInProgressDirPath())); + Mockito.verify(mockFs, times(1)).rename(Mockito.any(Path.class), Mockito.any(Path.class)); + // Ensure no listStatus() is called + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + + // Verify operation failed + assertFalse("markInProgress should return false for non-existent file", result.isPresent()); + } + + @Test + public void testMarkCompletedSuccessfulDeletion() throws IOException { + // Initialize tracker + tracker.init(); + + // Create original file in shard directory + Path originalFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789xyz.plog"); + localFs.create(originalFile, true).close(); + + // Verify original file exists + assertTrue("Original file should exist", localFs.exists(originalFile)); + + // Call markCompleted on the original file + boolean result = tracker.markCompleted(originalFile); + + // Verify file system operation counts + // markCompleted involves deleting the file from in-progress directory + // It should call exists() for in-progress directory (during init), delete() to remove file + Mockito.verify(mockFs, times(1)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).exists(Mockito.eq(tracker.getInProgressDirPath())); + Mockito.verify(mockFs, times(1)).delete(Mockito.any(Path.class), Mockito.eq(false)); + Mockito.verify(mockFs, times(1)).delete(Mockito.eq(originalFile), Mockito.eq(false)); + // Ensure no listStatus() is called + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + + // Verify operation was successful + assertTrue("markCompleted should return true for successful deletion", result); + + // Verify original file no longer exists + assertFalse("Original file should no longer exist", localFs.exists(originalFile)); + } + + @Test + public void testMarkCompletedIntermittentDeletionFailure() throws IOException { + // Initialize tracker + tracker.init(); + + // Create original file in in-progress directory with UUID + Path originalFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789pqr.plog"); + localFs.create(originalFile, true).close(); + + // Verify original file exists + assertTrue("Original file should exist", localFs.exists(originalFile)); + + // Create a spy on the real FileSystem that throws IOException for first 2 attempts, then + // succeeds + FileSystem mockFs = spy(localFs); + doThrow(new IOException("Simulated IO error")).doThrow(new IOException("Simulated IO error")) + .doReturn(true).when(mockFs).delete( + Mockito.argThat(path -> path.getName().equals(originalFile.getName())), Mockito.eq(false)); + + // Replace the tracker's filesystem with our mock + tracker.close(); + tracker = createReplicationLogFileTracker(conf, haGroupName, mockFs, rootURI); + tracker.init(); + + // Call markCompleted on the original file + boolean result = tracker.markCompleted(originalFile); + + // Verify file system operation counts + // markCompleted with intermittent failure involves retrying delete operations + // It should call exists() for in-progress directory (during init), and 2 times while fetching + // in progress files + Mockito.verify(mockFs, times(3)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(3)).exists(Mockito.eq(tracker.getInProgressDirPath())); + // delete() should be called 3 times (2 failures + 1 success) + Mockito.verify(mockFs, times(3)).delete(Mockito.any(Path.class), Mockito.eq(false)); + Mockito.verify(mockFs, times(3)).delete( + Mockito.argThat(path -> path.getName().equals(originalFile.getName())), Mockito.eq(false)); + // Ensure listStatus() is called exactly twice on in progress directory + Mockito.verify(mockFs, times(2)).listStatus(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(2)).listStatus(Mockito.eq(tracker.getInProgressDirPath())); + + // Verify operation was successful after retries + assertTrue("markCompleted should return true after successful retry", result); + + // clean up + tracker.close(); + } + + @Test + public void testMarkCompletedForPersistentDeletionFailure() throws IOException { + + // Create a spy on the real FileSystem that throws IOException for all attempts + FileSystem mockFs = spy(localFs); + + // Replace the tracker's filesystem with our spy + tracker = createReplicationLogFileTracker(conf, haGroupName, mockFs, rootURI); + tracker.init(); + + // Create original file in in-progress directory with UUID + Path originalFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789def.plog"); + + // Set up the mock BEFORE creating the file + // Mock delete for any file with the same name to throw IOException + doThrow(new IOException("Simulated IO error")).doThrow(new IOException("Simulated IO error")) + .doThrow(new IOException("Simulated IO error")).doThrow(new IOException("Simulated IO error")) + .when(mockFs).delete(Mockito.argThat(path -> path.getName().equals(originalFile.getName())), + Mockito.eq(false)); + + // Create the file using the spy filesystem + mockFs.create(originalFile, true).close(); + + // Call markCompleted on the original file + boolean result = tracker.markCompleted(originalFile); + + // Verify file system operation counts + // markCompleted with persistent failure involves retrying delete operations until all retries + // exhausted + // It should call exists() for in-progress directory (during init), and 3 times while fetching + // in progress files during retries + Mockito.verify(mockFs, times(4)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(4)).exists(Mockito.eq(tracker.getInProgressDirPath())); + // delete() should be called 4 times (1 initial + 3 retries, all failures) + Mockito.verify(mockFs, times(4)).delete(Mockito.any(Path.class), Mockito.eq(false)); + Mockito.verify(mockFs, times(4)).delete( + Mockito.argThat(path -> path.getName().equals(originalFile.getName())), Mockito.eq(false)); + // Ensure listStatus() is called exactly 3 times during retries on in progress directory + Mockito.verify(mockFs, times(3)).listStatus(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(3)).listStatus(Mockito.eq(tracker.getInProgressDirPath())); + + // Verify original file exists + assertTrue("Original file should exist", mockFs.exists(originalFile)); + + // Verify operation failed after all retries + assertFalse("markCompleted should return false after all retries fail", result); + + // clean up + tracker.close(); + } + + @Test + public void testMarkCompletedWhenFileDeletedByOtherProcessAlready() throws IOException { + // Initialize tracker + tracker.init(); + + // Create original file in in-progress directory with UUID + Path fileToBeDeleted = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789jkl.plog"); + localFs.create(fileToBeDeleted, true).close(); + + // Create in-progress file with different prefix + String uuid = "12345678-1234-1234-1234-123456789abc"; + Path anotherInProgressFile = + new Path(tracker.getInProgressDirPath(), "1704153600001_rs2_" + uuid + ".plog"); + localFs.create(anotherInProgressFile, true).close(); + + // Delete the first file + localFs.delete(fileToBeDeleted, true); + + // Assert first file is deleted and second file should exist + assertFalse("Original file to be deleted should not exist", localFs.exists(fileToBeDeleted)); + assertTrue("Second file should exist", localFs.exists(anotherInProgressFile)); + + // Call markCompleted on the deleted file + // Since the file is already deleted, it should search for other files with same prefix + // but find none, so it should return true (assuming file was deleted by another process) + boolean result = tracker.markCompleted(fileToBeDeleted); + + // Verify file system operation counts + // markCompleted when file is already deleted involves checking in-progress directory for + // matching files + // It should call exists() for in-progress directory (during init), and once while fetching in + // progress files + Mockito.verify(mockFs, times(2)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(2)).exists(Mockito.eq(tracker.getInProgressDirPath())); + // No delete() operations should be performed only once on input file + Mockito.verify(mockFs, times(1)).delete(Mockito.any(Path.class), Mockito.eq(false)); + Mockito.verify(mockFs, times(1)).delete(Mockito.eq(fileToBeDeleted), Mockito.eq(false)); + // Ensure listStatus() is called exactly once to check for matching files + Mockito.verify(mockFs, times(1)).listStatus(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(tracker.getInProgressDirPath())); + + // Verify operation was successful (no matching files found) + assertTrue( + "markCompleted should return true when file is already deleted and no matching files found", + result); + + // Verify the other in-progress file still exists (was not affected) + assertTrue("Other in-progress file should still exist", localFs.exists(anotherInProgressFile)); + } + + @Test + public void testMarkCompletedWithMultipleMatchingFiles() throws IOException { + // Initialize tracker + tracker.init(); + + // Create original file in in-progress directory with UUID + Path originalFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789ghi.plog"); + localFs.create(originalFile, true).close(); + + // Create multiple in-progress files with same prefix + Path inProgressDir = tracker.getInProgressDirPath(); + String uuid1 = "12345678-1234-1234-1234-123456789abc"; + String uuid2 = "87654321-4321-4321-4321-cba987654321"; + Path inProgressFile1 = new Path(inProgressDir, "1704153600000_rs1_" + uuid1 + ".plog"); + Path inProgressFile2 = new Path(inProgressDir, "1704153600000_rs1_" + uuid2 + ".plog"); + localFs.create(inProgressFile1, true).close(); + localFs.create(inProgressFile2, true).close(); + + // Delete the first file + localFs.delete(originalFile, true); + + // Verify files exist + assertFalse("Original file should not exist", localFs.exists(originalFile)); + assertTrue("In-progress file 1 should exist", localFs.exists(inProgressFile1)); + assertTrue("In-progress file 2 should exist", localFs.exists(inProgressFile2)); + + // Call markCompleted on the original file + // Should return false due to multiple matching files during retries + boolean result = tracker.markCompleted(originalFile); + + // Verify file system operation counts + // markCompleted with multiple matching files involves checking in-progress directory for + // matching files + // It should call exists() for in-progress directory (during init), and once while fetching in + // progress files + Mockito.verify(mockFs, times(2)).exists(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(2)).exists(Mockito.eq(tracker.getInProgressDirPath())); + // delete() should be called once on input file + Mockito.verify(mockFs, times(1)).delete(Mockito.any(Path.class), Mockito.eq(false)); + Mockito.verify(mockFs, times(1)).delete(Mockito.eq(originalFile), Mockito.eq(false)); + // Ensure listStatus() is called exactly once to check for matching files + Mockito.verify(mockFs, times(1)).listStatus(Mockito.any(Path.class)); + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(tracker.getInProgressDirPath())); + + // Verify operation failed + assertFalse("markCompleted should return false for multiple matching files", result); + + // Verify the files still exist + assertTrue("In-progress file 1 should still exist", localFs.exists(inProgressFile1)); + assertTrue("In-progress file 2 should still exist", localFs.exists(inProgressFile2)); + } + + @Test + public void testGetFileTimestamp() throws IOException { + // Initialize tracker + tracker.init(); + + // Test with new file (without UUID) + Path newFile = new Path(tracker.getInProgressDirPath(), "1704153600000_rs1.plog"); + long newFileTimestamp = tracker.getFileTimestamp(newFile); + assertEquals("New file timestamp should be extracted correctly", 1704153600000L, + newFileTimestamp); + + // Test with in-progress file (with UUID) + Path inProgressFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789abc.plog"); + long inProgressFileTimestamp = tracker.getFileTimestamp(inProgressFile); + assertEquals("In-progress file timestamp should be extracted correctly", 1704153600000L, + inProgressFileTimestamp); + + // Verify both timestamps are the same + assertEquals("Both files should have the same timestamp", newFileTimestamp, + inProgressFileTimestamp); + + // Test with different timestamp + Path anotherValidFile = new Path(tracker.getInProgressDirPath(), + "1704161280000_rs2_87654321-4321-4321-4321-cba987654321.plog"); + long anotherTimestamp = tracker.getFileTimestamp(anotherValidFile); + assertEquals("Should extract correct timestamp", 1704161280000L, anotherTimestamp); + } + + @Test(expected = NumberFormatException.class) + public void testGetFileTimestampWithInvalidFormat() throws IOException { + // Initialize tracker + tracker.init(); + + // Test file with invalid timestamp format (non-numeric) + Path invalidFile = new Path(tracker.getInProgressDirPath(), + "invalid_timestamp_rs1_12345678-1234-1234-1234-123456789abc.plog"); + tracker.getFileTimestamp(invalidFile); + // Should throw NumberFormatException + } + + @Test(expected = NumberFormatException.class) + public void testGetFileTimestampWithMissingTimestamp() throws IOException { + // Initialize tracker + tracker.init(); + + // Test file with missing timestamp part + Path invalidFile = + new Path(tracker.getInProgressDirPath(), "rs1_12345678-1234-1234-1234-123456789abc.plog"); + tracker.getFileTimestamp(invalidFile); + } + + @Test + public void testIsValidLogFile() throws IOException { + // Initialize tracker + tracker.init(); + + // 1. New File (valid) - without UUID, with .plog extension + Path newFile = new Path(tracker.getInProgressDirPath(), "1704153600000_rs1.plog"); + assertTrue("New file with .plog extension should be valid", tracker.isValidLogFile(newFile)); + + // 2. InProgressFile (valid) - with UUID, with .plog extension + Path inProgressFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789abc.plog"); + assertTrue("In-progress file with .plog extension should be valid", + tracker.isValidLogFile(inProgressFile)); + + // 3. Valid file (invalid extension) - without UUID, without .plog extension + Path newFileInvalidExt = new Path(tracker.getInProgressDirPath(), "1704153600000_rs1.txt"); + assertFalse("New file without .plog extension should be invalid", + tracker.isValidLogFile(newFileInvalidExt)); + + // 4. InProgress (invalid extension) - with UUID, without .plog extension + Path inProgressFileInvalidExt = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789abc.txt"); + assertFalse("In-progress file without .plog extension should be invalid", + tracker.isValidLogFile(inProgressFileInvalidExt)); + + // clean up + tracker.close(); + } + + @Test + public void testGetFileUUID() throws IOException { + // Initialize tracker + tracker.init(); + + // 1. For new File - without UUID + Path newFile = new Path(tracker.getInProgressDirPath(), "1704153600000_rs1.plog"); + Optional newFileUUID = tracker.getFileUUID(newFile); + assertFalse("New file without UUID should return empty Optional", newFileUUID.isPresent()); + + // 2. For in-progress file - with UUID + Path inProgressFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_12345678-1234-1234-1234-123456789abc.plog"); + Optional inProgressFileUUID = tracker.getFileUUID(inProgressFile); + assertTrue("In-progress file with UUID should return present Optional", + inProgressFileUUID.isPresent()); + assertEquals("In-progress file UUID should be extracted correctly", + "12345678-1234-1234-1234-123456789abc", inProgressFileUUID.get()); + + // Test with different UUID + Path anotherInProgressFile = new Path(tracker.getInProgressDirPath(), + "1704153600000_rs1_87654321-4321-4321-4321-cba987654321.plog"); + Optional anotherUUID = tracker.getFileUUID(anotherInProgressFile); + assertTrue("Another in-progress file with UUID should return present Optional", + anotherUUID.isPresent()); + assertEquals("Another in-progress file UUID should be extracted correctly", + "87654321-4321-4321-4321-cba987654321", anotherUUID.get()); + } + + @Test + public void testGetOlderInProgressFiles() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + // Create files with different timestamps + long baseTimestamp = 1704153600000L; // 2024-01-02 00:00:00 + long thresholdTimestamp = baseTimestamp + TimeUnit.HOURS.toMillis(1); // 1 hour later + + // Files older than threshold (should be returned) + Path oldFile1 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.MINUTES.toMillis(30)) + "_rs1.plog"); + Path oldFile2 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.MINUTES.toMillis(45)) + "_rs2.plog"); + + // Files newer than threshold (should not be returned) + Path newFile1 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.HOURS.toMillis(2)) + "_rs3.plog"); + Path newFile2 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.HOURS.toMillis(3)) + "_rs4.plog"); + + // Invalid files (should be skipped) + Path invalidFile = new Path(inProgressDir, "invalid_timestamp_rs5.plog"); + + // Create all files + localFs.create(oldFile1, true).close(); + localFs.create(oldFile2, true).close(); + localFs.create(newFile1, true).close(); + localFs.create(newFile2, true).close(); + localFs.create(invalidFile, true).close(); + + // Call getOlderInProgressFiles + List result = tracker.getOlderInProgressFiles(thresholdTimestamp); + + // Verify file system operations + Mockito.verify(mockFs, times(1)).listStatus(Mockito.eq(inProgressDir)); + + // Verify results - check by filename instead of full path comparison + assertEquals("Should return 2 old files", 2, result.size()); + + // Convert to sets of filenames for comparison + Set resultFilenames = result.stream().map(Path::getName).collect(Collectors.toSet()); + + assertTrue("Should contain oldFile1", resultFilenames.contains(oldFile1.getName())); + assertTrue("Should contain oldFile2", resultFilenames.contains(oldFile2.getName())); + assertFalse("Should not contain newFile1", resultFilenames.contains(newFile1.getName())); + assertFalse("Should not contain newFile2", resultFilenames.contains(newFile2.getName())); + assertFalse("Should not contain invalidFile", resultFilenames.contains(invalidFile.getName())); + } + + @Test + public void testGetOlderInProgressFilesWithNoOldFiles() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + // Create files all newer than threshold + long baseTimestamp = 1704153600000L; + long thresholdTimestamp = baseTimestamp + TimeUnit.HOURS.toMillis(1); + + Path newFile1 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.HOURS.toMillis(2)) + "_rs1.plog"); + Path newFile2 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.HOURS.toMillis(3)) + "_rs2.plog"); + + localFs.create(newFile1, true).close(); + localFs.create(newFile2, true).close(); + + // Call getOlderInProgressFiles + List result = tracker.getOlderInProgressFiles(thresholdTimestamp); + + // Verify empty list is returned + assertTrue("Should return empty list when no files are older than threshold", result.isEmpty()); + } + + @Test + public void testGetOlderInProgressFilesForEmptyDirectory() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + // Ensure directory exists but is empty + assertTrue("In-progress directory should exist", localFs.exists(inProgressDir)); + + // Call getOlderInProgressFiles + List result = tracker.getOlderInProgressFiles(1704153600000L); + + // Verify empty list is returned + assertTrue("Should return empty list for empty directory", result.isEmpty()); + } + + @Test + public void testGetOlderInProgressFilesForNonExistentDirectory() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + // Delete the in-progress directory to make it non-existent + localFs.delete(inProgressDir, true); + assertFalse("In-progress directory should not exist", localFs.exists(inProgressDir)); + + // Call getOlderInProgressFiles + List result = tracker.getOlderInProgressFiles(1704153600000L); + + // Verify file system operations + Mockito.verify(mockFs, times(0)).listStatus(Mockito.any(Path.class)); + + // Verify empty list is returned + assertTrue("Should return empty list for non-existent directory", result.isEmpty()); + } + + @Test + public void testGetOlderInProgressFilesWithInvalidFiles() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + long baseTimestamp = 1704153600000L; + long thresholdTimestamp = baseTimestamp + TimeUnit.HOURS.toMillis(1); + + // Valid old file (should be returned) + Path validOldFile = + new Path(inProgressDir, (baseTimestamp + TimeUnit.MINUTES.toMillis(30)) + "_rs1.plog"); + + // Invalid files (should be skipped) + Path invalidFile1 = new Path(inProgressDir, "invalid_timestamp_rs2.plog"); + Path invalidFile2 = new Path(inProgressDir, "not_a_timestamp_rs3.plog"); + Path invalidFile3 = new Path(inProgressDir, "1704153600000_rs4.txt"); // wrong extension + + localFs.create(validOldFile, true).close(); + localFs.create(invalidFile1, true).close(); + localFs.create(invalidFile2, true).close(); + localFs.create(invalidFile3, true).close(); + + // Call getOlderInProgressFiles + List result = tracker.getOlderInProgressFiles(thresholdTimestamp); + + // Verify results - should only contain the valid old file + assertEquals("Should return 1 valid old file", 1, result.size()); + + // Convert to set of filenames for comparison + Set resultFilenames = result.stream().map(Path::getName).collect(Collectors.toSet()); + + assertTrue("Should contain validOldFile", resultFilenames.contains(validOldFile.getName())); + assertFalse("Should not contain invalidFile1", + resultFilenames.contains(invalidFile1.getName())); + assertFalse("Should not contain invalidFile2", + resultFilenames.contains(invalidFile2.getName())); + assertFalse("Should not contain invalidFile3", + resultFilenames.contains(invalidFile3.getName())); + } + + @Test + public void testGetOlderInProgressFilesWithExactThreshold() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + long baseTimestamp = 1704153600000L; + long thresholdTimestamp = baseTimestamp + TimeUnit.HOURS.toMillis(1); + + // File with timestamp exactly at threshold (should NOT be returned - we want older than + // threshold) + Path fileAtThreshold = new Path(inProgressDir, thresholdTimestamp + "_rs1.plog"); + + // File with timestamp just before threshold (should be returned) + Path fileJustBeforeThreshold = new Path(inProgressDir, (thresholdTimestamp - 1) + "_rs2.plog"); + + localFs.create(fileAtThreshold, true).close(); + localFs.create(fileJustBeforeThreshold, true).close(); + + // Call getOlderInProgressFiles + List result = tracker.getOlderInProgressFiles(thresholdTimestamp); + + // Verify results - should only contain the file just before threshold + assertEquals("Should return 1 file older than threshold", 1, result.size()); + + // Convert to set of filenames for comparison + Set resultFilenames = result.stream().map(Path::getName).collect(Collectors.toSet()); + + assertTrue("Should contain fileJustBeforeThreshold", + resultFilenames.contains(fileJustBeforeThreshold.getName())); + assertFalse("Should not contain fileAtThreshold", + resultFilenames.contains(fileAtThreshold.getName())); + } + + @Test + public void testGetOlderInProgressFilesWithMixedFileTypes() throws IOException { + // Initialize tracker + tracker.init(); + + // Get the in-progress directory path + Path inProgressDir = tracker.getInProgressDirPath(); + + long baseTimestamp = 1704153600000L; + long thresholdTimestamp = baseTimestamp + TimeUnit.HOURS.toMillis(1); + + // Valid old files (should be returned) + Path oldFile1 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.MINUTES.toMillis(30)) + "_rs1.plog"); + Path oldFile2 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.MINUTES.toMillis(45)) + "_rs2.plog"); + + // Valid new files (should not be returned) + Path newFile1 = + new Path(inProgressDir, (baseTimestamp + TimeUnit.HOURS.toMillis(2)) + "_rs3.plog"); + + // Invalid files (should be skipped) + Path invalidFile1 = new Path(inProgressDir, "invalid_timestamp_rs4.plog"); + Path invalidFile2 = new Path(inProgressDir, "1704153600000_rs5.txt"); // wrong extension + Path invalidFile3 = new Path(inProgressDir, "not_a_number_rs6.plog"); + + // Create all files + localFs.create(oldFile1, true).close(); + localFs.create(oldFile2, true).close(); + localFs.create(newFile1, true).close(); + localFs.create(invalidFile1, true).close(); + localFs.create(invalidFile2, true).close(); + localFs.create(invalidFile3, true).close(); + + // Call getOlderInProgressFiles + List result = tracker.getOlderInProgressFiles(thresholdTimestamp); + + // Verify results - should only contain the valid old files + assertEquals("Should return 2 valid old files", 2, result.size()); + + // Convert to set of filenames for comparison + Set resultFilenames = result.stream().map(Path::getName).collect(Collectors.toSet()); + + assertTrue("Should contain oldFile1", resultFilenames.contains(oldFile1.getName())); + assertTrue("Should contain oldFile2", resultFilenames.contains(oldFile2.getName())); + assertFalse("Should not contain newFile1", resultFilenames.contains(newFile1.getName())); + assertFalse("Should not contain invalidFile1", + resultFilenames.contains(invalidFile1.getName())); + assertFalse("Should not contain invalidFile2", + resultFilenames.contains(invalidFile2.getName())); + assertFalse("Should not contain invalidFile3", + resultFilenames.contains(invalidFile3.getName())); + } + + private int countDirectories(FileSystem fs, Path path) throws IOException { + if (!fs.exists(path)) { + return 0; + } + int count = 0; + if (fs.isDirectory(path)) { + count = 1; // Count this directory + try { + for (org.apache.hadoop.fs.FileStatus status : fs.listStatus(path)) { + if (status.isDirectory()) { + count += countDirectories(fs, status.getPath()); + } + } + } catch (IOException e) { + // Ignore listing errors for test purposes + } + } + return count; + } + + private ReplicationLogTracker createReplicationLogFileTracker(final Configuration conf, + final String haGroupName, final FileSystem fileSystem, final URI rootURI) { + Path newFilesDirectory = + new Path(new Path(rootURI.getPath(), haGroupName), ReplicationLogReplay.IN_DIRECTORY_NAME); + ReplicationShardDirectoryManager replicationShardDirectoryManager = + new ReplicationShardDirectoryManager(conf, newFilesDirectory); + return new ReplicationLogTracker(conf, haGroupName, fileSystem, + replicationShardDirectoryManager, metrics); + } + +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationShardDirectoryManagerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationShardDirectoryManagerTest.java new file mode 100644 index 00000000000..9bb27e356f6 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationShardDirectoryManagerTest.java @@ -0,0 +1,657 @@ +/* + * 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.phoenix.replication; + +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class ReplicationShardDirectoryManagerTest { + + @ClassRule + public static TemporaryFolder testFolder = new TemporaryFolder(); + + private Configuration conf; + private FileSystem localFs; + private URI standbyUri; + private ReplicationShardDirectoryManager manager; + + @Before + public void setUp() throws IOException { + conf = HBaseConfiguration.create(); + localFs = FileSystem.getLocal(conf); + standbyUri = new Path(testFolder.toString()).toUri(); + conf.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + + // Create manager with default configuration + Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); + manager = new ReplicationShardDirectoryManager(conf, rootPath); + } + + @After + public void tearDown() throws IOException { + localFs.delete(new Path(testFolder.getRoot().toURI()), true); + } + + @Test + public void testGetShardDirectoryWithSpecificConditions() { + // Use a specific day for consistent testing + // 2024-01-01 00:00:00 UTC = 1704067200000L + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + + // Calculate expected base path + String expectedBasePath = testFolder.getRoot().getAbsolutePath() + "/shard"; + + // 1. Timestamp at start of min (00:00:00) + long timestampStartOfMin = dayStart; // 00:00:00 + Path shardPath1 = manager.getShardDirectory(timestampStartOfMin); + String expectedPath1 = expectedBasePath + "/000"; + assertEquals("Timestamp at start of min should map to shard 000", expectedPath1, + shardPath1.toString()); + + // 2. Timestamp at mid of min (00:00:30) + long timestampMidOfMin = dayStart + (30 * 1000L); // 00:00:30 + Path shardPath2 = manager.getShardDirectory(timestampMidOfMin); + String expectedPath2 = expectedBasePath + "/000"; // Still in first 60-second round + assertEquals("Timestamp at mid of min should map to shard 000", expectedPath2, + shardPath2.toString()); + + // 3. Timestamp at just end of min (00:00:59) + long timestampEndOfMin = dayStart + (59 * 1000L); // 00:00:59 + Path shardPath3 = manager.getShardDirectory(timestampEndOfMin); + String expectedPath3 = expectedBasePath + "/000"; // Still in first 60-second round + assertEquals("Timestamp at end of min should map to shard 000", expectedPath3, + shardPath3.toString()); + + // 4. Timestamp at which modulo logic is used (128th round = 128 minutes later) + // 128 * 60 seconds = 7680 seconds + long timestampModuloLogic = dayStart + (128 * 60 * 1000L); // 02:08:00 + Path shardPath4 = manager.getShardDirectory(timestampModuloLogic); + String expectedPath4 = expectedBasePath + "/000"; // Should wrap around to shard 0 + assertEquals("Timestamp at 128th round should wrap to shard 000 due to modulo", expectedPath4, + shardPath4.toString()); + + // Additional test: Second round (00:01:00) + long timestampSecondRound = dayStart + (60 * 1000L); // 00:01:00 + Path shardPath5 = manager.getShardDirectory(timestampSecondRound); + String expectedPath5 = expectedBasePath + "/001"; + assertEquals("Timestamp at second round should map to shard 001", expectedPath5, + shardPath5.toString()); + + // Additional test: 127th round (should be shard 127) + long timestamp127thRound = dayStart + (127 * 60 * 1000L); // 02:07:00 + Path shardPath6 = manager.getShardDirectory(timestamp127thRound); + String expectedPath6 = expectedBasePath + "/127"; + assertEquals("Timestamp at 127th round should map to shard 127", expectedPath6, + shardPath6.toString()); + + // Additional test: 129th round (should wrap to shard 1) + long timestamp129thRound = dayStart + (129 * 60 * 1000L); // 02:09:00 + Path shardPath7 = manager.getShardDirectory(timestamp129thRound); + String expectedPath7 = expectedBasePath + "/001"; + assertEquals("Timestamp at 129th round should wrap to shard 001", expectedPath7, + shardPath7.toString()); + } + + @Test + public void testGetShardDirectoryForTwoDayWindow() { + // Use a specific day for consistent testing + // 2024-01-01 00:00:00 UTC = 1704067200000L + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + + // Calculate expected base path + String expectedBasePath = testFolder.getRoot().getAbsolutePath() + "/shard"; + + // Test for each minute in the 2-day window + for (int day = 0; day < 2; day++) { + for (int hour = 0; hour < 24; hour++) { + for (int minute = 0; minute < 60; minute++) { + long minuteStart = dayStart + (day * 24 * 60 * 60 * 1000L) + (hour * 60 * 60 * 1000L) + + (minute * 60 * 1000L); + + // Calculate expected shard index based on the algorithm + // Convert to seconds since start of day + long secondsSinceEpoch = minuteStart / 1000L; + long secondsSinceStartOfDay = secondsSinceEpoch % TimeUnit.DAYS.toSeconds(1); + int shardIndex = (int) (secondsSinceStartOfDay / 60) % 128; // 60-second rounds, 128 + // shards + + // 1. Test start of minute (XX:XX:00) + long timestampStartOfMin = minuteStart; + Path shardPathStart = manager.getShardDirectory(timestampStartOfMin); + String expectedPathStart = expectedBasePath + "/" + String.format("%03d", shardIndex); + assertEquals( + String.format("Start of minute %02d:%02d:%02d (day %d) should map to shard %03d", hour, + minute, 0, day + 1, shardIndex), + expectedPathStart, shardPathStart.toString()); + + // 2. Test mid of minute (random between 2-58 seconds) + int randomSeconds = 2 + (int) (Math.random() * 57); // Random between 2-58 + long timestampMidOfMin = minuteStart + (randomSeconds * 1000L); + Path shardPathMid = manager.getShardDirectory(timestampMidOfMin); + String expectedPathMid = expectedBasePath + "/" + String.format("%03d", shardIndex); + assertEquals( + String.format("Mid of minute %02d:%02d:%02d (day %d) should map to shard %03d", hour, + minute, randomSeconds, day + 1, shardIndex), + expectedPathMid, shardPathMid.toString()); + + // 3. Test end of minute (XX:XX:59) + long timestampEndOfMin = minuteStart + (59 * 1000L); + Path shardPathEnd = manager.getShardDirectory(timestampEndOfMin); + String expectedPathEnd = expectedBasePath + "/" + String.format("%03d", shardIndex); + assertEquals( + String.format("End of minute %02d:%02d:%02d (day %d) should map to shard %03d", hour, + minute, 59, day + 1, shardIndex), + expectedPathEnd, shardPathEnd.toString()); + } + } + } + } + + @Test + public void testGetShardDirectoryWithReplicationRound() { + // Create a spy of the manager to verify method calls + ReplicationShardDirectoryManager spyManager = spy(manager); + + // Create a replication round + long startTime = 1704110400000L; // 2024-01-01 12:00:00 UTC + long endTime = startTime + (60 * 1000L); // 60 seconds later + ReplicationRound round = new ReplicationRound(startTime, endTime); + + // Call the method that takes ReplicationRound + spyManager.getShardDirectory(round); + + // Verify that it calls the timestamp version with the correct start time + verify(spyManager).getShardDirectory(eq(startTime)); + } + + @Test + public void testGetNearestRoundStartTimestamp() { + // Use a specific day for consistent testing + // 2024-01-01 00:00:00 UTC = 1704067200000L + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + + // Default configuration: 60-second rounds + long roundDurationMs = 60 * 1000L; // 60 seconds in milliseconds + + // Test 1: Exact round start time + long exactRoundStart = dayStart; // 00:00:00 + long result1 = manager.getNearestRoundStartTimestamp(exactRoundStart); + assertEquals("Exact round start time should return itself", exactRoundStart, result1); + + // Test 2: Middle of first round + long midFirstRound = dayStart + (30 * 1000L); // 00:00:30 + long result2 = manager.getNearestRoundStartTimestamp(midFirstRound); + assertEquals("Middle of first round should round down to start", dayStart, result2); + + // Test 3: End of first round + long endFirstRound = dayStart + (59 * 1000L); // 00:00:59 + long result3 = manager.getNearestRoundStartTimestamp(endFirstRound); + assertEquals("End of first round should round down to start", dayStart, result3); + + // Test 4: Start of second round + long startSecondRound = dayStart + (60 * 1000L); // 00:01:00 + long result4 = manager.getNearestRoundStartTimestamp(startSecondRound); + assertEquals("Start of second round should return itself", startSecondRound, result4); + + // Test 5: Middle of second round + long midSecondRound = dayStart + (90 * 1000L); // 00:01:30 + long result5 = manager.getNearestRoundStartTimestamp(midSecondRound); + assertEquals("Middle of second round should round down to start", startSecondRound, result5); + + // Test 6: End of second round + long endSecondRound = dayStart + (119 * 1000L); // 00:01:59 + long result6 = manager.getNearestRoundStartTimestamp(endSecondRound); + assertEquals("End of second round should round down to start", startSecondRound, result6); + + // Test 7: Multiple rounds later + long multipleRoundsLater = dayStart + (300 * 1000L); // 00:05:00 (5 minutes) + long expectedRoundStart = dayStart + (300 * 1000L); // Should be exact + long result7 = manager.getNearestRoundStartTimestamp(multipleRoundsLater); + assertEquals("Multiple rounds later should return exact round start", expectedRoundStart, + result7); + + // Test 8: Just before a round boundary + long justBeforeRound = dayStart + (299 * 1000L); // 00:04:59 + long expectedBeforeRound = dayStart + (240 * 1000L); // 00:04:00 + long result8 = manager.getNearestRoundStartTimestamp(justBeforeRound); + assertEquals("Just before round boundary should round down", expectedBeforeRound, result8); + + // Test 9: Just after a round boundary + long justAfterRound = dayStart + (301 * 1000L); // 00:05:01 + long expectedAfterRound = dayStart + (300 * 1000L); // 00:05:00 + long result9 = manager.getNearestRoundStartTimestamp(justAfterRound); + assertEquals("Just after round boundary should round down", expectedAfterRound, result9); + + // Test 11: Current time (should round down to nearest round) + long currentTime = System.currentTimeMillis(); + long result11 = manager.getNearestRoundStartTimestamp(currentTime); + Assert.assertTrue("Current time should round down to nearest round", result11 <= currentTime); + assertEquals("Result should be a multiple of round duration", 0, (result11 % roundDurationMs)); + } + + @Test + public void testGetNearestRoundStartTimestampWithCustomConfiguration() { + // Test with custom round duration + Configuration customConf = HBaseConfiguration.create(); + customConf + .setInt(ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, 30); + + Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); + ReplicationShardDirectoryManager customManager = + new ReplicationShardDirectoryManager(customConf, rootPath); + + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + + // Test with 30-second rounds + long midRound = dayStart + (45 * 1000L); // 00:00:45 + long result = customManager.getNearestRoundStartTimestamp(midRound); + long expected = dayStart + (30 * 1000L); // 00:00:30 + assertEquals("With 30-second rounds, should round down to 30-second boundary", expected, + result); + + // Test exact 30-second boundary + long exactBoundary = dayStart + (60 * 1000L); // 00:01:00 + long result2 = customManager.getNearestRoundStartTimestamp(exactBoundary); + assertEquals("Exact 30-second boundary should return itself", exactBoundary, result2); + } + + @Test + public void testGetReplicationRoundFromStartTime() { + // Use a specific day for consistent testing + // 2024-01-01 00:00:00 UTC = 1704067200000L + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + + // Default configuration: 60-second rounds + long roundDurationMs = 60 * 1000L; // 60 seconds in milliseconds + + // Test 1: Exact round start time + long exactRoundStart = dayStart; // 00:00:00 + ReplicationRound result1 = manager.getReplicationRoundFromStartTime(exactRoundStart); + assertEquals("Round start time should be validated and unchanged", exactRoundStart, + result1.getStartTime()); + assertEquals("Round end time should be start time + round duration", + exactRoundStart + roundDurationMs, result1.getEndTime()); + + // Test 2: Mid-round timestamp (should be rounded down) + long midRoundTimestamp = dayStart + (30 * 1000L); // 00:00:30 + ReplicationRound result2 = manager.getReplicationRoundFromStartTime(midRoundTimestamp); + long expectedStart2 = dayStart; // Should round down to 00:00:00 + assertEquals("Mid-round timestamp should round down to round start", expectedStart2, + result2.getStartTime()); + assertEquals("Round end time should be start time + round duration", + expectedStart2 + roundDurationMs, result2.getEndTime()); + + // Test 3: End of round timestamp (should round down) + long endRoundTimestamp = dayStart + (59 * 1000L); // 00:00:59 + ReplicationRound result3 = manager.getReplicationRoundFromStartTime(endRoundTimestamp); + long expectedStart3 = dayStart; // Should round down to 00:00:00 + assertEquals("End of round timestamp should round down to round start", expectedStart3, + result3.getStartTime()); + assertEquals("Round end time should be start time + round duration", + expectedStart3 + roundDurationMs, result3.getEndTime()); + + // Test 4: Start of second round + long secondRoundStart = dayStart + (60 * 1000L); // 00:01:00 + ReplicationRound result4 = manager.getReplicationRoundFromStartTime(secondRoundStart); + assertEquals("Second round start time should be validated and unchanged", secondRoundStart, + result4.getStartTime()); + assertEquals("Round end time should be start time + round duration", + secondRoundStart + roundDurationMs, result4.getEndTime()); + + // Test 5: Mid-second round timestamp + long midSecondRound = dayStart + (90 * 1000L); // 00:01:30 + ReplicationRound result5 = manager.getReplicationRoundFromStartTime(midSecondRound); + long expectedStart5 = dayStart + (60 * 1000L); // Should round down to 00:01:00 + assertEquals("Mid-second round timestamp should round down to round start", expectedStart5, + result5.getStartTime()); + assertEquals("Round end time should be start time + round duration", + expectedStart5 + roundDurationMs, result5.getEndTime()); + + // Test 6: Multiple rounds later + long multipleRoundsLater = dayStart + (300 * 1000L); // 00:05:00 + ReplicationRound result6 = manager.getReplicationRoundFromStartTime(multipleRoundsLater); + assertEquals("Multiple rounds later should be validated and unchanged", multipleRoundsLater, + result6.getStartTime()); + assertEquals("Round end time should be start time + round duration", + multipleRoundsLater + roundDurationMs, result6.getEndTime()); + + // Test 7: Just before round boundary + long justBeforeRound = dayStart + (299 * 1000L); // 00:04:59 + ReplicationRound result7 = manager.getReplicationRoundFromStartTime(justBeforeRound); + long expectedStart7 = dayStart + (240 * 1000L); // Should round down to 00:04:00 + assertEquals("Just before round boundary should round down", expectedStart7, + result7.getStartTime()); + assertEquals("Round end time should be start time + round duration", + expectedStart7 + roundDurationMs, result7.getEndTime()); + + // Test 8: Just after round boundary + long justAfterRound = dayStart + (301 * 1000L); // 00:05:01 + ReplicationRound result8 = manager.getReplicationRoundFromStartTime(justAfterRound); + long expectedStart8 = dayStart + (300 * 1000L); // Should round down to 00:05:00 + assertEquals("Just after round boundary should round down", expectedStart8, + result8.getStartTime()); + assertEquals("Round end time should be start time + round duration", + expectedStart8 + roundDurationMs, result8.getEndTime()); + + // Test 10: Current time (should round down to nearest round) + long currentTime = System.currentTimeMillis(); + ReplicationRound result10 = manager.getReplicationRoundFromStartTime(currentTime); + Assert.assertTrue("Current time should round down to nearest round", + result10.getStartTime() <= currentTime); + assertEquals("Round start should be a multiple of round duration", 0, + (result10.getStartTime() % roundDurationMs)); + assertEquals("Round end time should be start time + round duration", + result10.getStartTime() + roundDurationMs, result10.getEndTime()); + } + + @Test + public void testGetReplicationRoundFromStartTimeWithCustomConfiguration() { + // Test with custom round duration + Configuration customConf = HBaseConfiguration.create(); + customConf + .setInt(ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, 30); + + Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); + ReplicationShardDirectoryManager customManager = + new ReplicationShardDirectoryManager(customConf, rootPath); + + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + long roundDurationMs = 30 * 1000L; // 30 seconds in milliseconds + + // Test with 30-second rounds + long midRound = dayStart + (45 * 1000L); // 00:00:45 + ReplicationRound result = customManager.getReplicationRoundFromStartTime(midRound); + long expectedStart = dayStart + (30 * 1000L); // 00:00:30 + assertEquals("With 30-second rounds, should round down to 30-second boundary", expectedStart, + result.getStartTime()); + assertEquals("Round end time should be start time + round duration", + expectedStart + roundDurationMs, result.getEndTime()); + + // Test exact 30-second boundary + long exactBoundary = dayStart + (60 * 1000L); // 00:01:00 + ReplicationRound result2 = customManager.getReplicationRoundFromStartTime(exactBoundary); + assertEquals("Exact 30-second boundary should be validated and unchanged", exactBoundary, + result2.getStartTime()); + assertEquals("Round end time should be start time + round duration", + exactBoundary + roundDurationMs, result2.getEndTime()); + } + + @Test + public void testGetReplicationRoundFromStartTimeConsistency() { + // Test that the same input always produces the same output + long timestamp = 1704110400000L; // 2024-01-01 12:00:00 UTC + + ReplicationRound result1 = manager.getReplicationRoundFromStartTime(timestamp); + ReplicationRound result2 = manager.getReplicationRoundFromStartTime(timestamp); + ReplicationRound result3 = manager.getReplicationRoundFromStartTime(timestamp); + + assertEquals("Same input should produce same start time", result1.getStartTime(), + result2.getStartTime()); + assertEquals("Same input should produce same end time", result1.getEndTime(), + result2.getEndTime()); + assertEquals("Same input should produce same start time", result1.getStartTime(), + result3.getStartTime()); + assertEquals("Same input should produce same end time", result1.getEndTime(), + result3.getEndTime()); + + // Test that round duration is consistent + long roundDuration1 = result1.getEndTime() - result1.getStartTime(); + assertEquals("Round duration should be 60 seconds", 60 * 1000L, roundDuration1); + } + + @Test + public void testGetReplicationRoundFromEndTime() { + // Use a specific day for consistent testing + // 2024-01-01 00:00:00 UTC = 1704067200000L + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + + // Default configuration: 60-second rounds + long roundDurationMs = 60 * 1000L; // 60 seconds in milliseconds + + // Test 1: Exact round end time + long exactRoundEnd = dayStart + roundDurationMs; // 00:01:00 + ReplicationRound result1 = manager.getReplicationRoundFromEndTime(exactRoundEnd); + assertEquals("Round end time should be validated and unchanged", exactRoundEnd, + result1.getEndTime()); + assertEquals("Round start time should be end time - round duration", + exactRoundEnd - roundDurationMs, result1.getStartTime()); + + // Test 2: Mid-round timestamp (should be rounded down to round start, then calculate end) + long midRoundTimestamp = dayStart + (30 * 1000L); // 00:00:30 + ReplicationRound result2 = manager.getReplicationRoundFromEndTime(midRoundTimestamp); + long expectedEnd2 = dayStart; // Should round down to 00:00:00 (round start) + long expectedStart2 = expectedEnd2 - roundDurationMs; // Should be -60 seconds (edge case) + assertEquals("Mid-round timestamp should round down to round start", expectedEnd2, + result2.getEndTime()); + assertEquals("Round start time should be end time - round duration", expectedStart2, + result2.getStartTime()); + + // Test 3: End of first round + long endFirstRound = dayStart + (59 * 1000L); // 00:00:59 + ReplicationRound result3 = manager.getReplicationRoundFromEndTime(endFirstRound); + long expectedEnd3 = dayStart; // Should round down to 00:00:00 + long expectedStart3 = expectedEnd3 - roundDurationMs; // Should be -60 seconds + assertEquals("End of first round should round down to round start", expectedEnd3, + result3.getEndTime()); + assertEquals("Round start time should be end time - round duration", expectedStart3, + result3.getStartTime()); + + // Test 4: Start of second round (exact boundary) + long startSecondRound = dayStart + (60 * 1000L); // 00:01:00 + ReplicationRound result4 = manager.getReplicationRoundFromEndTime(startSecondRound); + assertEquals("Second round start time should be validated and unchanged", startSecondRound, + result4.getEndTime()); + assertEquals("Round start time should be end time - round duration", + startSecondRound - roundDurationMs, result4.getStartTime()); + + // Test 5: Mid-second round timestamp + long midSecondRound = dayStart + (90 * 1000L); // 00:01:30 + ReplicationRound result5 = manager.getReplicationRoundFromEndTime(midSecondRound); + long expectedEnd5 = dayStart + (60 * 1000L); // Should round down to 00:01:00 + long expectedStart5 = expectedEnd5 - roundDurationMs; // Should be 00:00:00 + assertEquals("Mid-second round timestamp should round down to round start", expectedEnd5, + result5.getEndTime()); + assertEquals("Round start time should be end time - round duration", expectedStart5, + result5.getStartTime()); + + // Test 6: Multiple rounds later + long multipleRoundsLater = dayStart + (360 * 1000L); // 00:06:00 + ReplicationRound result6 = manager.getReplicationRoundFromEndTime(multipleRoundsLater); + assertEquals("Multiple rounds later should be validated and unchanged", multipleRoundsLater, + result6.getEndTime()); + assertEquals("Round start time should be end time - round duration", + multipleRoundsLater - roundDurationMs, result6.getStartTime()); + + // Test 7: Just before round boundary + long justBeforeRound = dayStart + (299 * 1000L); // 00:04:59 + ReplicationRound result7 = manager.getReplicationRoundFromEndTime(justBeforeRound); + long expectedEnd7 = dayStart + (240 * 1000L); // Should round down to 00:04:00 + long expectedStart7 = expectedEnd7 - roundDurationMs; // Should be 00:03:00 + assertEquals("Just before round boundary should round down", expectedEnd7, + result7.getEndTime()); + assertEquals("Round start time should be end time - round duration", expectedStart7, + result7.getStartTime()); + + // Test 8: Just after round boundary + long justAfterRound = dayStart + (301 * 1000L); // 00:05:01 + ReplicationRound result8 = manager.getReplicationRoundFromEndTime(justAfterRound); + long expectedEnd8 = dayStart + (300 * 1000L); // Should round down to 00:05:00 + long expectedStart8 = expectedEnd8 - roundDurationMs; // Should be 00:04:00 + assertEquals("Just after round boundary should round down", expectedEnd8, result8.getEndTime()); + assertEquals("Round start time should be end time - round duration", expectedStart8, + result8.getStartTime()); + + // Test 9: Current time (should round down to nearest round) + long currentTime = System.currentTimeMillis(); + ReplicationRound result9 = manager.getReplicationRoundFromEndTime(currentTime); + Assert.assertTrue("Current time should round down to nearest round", + result9.getEndTime() <= currentTime); + assertEquals("Round end should be a multiple of round duration", 0, + (result9.getEndTime() % roundDurationMs)); + assertEquals("Round start time should be end time - round duration", + result9.getEndTime() - roundDurationMs, result9.getStartTime()); + } + + @Test + public void testGetReplicationRoundFromEndTimeWithCustomConfiguration() { + // Test with custom round duration + Configuration customConf = HBaseConfiguration.create(); + customConf + .setInt(ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, 30); + + Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); + ReplicationShardDirectoryManager customManager = + new ReplicationShardDirectoryManager(customConf, rootPath); + + long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC + long roundDurationMs = 30 * 1000L; // 30 seconds in milliseconds + + // Test with 30-second rounds + long midRound = dayStart + (45 * 1000L); // 00:00:45 + ReplicationRound result = customManager.getReplicationRoundFromEndTime(midRound); + long expectedEnd = dayStart + (30 * 1000L); // 00:00:30 + long expectedStart = expectedEnd - roundDurationMs; // 00:00:00 + assertEquals("With 30-second rounds, should round down to 30-second boundary", expectedEnd, + result.getEndTime()); + assertEquals("Round start time should be end time - round duration", expectedStart, + result.getStartTime()); + + // Test exact 30-second boundary + long exactBoundary = dayStart + (60 * 1000L); // 00:01:00 + ReplicationRound result2 = customManager.getReplicationRoundFromEndTime(exactBoundary); + assertEquals("Exact 30-second boundary should be validated and unchanged", exactBoundary, + result2.getEndTime()); + assertEquals("Round start time should be end time - round duration", + exactBoundary - roundDurationMs, result2.getStartTime()); + } + + @Test + public void testDefaultAndCustomConfigurationValues() { + // Test with default configuration (no custom values set) + ReplicationShardDirectoryManager defaultManager = + new ReplicationShardDirectoryManager(conf, new Path(testFolder.getRoot().getAbsolutePath())); + + // Validate default values + assertEquals("numShards should use default value when not configured", + ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS, + defaultManager.getNumShards()); + assertEquals("replicationRoundDurationSeconds should use default value when not configured", + ReplicationShardDirectoryManager.DEFAULT_REPLICATION_ROUND_DURATION_SECONDS, + defaultManager.getReplicationRoundDurationSeconds()); + + // Test with custom configuration values + Configuration customConf = new Configuration(conf); + int customNumShards = 64; + int customReplicationRoundDurationSeconds = 120; + + customConf.setInt(ReplicationShardDirectoryManager.REPLICATION_NUM_SHARDS_KEY, customNumShards); + customConf.setInt( + ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, + customReplicationRoundDurationSeconds); + + ReplicationShardDirectoryManager customManager = new ReplicationShardDirectoryManager( + customConf, new Path(testFolder.getRoot().getAbsolutePath())); + + // Validate custom values + assertEquals("numShards should use custom value when configured", customNumShards, + customManager.getNumShards()); + assertEquals("replicationRoundDurationSeconds should use custom value when configured", + customReplicationRoundDurationSeconds, customManager.getReplicationRoundDurationSeconds()); + } + + @Test + public void testGetReplicationRoundFromEndTimeConsistency() { + // Test that the same input always produces the same output + long timestamp = 1704110400000L; // 2024-01-01 12:00:00 UTC + + ReplicationRound result1 = manager.getReplicationRoundFromEndTime(timestamp); + ReplicationRound result2 = manager.getReplicationRoundFromEndTime(timestamp); + ReplicationRound result3 = manager.getReplicationRoundFromEndTime(timestamp); + + assertEquals("Same input should produce same end time", result1.getEndTime(), + result2.getEndTime()); + assertEquals("Same input should produce same start time", result1.getStartTime(), + result2.getStartTime()); + assertEquals("Same input should produce same end time", result1.getEndTime(), + result3.getEndTime()); + assertEquals("Same input should produce same start time", result1.getStartTime(), + result3.getStartTime()); + + // Test that round duration is consistent + long roundDuration1 = result1.getEndTime() - result1.getStartTime(); + assertEquals("Round duration should be 60 seconds", 60 * 1000L, roundDuration1); + } + + @Test + public void testGetAllShardPaths() { + testGetAllShardPathsHelper(conf, + ReplicationShardDirectoryManager.DEFAULT_REPLICATION_NUM_SHARDS); + } + + @Test + public void testGetAllShardPathsWithCustomConfiguration() { + // Test with custom number of shards + int customShardCount = 64; + Configuration customConf = HBaseConfiguration.create(); + customConf.setInt(ReplicationShardDirectoryManager.REPLICATION_NUM_SHARDS_KEY, + customShardCount); + testGetAllShardPathsHelper(customConf, customShardCount); + } + + private void testGetAllShardPathsHelper(final Configuration conf, final int expectedShardCount) { + + ReplicationShardDirectoryManager replicationShardDirectoryManager = + new ReplicationShardDirectoryManager(conf, new Path(testFolder.getRoot().getAbsolutePath())); + List shardPaths = replicationShardDirectoryManager.getAllShardPaths(); + + // Verify the number of shards + assertEquals("Should return exactly " + expectedShardCount + " shard paths", expectedShardCount, + shardPaths.size()); + + // Verify the base path + String expectedBasePath = testFolder.getRoot().getAbsolutePath() + "/shard"; + + // Verify each shard path + for (int i = 0; i < expectedShardCount; i++) { + Path shardPath = shardPaths.get(i); + String expectedShardName = String.format("%03d", i); + String expectedPath = expectedBasePath + "/" + expectedShardName; + + assertEquals("Shard " + i + " should have correct path", expectedPath, shardPath.toString()); + assertEquals("Shard " + i + " should have correct name", expectedShardName, + shardPath.getName()); + } + } +}