apurtell commented on code in PR #2188:
URL: https://github.com/apache/phoenix/pull/2188#discussion_r2167523892


##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogProcessor.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+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.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.phoenix.replication.log.LogFile;
+import org.apache.phoenix.replication.log.LogFileReader;
+import org.apache.phoenix.replication.log.LogFileReaderContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicationLogProcessor implements Closeable {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(ReplicationLogProcessor.class);
+
+    /**
+     * The maximum count of mutations to process in single batch while reading 
replication log file
+     */
+    public static final String REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE =
+            "phoenix.replication.log.standby.replay.batch.size";
+
+    /**
+     * The default batch size for reading the replication log file.
+     * Assuming each log record to be 10 KB (un-compressed) and allow at max 
64 MB of
+     * in-memory records to be processed
+     */
+    public static final int DEFAULT_REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE 
= 6400;
+
+    /**
+     * The maximum number of retries for HBase client operations while 
applying the mutations
+     */
+    public static final String REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT =
+            "phoenix.replication.standby.hbase.client.retries.number";
+
+    /**
+     * The default number of retries for HBase client operations while 
applying the mutations.
+     */
+    public static final int 
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT = 4;
+
+    /**
+     * The timeout for HBase client operations while applying the mutations.
+     */
+    public static final String 
REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS =
+            "phoenix.replication.standby.hbase.client.operations.timeout";
+
+    /**
+     * The default timeout for HBase client operations while applying the 
mutations.
+     */
+    public static final long 
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS = 8000;
+
+    /**
+     * The maximum number of retry attempts for failed batch operations.
+     */
+    public static final String REPLICATION_STANDBY_BATCH_RETRY_COUNT =
+            "phoenix.replication.standby.batch.retry.count";
+
+    /**
+     * The default number of retry attempts for failed batch operations.
+     */
+    public static final int DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_COUNT = 2;
+
+    /**
+     * The maximum delay for retry attempts in milliseconds.
+     */
+    public static final String REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS =
+            "phoenix.replication.standby.batch.retry.max.delay.ms";
+
+    /**
+     * The default maximum delay for retry attempts in milliseconds.
+     */
+    public static final long 
DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS = 10000;
+
+    private final Configuration conf;
+
+    private final ExecutorService executorService;
+
+    /**
+     * This {@link AsyncConnection} is used for handling mutations
+     */
+    private volatile AsyncConnection asyncConnection;
+
+    private final int batchSize;
+
+    private final int batchRetryCount;
+
+    private final long maxRetryDelayMs;
+
+    /**
+     * Creates a new ReplicationLogProcessor with the given configuration and 
executor service.
+     * @param conf The configuration to use
+     * @param executorService The executor service for processing mutations
+     */
+    public ReplicationLogProcessor(final Configuration conf,
+            final ExecutorService executorService) {
+        // Create a copy of configuration as some of the properties would be
+        // overridden
+        this.conf = HBaseConfiguration.create(conf);
+        this.executorService = executorService;
+        this.batchSize = 
this.conf.getInt(REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE,
+                DEFAULT_REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE);
+        this.batchRetryCount = 
this.conf.getInt(REPLICATION_STANDBY_BATCH_RETRY_COUNT,
+                DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_COUNT);
+        this.maxRetryDelayMs = 
this.conf.getLong(REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS,
+                DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS);
+        decorateConf();
+    }
+
+    /**
+     * Decorate the Configuration object to make replication more receptive to 
delays by
+     * reducing the timeout and number of retries.
+     */
+    private void decorateConf() {
+        this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+                
this.conf.getInt(REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT,
+                        
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT));
+        this.conf.setLong(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
+                
this.conf.getLong(REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS,
+                        
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS));
+    }
+
+    public void processLogFile(FileSystem fs, Path filePath) throws 
IOException {
+
+        // Map from Table Name to List of Mutations
+        Map<TableName, List<Mutation>> tableToMutationsMap = new HashMap<>();
+
+        // Track the total number of processed records from input log file
+        long totalProcessed = 0;
+
+        // Track the current batch size as records will be processed in batch 
size of
+        // {@link REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE}
+        long currentBatchSize = 0;
+
+        LogFileReader logFileReader = null;
+
+        try {
+            // Create the LogFileReader for given path
+            logFileReader = createLogFileReader(fs, filePath);
+
+            for (LogFile.Record record : logFileReader) {
+                final TableName tableName = 
TableName.valueOf(record.getHBaseTableName());
+                final Mutation mutation = record.getMutation();
+
+                tableToMutationsMap.computeIfAbsent(tableName, k -> new 
ArrayList<>())
+                        .add(mutation);
+                currentBatchSize++;
+
+                // Process when we reach the batch size and reset the batch 
size and
+                // table to mutations map
+                if (currentBatchSize >= getBatchSize()) {
+                    processReplicationLogBatch(tableToMutationsMap);
+                    totalProcessed += currentBatchSize;
+                    tableToMutationsMap.clear();
+                    currentBatchSize = 0;
+                }
+            }
+
+            // Process any remaining mutations
+            if (currentBatchSize > 0) {
+                processReplicationLogBatch(tableToMutationsMap);
+                totalProcessed += currentBatchSize;
+            }
+
+            LOG.info("Completed processing log file {}. Total mutations 
processed: {}",
+                    logFileReader.getContext().getFilePath(), totalProcessed);
+
+        } catch (Exception e) {
+            LOG.error("Error while processing replication log file", e);
+            throw new IOException("Failed to process log file " + filePath, e);
+        } finally {
+            closeReader(logFileReader);
+        }
+    }
+
+    /**
+     * Creates a LogFileReader for the specified file path.
+     * Validates that the file exists and initializes the reader with the 
given file system and path.
+     * @param fs The file system to use for reading
+     * @param filePath The path to the log file
+     * @return A configured LogFileReader instance
+     * @throws IOException if the file doesn't exist or initialization fails
+     */
+    protected LogFileReader createLogFileReader(FileSystem fs, Path filePath) 
throws IOException {
+        // Ensure that file exists. If we face exception while checking the 
path itself,
+        // method would throw same exception back to the caller
+        if (!fs.exists(filePath)) {
+            throw new IOException("Log file does not exist: " + filePath);
+        }
+        LogFileReader logFileReader = new LogFileReader();
+        try {
+            LogFileReaderContext logFileReaderContext = new 
LogFileReaderContext(conf)
+                    .setFileSystem(fs).setFilePath(filePath);
+            logFileReader.init(logFileReaderContext);
+        } catch (IOException exception) {
+            LOG.error("Failed to initialize new LogFileReader for path {}",
+                    filePath, exception);
+            throw exception;
+        }
+        return logFileReader;
+    }
+
+    /**
+     * Closes the given writer, logging any errors that occur during close.
+     */
+    protected  void closeReader(LogFileReader logFileReader) {
+        if (logFileReader == null) {
+            return;
+        }
+        try {
+            logFileReader.close();
+        } catch (IOException exception) {
+            LOG.error("Error while closing LogFileReader: {}",
+                    logFileReader, exception);
+        }
+    }
+
+    protected void processReplicationLogBatch(
+            Map<TableName, List<Mutation>> tableMutationMap) throws 
IOException {
+
+        if (tableMutationMap == null || tableMutationMap.isEmpty()) {
+            return;
+        }
+
+        // Track failed operations for retry
+        Map<TableName, List<Mutation>> currentOperations = tableMutationMap;
+        IOException lastError = null;
+
+        int attempt = 0;
+        while(attempt <= batchRetryCount && !currentOperations.isEmpty()) {
+            if (attempt > 0) {
+                LOG.warn("Retrying failed batch operations, attempt {} of {}",
+                        attempt, batchRetryCount);
+            }
+
+            try {
+                // Apply mutations and get any failed operations
+                Map<TableName, List<Mutation>> failedOperations = 
applyMutations(currentOperations);
+                
+                // If no failures, we're done
+                if (failedOperations.isEmpty()) {
+                    return;
+                }
+                
+                // Update current operations for next retry
+                currentOperations = failedOperations;
+                lastError = new IOException("Failed to apply the mutations");

Review Comment:
   No details?



##########
phoenix-core-server/src/main/java/org/apache/phoenix/replication/reader/ReplicationLogProcessor.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+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.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.phoenix.replication.log.LogFile;
+import org.apache.phoenix.replication.log.LogFileReader;
+import org.apache.phoenix.replication.log.LogFileReaderContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicationLogProcessor implements Closeable {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(ReplicationLogProcessor.class);
+
+    /**
+     * The maximum count of mutations to process in single batch while reading 
replication log file
+     */
+    public static final String REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE =
+            "phoenix.replication.log.standby.replay.batch.size";
+
+    /**
+     * The default batch size for reading the replication log file.
+     * Assuming each log record to be 10 KB (un-compressed) and allow at max 
64 MB of
+     * in-memory records to be processed
+     */
+    public static final int DEFAULT_REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE 
= 6400;
+
+    /**
+     * The maximum number of retries for HBase client operations while 
applying the mutations
+     */
+    public static final String REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT =
+            "phoenix.replication.standby.hbase.client.retries.number";
+
+    /**
+     * The default number of retries for HBase client operations while 
applying the mutations.
+     */
+    public static final int 
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT = 4;
+
+    /**
+     * The timeout for HBase client operations while applying the mutations.
+     */
+    public static final String 
REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS =
+            "phoenix.replication.standby.hbase.client.operations.timeout";
+
+    /**
+     * The default timeout for HBase client operations while applying the 
mutations.
+     */
+    public static final long 
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS = 8000;
+
+    /**
+     * The maximum number of retry attempts for failed batch operations.
+     */
+    public static final String REPLICATION_STANDBY_BATCH_RETRY_COUNT =
+            "phoenix.replication.standby.batch.retry.count";
+
+    /**
+     * The default number of retry attempts for failed batch operations.
+     */
+    public static final int DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_COUNT = 2;
+
+    /**
+     * The maximum delay for retry attempts in milliseconds.
+     */
+    public static final String REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS =
+            "phoenix.replication.standby.batch.retry.max.delay.ms";
+
+    /**
+     * The default maximum delay for retry attempts in milliseconds.
+     */
+    public static final long 
DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS = 10000;
+
+    private final Configuration conf;
+
+    private final ExecutorService executorService;
+
+    /**
+     * This {@link AsyncConnection} is used for handling mutations
+     */
+    private volatile AsyncConnection asyncConnection;
+
+    private final int batchSize;
+
+    private final int batchRetryCount;
+
+    private final long maxRetryDelayMs;
+
+    /**
+     * Creates a new ReplicationLogProcessor with the given configuration and 
executor service.
+     * @param conf The configuration to use
+     * @param executorService The executor service for processing mutations
+     */
+    public ReplicationLogProcessor(final Configuration conf,
+            final ExecutorService executorService) {
+        // Create a copy of configuration as some of the properties would be
+        // overridden
+        this.conf = HBaseConfiguration.create(conf);
+        this.executorService = executorService;
+        this.batchSize = 
this.conf.getInt(REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE,
+                DEFAULT_REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE);
+        this.batchRetryCount = 
this.conf.getInt(REPLICATION_STANDBY_BATCH_RETRY_COUNT,
+                DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_COUNT);
+        this.maxRetryDelayMs = 
this.conf.getLong(REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS,
+                DEFAULT_REPLICATION_STANDBY_BATCH_RETRY_MAX_DELAY_MS);
+        decorateConf();
+    }
+
+    /**
+     * Decorate the Configuration object to make replication more receptive to 
delays by
+     * reducing the timeout and number of retries.
+     */
+    private void decorateConf() {
+        this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+                
this.conf.getInt(REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT,
+                        
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_RETRIES_COUNT));
+        this.conf.setLong(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
+                
this.conf.getLong(REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS,
+                        
DEFAULT_REPLICATION_STANDBY_HBASE_CLIENT_OPERATION_TIMEOUT_MS));
+    }
+
+    public void processLogFile(FileSystem fs, Path filePath) throws 
IOException {
+
+        // Map from Table Name to List of Mutations
+        Map<TableName, List<Mutation>> tableToMutationsMap = new HashMap<>();
+
+        // Track the total number of processed records from input log file
+        long totalProcessed = 0;
+
+        // Track the current batch size as records will be processed in batch 
size of
+        // {@link REPLICATION_STANDBY_LOG_REPLAY_BATCH_SIZE}
+        long currentBatchSize = 0;
+
+        LogFileReader logFileReader = null;
+
+        try {
+            // Create the LogFileReader for given path
+            logFileReader = createLogFileReader(fs, filePath);
+
+            for (LogFile.Record record : logFileReader) {
+                final TableName tableName = 
TableName.valueOf(record.getHBaseTableName());
+                final Mutation mutation = record.getMutation();
+
+                tableToMutationsMap.computeIfAbsent(tableName, k -> new 
ArrayList<>())
+                        .add(mutation);
+                currentBatchSize++;
+
+                // Process when we reach the batch size and reset the batch 
size and
+                // table to mutations map
+                if (currentBatchSize >= getBatchSize()) {
+                    processReplicationLogBatch(tableToMutationsMap);
+                    totalProcessed += currentBatchSize;
+                    tableToMutationsMap.clear();
+                    currentBatchSize = 0;
+                }
+            }
+
+            // Process any remaining mutations
+            if (currentBatchSize > 0) {
+                processReplicationLogBatch(tableToMutationsMap);
+                totalProcessed += currentBatchSize;
+            }
+
+            LOG.info("Completed processing log file {}. Total mutations 
processed: {}",
+                    logFileReader.getContext().getFilePath(), totalProcessed);
+
+        } catch (Exception e) {
+            LOG.error("Error while processing replication log file", e);
+            throw new IOException("Failed to process log file " + filePath, e);
+        } finally {
+            closeReader(logFileReader);
+        }
+    }
+
+    /**
+     * Creates a LogFileReader for the specified file path.
+     * Validates that the file exists and initializes the reader with the 
given file system and path.
+     * @param fs The file system to use for reading
+     * @param filePath The path to the log file
+     * @return A configured LogFileReader instance
+     * @throws IOException if the file doesn't exist or initialization fails
+     */
+    protected LogFileReader createLogFileReader(FileSystem fs, Path filePath) 
throws IOException {
+        // Ensure that file exists. If we face exception while checking the 
path itself,
+        // method would throw same exception back to the caller
+        if (!fs.exists(filePath)) {
+            throw new IOException("Log file does not exist: " + filePath);
+        }
+        LogFileReader logFileReader = new LogFileReader();
+        try {
+            LogFileReaderContext logFileReaderContext = new 
LogFileReaderContext(conf)
+                    .setFileSystem(fs).setFilePath(filePath);
+            logFileReader.init(logFileReaderContext);
+        } catch (IOException exception) {
+            LOG.error("Failed to initialize new LogFileReader for path {}",
+                    filePath, exception);
+            throw exception;
+        }
+        return logFileReader;
+    }
+
+    /**
+     * Closes the given writer, logging any errors that occur during close.
+     */
+    protected  void closeReader(LogFileReader logFileReader) {
+        if (logFileReader == null) {
+            return;
+        }
+        try {
+            logFileReader.close();
+        } catch (IOException exception) {
+            LOG.error("Error while closing LogFileReader: {}",
+                    logFileReader, exception);
+        }
+    }
+
+    protected void processReplicationLogBatch(
+            Map<TableName, List<Mutation>> tableMutationMap) throws 
IOException {
+
+        if (tableMutationMap == null || tableMutationMap.isEmpty()) {
+            return;
+        }
+
+        // Track failed operations for retry
+        Map<TableName, List<Mutation>> currentOperations = tableMutationMap;
+        IOException lastError = null;
+
+        int attempt = 0;
+        while(attempt <= batchRetryCount && !currentOperations.isEmpty()) {
+            if (attempt > 0) {
+                LOG.warn("Retrying failed batch operations, attempt {} of {}",
+                        attempt, batchRetryCount);
+            }
+
+            try {
+                // Apply mutations and get any failed operations
+                Map<TableName, List<Mutation>> failedOperations = 
applyMutations(currentOperations);
+                
+                // If no failures, we're done
+                if (failedOperations.isEmpty()) {
+                    return;
+                }
+                
+                // Update current operations for next retry
+                currentOperations = failedOperations;
+                lastError = new IOException("Failed to apply the mutations");
+            } catch (IOException e) {
+                lastError = e;
+            }
+            attempt++;
+            // Add delay between retries (exponential backoff)
+            if (attempt <= batchRetryCount && !currentOperations.isEmpty()) {
+                try {
+                    long delayMs = calculateRetryDelay(attempt);
+                    Thread.sleep(delayMs);
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw new IOException("Interrupted during retry delay", e);
+                }
+            }
+        }
+
+        // If we still have failed operations after all retries, throw the 
last error
+        if (!currentOperations.isEmpty() && lastError != null) {
+            LOG.error("Failed to process batch operations after {} retries. 
Failed tables: {}", 
+                    batchRetryCount, currentOperations.keySet());
+            throw lastError;
+        }
+    }
+
+    /**
+     * Calculates the delay time for retry attempts using exponential backoff.
+     * @param attempt The current retry attempt number (0-based)
+     * @return The delay time in milliseconds
+     */
+    protected long calculateRetryDelay(int attempt) {
+        return Math.min(1000L * (1L << attempt), maxRetryDelayMs);
+    }
+
+    /**
+     * Applies mutations to HBase tables and returns any failed operations.
+     * @param tableMutationMap Map of table names to their mutations
+     * @return Map of table names to their failed mutations, empty if all 
succeeded
+     * @throws IOException if there's an error applying mutations
+     */
+    protected Map<TableName, List<Mutation>> applyMutations(
+            Map<TableName, List<Mutation>> tableMutationMap) throws 
IOException {
+
+        if(tableMutationMap == null || tableMutationMap.isEmpty()) {
+            return Collections.emptyMap();
+        }
+
+        Map<TableName, List<Mutation>> failedOperations = new HashMap<>();
+        Map<TableName, Future<?>> futures = new HashMap<>();
+
+        // Submit batch operations
+        for (Map.Entry<TableName, List<Mutation>> entry : 
tableMutationMap.entrySet()) {
+            TableName tableName = entry.getKey();
+            List<Mutation> mutations = entry.getValue();
+            AsyncTable<?> table = getAsyncConnection()
+                    .getTable(tableName, executorService);
+            futures.put(tableName, table.batchAll(mutations));
+        }
+
+        // Check results and track failures
+        for (Map.Entry<TableName, Future<?>> entry : futures.entrySet()) {
+            TableName tableName = entry.getKey();
+            Future<?> future = entry.getValue();
+            try {
+                FutureUtils.get(future);
+            } catch (IOException e) {
+                // Add failed mutations to retry list
+                failedOperations.put(tableName, 
tableMutationMap.get(tableName));
+                LOG.warn("Failed to apply mutations for table {}: {}", 
tableName, e.getMessage());

Review Comment:
   This will probably be pretty noisy? Can we do something else? I would 
recommend the detail log at DEBUG and a metric for failed mutation application 
we can track instead of a likely flood of WARN level messages every time there 
is an issue.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to