From 6ad1b33d8d34fd47281399ee6b1374bcdf400391 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Thu, 6 Nov 2025 23:58:23 -0800 Subject: [PATCH 01/13] PHOENIX-7602 Replication Log Writer (Store and Forward mode) This patch implements the replication mode transitions and log forwarding from fallback cluster to the standby cluster. --- .../jdbc/FailoverPhoenixConnection.java | 2 +- .../phoenix/jdbc/HAGroupStoreManager.java | 2 +- .../org/apache/phoenix/util/SchemaUtil.java | 1 + .../hbase/index/IndexRegionObserver.java | 160 ++- .../phoenix/replication/ReplicationLog.java | 471 ++++++++ .../replication/ReplicationLogDiscovery.java | 17 +- .../ReplicationLogDiscoveryForwarder.java | 183 +++ .../replication/ReplicationLogGroup.java | 1061 ++++++++++++++--- .../ReplicationLogGroupWriter.java | 821 ------------- .../replication/ReplicationLogTracker.java | 4 +- .../replication/ReplicationModeImpl.java | 94 ++ .../ReplicationShardDirectoryManager.java | 59 +- .../replication/StandbyLogGroupWriter.java | 129 -- .../StoreAndForwardLogGroupWriter.java | 81 -- .../replication/StoreAndForwardModeImpl.java | 114 ++ .../replication/SyncAndForwardModeImpl.java | 68 ++ .../phoenix/replication/SyncModeImpl.java | 64 + ...sReplicationLogDiscoveryForwarderImpl.java | 39 + ...sReplicationLogForwarderSourceFactory.java | 28 + ...icsReplicationLogTrackerForwarderImpl.java | 39 + .../reader/ReplicationLogReplay.java | 4 +- .../replication/ReplicationLogGroupIT.java | 185 ++- .../ReplicationLogDiscoveryReplayTestIT.java | 4 +- .../org/apache/phoenix/query/BaseTest.java | 4 +- .../phoenix/query/PhoenixTestBuilder.java | 11 +- .../replication/ReplicationLogBaseTest.java | 170 +++ .../ReplicationLogDiscoveryForwarderTest.java | 144 +++ .../ReplicationLogDiscoveryTest.java | 4 +- .../replication/ReplicationLogGroupTest.java | 384 +++--- .../ReplicationLogTrackerTest.java | 4 +- .../ReplicationShardDirectoryManagerTest.java | 22 +- 31 files changed, 2853 insertions(+), 1520 deletions(-) create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java delete mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroupWriter.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java delete mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/StandbyLogGroupWriter.java delete mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardLogGroupWriter.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryForwarderImpl.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java create mode 100644 phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java create mode 100644 phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java create mode 100644 phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection.java index 10f88124e3a..c344301ae04 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection.java @@ -662,7 +662,7 @@ public int getNetworkTimeout() throws SQLException { * @return the currently wrapped connection. */ @VisibleForTesting - PhoenixConnection getWrappedConnection() { + public PhoenixConnection getWrappedConnection() { return connection; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreManager.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreManager.java index fb3d7d5ad20..362804c2c3e 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreManager.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreManager.java @@ -189,7 +189,7 @@ public static HAGroupStoreManager getInstanceForZkUrl(final Configuration conf, } @VisibleForTesting - HAGroupStoreManager(final Configuration conf) { + protected HAGroupStoreManager(final Configuration conf) { this(conf, getLocalZkUrl(conf)); } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java index eb0e2a4ad43..36d86abbc99 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java @@ -1459,6 +1459,7 @@ public static boolean areSeparatorBytesForVarBinaryEncoded(byte[] bytes, int off * SYSTEM.TRANSFORM * SYSTEM.CDC_STREAM_STATUS * SYSTEM.CDC_STREAM + * SYSTEM.HA_GROUP * For SYSTEM.CATALOG and SYSTEM.CHILD_LINK we only replicate rows with tenant information. * Non tenant (Global) rows are assumed to be executed by an admin or an admin process in each * cluster separately and thus not replicated. diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java index f11f15d3ab7..8eef8d746da 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java @@ -178,8 +178,6 @@ public class IndexRegionObserver implements RegionCoprocessor, RegionObserver { // Mutation attribute to ignore the mutation for replication public static final String IGNORE_REPLICATION_ATTRIB = "_IGNORE_REPLICATION"; private static final byte[] IGNORE_REPLICATION_ATTRIB_VAL = new byte[]{0}; - // TODO hardcoded for now, will fix later - public static final String DEFAULT_HA_GROUP = "DEFAULT_HA_GROUP"; /** * Class to represent pending data table rows @@ -324,6 +322,8 @@ public static class BatchMutateContext { private boolean returnResult; private boolean returnOldRow; private boolean hasConditionalTTL; // table has Conditional TTL + // HAGroup associated with the batch + private Optional logGroup = Optional.empty(); public BatchMutateContext() { this.clientVersion = 0; @@ -431,7 +431,6 @@ public int getMaxPendingRowCount() { private static final int DEFAULT_CONCURRENT_MUTATION_WAIT_DURATION_IN_MS = 100; private byte[] encodedRegionName; private boolean shouldReplicate; - private ReplicationLogGroup replicationLog; // Don't replicate the mutation if this attribute is set private static final Predicate IGNORE_REPLICATION = mutation -> @@ -527,8 +526,6 @@ public void start(CoprocessorEnvironment e) throws IOException { this.shouldReplicate = SchemaUtil.shouldReplicateTable(tableName); } if (this.shouldReplicate) { - this.replicationLog = ReplicationLogGroup.get(env.getConfiguration(), - env.getServerName(), DEFAULT_HA_GROUP); this.ignoreReplicationFilter = getSynchronousReplicationFilter(tableName); } } catch (NoSuchMethodError ex) { @@ -631,19 +628,21 @@ public void preBatchMutate(ObserverContext c, throw new IOException("HAGroupStoreManager is null " + "for current cluster, check configuration"); } - String tableName - = c.getEnvironment().getRegion().getRegionInfo().getTable().getNameAsString(); + Optional logGroup = Optional.empty(); + if (shouldReplicate) { + // Extract HAGroupName from the mutations + logGroup = getHAGroupFromBatch(c.getEnvironment(), miniBatchOp); + } // We don't want to check for mutation blocking for the system ha group table - if (!tableName.equals(SYSTEM_HA_GROUP_NAME)) { - // Extract HAGroupName from the mutations - final String haGroupName = extractHAGroupNameAttribute(miniBatchOp); - // Check if mutation is blocked for any of the HAGroupNames - if (StringUtils.isNotBlank(haGroupName)) { - //TODO: Below approach might be slow need to figure out faster way, - // slower part is getting haGroupStoreClient We can also cache - // roleRecord (I tried it and still it's slow due to haGroupStoreClient - // initialization) and caching will give us old result in case one cluster - // is unreachable instead of UNKNOWN. + if (!dataTableName.equals(SYSTEM_HA_GROUP_NAME)) { + // Check if mutation is blocked for the HA Group + if (logGroup.isPresent()) { + String haGroupName = logGroup.get().getHAGroupName(); + //TODO: Below approach might be slow need to figure out faster way, + // slower part is getting haGroupStoreClient We can also cache + // roleRecord (I tried it and still it's slow due to haGroupStoreClient + // initialization) and caching will give us old result in case one cluster + // is unreachable instead of UNKNOWN. boolean isHAGroupOnClientStale = haGroupStoreManager .isHAGroupOnClientStale(haGroupName); @@ -662,7 +661,7 @@ public void preBatchMutate(ObserverContext c, } } } - preBatchMutateWithExceptions(c, miniBatchOp); + preBatchMutateWithExceptions(c, miniBatchOp, logGroup); return; } catch (Throwable t) { rethrowIndexingException(t); @@ -671,17 +670,44 @@ public void preBatchMutate(ObserverContext c, "Somehow didn't return an index update but also didn't propagate the failure to the client!"); } - private String extractHAGroupNameAttribute( - MiniBatchOperationInProgress miniBatchOp) { - for (int i = 0; i < miniBatchOp.size(); i++) { - Mutation m = miniBatchOp.getOperation(i); - byte[] haGroupName = m.getAttribute( - BaseScannerRegionObserverConstants.HA_GROUP_NAME_ATTRIB); - if (haGroupName != null) { - return String.valueOf(haGroupName); - } - } - return null; + /** + * Get the HA group associated with the batch. We assume that all the mutations in the + * batch will have the same HA group. + * @return HA group if present or empty if missing + */ + private Optional getHAGroupFromBatch( + RegionCoprocessorEnvironment env, + MiniBatchOperationInProgress miniBatchOp) { + if (miniBatchOp.size() > 0) { + Mutation m = miniBatchOp.getOperation(0); + byte[] haGroupName = m.getAttribute( + BaseScannerRegionObserverConstants.HA_GROUP_NAME_ATTRIB); + if (haGroupName != null) { + ReplicationLogGroup logGroup = + ReplicationLogGroup.get(env.getConfiguration(), + env.getServerName(), Bytes.toString(haGroupName)); + return Optional.of(logGroup); + } + } + return Optional.empty(); + } + + /** + * Get the HA group associated with the WAL key. A batch of mutations is recorded + * in a single WAL edit. + * @return HA group if present or empty if missing + */ + private Optional getHAGroupFromWALKey( + RegionCoprocessorEnvironment env, + org.apache.hadoop.hbase.wal.WALKey logKey) { + byte[] haGroupName = logKey.getExtendedAttribute( + BaseScannerRegionObserverConstants.HA_GROUP_NAME_ATTRIB); + if (haGroupName != null) { + ReplicationLogGroup logGroup = ReplicationLogGroup.get(env.getConfiguration(), + env.getServerName(), Bytes.toString(haGroupName)); + return Optional.of(logGroup); + } + return Optional.empty(); } @Override @@ -696,9 +722,14 @@ public void preWALRestore( if (!shouldReplicate) { return; } + Optional logGroup = + getHAGroupFromWALKey(ctx.getEnvironment(), logKey); + if (!logGroup.isPresent()) { + return; + } long start = EnvironmentEdgeManager.currentTimeMillis(); try { - replicateEditOnWALRestore(logKey, logEdit); + replicateEditOnWALRestore(logGroup.get(), logKey, logEdit); } finally { long duration = EnvironmentEdgeManager.currentTimeMillis() - start; metricSource.updatePreWALRestoreTime(dataTableName, duration); @@ -709,11 +740,14 @@ public void preWALRestore( * A batch of mutations is recorded in a single WAL edit so a WAL edit can have cells * belonging to multiple rows. Further, for one mutation the WAL edit contains the individual * cells that are part of the mutation. - * @param logKey - * @param logEdit + * + * @param logGroup HA Group + * @param logKey WAL log key + * @param logEdit WAL edit record * @throws IOException */ - private void replicateEditOnWALRestore(org.apache.hadoop.hbase.wal.WALKey logKey, + private void replicateEditOnWALRestore(ReplicationLogGroup logGroup, + WALKey logKey, WALEdit logEdit) throws IOException { ImmutableBytesPtr prevKey = null, currentKey = null; Put put = null; @@ -721,7 +755,7 @@ private void replicateEditOnWALRestore(org.apache.hadoop.hbase.wal.WALKey logKey for (Cell kv : logEdit.getCells()) { if (kv instanceof IndexedKeyValue) { IndexedKeyValue ikv = (IndexedKeyValue) kv; - replicationLog.append(Bytes.toString(ikv.getIndexTable()), -1, ikv.getMutation()); + logGroup.append(Bytes.toString(ikv.getIndexTable()), -1, ikv.getMutation()); } else { // While we can generate a separate mutation for every cell that is part of the // WAL edit and replicate each such mutation. Doing that will not be very efficient @@ -732,10 +766,10 @@ private void replicateEditOnWALRestore(org.apache.hadoop.hbase.wal.WALKey logKey kv.getRowOffset(), kv.getRowLength()); if (!currentKey.equals(prevKey)) { if (put != null && !this.ignoreReplicationFilter.test(put)) { - replicationLog.append(logKey.getTableName().getNameAsString(), -1, put); + logGroup.append(logKey.getTableName().getNameAsString(), -1, put); } if (del != null && !this.ignoreReplicationFilter.test(del)) { - replicationLog.append(logKey.getTableName().getNameAsString(), -1, del); + logGroup.append(logKey.getTableName().getNameAsString(), -1, del); } // reset put = null; @@ -759,12 +793,12 @@ private void replicateEditOnWALRestore(org.apache.hadoop.hbase.wal.WALKey logKey } // append the last one if (put != null && !this.ignoreReplicationFilter.test(put)) { - replicationLog.append(logKey.getTableName().getNameAsString(), -1, put); + logGroup.append(logKey.getTableName().getNameAsString(), -1, put); } if (del != null && !this.ignoreReplicationFilter.test(del)) { - replicationLog.append(logKey.getTableName().getNameAsString(), -1, del); + logGroup.append(logKey.getTableName().getNameAsString(), -1, del); } - replicationLog.sync(); + logGroup.sync(); } private void populateRowsToLock(MiniBatchOperationInProgress miniBatchOp, @@ -1671,9 +1705,11 @@ private long getBatchTimestamp(BatchMutateContext context, TableName table) } public void preBatchMutateWithExceptions(ObserverContext c, - MiniBatchOperationInProgress miniBatchOp) throws Throwable { + MiniBatchOperationInProgress miniBatchOp, + Optional logGroup) throws Throwable { PhoenixIndexMetaData indexMetaData = getPhoenixIndexMetaData(c, miniBatchOp); BatchMutateContext context = new BatchMutateContext(indexMetaData.getClientVersion()); + context.logGroup = logGroup; setBatchMutateContext(c, context); identifyIndexMaintainerTypes(indexMetaData, context); identifyMutationTypes(miniBatchOp, context); @@ -1874,10 +1910,15 @@ public void preWALAppend(ObserverContext c, WALKey BatchMutateContext context = getBatchMutateContext(c); appendMutationAttributesToWALKey(key, context); } + + if (shouldReplicate) { + BatchMutateContext context = getBatchMutateContext(c); + appendHAGroupAttributeToWALKey(key, context); + } } - public void appendMutationAttributesToWALKey(WALKey key, - IndexRegionObserver.BatchMutateContext context) { + private void appendMutationAttributesToWALKey(WALKey key, + IndexRegionObserver.BatchMutateContext context) { if (context != null && context.getOriginalMutations().size() > 0) { Mutation firstMutation = context.getOriginalMutations().get(0); Map attrMap = firstMutation.getAttributesMap(); @@ -1892,6 +1933,20 @@ public void appendMutationAttributesToWALKey(WALKey key, } } + /** + * Save the HA group name if present in the WAL key so that we can use it when restoring + * from the WAL + */ + private void appendHAGroupAttributeToWALKey(WALKey key, + IndexRegionObserver.BatchMutateContext context) { + if (context != null && context.logGroup.isPresent()) { + String haGroupName = context.logGroup.get().getHAGroupName(); + IndexRegionObserver.appendToWALKey(key, + BaseScannerRegionObserverConstants.HA_GROUP_NAME_ATTRIB, + Bytes.toBytes(haGroupName)); + } + } + /** * When this hook is called, all the rows in the batch context are locked if the batch of * mutations is successful. Because the rows are locked, we can safely make updates to @@ -1957,7 +2012,7 @@ public void postBatchMutateIndispensably(ObserverContext doPost(c, context)); long start = EnvironmentEdgeManager.currentTimeMillis(); try { - replicateMutations(miniBatchOp, context); + replicateMutations(c.getEnvironment(), miniBatchOp, context); } finally { long duration = EnvironmentEdgeManager.currentTimeMillis() - start; metricSource.updateReplicationSyncTime(dataTableName, duration); @@ -2505,7 +2560,8 @@ public static boolean isAtomicOperationComplete(OperationStatus status) { return status.getOperationStatusCode() == SUCCESS && status.getResult() != null; } - private void replicateMutations(MiniBatchOperationInProgress miniBatchOp, + private void replicateMutations(RegionCoprocessorEnvironment env, + MiniBatchOperationInProgress miniBatchOp, BatchMutateContext context) throws IOException { if (!this.shouldReplicate) { @@ -2514,18 +2570,22 @@ private void replicateMutations(MiniBatchOperationInProgress miniBatch if (ignoreSyncReplicationForTesting) { return; } - assert this.replicationLog != null; + + Optional logGroup = getHAGroupFromBatch(env, miniBatchOp); + if (!logGroup.isPresent()) { + return; + } for (Integer i = 0; i < miniBatchOp.size(); i++) { Mutation m = miniBatchOp.getOperation(i); if (this.ignoreReplicationFilter.test(m)) { continue; } - this.replicationLog.append(this.dataTableName, -1, m); + logGroup.get().append(this.dataTableName, -1, m); Mutation[] mutationsAddedByCP = miniBatchOp.getOperationsFromCoprocessors(i); if (mutationsAddedByCP != null) { for (Mutation addedMutation : mutationsAddedByCP) { - this.replicationLog.append(this.dataTableName, -1, addedMutation); + logGroup.get().append(this.dataTableName, -1, addedMutation); } } } @@ -2535,7 +2595,7 @@ private void replicateMutations(MiniBatchOperationInProgress miniBatch if (this.ignoreReplicationFilter.test(entry.getValue())) { continue; } - this.replicationLog.append(entry.getKey().getTableName(), -1, entry.getValue()); + logGroup.get().append(entry.getKey().getTableName(), -1, entry.getValue()); } } if (context.postIndexUpdates != null) { @@ -2544,9 +2604,9 @@ private void replicateMutations(MiniBatchOperationInProgress miniBatch if (this.ignoreReplicationFilter.test(entry.getValue())) { continue; } - this.replicationLog.append(entry.getKey().getTableName(), -1, entry.getValue()); + logGroup.get().append(entry.getKey().getTableName(), -1, entry.getValue()); } } - this.replicationLog.sync(); + logGroup.get().sync(); } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java new file mode 100644 index 00000000000..9419c967523 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java @@ -0,0 +1,471 @@ +/* + * 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.io.InterruptedIOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.phoenix.replication.ReplicationLogGroup.Record; +import org.apache.phoenix.replication.log.LogFileWriter; +import org.apache.phoenix.replication.log.LogFileWriterContext; +import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ReplicationLog manages the underlying replication log. + *

+ * This class implements the functionality for managing the replication log including log rotation + * based on time or size, retries on file operations and the lifecycle of the underlying + * LogFileWriter. + *

+ */ +@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = { "EI_EXPOSE_REP", "EI_EXPOSE_REP2", + "MS_EXPOSE_REP" }, justification = "Intentional") +public class ReplicationLog { + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLog.class); + + protected final ReplicationLogGroup logGroup; + protected final long rotationTimeMs; + protected final long rotationSizeBytes; + protected final int maxRotationRetries; + protected final Compression.Algorithm compression; + protected final ReentrantLock lock = new ReentrantLock(); + protected final int maxAttempts; // Configurable max attempts for sync + protected final long retryDelayMs; // Configurable delay between attempts + // Underlying file writer + protected volatile LogFileWriter currentWriter; + protected final AtomicLong lastRotationTime = new AtomicLong(); + protected final AtomicLong writerGeneration = new AtomicLong(); + protected final AtomicLong rotationFailures = new AtomicLong(0); + protected ScheduledExecutorService rotationExecutor; + protected volatile boolean closed = false; + // Manages the creation of the actual log file in the shard directory + protected ReplicationShardDirectoryManager replicationShardDirectoryManager; + // List of in-flight appends which are successful but haven't been synced yet + private final List currentBatch = new ArrayList<>(); + // Current version of the writer being used for file operations. It is needed for detecting + // when the writer changes because of rotation while we are in the middle of a write operation. + private long generation; + + public ReplicationLog(ReplicationLogGroup logGroup, ReplicationShardDirectoryManager shardManager) { + this.logGroup = logGroup; + Configuration conf = logGroup.getConfiguration(); + this.maxAttempts = conf.getInt(ReplicationLogGroup.REPLICATION_LOG_SYNC_RETRIES_KEY, + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_SYNC_RETRIES) + 1; + this.retryDelayMs = + conf.getLong(ReplicationLogGroup.REPLICATION_LOG_RETRY_DELAY_MS_KEY, + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_RETRY_DELAY_MS); + this.rotationTimeMs = + conf.getLong(ReplicationLogGroup.REPLICATION_LOG_ROTATION_TIME_MS_KEY, + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_TIME_MS); + long rotationSize = + conf.getLong(ReplicationLogGroup.REPLICATION_LOG_ROTATION_SIZE_BYTES_KEY, + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_SIZE_BYTES); + double rotationSizePercent = + conf.getDouble(ReplicationLogGroup.REPLICATION_LOG_ROTATION_SIZE_PERCENTAGE_KEY, + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_SIZE_PERCENTAGE); + this.rotationSizeBytes = (long) (rotationSize * rotationSizePercent); + this.maxRotationRetries = + conf.getInt(ReplicationLogGroup.REPLICATION_LOG_ROTATION_RETRIES_KEY, + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_RETRIES); + String compressionName = + conf.get(ReplicationLogGroup.REPLICATION_LOG_COMPRESSION_ALGORITHM_KEY, + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_COMPRESSION_ALGORITHM); + Compression.Algorithm compression = Compression.Algorithm.NONE; + if (!compressionName.equals( + ReplicationLogGroup.DEFAULT_REPLICATION_LOG_COMPRESSION_ALGORITHM)) { + try { + compression = Compression.getCompressionAlgorithmByName(compressionName); + } catch (IllegalArgumentException e) { + LOG.warn("Unknown compression type " + compressionName + ", using NONE", e); + } + } + this.compression = compression; + this.replicationShardDirectoryManager = shardManager; + } + + /** The reason for requesting a log rotation. */ + protected enum RotationReason { + /** Rotation requested due to time threshold being exceeded. */ + TIME, + /** Rotation requested due to size threshold being exceeded. */ + SIZE, + /** Rotation requested due to an error condition. */ + ERROR + } + + /** Initialize the writer. */ + public void init() throws IOException { + // Start time based rotation. + lastRotationTime.set(EnvironmentEdgeManager.currentTimeMillis()); + startRotationExecutor(); + // Create the initial writer + currentWriter = createNewWriter(); + generation = currentWriter.getGeneration(); + } + + /** Creates and initializes a new LogFileWriter. */ + protected LogFileWriter createNewWriter() throws IOException { + long timestamp = EnvironmentEdgeManager.currentTimeMillis(); + Path filePath = replicationShardDirectoryManager.getWriterPath(timestamp, + logGroup.getServerName().getServerName()); + LogFileWriterContext writerContext = new LogFileWriterContext(logGroup.getConfiguration()) + .setFileSystem(replicationShardDirectoryManager.getFileSystem()) + .setFilePath(filePath).setCompression(compression); + LogFileWriter newWriter = new LogFileWriter(); + newWriter.init(writerContext); + newWriter.setGeneration(writerGeneration.incrementAndGet()); + return newWriter; + } + + protected void startRotationExecutor() { + long rotationCheckInterval = getRotationCheckInterval(rotationTimeMs); + rotationExecutor = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("ReplicationLogRotation-" + logGroup.getHAGroupName() + "-%d") + .setDaemon(true).build()); + rotationExecutor.scheduleAtFixedRate(new LogRotationTask(), rotationCheckInterval, + rotationCheckInterval, TimeUnit.MILLISECONDS); + LOG.debug("Started rotation executor with interval {}ms", rotationCheckInterval); + } + + protected long getRotationCheckInterval(long rotationTimeMs) { + long interval = Math.max(10 * 1000L, Math.min(60 * 1000L, rotationTimeMs / 10)); + return interval; + } + + protected void stopRotationExecutor() { + if (rotationExecutor != null) { + rotationExecutor.shutdown(); + try { + if (!rotationExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + rotationExecutor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + rotationExecutor.shutdownNow(); + } + } + } + + /** Gets the current writer, rotating it if necessary based on size thresholds. */ + protected LogFileWriter getWriter() throws IOException { + lock.lock(); + try { + if (shouldRotate()) { + rotateLog(RotationReason.SIZE); + } + return currentWriter; + } finally { + lock.unlock(); + } + } + + /** + * Checks if the current log file needs to be rotated based on time or size. Must be called + * under lock. + * + * @return true if rotation is needed, false otherwise. + * @throws IOException If an error occurs checking the file size. + */ + protected boolean shouldRotate() throws IOException { + if (currentWriter == null) { + LOG.warn("Current writer is null, forcing rotation."); + return true; + } + // Check time threshold + long now = EnvironmentEdgeManager.currentTimeMillis(); + long last = lastRotationTime.get(); + if (now - last >= rotationTimeMs) { + LOG.debug("Rotating log file due to time threshold ({} ms elapsed, threshold {} ms)", + now - last, rotationTimeMs); + return true; + } + + // Check size threshold (using actual file size for accuracy) + long currentSize = currentWriter.getLength(); + if (currentSize >= rotationSizeBytes) { + LOG.debug("Rotating log file due to size threshold ({} bytes, threshold {} bytes)", + currentSize, rotationSizeBytes); + return true; + } + + return false; + } + + /** + * Closes the current log writer and opens a new one, updating rotation metrics. + *

+ * This method handles the rotation of log files, which can be triggered by: + *

    + *
  • Time threshold exceeded (TIME)
  • + *
  • Size threshold exceeded (SIZE)
  • + *
  • Error condition requiring rotation (ERROR)
  • + *
+ *

+ * The method implements retry logic for handling rotation failures. If rotation fails, it + * retries up to maxRotationRetries times. If the number of failures exceeds + * maxRotationRetries, an exception is thrown. Otherwise, it logs a warning and continues with + * the current writer. + *

+ * The method is thread-safe and uses a lock to ensure atomic rotation operations. + * + * @param reason The reason for requesting log rotation + * @return The new LogFileWriter instance if rotation succeeded, or the current writer if + * rotation failed + * @throws IOException if rotation fails after exceeding maxRotationRetries + */ + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UL_UNRELEASED_LOCK", + justification = "False positive") + protected LogFileWriter rotateLog(RotationReason reason) throws IOException { + lock.lock(); + try { + // Try to get the new writer first. If it fails we continue using the current writer. + // Increment the writer generation + LogFileWriter newWriter = createNewWriter(); + LOG.debug("Created new writer: {}", newWriter); + // Close the current writer + closeWriter(currentWriter); + currentWriter = newWriter; + lastRotationTime.set(EnvironmentEdgeManager.currentTimeMillis()); + rotationFailures.set(0); + logGroup.getMetrics().incrementRotationCount(); + switch (reason) { + case TIME: + logGroup.getMetrics().incrementTimeBasedRotationCount(); + break; + case SIZE: + logGroup.getMetrics().incrementSizeBasedRotationCount(); + break; + case ERROR: + logGroup.getMetrics().incrementErrorBasedRotationCount(); + break; + } + } catch (IOException e) { + // If we fail to rotate the log, we increment the failure counter. If we have exceeded + // the maximum number of retries, we close the log and throw the exception. Otherwise + // we log a warning and continue. + logGroup.getMetrics().incrementRotationFailureCount(); + long numFailures = rotationFailures.getAndIncrement(); + if (numFailures >= maxRotationRetries) { + LOG.warn("Failed to rotate log (attempt {}/{}), closing log", numFailures, + maxRotationRetries, e); + closeOnError(); + throw e; + } + LOG.warn("Failed to rotate log (attempt {}/{}), retrying...", numFailures, + maxRotationRetries, e); + } finally { + lock.unlock(); + } + return currentWriter; + } + + /** Closes the given writer, logging any errors that occur during close. */ + private void closeWriter(LogFileWriter writer) { + if (writer == null) { + return; + } + LOG.debug("Closing writer: {}", writer); + try { + writer.close(); + } catch (IOException e) { + // For now, just log and continue + LOG.error("Error closing log writer: " + writer, e); + } + } + + /** + * Check if this replication log is closed. + * + * @return true if closed, false otherwise + */ + public boolean isClosed() { + return closed; + } + + private interface Action { + void action(LogFileWriter writer) throws IOException; + } + + private void apply(Action action) throws IOException { + LogFileWriter writer = getWriter(); + for (int attempt = 1; attempt <= maxAttempts; attempt++) { + if (isClosed()) { + throw new IOException("Closed"); + } + try { + if (writer.getGeneration() > generation) { + generation = writer.getGeneration(); + // If the writer has been rotated, we need to replay the current batch of + // in-flight appends into the new writer. + if (!currentBatch.isEmpty()) { + LOG.trace("Writer has been rotated, replaying in-flight batch"); + for (Record r: currentBatch) { + writer.append(r.tableName, r.commitId, r.mutation); + } + } + } + action.action(writer); + break; + } catch (IOException e) { + // IO exception, force a rotation. + LOG.debug("Attempt " + attempt + "/" + maxAttempts + " failed", e); + if (attempt == maxAttempts) { + // TODO: Add log + closeOnError(); + throw e; + } + // Add delay before retrying to prevent tight loops + try { + Thread.sleep(retryDelayMs); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new InterruptedIOException("Interrupted during retry delay"); + } + writer = rotateLog(RotationReason.ERROR); + } + } + } + + protected void append(Record r) throws IOException { + apply(writer -> writer.append(r.tableName, r.commitId, r.mutation)); + // Add to current batch only after we succeed at appending + currentBatch.add(r); + } + + protected void append(String tableName, long commitId, Mutation mutation) throws IOException { + apply(writer -> writer.append(tableName, commitId, mutation)); + } + + protected void sync() throws IOException { + apply(writer -> writer.sync()); + // Sync completed, clear the list of in-flight appends. + currentBatch.clear(); + } + + /** + * Return the current batch of in-flight appends. It is used when we switch replication mode. + * + * @return List of in-flight successful append records + */ + protected List getCurrentBatch() { + return currentBatch; + } + + /** + * Force closes the log upon an unrecoverable internal error. This is a fail-stop behavior: + * once called, the log is marked as closed, the Disruptor is halted, and all subsequent + * append() and sync() calls will throw an IOException("Closed"). This ensures that no + * further operations are attempted on a log that has encountered a critical error. + */ + protected void closeOnError() { + lock.lock(); + try { + if (closed) { + return; + } + closed = true; + } finally { + lock.unlock(); + } + // Stop the time based rotation check. + stopRotationExecutor(); + // We expect a final sync will not work. Just close the inner writer. + closeWriter(currentWriter); + } + + /** Closes the log. */ + public void close() { + lock.lock(); + try { + if (closed) { + return; + } + closed = true; + } finally { + lock.unlock(); + } + // Stop the time based rotation check. + stopRotationExecutor(); + // We must for the disruptor before closing the current writer. + closeWriter(currentWriter); + } + + protected FileSystem getFileSystem(URI uri) throws IOException { + return FileSystem.get(uri, logGroup.getConfiguration()); + } + + /** Implements time based rotation independent of in-line checking. */ + protected class LogRotationTask implements Runnable { + @Override + public void run() { + if (closed) { + return; + } + // Use tryLock with a timeout to avoid blocking indefinitely if another thread holds + // the lock for an unexpectedly long time (e.g., during a problematic rotation). + boolean acquired = false; + try { + // Wait a short time for the lock + acquired = lock.tryLock(1, TimeUnit.SECONDS); + if (acquired) { + // Check only the time condition here, size is handled by getWriter + long now = EnvironmentEdgeManager.currentTimeMillis(); + long last = lastRotationTime.get(); + if (!closed && now - last >= rotationTimeMs) { + LOG.debug("Time based rotation needed ({} ms elapsed, threshold {} ms).", + now - last, rotationTimeMs); + try { + rotateLog(RotationReason.TIME); // rotateLog updates lastRotationTime + } catch (IOException e) { + LOG.error("Failed to rotate log, currentWriter is {}", currentWriter, + e); + // More robust error handling goes here once the store-and-forward + // fallback is implemented. For now we just log the error and continue. + } + } + } else { + LOG.warn("LogRotationTask could not acquire lock, skipping check this time."); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); // Preserve interrupt status + LOG.warn("LogRotationTask interrupted while trying to acquire lock."); + } finally { + if (acquired) { + lock.unlock(); + } + } + } + } +} 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 index 01e9e13651b..3d454409679 100644 --- 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 @@ -158,9 +158,8 @@ public void start() throws IOException { /** * 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 { + public void stop() { ScheduledExecutorService schedulerToShutdown; synchronized (this) { @@ -205,6 +204,7 @@ public void stop() throws IOException { */ public void replay() throws IOException { Optional optionalNextRound = getNextRoundToProcess(); + LOG.info("replay round={}", optionalNextRound.isPresent()); while (optionalNextRound.isPresent()) { ReplicationRound replicationRound = optionalNextRound.get(); try { @@ -217,8 +217,18 @@ public void replay() throws IOException { setLastRoundProcessed(replicationRound); optionalNextRound = getNextRoundToProcess(); } + if (!optionalNextRound.isPresent()) { + // no more rounds to process + processNoMoreRoundsLeft(); + } } + /** + * Individual implementations can take specific actions when there are no + * more rounds ready to process. + */ + protected void processNoMoreRoundsLeft() throws IOException {} + /** * Returns the next replication round to process based on lastRoundProcessed. * Ensures sufficient time (round duration + buffer) has elapsed before returning the next @@ -229,6 +239,7 @@ public void replay() throws IOException { protected Optional getNextRoundToProcess() { long lastRoundEndTimestamp = getLastRoundProcessed().getEndTime(); long currentTime = EnvironmentEdgeManager.currentTime(); + LOG.info("last={} current={}", lastRoundEndTimestamp, currentTime); if (currentTime - lastRoundEndTimestamp < roundTimeMills + bufferMillis) { // nothing more to process return Optional.empty(); @@ -391,7 +402,7 @@ protected void initializeLastRoundProcessed() throws IOException { haGroupName, currentTime); this.lastRoundProcessed = replicationLogTracker .getReplicationShardDirectoryManager() - .getReplicationRoundFromEndTime(EnvironmentEdgeManager.currentTime()); + .getReplicationRoundFromEndTime(currentTime); } } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java new file mode 100644 index 00000000000..7f4ff3e5bd6 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java @@ -0,0 +1,183 @@ +package org.apache.phoenix.replication; + +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; + +import java.io.IOException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.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.metrics.MetricsReplicationLogDiscovery; +import org.apache.phoenix.replication.metrics.MetricsReplicationLogForwarderSourceFactory; +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ReplicationLogDiscoveryForwarder manages the forwarding of the replication log + * from the fallback cluster to the remote cluster. + */ +public class ReplicationLogDiscoveryForwarder extends ReplicationLogDiscovery { + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogDiscoveryForwarder.class); + + private final ReplicationLogGroup logGroup; + + /** + * Create a tracker for the replication logs in the fallback cluster. + * + * @param logGroup HAGroup + * @return ReplicationLogTracker + */ + private static ReplicationLogTracker createLogTracker(ReplicationLogGroup logGroup) { + ReplicationShardDirectoryManager localShardManager = logGroup.getFallbackShardManager(); + return new ReplicationLogTracker( + logGroup.conf, + logGroup.getHAGroupName(), + localShardManager, + MetricsReplicationLogForwarderSourceFactory. + getInstanceForTracker(logGroup.getHAGroupName())); + } + + public ReplicationLogDiscoveryForwarder(ReplicationLogGroup logGroup) { + super(createLogTracker(logGroup)); + this.logGroup = logGroup; + } + + @Override + public String getExecutorThreadNameFormat() { + return "ReplicationLogDiscoveryForwarder-" + logGroup.getHAGroupName() + "-%d"; + } + + public void init() throws IOException { + replicationLogTracker.init(); + // Initialize the discovery only. Forwarding begins only when we switch to the + // STORE_AND_FORWARD mode or SYNC_AND_FORWARD mode. + super.init(); + + // Set up a listener to the ACTIVE_NOT_IN_SYNC state. This is needed because whenever any + // RS switches to STORE_AND_FORWARD mode, other RS's in the cluster must move out of SYNC + // mode. + HAGroupStateListener activeNotInSync = (groupName, + fromState, + toState, + modifiedTime, + clusterType, + lastSyncStateTimeInMs) -> { + if (clusterType == ClusterType.LOCAL && HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC.equals(toState)) { + LOG.info("Received ACTIVE_NOT_IN_SYNC event for {}", logGroup); + // If the current mode is SYNC only then switch to SYNC_AND_FORWARD mode + if (logGroup.checkAndSetMode(SYNC, SYNC_AND_FORWARD)) { + // replication mode switched, notify the event handler + try { + logGroup.sync(); + } catch (IOException e) { + LOG.info("Failed to send sync event to {}", logGroup); + } + } + } + }; + + // Set up a listener to the ACTIVE_IN_SYNC state. This is needed because when the RS + // switches back to SYNC mode, the other RS's in the cluster must move out of + // SYNC_AND_FORWARD mode to SYNC mode. + HAGroupStateListener activeInSync = (groupName, + fromState, + toState, + modifiedTime, + clusterType, + lastSyncStateTimeInMs) -> { + if (clusterType == ClusterType.LOCAL && HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC.equals(toState)) { + LOG.info("Received ACTIVE_IN_SYNC event for {}", logGroup); + // Set the current mode to SYNC + if (logGroup.checkAndSetMode(SYNC_AND_FORWARD, SYNC)) { + // replication mode switched, notify the event handler + try { + logGroup.sync(); + } catch (IOException e) { + LOG.info("Failed to send sync event to {}", logGroup); + } + } + } + }; + + HAGroupStoreManager haGroupStoreManager = logGroup.getHAGroupStoreManager(); + haGroupStoreManager.subscribeToTargetState(logGroup.getHAGroupName(), + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC, ClusterType.LOCAL, activeNotInSync); + haGroupStoreManager.subscribeToTargetState(logGroup.getHAGroupName(), + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, ClusterType.LOCAL, activeInSync); + } + + @Override + protected void processFile(Path src) throws IOException { + FileSystem srcFS = replicationLogTracker.getFileSystem(); + FileStatus srcStat = srcFS.getFileStatus(src); + long ts = EnvironmentEdgeManager.currentTimeMillis(); + ReplicationShardDirectoryManager remoteShardManager = logGroup.getStandbyShardManager(); + Path dst = remoteShardManager.getWriterPath(ts, logGroup.getServerName().getServerName()); + long startTime = EnvironmentEdgeManager.currentTimeMillis(); + FileUtil.copy(srcFS, srcStat, remoteShardManager.getFileSystem(), dst, false, false, conf); + // successfully copied the file + long endTime = EnvironmentEdgeManager.currentTimeMillis(); + long copyTime = endTime - startTime; + LOG.info("Copying file src={} dst={} size={} took {}ms", src, dst, srcStat.getLen(), copyTime); + if (logGroup.getMode() == STORE_AND_FORWARD && + isLogCopyThroughputAboveThreshold(srcStat.getLen(), copyTime)) { + // start recovery by switching to SYNC_AND_FORWARD + if (logGroup.checkAndSetMode(STORE_AND_FORWARD, SYNC_AND_FORWARD)) { + // replication mode switched, notify the event handler + try { + logGroup.sync(); + } catch (IOException e) { + LOG.info("Failed to send sync event to {}", logGroup); + } + } + } + } + + @Override + protected void processNoMoreRoundsLeft() throws IOException { + // check if we are caught up so that we can transition to SYNC state + // we are caught up when there are no files currently in the out progress directory + // and no new files exist for ongoing round + if (replicationLogTracker.getInProgressFiles().isEmpty() + && replicationLogTracker.getNewFilesForRound(replicationLogTracker + .getReplicationShardDirectoryManager() + .getNextRound(getLastRoundProcessed())).isEmpty()) { + LOG.info("Processed all the replication log files for {}", logGroup); + // TODO ensure the mTime on the group store record is older than the wait sync timeout + logGroup.setHAGroupStatusToSync(); + } + } + + /** + * Determine if the throughput is above the configured threshold. If it is, then we can switch + * to the SYNC_AND_FORWARD mode + * + * @param fileSize + * @param copyTime + * @return True if the throughput is good else false + */ + private boolean isLogCopyThroughputAboveThreshold(long fileSize, long copyTime) { + // TODO: calculate throughput and check if is above configured threshold + return true; + } + + @Override + protected MetricsReplicationLogDiscovery createMetricsSource() { + return MetricsReplicationLogForwarderSourceFactory. + getInstanceForDiscovery(logGroup.getHAGroupName()); + } + + @VisibleForTesting + protected ReplicationLogTracker getReplicationLogTracker() { + return replicationLogTracker; + } +} 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 8fdd80c93c9..64842eb25a3 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 @@ -17,37 +17,117 @@ */ package org.apache.phoenix.replication; +import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; +import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; +import static org.apache.phoenix.replication.ReplicationLogGroup.LogEvent.EVENT_TYPE_DATA; +import static org.apache.phoenix.replication.ReplicationLogGroup.LogEvent.EVENT_TYPE_SYNC; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.INIT; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; + import java.io.IOException; +import java.io.InterruptedIOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; +import org.apache.phoenix.jdbc.HAGroupStoreManager; +import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.jdbc.HAGroupStoreRecord.HAGroupState; import org.apache.phoenix.replication.metrics.MetricsReplicationLogGroupSource; import org.apache.phoenix.replication.metrics.MetricsReplicationLogGroupSourceImpl; +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.phoenix.thirdparty.com.google.common.base.Throwables; +import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.lmax.disruptor.EventFactory; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.ExceptionHandler; +import com.lmax.disruptor.LifecycleAware; +import com.lmax.disruptor.RingBuffer; +import com.lmax.disruptor.YieldingWaitStrategy; +import com.lmax.disruptor.dsl.Disruptor; +import com.lmax.disruptor.dsl.ProducerType; + /** - * ReplicationLogGroup manages a group of replication logs for a given HA Group. + * ReplicationLogGroup manages replication logs for a given HA Group. *

* This class provides an API for replication operations and delegates to either synchronous - * replication (StandbyLogGroupWriter) or store-and-forward replication - * (StoreAndForwardLogGroupWriter) based on the current replication mode. - *

+ * replication or store-and-forward replication based on the current replication mode. + *

+ * Architecture Overview: + *
+ *  ┌──────────────────────────────────────────────────────────────────────┐
+ *  │                       ReplicationLogGroup                            │
+ *  │                                                                      │
+ *  │  ┌─────────────┐     ┌────────────────────────────────────────────┐  │
+ *  │  │             │     │                                            │  │
+ *  │  │  Producers  │     │  Disruptor Ring Buffer                     │  │
+ *  │  │  (append/   │────▶│  ┌─────────┐ ┌─────────┐ ┌─────────┐       │  │
+ *  │  │   sync)     │     │  │ Event 1 │ │ Event 2 │ │ Event 3 │ ...   │  │
+ *  │  │             │     │  └─────────┘ └─────────┘ └─────────┘       │  │
+ *  │  └─────────────┘     └────────────────────────────────────────────┘  │
+ *  │                                │                                     │
+ *  │                                ▼                                     │
+ *  │  ┌─────────────────────────────────────────────────────────────┐     │
+ *  │  │                                                             │     │
+ *  │  │  LogEventHandler                                            │     │
+ *  │  │  - Batch Management                                         │     │
+ *  │  │  - Replay events on errors                                  │     │
+ *  │  │  - Mode Transitions                                         │     │
+ *  │  │                                                             │     │
+ *  │  │  ┌──────────────────────────────────────────────────────┐   │     │
+ *  │  │  │  ReplicationModeImpl                                 │   │     │
+ *  │  │  │  - Mode entry/exit                                   │   │     │
+ *  │  │  │  - Failure Handling                                  │   │     │
+ *  │  │  │                                                      │   │     │
+ *  │  │  │  ┌────────────────────────────────────────────────┐  │   │     │
+ *  │  │  │  │  ReplicationLog                                │  │   │     │
+ *  │  │  │  │  ┌──────────────────────────────────────────┐  │  │   │     │
+ *  │  │  │  │  │  LogFileWriter                           │  │  │   │     │
+ *  │  │  │  │  │  - File Management                       │  │  │   │     │
+ *  │  │  │  │  │  - Compression                           │  │  │   │     │
+ *  │  │  │  │  │  - HDFS Operations                       │  │  │   │     │
+ *  │  │  │  │  └──────────────────────────────────────────┘  │  │   │     │
+ *  │  │  │  └────────────────────────────────────────────────┘  │   │     │
+ *  │  │  └──────────────────────────────────────────────────────┘   │     │
+ *  │  └─────────────────────────────────────────────────────────────┘     │
+ *  └──────────────────────────────────────────────────────────────────────┘
+ * 
* Key features: *
    - *
  • Manages multiple replication logs for an HA Group
  • *
  • Provides append() and sync() API for higher layers
  • - *
  • Delegates to appropriate writer implementation based on replication mode
  • + *
  • Manages the replication mode
  • + *
  • Delegates the append and sync events to the current replication mode
  • *
  • Thread-safe operations
  • *
*

- * The class delegates actual replication work to implementations of ReplicationLogGroupWriter: - *

    - *
  • StandbyLogGroupWriter: Synchronous replication to standby cluster
  • - *
  • StoreAndForwardLogGroupWriter: Local storage with forwarding when available
  • - *
+ * A high-performance ring buffer decouples the API from the complexity of writer management. + * Callers of append and sync generally return quickly, except for sync, where the writer must + * suspend the caller until the sync operation is successful (or times out). An internal single + * threaded process handles these events, encapsulating the complexity of batching mutations for + * efficiency, consolidating multiple in-flight syncs and mode transitions. */ public class ReplicationLogGroup { @@ -75,18 +155,19 @@ public class ReplicationLogGroup { public static final int DEFAULT_REPLICATION_LOG_RINGBUFFER_SIZE = 1024 * 32; public static final String REPLICATION_LOG_SYNC_TIMEOUT_KEY = "phoenix.replication.log.sync.timeout.ms"; - public static final long DEFAULT_REPLICATION_LOG_SYNC_TIMEOUT = 1000 * 30; public static final String REPLICATION_LOG_SYNC_RETRIES_KEY = "phoenix.replication.log.sync.retries"; - public static final int DEFAULT_REPLICATION_LOG_SYNC_RETRIES = 5; + public static final int DEFAULT_REPLICATION_LOG_SYNC_RETRIES = 4; public static final String REPLICATION_LOG_ROTATION_RETRIES_KEY = "phoenix.replication.log.rotation.retries"; public static final int DEFAULT_REPLICATION_LOG_ROTATION_RETRIES = 5; public static final String REPLICATION_LOG_RETRY_DELAY_MS_KEY = "phoenix.replication.log.retry.delay.ms"; public static final long DEFAULT_REPLICATION_LOG_RETRY_DELAY_MS = 100L; + private static final long DEFAULT_HDFS_WRITE_RPC_TIMEOUT_MS = 30*1000; - public static final String FILE_NAME_FORMAT = "%d_%s.plog"; + public static final String STANDBY_DIR = "in"; + public static final String FALLBACK_DIR = "out"; /** Cache of ReplicationLogGroup instances by HA Group ID */ protected static final ConcurrentHashMap INSTANCES = @@ -95,42 +176,51 @@ public class ReplicationLogGroup { protected final Configuration conf; protected final ServerName serverName; protected final String haGroupName; - protected ReplicationLogGroupWriter remoteWriter; - protected ReplicationLogGroupWriter localWriter; - protected ReplicationMode mode; - protected volatile boolean closed = false; + protected final HAGroupStoreManager haGroupStoreManager; protected final MetricsReplicationLogGroupSource metrics; + protected ReplicationShardDirectoryManager standbyShardManager; + protected ReplicationShardDirectoryManager fallbackShardManager; + protected ReplicationLogDiscoveryForwarder logForwarder; + protected long syncTimeoutMs; + protected volatile boolean closed = false; /** - * Tracks the current replication mode of the ReplicationLog. - *

- * The replication mode determines how mutations are handled: - *

    - *
  • SYNC: Normal operation where mutations are written directly to the standby cluster's - * HDFS. - * This is the default and primary mode of operation.
  • - *
  • STORE_AND_FORWARD: Fallback mode when the standby cluster's HDFS is unavailable. - * Mutations are stored locally and will be forwarded when connectivity is restored.
  • - *
  • SYNC_AND_FORWARD: Transitional mode where new mutations are written directly to the - * standby cluster while concurrently draining the local queue of previously stored - * mutations.
  • - *
- *

+ * The replication mode determines how mutations are handled. * Mode transitions occur automatically based on the availability of the standby cluster's HDFS * and the state of the local mutation queue. */ - protected enum ReplicationMode { + public enum ReplicationMode { + /** + * Dummy mode before we transition to the mode based on the HAGroupStore state + */ + INIT { + @Override + ReplicationModeImpl createModeImpl(ReplicationLogGroup logGroup) { + return new Init(logGroup); + } + }, + /** * Normal operation where mutations are written directly to the standby cluster's HDFS. * This is the default and primary mode of operation. */ - SYNC, + SYNC { + @Override + ReplicationModeImpl createModeImpl(ReplicationLogGroup logGroup) { + return new SyncModeImpl(logGroup); + } + }, /** * Fallback mode when the standby cluster's HDFS is unavailable. Mutations are stored * locally and will be forwarded when connectivity is restored. */ - STORE_AND_FORWARD, + STORE_AND_FORWARD { + @Override + ReplicationModeImpl createModeImpl(ReplicationLogGroup logGroup) { + return new StoreAndForwardModeImpl(logGroup); + } + }, /** * Transitional mode where new mutations are written directly to the standby cluster @@ -138,8 +228,98 @@ protected enum ReplicationMode { * is entered when connectivity to the standby cluster is restored and there are still * mutations in the local queue. */ - SYNC_AND_FORWARD; + SYNC_AND_FORWARD { + @Override + ReplicationModeImpl createModeImpl(ReplicationLogGroup logGroup) { + return new SyncAndForwardModeImpl(logGroup); + } + }; + + abstract ReplicationModeImpl createModeImpl(ReplicationLogGroup logGroup); + } + // Tracks the current replication mode of the ReplicationLogGroup. + private AtomicReference mode; + + /* + * Dummy mode before we transition to the mode based on the HAGroupStore state + */ + private static class Init extends ReplicationModeImpl { + Init(ReplicationLogGroup logGroup) { + super(logGroup); + } + + @Override + void onEnter() throws IOException {} + + @Override + void onExit(boolean gracefulShutdown) {} + + @Override + ReplicationMode onFailure(Throwable e) throws IOException { + throw new UnsupportedOperationException("Not supported for " + this); + } + + @Override + void append(Record r) throws IOException { + throw new UnsupportedOperationException("Not supported for " + this); + } + + @Override + void sync() throws IOException { + throw new UnsupportedOperationException("Not supported for " + this); + } + + @Override + ReplicationMode getMode() { + return INIT; + } + } + + private static final ImmutableMap> allowedTransition = + Maps.immutableEnumMap(ImmutableMap.of( + INIT, EnumSet.of(SYNC, STORE_AND_FORWARD), + SYNC, EnumSet.of(STORE_AND_FORWARD, SYNC_AND_FORWARD), + STORE_AND_FORWARD, EnumSet.of(SYNC_AND_FORWARD), + SYNC_AND_FORWARD, EnumSet.of(SYNC, STORE_AND_FORWARD)) + ); + + + /** Event structure for the Disruptor ring buffer containing data and sync operations. */ + protected static class LogEvent { + protected static final EventFactory EVENT_FACTORY = LogEvent::new; + + protected int type; + protected Record record; + protected CompletableFuture syncFuture; // Used only for SYNC events + protected long timestampNs; // Timestamp when event was created + + public static final byte EVENT_TYPE_DATA = 0; + public static final byte EVENT_TYPE_SYNC = 1; + + public void setValues(int type, Record record, CompletableFuture syncFuture) { + this.type = type; + this.record = record; + this.syncFuture = syncFuture; + this.timestampNs = System.nanoTime(); + } + } + + protected static class Record { + public String tableName; + public long commitId; + public Mutation mutation; + + public Record(String tableName, long commitId, Mutation mutation) { + this.tableName = tableName; + this.commitId = commitId; + this.mutation = mutation; + } } + protected Disruptor disruptor; + protected RingBuffer ringBuffer; + protected LogEventHandler eventHandler; + // Used to inform the disruptor event thread whether this is a graceful or a forced shutdown + private final AtomicBoolean gracefulShutdownEventHandlerFlag = new AtomicBoolean(); /** * Get or create a ReplicationLogGroup instance for the given HA Group. @@ -150,8 +330,9 @@ protected enum ReplicationMode { * @return ReplicationLogGroup instance * @throws RuntimeException if initialization fails */ - public static ReplicationLogGroup get(Configuration conf, ServerName serverName, - String haGroupName) { + public static ReplicationLogGroup get(Configuration conf, + ServerName serverName, + String haGroupName) { return INSTANCES.computeIfAbsent(haGroupName, k -> { try { ReplicationLogGroup group = new ReplicationLogGroup(conf, serverName, haGroupName); @@ -164,102 +345,193 @@ public static ReplicationLogGroup get(Configuration conf, ServerName serverName, }); } + /** + * Get or create a ReplicationLogGroup instance for the given HA Group. + * Used mainly for testing + * + * @param conf Configuration object + * @param serverName The server name + * @param haGroupName The HA Group name + * @param haGroupStoreManager HA Group Store Manager instance + * @return ReplicationLogGroup instance + * @throws RuntimeException if initialization fails + */ + public static ReplicationLogGroup get(Configuration conf, + ServerName serverName, + String haGroupName, + HAGroupStoreManager haGroupStoreManager) { + return INSTANCES.computeIfAbsent(haGroupName, k -> { + try { + ReplicationLogGroup group = new ReplicationLogGroup(conf, + serverName, haGroupName, haGroupStoreManager); + group.init(); + return group; + } catch (IOException e) { + LOG.error("Failed to create ReplicationLogGroup for HA Group: {}", haGroupName, e); + throw new RuntimeException(e); + } + }); + } + + /** + * Protected constructor for ReplicationLogGroup. + * + * @param conf Configuration object + * @param serverName The server name + * @param haGroupName The HA Group name + */ + protected ReplicationLogGroup(Configuration conf, + ServerName serverName, + String haGroupName) { + this(conf, serverName, haGroupName, HAGroupStoreManager.getInstance(conf)); + } + /** * Protected constructor for ReplicationLogGroup. * * @param conf Configuration object * @param serverName The server name * @param haGroupName The HA Group name + * @param haGroupStoreManager HA Group Store Manager instance */ - protected ReplicationLogGroup(Configuration conf, ServerName serverName, String haGroupName) { + protected ReplicationLogGroup(Configuration conf, + ServerName serverName, + String haGroupName, + HAGroupStoreManager haGroupStoreManager) { this.conf = conf; this.serverName = serverName; this.haGroupName = haGroupName; + this.haGroupStoreManager = haGroupStoreManager; this.metrics = createMetricsSource(); + this.mode = new AtomicReference<>(INIT); } /** - * Initialize the ReplicationLogGroup by creating the appropriate writer implementation. + * Initialize the ReplicationLogGroup * * @throws IOException if initialization fails */ protected void init() throws IOException { - // We need the local writer created first if we intend to fall back to it should the init - // of the remote writer fail. - localWriter = createLocalWriter(); - // Initialize the remote writer and set the mode to SYNC. TODO: switch instead of set - mode = ReplicationMode.SYNC; - remoteWriter = createRemoteWriter(); - // TODO: Switch the initial mode to STORE_AND_FORWARD if the remote writer fails to - // initialize. - LOG.info("Started ReplicationLogGroup for HA Group: {}", haGroupName); + // First initialize the shard managers + this.standbyShardManager = createStandbyShardManager(); + this.fallbackShardManager = createFallbackShardManager(); + // Initialize the replication log forwarder. The log forwarder is only activated when + // we switch to STORE_AND_FORWARD or SYNC_AND_FORWARD mode + this.logForwarder = new ReplicationLogDiscoveryForwarder(this); + this.logForwarder.init(); + // Initialize the replication mode based on the HAGroupStore state + initializeReplicationMode(); + // Use the override value if provided in the config, else use a derived value + this.syncTimeoutMs = conf.getLong(REPLICATION_LOG_SYNC_TIMEOUT_KEY, + calculateSyncTimeout()); + // Initialize the disruptor so that we start processing events + initializeDisruptor(); + LOG.info("HAGroup {} started with mode={}", this, mode); } /** - * Get the name for this HA Group. + * Calculate how long the application thread should wait for a sync to finish. + * The application thread here is the write rpc handler thread. It takes into account + * the number of retries, pause between successive attempts, dfs write timeout and zk + * session timeouts. * - * @return The name for this HA Group - */ - public String getHaGroupName() { - return haGroupName; + * @return sync timeout in ms + */ + protected long calculateSyncTimeout() { + int maxAttempts = conf.getInt(REPLICATION_LOG_SYNC_RETRIES_KEY, + DEFAULT_REPLICATION_LOG_SYNC_RETRIES) + 1; + long retryDelayMs = conf.getLong(REPLICATION_LOG_RETRY_DELAY_MS_KEY, + DEFAULT_REPLICATION_LOG_RETRY_DELAY_MS); + long wrtiteRpcTimeout = conf.getLong(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, + DEFAULT_HDFS_WRITE_RPC_TIMEOUT_MS); + // account for HAGroupStore update when we switch replication mode + long zkTimeoutMs = conf.getLong(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT); + long totalRpcTimeout = maxAttempts*wrtiteRpcTimeout + (maxAttempts - 1)*retryDelayMs; + return 2*totalRpcTimeout + zkTimeoutMs; } - protected Configuration getConfiguration() { - return conf; + /** + * Initialize the replication mode based on the HAGroupStore state + * + * @throws IOException + */ + protected void initializeReplicationMode() throws IOException { + Optional haGroupStoreRecord = + haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + if (haGroupStoreRecord.isPresent()) { + HAGroupStoreRecord record = haGroupStoreRecord.get(); + HAGroupState haGroupState = record.getHAGroupState(); + if (haGroupState.equals(HAGroupState.ACTIVE_IN_SYNC)) { + setMode(SYNC); + } else if (haGroupState.equals(HAGroupState.ACTIVE_NOT_IN_SYNC)) { + setMode(STORE_AND_FORWARD); + } else { + String message = String.format("HAGroup %s got an unexpected state %s while " + + "initializing mode", this, haGroupState); + LOG.error(message); + throw new IOException(message); + } + } else { + String message = String.format("HAGroup %s got an empty group store record while " + + "initializing mode", this); + LOG.error(message); + throw new IOException(message); + } } - protected ServerName getServerName() { - return serverName; + /** Initialize the Disruptor. */ + @SuppressWarnings("unchecked") + protected void initializeDisruptor() throws IOException { + int ringBufferSize = conf.getInt(REPLICATION_LOG_RINGBUFFER_SIZE_KEY, + DEFAULT_REPLICATION_LOG_RINGBUFFER_SIZE); + disruptor = new Disruptor<>(LogEvent.EVENT_FACTORY, ringBufferSize, + new ThreadFactoryBuilder() + .setNameFormat("ReplicationLogGroup-" + getHAGroupName() + "-%d") + .setDaemon(true).build(), + ProducerType.MULTI, new YieldingWaitStrategy()); + eventHandler = new LogEventHandler(); + eventHandler.init(); + disruptor.handleEventsWith(eventHandler); + LogExceptionHandler exceptionHandler = new LogExceptionHandler(); + disruptor.setDefaultExceptionHandler(exceptionHandler); + ringBuffer = disruptor.start(); } /** - * Append a mutation to the replication log group. This operation is normally non-blocking - * unless the ring buffer is full. + * Append a mutation to the log. This method is non-blocking and returns quickly, unless the + * ring buffer is full. The actual write happens asynchronously. We expect multiple append() + * calls followed by a sync(). The appends will be batched by the Disruptor. Should the ring + * buffer become full, which is not expected under normal operation but could (and should) + * happen if the log file writer is unable to make progress, due to a HDFS level disruption. + * Should we enter that condition this method will block until the append can be inserted. + *

+ * An internal error may trigger fail-stop behavior. Subsequent to fail-stop, this method will + * throw an IOException("Closed"). No further appends are allowed. * - * @param tableName The name of the HBase table the mutation applies to - * @param commitId The commit identifier (e.g., SCN) associated with the mutation - * @param mutation The HBase Mutation (Put or Delete) to be logged - * @throws IOException If the operation fails + * @param tableName The name of the HBase table the mutation applies to. + * @param commitId The commit identifier (e.g., SCN) associated with the mutation. + * @param mutation The HBase Mutation (Put or Delete) to be logged. + * @throws IOException If the writer is closed or if the ring buffer is full. */ public void append(String tableName, long commitId, Mutation mutation) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Append: table={}, commitId={}, mutation={}", tableName, commitId, mutation); + } if (closed) { throw new IOException("Closed"); } long startTime = System.nanoTime(); try { - switch (mode) { - case SYNC: - // In sync mode, we only write to the remote writer. - try { - remoteWriter.append(tableName, commitId, mutation); - } catch (IOException e) { - // TODO: If the remote writer fails, we must switch to store and forward. - LOG.warn("Mode switching not implemented"); - throw e; - } - break; - case SYNC_AND_FORWARD: - // In sync and forward mode, we write to only the remote writer, while in the - // background we are draining the local queue. - try { - remoteWriter.append(tableName, commitId, mutation); - } catch (IOException e) { - // TODO: If the remote writer fails again, we must switch back to store and - // forward. - LOG.warn("Mode switching not implemented"); - throw e; - } - break; - case STORE_AND_FORWARD: - // In store and forward mode, we append to the local writer. If we fail it's a - // critical failure. - localWriter.append(tableName, commitId, mutation); - // TODO: Probe the state of the remoteWriter. Can we switch back? - // TODO: This suggests the ReplicationLogGroupWriter interface should have a status - // probe API. - break; - default: - throw new IllegalStateException("Invalid replication mode: " + mode); + // ringBuffer.next() claims the next sequence number. Because we initialize the Disruptor + // with ProducerType.MULTI and the blocking YieldingWaitStrategy this call WILL BLOCK if + // the ring buffer is full, thus providing backpressure to the callers. + long sequence = ringBuffer.next(); + try { + LogEvent event = ringBuffer.get(sequence); + event.setValues(EVENT_TYPE_DATA, new Record(tableName, commitId, mutation), null); + } finally { + // Update ring buffer events metric + ringBuffer.publish(sequence); } } finally { metrics.updateAppendTime(System.nanoTime() - startTime); @@ -267,56 +539,70 @@ public void append(String tableName, long commitId, Mutation mutation) throws IO } /** - * Ensure all previously appended records are durably persisted. This method blocks until the - * sync operation completes or fails. - * - * @throws IOException If the sync operation fails + * Ensures all previously appended records are durably persisted. This method blocks until the + * sync operation completes or fails, potentially after internal retries. All in flight appends + * are batched and provided to the underlying LogWriter, which will then be synced. If there is + * a problem syncing the LogWriter we will retry, up to the retry limit, rolling the writer for + * each retry. If the operation is still not successful and if we are in SYNC mode or in + * SYNC_AND_FORWARD mode then we switch the mode to STORE_AND_FORWARD and try again. If the + * sync operation still fails after switching to the STORE_AND_FORWARD mode then we treat this + * as a fatal error and abort the region server. + *

+ * An internal error may trigger fail-stop behavior. Subsequent to fail-stop, this method will + * throw an IOException("Closed"). No further syncs are allowed. + *

+ + * @throws IOException If the sync operation fails after retries, or if interrupted. */ public void sync() throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Sync"); + } if (closed) { throw new IOException("Closed"); } long startTime = System.nanoTime(); try { - switch (mode) { - case SYNC: - // In sync mode, we only write to the remote writer. - try { - remoteWriter.sync(); - } catch (IOException e) { - // TODO: If the remote writer fails, we must switch to store and forward. - LOG.warn("Mode switching not implemented"); - throw e; - } - break; - case SYNC_AND_FORWARD: - // In sync and forward mode, we write to only the remote writer, while in the - // background we are draining the local queue. - try { - remoteWriter.sync(); - } catch (IOException e) { - // TODO: If the remote writer fails again, we must switch back to store and - // forward. - LOG.warn("Mode switching not implemented"); - throw e; - } - break; - case STORE_AND_FORWARD: - // In store and forward mode, we sync the local writer. If we fail it's a critical - // failure. - localWriter.sync(); - // TODO: Probe the state of the remoteWriter. Can we switch back? - // TODO: This suggests the ReplicationLogGroupWriter interface should have a - // status probe API. - break; - default: - throw new IllegalStateException("Invalid replication mode: " + mode); - } + syncInternal(); } finally { metrics.updateSyncTime(System.nanoTime() - startTime); } } + /** + * Internal implementation of sync that publishes a sync event to the ring buffer and waits + * for completion. + */ + protected void syncInternal() throws IOException { + CompletableFuture syncFuture = new CompletableFuture<>(); + long sequence = ringBuffer.next(); + try { + LogEvent event = ringBuffer.get(sequence); + event.setValues(EVENT_TYPE_SYNC, null, syncFuture); + } finally { + ringBuffer.publish(sequence); + } + LOG.trace("Published EVENT_TYPE_SYNC at sequence {}", sequence); + try { + // Wait for the event handler to process up to and including this sync event + syncFuture.get(syncTimeoutMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new InterruptedIOException("Interrupted while waiting for sync"); + } catch (ExecutionException e) { + // After exhausting all attempts to sync to the standby cluster we switch mode + // and then retry again. If that also fails, it is a fatal error + String message = String.format("HAGroup %s sync operation failed", this); + LOG.error(message, e); + abort(message, e); + } catch (TimeoutException e) { + String message = String.format("HAGroup %s sync operation timed out", this); + LOG.error(message); + // sync timeout is a fatal error + abort(message, e); + } + } + /** * Check if this ReplicationLogGroup is closed. * @@ -326,6 +612,31 @@ public boolean isClosed() { return closed; } + /** + * Force closes the log group upon an unrecoverable internal error. + * This is a fail-stop behavior: once called, the log group is marked as closed, + * the Disruptor is halted, and all subsequent append() and sync() calls will + * throw an IOException("Closed"). This ensures that no further operations are attempted on a + * log group that has encountered a critical error. + */ + protected void closeOnError() { + if (closed) { + return; + } + synchronized (this) { + if (closed) { + return; + } + closed = true; + } + // Directly halt the disruptor. shutdown() would wait for events to drain. We are expecting + // that will not work. + gracefulShutdownEventHandlerFlag.set(false); + disruptor.halt(); + metrics.close(); + LOG.info("HAGroup {} closed on error", this); + } + /** * Close the ReplicationLogGroup and all associated resources. This method is thread-safe and * can be called multiple times. @@ -341,31 +652,61 @@ public void close() { closed = true; // Remove from instances cache INSTANCES.remove(haGroupName); - // Close the writers, remote first. If there are any problems closing the remote writer - // the pending writes will be sent to the local writer instead, during the appropriate - // mode switch. - closeWriter(remoteWriter); - closeWriter(localWriter); + // Sync before shutting down to flush all pending appends. + try { + syncInternal(); + gracefulShutdownEventHandlerFlag.set(true); + disruptor.shutdown(); // Wait for a clean shutdown. + } catch (IOException e) { + LOG.warn("Error during final sync on close", e); + gracefulShutdownEventHandlerFlag.set(false); + disruptor.halt(); // Go directly to halt. + } + // TODO revisit close logic and the below comment + // We must wait for the disruptor before closing the writers. metrics.close(); - LOG.info("Closed ReplicationLogGroup for HA Group: {}", haGroupName); + LOG.info("HAGroup {} closed", this); } } /** - * Switch the replication mode. + * Switch the replication mode to the new mode * - * @param mode The new replication mode - * @param reason The reason for the mode switch - * @throws IOException If the mode switch fails + * @param newReplicationMode The new replication mode + * @return previous replication mode */ - public void switchMode(ReplicationMode mode, Throwable reason) throws IOException { - // TODO: Implement mode switching guardrails and transition logic. - // TODO: We will be interacting with the HA Group Store to switch modes. + protected ReplicationMode setMode(ReplicationMode newReplicationMode) { + ReplicationMode previous = mode.getAndUpdate( current -> newReplicationMode); + if (previous != newReplicationMode) { + LOG.info("HAGroup {} switched from {} to {}", this, previous, newReplicationMode); + } + return previous; + } - // TODO: Drain the disruptor ring from the remote writer to the local writer when making - // transitions from SYNC or SYNC_AND_FORWARD to STORE_AND_FORWARD. + /** + * Switch the replication mode only if the current mode matches the expected mode. + * + * @param expectedReplicationMode + * @param newReplicationMode + * @return true if the mode was updated else false + */ + protected boolean checkAndSetMode(ReplicationMode expectedReplicationMode, + ReplicationMode newReplicationMode) { + boolean updated = mode.compareAndSet(expectedReplicationMode, newReplicationMode); + if (updated) { + LOG.info("HAGroup {} conditionally switched from {} to {}", this, + expectedReplicationMode, newReplicationMode); + } else { + LOG.info("HAGroup {} ignoring attempt to switch replication mode to {} " + + "because expected={} != actual={}", this, newReplicationMode, + expectedReplicationMode, getMode()); + } + return updated; + } - throw new UnsupportedOperationException("Mode switching is not implemented"); + /** Get the current replication mode */ + protected ReplicationMode getMode() { + return mode.get(); } /** Get the current metrics source for monitoring operations. */ @@ -378,38 +719,408 @@ protected MetricsReplicationLogGroupSource createMetricsSource() { return new MetricsReplicationLogGroupSourceImpl(haGroupName); } - /** Close the given writer. */ - protected void closeWriter(ReplicationLogGroupWriter writer) { - if (writer != null) { - writer.close(); + /** + * Get the name for this HA Group. + * + * @return The name for this HA Group + */ + public String getHAGroupName() { + return haGroupName; + } + + protected HAGroupStoreManager getHAGroupStoreManager() { + return haGroupStoreManager; + } + + protected Configuration getConfiguration() { + return conf; + } + + protected ServerName getServerName() { + return serverName; + } + + @Override + public String toString() { + return getHAGroupName(); + } + + /** + * Creates the top level directory on the cluster determined by the URI + * + * @param urlKey Config property for the URI + * @param logDirName Top level directory underneath which the shards will be created + * + * @return ReplicationShardDirectoryManager + * @throws IOException + */ + private ReplicationShardDirectoryManager createShardManager( + String urlKey, String logDirName) throws IOException { + URI rootURI = getLogURI(urlKey); + FileSystem fs = getFileSystem(rootURI); + LOG.info("HAGroup {} initialized filesystem at {}", this, rootURI); + // root dir path is //[in|out] + Path rootDirPath = new Path(new Path(rootURI.getPath(), getHAGroupName()), logDirName); + if (!fs.exists(rootDirPath)) { + LOG.info("HAGroup {} creating root directory at {}", this, rootDirPath); + if (!fs.mkdirs(rootDirPath)) { + throw new IOException("Failed to create directory: " + rootDirPath); + } + } + return new ReplicationShardDirectoryManager(conf, fs, rootDirPath); + } + + /** create shard manager for the standby cluster */ + protected ReplicationShardDirectoryManager createStandbyShardManager() throws IOException { + return createShardManager(REPLICATION_STANDBY_HDFS_URL_KEY, STANDBY_DIR); + } + + /** create shard manager for the fallback cluster */ + protected ReplicationShardDirectoryManager createFallbackShardManager() throws IOException { + return createShardManager(REPLICATION_FALLBACK_HDFS_URL_KEY, FALLBACK_DIR); + } + + /** return shard manager for the standby cluster */ + protected ReplicationShardDirectoryManager getStandbyShardManager() { + return standbyShardManager; + } + + /** return shard manager for the fallback cluster */ + protected ReplicationShardDirectoryManager getFallbackShardManager() { + return fallbackShardManager; + } + + private URI getLogURI(String urlKey) throws IOException { + String urlString = conf.get(urlKey); + if (urlString == null || urlString.trim().isEmpty()) { + throw new IOException("HDFS URL not configured: " + urlKey); + } + try { + return new URI(urlString); + } catch (URISyntaxException e) { + throw new IOException("Invalid HDFS URL: " + urlString, e); } } - /** Create the remote (synchronous) writer. Mainly for tests. */ - protected ReplicationLogGroupWriter createRemoteWriter() throws IOException { - ReplicationLogGroupWriter writer = new StandbyLogGroupWriter(this); - writer.init(); - return writer; + private FileSystem getFileSystem(URI uri) throws IOException { + return FileSystem.get(uri, conf); + } + + /** Create the standby(synchronous) writer */ + protected ReplicationLog createStandbyLog() throws IOException { + return new ReplicationLog(this, standbyShardManager); + } + + /** Create the fallback writer */ + protected ReplicationLog createFallbackLog() throws IOException { + return new ReplicationLog(this, fallbackShardManager); } - /** Create the local (store and forward) writer. Mainly for tests. */ - protected ReplicationLogGroupWriter createLocalWriter() throws IOException { - ReplicationLogGroupWriter writer = new StoreAndForwardLogGroupWriter(this); - writer.init(); - return writer; + /** Returns the log forwarder for this replication group */ + protected ReplicationLogDiscoveryForwarder getLogForwarder() { + return logForwarder; } /** Returns the currently active writer. Mainly for tests. */ - protected ReplicationLogGroupWriter getActiveWriter() { - switch (mode) { - case SYNC: - return remoteWriter; - case SYNC_AND_FORWARD: - return remoteWriter; - case STORE_AND_FORWARD: - return localWriter; - default: - throw new IllegalStateException("Invalid replication mode: " + mode); + protected ReplicationLog getActiveLog() { + return eventHandler.getCurrentModeImpl().getReplicationLog(); + } + + protected void setHAGroupStatusToStoreAndForward() throws Exception { + try { + haGroupStoreManager.setHAGroupStatusToStoreAndForward(haGroupName); + } + catch (Exception ex) { + LOG.info("HAGroup {} failed to set status to STORE_AND_FORWARD", this, ex); + throw ex; + } + } + + protected void setHAGroupStatusToSync() throws IOException { + try { + haGroupStoreManager.setHAGroupStatusToSync(haGroupName); + } catch (IOException ex) { + // TODO logging + throw ex; + } + catch (Exception ex) { + // TODO logging + throw new IOException(ex); + } + } + + /** + * Abort when we hit a fatal error + * + * @param reason + * @param cause + */ + protected void abort(String reason, Throwable cause) { + // TODO better to use abort using RegionServerServices + String msg = "***** ABORTING region server: " + reason + " *****"; + if (cause != null) { + msg += "\nCause:\n" + Throwables.getStackTraceAsString(cause); + } + LOG.error(msg); + if (cause != null) { + throw new RuntimeException(msg, cause); + } else { + throw new RuntimeException(msg); + } + } + + /** + * Handles events from the Disruptor, + */ + protected class LogEventHandler implements EventHandler, LifecycleAware { + private final List> pendingSyncFutures = new ArrayList<>(); + // Current replication mode implementation which will handle the events + private ReplicationModeImpl currentModeImpl; + + public LogEventHandler() { + } + + public void init() throws IOException { + initializeMode(getMode()); + } + + @VisibleForTesting + public ReplicationModeImpl getCurrentModeImpl() { + return currentModeImpl; + } + + private void initializeMode(ReplicationMode newMode) throws IOException { + try { + currentModeImpl = newMode.createModeImpl(ReplicationLogGroup.this); + currentModeImpl.onEnter(); + } catch (IOException e) { + LOG.error("HAGroup {} couldn't initialize mode {}", + ReplicationLogGroup.this, currentModeImpl, e); + updateModeOnFailure(e); + } + } + + private void updateModeOnFailure(IOException e) throws IOException { + // send the failed event to the current mode + ReplicationMode newMode = currentModeImpl.onFailure(e); + setMode(newMode); + currentModeImpl.onExit(true); + initializeMode(newMode); + } + + /** + * Processes all pending sync operations by syncing the current writer and completing + * their associated futures. This method is called when we are ready to process a set of + * consolidated sync requests and performs the following steps: + *

    + *
  1. Syncs the current writer to ensure all data is durably written.
  2. + *
  3. Completes all pending sync futures successfully.
  4. + *
  5. Clears the list of pending sync futures.
  6. + *
  7. Clears the current batch of records since they have been successfully synced.
  8. + *
+ * @param sequence The sequence number of the last processed event + * @throws IOException if the sync operation fails + */ + private void processPendingSyncs(long sequence) throws IOException { + if (pendingSyncFutures.isEmpty()) { + return; + } + // call sync on the current mode + currentModeImpl.sync(); + // Complete all pending sync futures + for (CompletableFuture future : pendingSyncFutures) { + future.complete(null); + } + pendingSyncFutures.clear(); + LOG.info("Sync operation completed successfully up to sequence {}", sequence); + } + + /** + * Fails all pending sync operations with the given exception. This method is called when + * we encounter an unrecoverable error during the sync of the inner writer. It completes + * all pending sync futures that were consolidated exceptionally. + *

+ * Note: This method does not clear the currentBatch list. The currentBatch must be + * preserved as it contains records that may need to be replayed if we successfully + * rotate to a new writer. + * + * @param sequence The sequence number of the last processed event + * @param e The IOException that caused the failure + */ + private void failPendingSyncs(long sequence, IOException e) { + if (pendingSyncFutures.isEmpty()) { + return; + } + for (CompletableFuture future : pendingSyncFutures) { + future.completeExceptionally(e); + } + pendingSyncFutures.clear(); + LOG.warn("Failed to process syncs at sequence {}", sequence, e); + } + + /** + * Handle the failure while processing an event + * + * @param failedEvent Event which triggered the failure + * @param sequence Sequence number of the failed event + * @param cause Reason of failure + */ + private void onFailure(LogEvent failedEvent, + long sequence, + IOException cause) throws IOException { + // fetch the in-flight appends + List unsyncedAppends = currentModeImpl.log.getCurrentBatch(); + // try updating the mode + updateModeOnFailure(cause); + // retry the batch after updating the mode + replayBatch(unsyncedAppends); + // retry the failed event + replayFailedEvent(failedEvent, sequence); + } + + /** Replay all append events which were not yet synced */ + private void replayBatch(List unsyncedAppends) throws IOException { + for (Record r : unsyncedAppends) { + currentModeImpl.append(r); + } + } + + /** Retry the failed event after switching mode */ + private void replayFailedEvent(LogEvent failedEvent, + long sequence) throws IOException { + // now retry the event which failed + // only need to retry append event since for sync event we have already added the + // sync event future to the pending future list before the sync event can potentially + // fail. + if (failedEvent.type == EVENT_TYPE_DATA) { + currentModeImpl.append(failedEvent.record); + } + processPendingSyncs(sequence); + } + + /** + * Processes a single event from the Disruptor ring buffer. This method handles both data + * and sync events. + *

+ * For data events, it: + *

    + *
  1. Sends the append event to the current mode.
  2. + *
  3. Processes any pending syncs if this is the end of a batch.
  4. + *
+ *

+ * For sync events, it: + *

    + *
  1. Adds the sync future to the pending list.
  2. + *
  3. Processes any pending syncs if this is the end of a batch.
  4. + *
+ *

+ * If an IOException occurs, it sends the failure event to the current replication mode + * and then switches to the new mode. After switching to the new mode, it replays the + * pending batch of un-synced appends and the failed event. + *

+ * + * @param event The event to process + * @param sequence The sequence number of the event + * @param endOfBatch Whether this is the last event in the current batch + * @throws Exception if the operation fails after all retries + */ + @Override + public void onEvent(LogEvent event, long sequence, boolean endOfBatch) throws Exception { + // Calculate time spent in ring buffer + long currentTimeNs = System.nanoTime(); + long ringBufferTimeNs = currentTimeNs - event.timestampNs; + metrics.updateRingBufferTime(ringBufferTimeNs); + try { + switch (event.type) { + case EVENT_TYPE_DATA: + currentModeImpl.append(event.record); + // Process any pending syncs at the end of batch. + if (endOfBatch) { + processPendingSyncs(sequence); + } + return; + case EVENT_TYPE_SYNC: + // Add this sync future to the pending list + // OK, to add the same future multiple times when we rewind the batch + // as completing an already completed future is a no-op + pendingSyncFutures.add(event.syncFuture); + // Process any pending syncs at the end of batch. + if (endOfBatch) { + processPendingSyncs(sequence); + } + // after a successful sync check the mode set on the replication group + // Doing the mode check on sync points makes the implementation more robust + // since we can guarantee that all unsynced appends have been flushed to the + // replication log before we switch the replication mode + ReplicationMode newMode = getMode(); + if (newMode != currentModeImpl.getMode()) { + // some other thread switched the mode on the replication group + LOG.info("Mode switched at sequence {} from {} to {}", + sequence, currentModeImpl, newMode); + // call exit on the last mode here since we can guarantee that the lastMode + // is not processing any event like append/sync because this is the only thread + // that is consuming the events from the ring buffer and handing them off to the + // mode + currentModeImpl.onExit(true); + initializeMode(newMode); + } + return; + default: + throw new UnsupportedOperationException("Unknown event type: " + + event.type); + } + } catch (IOException e) { + try { + LOG.info("Failed to process event at sequence {} on mode {}", sequence, currentModeImpl, e); + onFailure(event, sequence, e); + } catch (Exception fatalEx) { + // Either we failed to switch the mode or we are in STORE_AND_FORWARD mode + // and got an exception. This is a fatal exception so halt the disruptor + // fail the pending sync events with the original exception + failPendingSyncs(sequence, e); + // halt the disruptor with the fatal exception + throw fatalEx; + } + } + } + + @Override + public void onStart() { + // no-op + } + + @Override + public void onShutdown() { + boolean isGracefulShutdown = gracefulShutdownEventHandlerFlag.get(); + LOG.info("HAGroup {} shutting down event handler graceful={}", + ReplicationLogGroup.this, isGracefulShutdown); + currentModeImpl.onExit(isGracefulShutdown); + } + } + + /** + * Handler for critical errors during the Disruptor lifecycle that closes the writer to prevent + * data loss. + */ + protected class LogExceptionHandler implements ExceptionHandler { + @Override + public void handleEventException(Throwable e, long sequence, LogEvent event) { + String message = "Exception processing sequence " + sequence + " for event " + event; + LOG.error(message, e); + closeOnError(); + } + + @Override + public void handleOnStartException(Throwable e) { + LOG.error("Exception during Disruptor startup", e); + closeOnError(); + } + + @Override + public void handleOnShutdownException(Throwable e) { + // Should not happen, but if it does, the regionserver is aborting or shutting down. + LOG.error("Exception during Disruptor shutdown", e); + closeOnError(); } } } 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 deleted file mode 100644 index ffc7762f9ba..00000000000 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogGroupWriter.java +++ /dev/null @@ -1,821 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.phoenix.replication; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReentrantLock; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hbase.client.Mutation; -import org.apache.hadoop.hbase.io.compress.Compression; -import org.apache.phoenix.query.QueryServices; -import org.apache.phoenix.query.QueryServicesOptions; -import org.apache.phoenix.replication.log.LogFileWriter; -import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.phoenix.util.EnvironmentEdgeManager; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.lmax.disruptor.EventFactory; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.ExceptionHandler; -import com.lmax.disruptor.RingBuffer; -import com.lmax.disruptor.YieldingWaitStrategy; -import com.lmax.disruptor.dsl.Disruptor; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * Base class for replication log group writers. - *

- * This abstract class contains most of the common functionality for managing replication logs - * including the disruptor ring buffer, log rotation, file system management, and metrics. - * Concrete implementations provide specific replication behavior (synchronous vs store-and- - * forward). - *

- * Architecture Overview: - *

- * ┌──────────────────────────────────────────────────────────────────────┐
- * │                       ReplicationLogGroup                            │
- * │                                                                      │
- * │  ┌─────────────┐     ┌────────────────────────────────────────────┐  │
- * │  │             │     │                                            │  │
- * │  │  Producers  │     │  Disruptor Ring Buffer                     │  │
- * │  │  (append/   │────▶│  ┌─────────┐ ┌─────────┐ ┌─────────┐       │  │
- * │  │   sync)     │     │  │ Event 1 │ │ Event 2 │ │ Event 3 │ ...   │  │
- * │  │             │     │  └─────────┘ └─────────┘ └─────────┘       │  │
- * │  └─────────────┘     └────────────────────────────────────────────┘  │
- * │                                │                                     │
- * │                                ▼                                     │
- * │  ┌─────────────────────────────────────────────────────────────┐     │
- * │  │                                                             │     │
- * │  │  LogEventHandler                                            │     │
- * │  │  ┌──────────────────────────────────────────────────────┐   │     │
- * │  │  │                                                      │   │     │
- * │  │  │  - Batch Management                                  │   │     │
- * │  │  │  - Writer Rotation                                   │   │     │
- * │  │  │  - Error Handling                                    │   │     │
- * │  │  │  - Mode Transitions                                  │   │     │
- * │  │  │                                                      │   │     │
- * │  │  └──────────────────────────────────────────────────────┘   │     │
- * │  │                             │                               │     │
- * │  │                             ▼                               │     │
- * │  │  ┌──────────────────────────────────────────────────────┐   │     │
- * │  │  │                                                      │   │     │
- * │  │  │  LogFileWriter                                       │   │     │
- * │  │  │  - File Management                                   │   │     │
- * │  │  │  - Compression                                       │   │     │
- * │  │  │  - HDFS Operations                                   │   │     │
- * │  │  │                                                      │   │     │
- * │  │  └──────────────────────────────────────────────────────┘   │     │
- * │  └─────────────────────────────────────────────────────────────┘     │
- * └──────────────────────────────────────────────────────────────────────┘
- * 
- *

- * A high-performance ring buffer decouples the API from the complexity of writer management. - * Callers of append and sync generally return quickly, except for sync, where the writer must - * suspend the caller until the sync operation is successful (or times out). An internal single - * threaded process handles these events, encapsulating the complexity of batching mutations for - * efficiency, consolidating multiple in-flight syncs, rotating writers based on time or size, - * error handling and retries, and mode transitions. - */ -@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = { "EI_EXPOSE_REP", "EI_EXPOSE_REP2", - "MS_EXPOSE_REP" }, justification = "Intentional") -public abstract class ReplicationLogGroupWriter { - - private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogGroupWriter.class); - - protected final ReplicationLogGroup logGroup; - protected final long rotationTimeMs; - protected final long rotationSizeBytes; - protected final int maxRotationRetries; - protected final Compression.Algorithm compression; - protected final int ringBufferSize; - protected final long syncTimeoutMs; - protected final ReentrantLock lock = new ReentrantLock(); - protected volatile LogFileWriter currentWriter; - protected final AtomicLong lastRotationTime = new AtomicLong(); - protected final AtomicLong writerGeneration = new AtomicLong(); - protected final AtomicLong rotationFailures = new AtomicLong(0); - protected ScheduledExecutorService rotationExecutor; - protected Disruptor disruptor; - protected RingBuffer ringBuffer; - protected volatile boolean closed = false; - protected ReplicationShardDirectoryManager replicationShardDirectoryManager; - - /** The reason for requesting a log rotation. */ - protected enum RotationReason { - /** Rotation requested due to time threshold being exceeded. */ - TIME, - /** Rotation requested due to size threshold being exceeded. */ - SIZE, - /** Rotation requested due to an error condition. */ - ERROR; - } - - protected static final byte EVENT_TYPE_DATA = 0; - protected static final byte EVENT_TYPE_SYNC = 1; - - protected ReplicationLogGroupWriter(ReplicationLogGroup logGroup) { - this.logGroup = logGroup; - Configuration conf = logGroup.getConfiguration(); - this.rotationTimeMs = - conf.getLong(QueryServices.REPLICATION_LOG_ROTATION_TIME_MS_KEY, - QueryServicesOptions.DEFAULT_REPLICATION_LOG_ROTATION_TIME_MS); - long rotationSize = - conf.getLong(ReplicationLogGroup.REPLICATION_LOG_ROTATION_SIZE_BYTES_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_SIZE_BYTES); - double rotationSizePercent = - conf.getDouble(ReplicationLogGroup.REPLICATION_LOG_ROTATION_SIZE_PERCENTAGE_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_SIZE_PERCENTAGE); - this.rotationSizeBytes = (long) (rotationSize * rotationSizePercent); - this.maxRotationRetries = - conf.getInt(ReplicationLogGroup.REPLICATION_LOG_ROTATION_RETRIES_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_RETRIES); - String compressionName = - conf.get(ReplicationLogGroup.REPLICATION_LOG_COMPRESSION_ALGORITHM_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_COMPRESSION_ALGORITHM); - Compression.Algorithm compression = Compression.Algorithm.NONE; - if (!compressionName.equals( - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_COMPRESSION_ALGORITHM)) { - try { - compression = Compression.getCompressionAlgorithmByName(compressionName); - } catch (IllegalArgumentException e) { - LOG.warn("Unknown compression type " + compressionName + ", using NONE", e); - } - } - this.compression = compression; - this.ringBufferSize = conf.getInt(ReplicationLogGroup.REPLICATION_LOG_RINGBUFFER_SIZE_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_RINGBUFFER_SIZE); - this.syncTimeoutMs = conf.getLong(ReplicationLogGroup.REPLICATION_LOG_SYNC_TIMEOUT_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_SYNC_TIMEOUT); - } - - /** Initialize the writer. */ - public void init() throws IOException { - initializeFileSystems(); - initializeReplicationShardDirectoryManager(); - // Start time based rotation. - lastRotationTime.set(EnvironmentEdgeManager.currentTimeMillis()); - startRotationExecutor(); - // Create the initial writer. Do this before we initialize the Disruptor. - currentWriter = createNewWriter(); - initializeDisruptor(); - } - - /** - * Append a mutation to the log. This method is non-blocking and returns quickly, unless the - * ring buffer is full. The actual write happens asynchronously. We expect multiple append() - * calls followed by a sync(). The appends will be batched by the Disruptor. Should the ring - * buffer become full, which is not expected under normal operation but could (and should) - * happen if the log file writer is unable to make progress, due to a HDFS level disruption. - * Should we enter that condition this method will block until the append can be inserted. - *

- * An internal error may trigger fail-stop behavior. Subsequent to fail-stop, this method will - * throw an IOException("Closed"). No further appends are allowed. - * - * @param tableName The name of the HBase table the mutation applies to. - * @param commitId The commit identifier (e.g., SCN) associated with the mutation. - * @param mutation The HBase Mutation (Put or Delete) to be logged. - * @throws IOException If the writer is closed or if the ring buffer is full. - */ - public void append(String tableName, long commitId, Mutation mutation) throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace("Append: table={}, commitId={}, mutation={}", tableName, commitId, mutation); - } - if (closed) { - throw new IOException("Closed"); - } - // ringBuffer.next() claims the next sequence number. Because we initialize the Disruptor - // with ProducerType.MULTI and the blocking YieldingWaitStrategy this call WILL BLOCK if - // the ring buffer is full, thus providing backpressure to the callers. - long sequence = ringBuffer.next(); - try { - LogEvent event = ringBuffer.get(sequence); - event.setValues(EVENT_TYPE_DATA, new Record(tableName, commitId, mutation), null); - } finally { - // Update ring buffer events metric - ringBuffer.publish(sequence); - } - } - - /** - * Ensures all previously appended records are durably persisted. This method blocks until the - * sync operation completes or fails, potentially after internal retries. All in flight appends - * are batched and provided to the underlying LogWriter, which will then be synced. If there is - * a problem syncing the LogWriter we will retry, up to the retry limit, rolling the writer for - * each retry. - *

- * An internal error may trigger fail-stop behavior. Subsequent to fail-stop, this method will - * throw an IOException("Closed"). No further syncs are allowed. - *

- * NOTE: When the ReplicationLogManager is capable of switching between synchronous and - * fallback (store-and-forward) writers, then this will be pretty bullet proof. Right now we - * will still try to roll the synchronous writer a few times before giving up. - * @throws IOException If the sync operation fails after retries, or if interrupted. - */ - public void sync() throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace("Sync"); - } - if (closed) { - throw new IOException("Closed"); - } - syncInternal(); - } - - /** 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. - */ - protected abstract LogFileWriter createNewWriter() throws IOException; - - /** Initialize the Disruptor. */ - @SuppressWarnings("unchecked") - protected void initializeDisruptor() throws IOException { - disruptor = new Disruptor<>(LogEvent.EVENT_FACTORY, ringBufferSize, - new ThreadFactoryBuilder() - .setNameFormat("ReplicationLogGroupWriter-" + logGroup.getHaGroupName() + "-%d") - .setDaemon(true).build(), - ProducerType.MULTI, new YieldingWaitStrategy()); - LogEventHandler eventHandler = new LogEventHandler(); - eventHandler.init(); - disruptor.handleEventsWith(eventHandler); - LogExceptionHandler exceptionHandler = new LogExceptionHandler(); - disruptor.setDefaultExceptionHandler(exceptionHandler); - ringBuffer = disruptor.start(); - } - - /** - * Internal implementation of sync that publishes a sync event to the ring buffer and waits - * for completion. - */ - protected void syncInternal() throws IOException { - CompletableFuture syncFuture = new CompletableFuture<>(); - long sequence = ringBuffer.next(); - try { - LogEvent event = ringBuffer.get(sequence); - event.setValues(EVENT_TYPE_SYNC, null, syncFuture); - } finally { - ringBuffer.publish(sequence); - } - LOG.trace("Published EVENT_TYPE_SYNC at sequence {}", sequence); - try { - // Wait for the event handler to process up to and including this sync event - syncFuture.get(syncTimeoutMs, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new InterruptedIOException("Interrupted while waiting for sync"); - } catch (ExecutionException e) { - LOG.error("Sync operation failed", e.getCause()); - if (e.getCause() instanceof IOException) { - throw (IOException) e.getCause(); - } else { - throw new IOException("Sync operation failed", e.getCause()); - } - } catch (TimeoutException e) { - String message = "Sync operation timed out"; - LOG.error(message); - throw new IOException(message, e); - } - } - - protected void startRotationExecutor() { - long rotationCheckInterval = getRotationCheckInterval(rotationTimeMs); - rotationExecutor = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder() - .setNameFormat("ReplicationLogRotation-" + logGroup.getHaGroupName() + "-%d") - .setDaemon(true).build()); - rotationExecutor.scheduleAtFixedRate(new LogRotationTask(), rotationCheckInterval, - rotationCheckInterval, TimeUnit.MILLISECONDS); - LOG.debug("Started rotation executor with interval {}ms", rotationCheckInterval); - } - - protected long getRotationCheckInterval(long rotationTimeMs) { - long interval = Math.max(10 * 1000L, Math.min(60 * 1000L, rotationTimeMs / 10)); - return interval; - } - - protected void stopRotationExecutor() { - if (rotationExecutor != null) { - rotationExecutor.shutdown(); - try { - if (!rotationExecutor.awaitTermination(5, TimeUnit.SECONDS)) { - rotationExecutor.shutdownNow(); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - rotationExecutor.shutdownNow(); - } - } - } - - /** Gets the current writer, rotating it if necessary based on size thresholds. */ - protected LogFileWriter getWriter() throws IOException { - lock.lock(); - try { - if (shouldRotate()) { - rotateLog(RotationReason.SIZE); - } - return currentWriter; - } finally { - lock.unlock(); - } - } - - /** - * Checks if the current log file needs to be rotated based on time or size. Must be called - * under lock. - * @return true if rotation is needed, false otherwise. - * @throws IOException If an error occurs checking the file size. - */ - protected boolean shouldRotate() throws IOException { - if (currentWriter == null) { - LOG.warn("Current writer is null, forcing rotation."); - return true; - } - // Check time threshold - long now = EnvironmentEdgeManager.currentTimeMillis(); - long last = lastRotationTime.get(); - if (now - last >= rotationTimeMs) { - LOG.debug("Rotating log file due to time threshold ({} ms elapsed, threshold {} ms)", - now - last, rotationTimeMs); - return true; - } - - // Check size threshold (using actual file size for accuracy) - long currentSize = currentWriter.getLength(); - if (currentSize >= rotationSizeBytes) { - LOG.debug("Rotating log file due to size threshold ({} bytes, threshold {} bytes)", - currentSize, rotationSizeBytes); - return true; - } - - return false; - } - - /** - * Closes the current log writer and opens a new one, updating rotation metrics. - *

- * This method handles the rotation of log files, which can be triggered by: - *

    - *
  • Time threshold exceeded (TIME)
  • - *
  • Size threshold exceeded (SIZE)
  • - *
  • Error condition requiring rotation (ERROR)
  • - *
- *

- * The method implements retry logic for handling rotation failures. If rotation fails, it - * retries up to maxRotationRetries times. If the number of failures exceeds - * maxRotationRetries, an exception is thrown. Otherwise, it logs a warning and continues with - * the current writer. - *

- * The method is thread-safe and uses a lock to ensure atomic rotation operations. - * - * @param reason The reason for requesting log rotation - * @return The new LogFileWriter instance if rotation succeeded, or the current writer if - * rotation failed - * @throws IOException if rotation fails after exceeding maxRotationRetries - */ - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UL_UNRELEASED_LOCK", - justification = "False positive") - protected LogFileWriter rotateLog(RotationReason reason) throws IOException { - lock.lock(); - try { - // Try to get the new writer first. If it fails we continue using the current writer. - // Increment the writer generation - LogFileWriter newWriter = createNewWriter(); - LOG.debug("Created new writer: {}", newWriter); - // Close the current writer - if (currentWriter != null) { - LOG.debug("Closing current writer: {}", currentWriter); - closeWriter(currentWriter); - } - currentWriter = newWriter; - lastRotationTime.set(EnvironmentEdgeManager.currentTimeMillis()); - rotationFailures.set(0); - logGroup.getMetrics().incrementRotationCount(); - switch (reason) { - case TIME: - logGroup.getMetrics().incrementTimeBasedRotationCount(); - break; - case SIZE: - logGroup.getMetrics().incrementSizeBasedRotationCount(); - break; - case ERROR: - logGroup.getMetrics().incrementErrorBasedRotationCount(); - break; - } - } catch (IOException e) { - // If we fail to rotate the log, we increment the failure counter. If we have exceeded - // the maximum number of retries, we close the log and throw the exception. Otherwise - // we log a warning and continue. - logGroup.getMetrics().incrementRotationFailureCount(); - long numFailures = rotationFailures.getAndIncrement(); - if (numFailures >= maxRotationRetries) { - LOG.warn("Failed to rotate log (attempt {}/{}), closing log", numFailures, - maxRotationRetries, e); - closeOnError(); - throw e; - } - LOG.warn("Failed to rotate log (attempt {}/{}), retrying...", numFailures, - maxRotationRetries, e); - } finally { - lock.unlock(); - } - return currentWriter; - } - - /** Closes the given writer, logging any errors that occur during close. */ - protected void closeWriter(LogFileWriter writer) { - if (writer == null) { - return; - } - try { - writer.close(); - } catch (IOException e) { - // For now, just log and continue - LOG.error("Error closing log writer: " + writer, e); - } - } - - /** Close the currentWriter. - * Needed by tests so that we can close the log file and then read it - */ - protected void closeCurrentWriter() { - lock.lock(); - try { - closeWriter(currentWriter); - currentWriter = null; - } finally { - lock.unlock(); - } - } - - /** - * Check if this ReplicationLogGroup is closed. - * - * @return true if closed, false otherwise - */ - public boolean isClosed() { - return closed; - } - - /** - * Force closes the log upon an unrecoverable internal error. This is a fail-stop behavior: - * once called, the log is marked as closed, the Disruptor is halted, and all subsequent - * append() and sync() calls will throw an IOException("Closed"). This ensures that no - * further operations are attempted on a log that has encountered a critical error. - */ - protected void closeOnError() { - lock.lock(); - try { - if (closed) { - return; - } - closed = true; - } finally { - lock.unlock(); - } - // Stop the time based rotation check. - stopRotationExecutor(); - // We expect a final sync will not work. Just close the inner writer. - closeWriter(currentWriter); - // Directly halt the disruptor. shutdown() would wait for events to drain. We are expecting - // that will not work. - disruptor.halt(); - } - - /** Closes the log. */ - public void close() { - lock.lock(); - try { - if (closed) { - return; - } - closed = true; - } finally { - lock.unlock(); - } - // Stop the time based rotation check. - stopRotationExecutor(); - // Sync before shutting down to flush all pending appends. - try { - syncInternal(); - disruptor.shutdown(); // Wait for a clean shutdown. - } catch (IOException e) { - LOG.warn("Error during final sync on close", e); - disruptor.halt(); // Go directly to halt. - } - // We must for the disruptor before closing the current writer. - closeWriter(currentWriter); - } - - protected FileSystem getFileSystem(URI uri) throws IOException { - return FileSystem.get(uri, logGroup.getConfiguration()); - } - - /** Implements time based rotation independent of in-line checking. */ - protected class LogRotationTask implements Runnable { - @Override - public void run() { - if (closed) { - return; - } - // Use tryLock with a timeout to avoid blocking indefinitely if another thread holds - // the lock for an unexpectedly long time (e.g., during a problematic rotation). - boolean acquired = false; - try { - // Wait a short time for the lock - acquired = lock.tryLock(1, TimeUnit.SECONDS); - if (acquired) { - // Check only the time condition here, size is handled by getWriter - long now = EnvironmentEdgeManager.currentTimeMillis(); - long last = lastRotationTime.get(); - if (!closed && now - last >= rotationTimeMs) { - LOG.debug("Time based rotation needed ({} ms elapsed, threshold {} ms).", - now - last, rotationTimeMs); - try { - rotateLog(RotationReason.TIME); // rotateLog updates lastRotationTime - } catch (IOException e) { - LOG.error("Failed to rotate log, currentWriter is {}", currentWriter, - e); - // More robust error handling goes here once the store-and-forward - // fallback is implemented. For now we just log the error and continue. - } - } - } else { - LOG.warn("LogRotationTask could not acquire lock, skipping check this time."); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); // Preserve interrupt status - LOG.warn("LogRotationTask interrupted while trying to acquire lock."); - } finally { - if (acquired) { - lock.unlock(); - } - } - } - } - - protected static class Record { - public String tableName; - public long commitId; - public Mutation mutation; - - public Record(String tableName, long commitId, Mutation mutation) { - this.tableName = tableName; - this.commitId = commitId; - this.mutation = mutation; - } - } - - /** Event structure for the Disruptor ring buffer containing data and sync operations. */ - protected static class LogEvent { - protected static final EventFactory EVENT_FACTORY = LogEvent::new; - - protected int type; - protected Record record; - protected CompletableFuture syncFuture; // Used only for SYNC events - protected long timestampNs; // Timestamp when event was created - - public void setValues(int type, Record record, CompletableFuture syncFuture) { - this.type = type; - this.record = record; - this.syncFuture = syncFuture; - this.timestampNs = System.nanoTime(); - } - } - - /** - * Handles events from the Disruptor, managing batching, writer rotation, and error handling. - */ - protected class LogEventHandler implements EventHandler { - protected final int maxRetries; // Configurable max retries for sync - protected final long retryDelayMs; // Configurable delay between retries - protected final List currentBatch = new ArrayList<>(); - protected final List> pendingSyncFutures = new ArrayList<>(); - protected LogFileWriter writer; - protected long generation; - - protected LogEventHandler() { - Configuration conf = logGroup.getConfiguration(); - this.maxRetries = conf.getInt(ReplicationLogGroup.REPLICATION_LOG_SYNC_RETRIES_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_SYNC_RETRIES); - this.retryDelayMs = - conf.getLong(ReplicationLogGroup.REPLICATION_LOG_RETRY_DELAY_MS_KEY, - ReplicationLogGroup.DEFAULT_REPLICATION_LOG_RETRY_DELAY_MS); - } - - protected void init() throws IOException { - this.writer = getWriter(); - this.generation = writer.getGeneration(); - } - - /** - * Processes all pending sync operations by syncing the current writer and completing - * their associated futures. This method is called when we are ready to process a set of - * consolidated sync requests and performs the following steps: - *

    - *
  1. Syncs the current writer to ensure all data is durably written.
  2. - *
  3. Completes all pending sync futures successfully.
  4. - *
  5. Clears the list of pending sync futures.
  6. - *
  7. Clears the current batch of records since they have been successfully synced.
  8. - *
- * - * @param sequence The sequence number of the last processed event - * @throws IOException if the sync operation fails - */ - protected void processPendingSyncs(long sequence) throws IOException { - if (pendingSyncFutures.isEmpty()) { - return; - } - writer.sync(); - // Complete all pending sync futures - for (CompletableFuture future : pendingSyncFutures) { - future.complete(null); - } - pendingSyncFutures.clear(); - // Sync completed, clear the list of in-flight appends. - currentBatch.clear(); - LOG.trace("Sync operation completed successfully up to sequence {}", sequence); - } - - /** - * Fails all pending sync operations with the given exception. This method is called when - * we encounter an unrecoverable error during the sync of the inner writer. It completes - * all pending sync futures that were consolidated exceptionally. - *

- * Note: This method does not clear the currentBatch list. The currentBatch must be - * preserved as it contains records that may need to be replayed if we successfully - * rotate to a new writer. - * - * @param sequence The sequence number of the last processed event - * @param e The IOException that caused the failure - */ - protected void failPendingSyncs(long sequence, IOException e) { - if (pendingSyncFutures.isEmpty()) { - return; - } - for (CompletableFuture future : pendingSyncFutures) { - future.completeExceptionally(e); - } - pendingSyncFutures.clear(); - LOG.warn("Failed to process syncs at sequence {}", sequence, e); - } - - /** - * Processes a single event from the Disruptor ring buffer. This method handles both data - * and sync events, with retry logic for handling IO failures. - *

- * For data events, it: - *

    - *
  1. Checks if the writer has been rotated and replays any in-flight records.
  2. - *
  3. Appends the record to the current writer.
  4. - *
  5. Adds the record to the current batch for potential replay.
  6. - *
  7. Processes any pending syncs if this is the end of a batch.
  8. - *
- *

- * For sync events, it: - *

    - *
  1. Adds the sync future to the pending list.
  2. - *
  3. Processes any pending syncs if this is the end of a batch.
  4. - *
- * If an IOException occurs, the method will attempt to rotate the writer and retry the - * operation up to the configured maximum number of retries. If all retries fail, it will - * fail all pending syncs and throw the exception. - *

- * The retry logic includes a configurable delay between attempts to prevent tight loops - * when there are persistent HDFS issues. This delay helps mitigate the risk of rapid - * cycling through writers when the underlying storage system is experiencing problems. - * - * @param event The event to process - * @param sequence The sequence number of the event - * @param endOfBatch Whether this is the last event in the current batch - * @throws Exception if the operation fails after all retries - */ - @Override - public void onEvent(LogEvent event, long sequence, boolean endOfBatch) throws Exception { - // Calculate time spent in ring buffer - long currentTimeNs = System.nanoTime(); - long ringBufferTimeNs = currentTimeNs - event.timestampNs; - logGroup.getMetrics().updateRingBufferTime(ringBufferTimeNs); - writer = getWriter(); - int attempt = 0; - while (attempt < maxRetries) { - try { - if (writer.getGeneration() > generation) { - generation = writer.getGeneration(); - // If the writer has been rotated, we need to replay the current batch of - // in-flight appends into the new writer. - if (!currentBatch.isEmpty()) { - LOG.trace("Writer has been rotated, replaying in-flight batch"); - for (Record r: currentBatch) { - writer.append(r.tableName, r.commitId, r.mutation); - } - } - } - switch (event.type) { - case EVENT_TYPE_DATA: - writer.append(event.record.tableName, event.record.commitId, - event.record.mutation); - // Add to current batch only after we succeed at appending, so we don't - // replay it twice. - currentBatch.add(event.record); - // Process any pending syncs at the end of batch. - if (endOfBatch) { - processPendingSyncs(sequence); - } - return; - case EVENT_TYPE_SYNC: - // Add this sync future to the pending list. - pendingSyncFutures.add(event.syncFuture); - // Process any pending syncs at the end of batch. - if (endOfBatch) { - processPendingSyncs(sequence); - } - return; - default: - throw new UnsupportedOperationException("Unknown event type: " - + event.type); - } - } catch (IOException e) { - // IO exception, force a rotation. - LOG.debug("Attempt " + (attempt + 1) + "/" + maxRetries + " failed", e); - if (attempt >= maxRetries) { - failPendingSyncs(sequence, e); - throw e; - } - attempt++; - // Add delay before retrying to prevent tight loops - try { - Thread.sleep(retryDelayMs); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - throw new InterruptedIOException("Interrupted during retry delay"); - } - writer = rotateLog(RotationReason.ERROR); - } - } - } - } - - /** - * Handler for critical errors during the Disruptor lifecycle that closes the writer to prevent - * data loss. - */ - protected class LogExceptionHandler implements ExceptionHandler { - @Override - public void handleEventException(Throwable e, long sequence, LogEvent event) { - String message = "Exception processing sequence " + sequence + " for event " + event; - LOG.error(message, e); - closeOnError(); - } - - @Override - public void handleOnStartException(Throwable e) { - LOG.error("Exception during Disruptor startup", e); - closeOnError(); - } - - @Override - public void handleOnShutdownException(Throwable e) { - // Should not happen, but if it does, the regionserver is aborting or shutting down. - LOG.error("Exception during Disruptor shutdown", e); - closeOnError(); - } - } -} 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 index 5838a9de4ba..cdb078dce3e 100644 --- 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 @@ -70,12 +70,12 @@ public class ReplicationLogTracker { protected final String haGroupName; protected MetricsReplicationLogTracker metrics; - public ReplicationLogTracker(final Configuration conf, final String haGroupName, final FileSystem fileSystem, + public ReplicationLogTracker(final Configuration conf, final String haGroupName, final ReplicationShardDirectoryManager replicationShardDirectoryManager, final MetricsReplicationLogTracker metrics) { this.conf = conf; this.haGroupName = haGroupName; - this.fileSystem = fileSystem; + this.fileSystem = replicationShardDirectoryManager.getFileSystem(); this.replicationShardDirectoryManager = replicationShardDirectoryManager; this.metrics = metrics; } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java new file mode 100644 index 00000000000..312a8cfbde2 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java @@ -0,0 +1,94 @@ +package org.apache.phoenix.replication; + +import java.io.IOException; + +import org.apache.phoenix.replication.ReplicationLogGroup.Record; +import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * Base class for different replication modes. + *

+ * This abstract class manages the lifecycle of a replication mode. It also delegates the + * append and sync events to the underlying ReplicationLog object. + *

+ */ +public abstract class ReplicationModeImpl { + protected final ReplicationLogGroup logGroup; + + // The mode manages the underlying log to which the append and sync events will be sent + protected ReplicationLog log; + + protected ReplicationModeImpl(ReplicationLogGroup logGroup) { + this.logGroup = logGroup; + } + + /** + * Invoked when we switch to this mode + * + * @throws IOException + */ + abstract void onEnter() throws IOException; + + /** + * Invoked when we switch out from this mode + * + * @param gracefulShutdown True if graceful, False if forced + */ + abstract void onExit(boolean gracefulShutdown); + + /** + * Invoked when there is a failure event on this mode + * + * @param cause Failure exception + * @return new mode to switch to + * @throws IOException + */ + abstract ReplicationMode onFailure(Throwable cause) throws IOException; + + abstract ReplicationMode getMode(); + + @Override + public String toString() { + return getMode().name(); + } + + /** Returns the underlying log abstraction */ + @VisibleForTesting + ReplicationLog getReplicationLog() { + return log; + } + + /** + * Delegates the append event to the underlying log + * + * @param r Mutation + * @throws IOException + */ + void append(Record r) throws IOException { + getReplicationLog().append(r); + } + + /** + * Delegates the sync event to the underlying log + * + * @throws IOException + */ + void sync() throws IOException { + getReplicationLog().sync(); + } + + /** Graceful close */ + void closeReplicationLog() { + if (log != null) { + log.close(); + } + } + + /** Forced close */ + void closeReplicationLogOnError() { + if (log != null) { + log.closeOnError(); + } + } +} 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 index 7010131b36f..55ffea2aa21 100644 --- 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 @@ -17,11 +17,14 @@ */ package org.apache.phoenix.replication; +import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; /** @@ -60,6 +63,12 @@ public class ReplicationShardDirectoryManager { */ public static final String SHARD_DIR_FORMAT = "%03d"; + /* + * Format string for log file names. _.plog + * Example 1762470665995_localhost,54575,1762470584502.plog + */ + public static final String FILE_NAME_FORMAT = "%d_%s.plog"; + /** * Configuration key for the duration of each replication round in seconds. */ @@ -73,25 +82,24 @@ public class ReplicationShardDirectoryManager { * 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; + private final FileSystem shardFS; + private final ConcurrentHashMap shardMap = new ConcurrentHashMap<>(); - public ReplicationShardDirectoryManager(final Configuration conf, final Path rootPath) { + public ReplicationShardDirectoryManager(Configuration conf, FileSystem fs, Path rootPath) { + this.shardFS = fs; this.rootDirectoryPath = rootPath; this.shardDirectoryPath = new Path(rootPath.toUri().getPath(), - REPLICATION_SHARD_SUB_DIRECTORY_NAME); + 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); + PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, + DEFAULT_REPLICATION_ROUND_DURATION_SECONDS); } /** @@ -131,6 +139,37 @@ public Path getShardDirectory(ReplicationRound replicationRound) { return getShardDirectory(replicationRound.getStartTime()); } + /** + * Creates a new log file path in a sharded directory structure + * File path: [root_path]/[ha_group_name]/[in|out]/shard/[shard_directory]/[file_name] + * @param timestamp current time + * @param serverName name of the server creating the log file + * @return Path to the replication log file + */ + public Path getWriterPath(long timestamp, String serverName) throws IOException { + Path shardPath = 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 (!shardFS.exists(p)) { + if (!shardFS.mkdirs(shardPath)) { + throw new IOException("Could not create path: " + p); + } + } + } catch (IOException e) { + exception[0] = e; + return null; // Don't cache the path if we can't create it. + } + return p; + }); + // If we faced an exception in computeIfAbsent, throw it + if (exception[0] != null) { + throw exception[0]; + } + return new Path(shardPath, String.format(FILE_NAME_FORMAT, timestamp, serverName)); + } /** * Returns a ReplicationRound object based on the given round start time, @@ -207,4 +246,8 @@ public Path getRootDirectoryPath() { public int getNumShards() { return this.numShards; } + + public FileSystem getFileSystem() { + return this.shardFS; + } } 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 deleted file mode 100644 index f1290eb574a..00000000000 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StandbyLogGroupWriter.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.phoenix.replication; - -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.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; - -/** - * Synchronous replication implementation of ReplicationLogGroupWriter. - *

- * This class implements synchronous replication to a standby cluster's HDFS. It writes replication - * logs directly to the standby cluster in synchronous mode, providing immediate consistency for - * failover scenarios. - */ -public class StandbyLogGroupWriter extends ReplicationLogGroupWriter { - - private static final Logger LOG = LoggerFactory.getLogger(StandbyLogGroupWriter.class); - - private FileSystem standbyFs; - private URI standbyUrl; - private Path haGroupLogFilesPath; - protected final ConcurrentHashMap shardMap = new ConcurrentHashMap<>(); - - /** - * Constructor for StandbyLogGroupWriter. - */ - public StandbyLogGroupWriter(ReplicationLogGroup logGroup) { - super(logGroup); - LOG.debug("Created StandbyLogGroupWriter for HA Group: {}", logGroup.getHaGroupName()); - } - - @Override - protected void initializeFileSystems() throws IOException { - Configuration conf = logGroup.getConfiguration(); - String standbyUrlString = conf.get(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY); - if (standbyUrlString == null || standbyUrlString.trim().isEmpty()) { - throw new IOException("Standby HDFS URL not configured: " - + ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY); - } - try { - standbyUrl = new URI(standbyUrlString); - standbyFs = getFileSystem(standbyUrl); - LOG.info("Initialized standby filesystem: {}", standbyUrl); - } catch (URISyntaxException e) { - throw new IOException("Invalid standby HDFS URL: " + standbyUrlString, e); - } - } - - @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 using - * {@link ReplicationShardDirectoryManager}. - * Directory Structure: [root_path]/[ha_group_name]/in/shard/[shard_directory]/[file_name] - */ - protected Path makeWriterPath(FileSystem fs) throws IOException { - long timestamp = EnvironmentEdgeManager.currentTimeMillis(); - 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(haGroupLogFilesPath); // This probably exists, but just in case. - if (!fs.mkdirs(shardPath)) { - throw new IOException("Could not create path: " + p); - } - } - } catch (IOException e) { - exception[0] = e; - return null; // Don't cache the path if we can't create it. - } - return p; - }); - // If we faced an exception in computeIfAbsent, throw it - if (exception[0] != null) { - throw exception[0]; - } - Path filePath = new Path(shardPath, String.format(ReplicationLogGroup.FILE_NAME_FORMAT, - timestamp, logGroup.getServerName())); - return filePath; - } - - /** Creates and initializes a new LogFileWriter. */ - protected LogFileWriter createNewWriter() throws IOException { - Path filePath = makeWriterPath(standbyFs); - LogFileWriterContext writerContext = new LogFileWriterContext(logGroup.getConfiguration()) - .setFileSystem(standbyFs) - .setFilePath(filePath).setCompression(compression); - LogFileWriter newWriter = new LogFileWriter(); - newWriter.init(writerContext); - newWriter.setGeneration(writerGeneration.incrementAndGet()); - return newWriter; - } -} 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 deleted file mode 100644 index 26bd02ff45f..00000000000 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardLogGroupWriter.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.phoenix.replication; - -import java.io.IOException; - -import org.apache.phoenix.replication.log.LogFileWriter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Store-and-forward replication implementation of ReplicationLogGroupWriter. - *

- * This class is a stub implementation for future store-and-forward replication functionality. - * Store-and-forward mode is used when the standby cluster is temporarily unavailable - mutations - * are stored locally and forwarded when connectivity is restored. - *

- * Currently this is a stub that throws UnsupportedOperationException for the abstract methods. - * Future implementation will include: - *

    - *
  • Local storage of mutations when standby is unavailable
  • - *
  • Background forwarding when connectivity is restored
  • - *
  • Proper error handling and retry logic
  • - *
  • Integration with HA state management
  • - *
  • Dual-mode operation: local storage + forwarding
  • - *
- */ -public class StoreAndForwardLogGroupWriter extends ReplicationLogGroupWriter { - - private static final Logger LOG = LoggerFactory.getLogger(StoreAndForwardLogGroupWriter.class); - - /** - * Constructor for StoreAndForwardLogGroupWriter. - */ - public StoreAndForwardLogGroupWriter(ReplicationLogGroup logGroup) { - super(logGroup); - LOG.debug("Created StoreAndForwardLogGroupWriter for HA Group: {}", - logGroup.getHaGroupName()); - } - - @Override - public void init() throws IOException { - // TODO - } - - @Override - public void close() { - // TODO - } - - @Override - protected void initializeFileSystems() throws IOException { - // TODO - } - - @Override - protected void initializeReplicationShardDirectoryManager() { - // TODO - } - - @Override - protected LogFileWriter createNewWriter() throws IOException { - // TODO - return null; - } -} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java new file mode 100644 index 00000000000..ba86f32c9a2 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java @@ -0,0 +1,114 @@ +package org.apache.phoenix.replication; + +import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; +import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; + +import java.io.IOException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; +import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Store and Forward mode implementation + *

+ * This class implements the store and forward replication mode. It delegates the + * append and sync events to the replication log on the fallback cluster. In the background, + * it also forwards the replication log from the fallback cluster to the standby cluster. + *

+ */ +public class StoreAndForwardModeImpl extends ReplicationModeImpl { + private static final Logger LOG = LoggerFactory.getLogger(StoreAndForwardModeImpl.class); + + // fraction of the zk session timeout to re-update the HA Group state to ACTIVE_NOT_IN_SYNC + private static final double HA_GROUP_STORE_UPDATE_MULTIPLIER = 0.7; + + private ScheduledExecutorService haGroupStoreUpdateExecutor; + + protected StoreAndForwardModeImpl(ReplicationLogGroup logGroup) { + super(logGroup); + } + + @Override + void onEnter() throws IOException { + LOG.info("HAGroup {} entered mode {}", logGroup, this); + // create a log on the fallback cluster + log = logGroup.createFallbackLog(); + log.init(); + // Schedule task to periodically set the HAGroupStore state to ACTIVE_NOT_IN_SYNC + startHAGroupStoreUpdateTask(); + // start the log forwarder + logGroup.getLogForwarder().start(); + } + + private long getHAGroupStoreUpdateInterval() { + return (long)Math.ceil(logGroup.conf.getLong(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT) + * HA_GROUP_STORE_UPDATE_MULTIPLIER); + } + + private void startHAGroupStoreUpdateTask() { + long haGroupStoreUpdateInterval = getHAGroupStoreUpdateInterval(); + haGroupStoreUpdateExecutor = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("StoreAndForwardStatusUpdate-" + logGroup.getHAGroupName() + "-%d") + .setDaemon(true).build()); + haGroupStoreUpdateExecutor.scheduleAtFixedRate(() -> { + try { + logGroup.setHAGroupStatusToStoreAndForward(); + } catch (Exception e) { + // retry again in the next interval + LOG.info("HAGroup {} failed to re-update the status to STORE_AND_FORWARD", + logGroup, e); + } + }, 0, haGroupStoreUpdateInterval, TimeUnit.MILLISECONDS); + LOG.info("HAGroup {} started haGroupStoreUpdateExecutor with interval {}ms", + logGroup, haGroupStoreUpdateInterval); + } + + private void stopHAGroupStoreUpdateTask() { + if (haGroupStoreUpdateExecutor != null) { + haGroupStoreUpdateExecutor.shutdown(); + try { + if (!haGroupStoreUpdateExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + haGroupStoreUpdateExecutor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + haGroupStoreUpdateExecutor.shutdownNow(); + } + LOG.info("HAGroup {} stopped haGroupStoreUpdateExecutor ", logGroup); + } + } + + @Override + void onExit(boolean gracefulShutdown) { + LOG.info("HAGroup {} exiting mode {} graceful={}", logGroup, this, gracefulShutdown); + stopHAGroupStoreUpdateTask(); + if (gracefulShutdown) { + closeReplicationLog(); + } else { + closeReplicationLogOnError(); + } + } + + @Override + ReplicationMode onFailure(Throwable e) throws IOException { + // Treating failures in STORE_AND_FORWARD mode as fatal errors + String message = String.format( + "HAGroup %s mode=%s got error", logGroup, this); + LOG.error(message, e); + logGroup.abort(message, e); + // unreachable, we remain in the same mode + return STORE_AND_FORWARD; + } + + @Override + ReplicationMode getMode() { + return STORE_AND_FORWARD; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java new file mode 100644 index 00000000000..508498409cf --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java @@ -0,0 +1,68 @@ +package org.apache.phoenix.replication; + +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; + +import java.io.IOException; + +import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Sync and Forward mode implementation + *

+ * This class implements the sync and forward replication mode. It delegates the + * append and sync events to the replication log on the standby cluster. In the background, + * it also forwards the replication log from the fallback cluster to the standby cluster. + *

+ */ +public class SyncAndForwardModeImpl extends ReplicationModeImpl { + private static final Logger LOG = LoggerFactory.getLogger(SyncAndForwardModeImpl.class); + + protected SyncAndForwardModeImpl(ReplicationLogGroup logGroup) { + super(logGroup); + } + + @Override + void onEnter() throws IOException { + LOG.info("HAGroup {} entered mode {}", logGroup, this); + // create a log on the standby cluster + log = logGroup.createStandbyLog(); + log.init(); + // no-op if the forwarder is already started + logGroup.getLogForwarder().start(); + } + + @Override + void onExit(boolean gracefulShutdown) { + LOG.info("HAGroup {} exiting mode {} graceful={}", logGroup, this, gracefulShutdown); + // stop the replication log forwarding + logGroup.getLogForwarder().stop(); + if (gracefulShutdown) { + closeReplicationLog(); + } else { + closeReplicationLogOnError(); + } + } + + @Override + ReplicationMode onFailure(Throwable e) throws IOException { + LOG.info("HAGroup {} mode={} got error", logGroup, this, e); + try { + logGroup.setHAGroupStatusToStoreAndForward(); + } catch (Exception ex) { + // Fatal error when we can't update the HAGroup status + String message = String.format( + "HAGroup %s could not update status to STORE_AND_FORWARD", logGroup); + LOG.error(message, ex); + logGroup.abort(message, ex); + } + return STORE_AND_FORWARD; + } + + @Override + ReplicationMode getMode() { + return SYNC_AND_FORWARD; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java new file mode 100644 index 00000000000..2d6ef349cdb --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java @@ -0,0 +1,64 @@ +package org.apache.phoenix.replication; + +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; + +import java.io.IOException; + +import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Synchronous mode implementation + *

+ * This class implements the synchronous replication mode. It delegates the + * append and sync events to the replication log on the standby cluster. + *

+ */ +public class SyncModeImpl extends ReplicationModeImpl { + private static final Logger LOG = LoggerFactory.getLogger(SyncModeImpl.class); + + protected SyncModeImpl(ReplicationLogGroup logGroup) { + super(logGroup); + } + + @Override + void onEnter() throws IOException { + LOG.info("HAGroup {} entered mode {}", logGroup, this); + // create a log on the standby cluster + log = logGroup.createStandbyLog(); + log.init(); + } + + @Override + void onExit(boolean gracefulShutdown) { + LOG.info("HAGroup {} exiting mode {} graceful={}", logGroup, this, gracefulShutdown); + if (gracefulShutdown) { + closeReplicationLog(); + } else { + closeReplicationLogOnError(); + } + } + + @Override + ReplicationMode onFailure(Throwable e) throws IOException { + LOG.info("HAGroup {} mode={} got error", logGroup, this, e); + try { + // first update the HAGroupStore state + logGroup.setHAGroupStatusToStoreAndForward(); + } catch (Exception ex) { + // Fatal error when we can't update the HAGroup status + String message = String.format( + "HAGroup %s could not update status to STORE_AND_FORWARD", logGroup); + LOG.error(message, ex); + logGroup.abort(message, ex); + } + return STORE_AND_FORWARD; + } + + @Override + ReplicationMode getMode() { + return SYNC; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryForwarderImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryForwarderImpl.java new file mode 100644 index 00000000000..6b578574bfe --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogDiscoveryForwarderImpl.java @@ -0,0 +1,39 @@ +/* + * 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 ReplicationLogDiscoveryForwarder operations. */ +public class MetricsReplicationLogDiscoveryForwarderImpl + extends MetricsReplicationLogDiscoveryImpl { + + private static final String METRICS_NAME = "ReplicationLogDiscoveryForwarder"; + private static final String METRICS_DESCRIPTION = + "Metrics about Replication Log Discovery Forwarder for a HA Group"; + private static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + public MetricsReplicationLogDiscoveryForwarderImpl(final String haGroupName) { + super(MetricsReplicationLogDiscoveryForwarderImpl.METRICS_NAME, + MetricsReplicationLogDiscoveryForwarderImpl.METRICS_DESCRIPTION, + MetricsReplicationLogDiscoveryImpl.METRICS_CONTEXT, + MetricsReplicationLogDiscoveryForwarderImpl.METRICS_JMX_CONTEXT + + ",haGroup=" + haGroupName); + super.groupMetricsContext = + MetricsReplicationLogDiscoveryForwarderImpl.METRICS_JMX_CONTEXT + + ",haGroup=" + haGroupName; + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java new file mode 100644 index 00000000000..827751c9a11 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java @@ -0,0 +1,28 @@ +package org.apache.phoenix.replication.metrics; + +import java.util.concurrent.ConcurrentHashMap; + + + +/** + * Factory class for creating log forwarder metrics + */ +public class MetricsReplicationLogForwarderSourceFactory { + /** Cache of ReplicationLogTrackerForwarderImpl instances by HA Group ID */ + private static final ConcurrentHashMap TRACKER_INSTANCES = + new ConcurrentHashMap<>(); + + /** Cache of ReplicationLogDiscoveryForwarderImpl instances by HA Group ID */ + private static final ConcurrentHashMap DISCOVERY_INSTANCES = + new ConcurrentHashMap<>(); + + public static MetricsReplicationLogTrackerForwarderImpl getInstanceForTracker(String haGroupName) { + return TRACKER_INSTANCES.computeIfAbsent(haGroupName, k -> + new MetricsReplicationLogTrackerForwarderImpl(haGroupName)); + } + + public static MetricsReplicationLogDiscoveryForwarderImpl getInstanceForDiscovery(String haGroupName) { + return DISCOVERY_INSTANCES.computeIfAbsent(haGroupName, k -> + new MetricsReplicationLogDiscoveryForwarderImpl(haGroupName)); + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java new file mode 100644 index 00000000000..030a44f964b --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java @@ -0,0 +1,39 @@ +/* + * 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 MetricsReplicationLogTrackerForwarderImpl + extends MetricsReplicationLogTrackerImpl { + + private static final String METRICS_NAME = "ReplicationLogTrackerForwarder"; + private static final String METRICS_DESCRIPTION = + "Metrics about Replication Log tracker for forwarding files in a HA Group"; + private static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + public MetricsReplicationLogTrackerForwarderImpl(final String haGroupName) { + super(MetricsReplicationLogTrackerForwarderImpl.METRICS_NAME, + MetricsReplicationLogTrackerForwarderImpl.METRICS_DESCRIPTION, + MetricsReplicationLogTracker.METRICS_CONTEXT, + MetricsReplicationLogTrackerForwarderImpl.METRICS_JMX_CONTEXT + + ",haGroup=" + haGroupName); + super.groupMetricsContext = + MetricsReplicationLogTrackerForwarderImpl.METRICS_JMX_CONTEXT + + ",haGroup=" + haGroupName; + } + +} 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 index 587f9bdb02a..b79a8f4f8dd 100644 --- 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 @@ -111,9 +111,9 @@ 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); + new ReplicationShardDirectoryManager(conf, fileSystem, newFilesDirectory); ReplicationLogTracker replicationLogReplayFileTracker = new ReplicationLogTracker( - conf, haGroupName, fileSystem, replicationShardDirectoryManager, + conf, haGroupName, replicationShardDirectoryManager, new MetricsReplicationLogTrackerReplayImpl(haGroupName)); replicationLogReplayFileTracker.init(); this.replicationLogDiscoveryReplay = diff --git a/phoenix-core/src/it/java/org/apache/phoenix/replication/ReplicationLogGroupIT.java b/phoenix-core/src/it/java/org/apache/phoenix/replication/ReplicationLogGroupIT.java index f111d141f67..94311ee4c62 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/replication/ReplicationLogGroupIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/replication/ReplicationLogGroupIT.java @@ -17,9 +17,13 @@ */ package org.apache.phoenix.replication; -import static org.apache.phoenix.hbase.index.IndexRegionObserver.DEFAULT_HA_GROUP; +import static org.apache.phoenix.jdbc.HighAvailabilityGroup.PHOENIX_HA_GROUP_ATTR; +import static org.apache.phoenix.jdbc.HighAvailabilityTestingUtility.getHighAvailibilityGroup; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME; +import static org.apache.phoenix.query.BaseTest.generateUniqueName; +import static org.apache.phoenix.query.QueryServices.SYNCHRONOUS_REPLICATION_ENABLED; +import static org.apache.phoenix.replication.ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -30,10 +34,10 @@ import java.sql.PreparedStatement; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -45,82 +49,125 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Threads; -import org.apache.phoenix.end2end.IndexToolIT; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; -import org.apache.phoenix.end2end.ParallelStatsDisabledIT; import org.apache.phoenix.hbase.index.IndexRegionObserver; +import org.apache.phoenix.jdbc.FailoverPhoenixConnection; +import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.jdbc.HighAvailabilityGroup; +import org.apache.phoenix.jdbc.HighAvailabilityPolicy; +import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility; +import org.apache.phoenix.jdbc.PhoenixDriver; +import org.apache.phoenix.jdbc.PhoenixHAAdmin; import org.apache.phoenix.query.PhoenixTestBuilder; import org.apache.phoenix.query.QueryConstants; -import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.replication.tool.LogFileAnalyzer; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; -import org.apache.phoenix.util.ReadOnlyProps; import org.apache.phoenix.util.TestUtil; import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Ignore; 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.slf4j.Logger; import org.slf4j.LoggerFactory; @Category(NeedsOwnMiniClusterTest.class) -public class ReplicationLogGroupIT extends ParallelStatsDisabledIT { +public class ReplicationLogGroupIT { private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogGroupIT.class); + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + + @ClassRule + public static TemporaryFolder standbyFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder localFolder = new TemporaryFolder(); + + private static Configuration conf1; + private static Configuration conf2; + private static URI standbyUri; + private static URI fallbackUri; + private static String zkUrl; + private static String peerZkUrl; @Rule public TestName name = new TestName(); + private Properties clientProps = new Properties(); + private String haGroupName; + private PhoenixHAAdmin haAdmin1; + private HighAvailabilityGroup haGroup; + private ReplicationLogGroup logGroup; + @BeforeClass - public static synchronized void doSetup() throws Exception { - Map props = Maps.newHashMapWithExpectedSize(1); - props.put(QueryServices.SYNCHRONOUS_REPLICATION_ENABLED, Boolean.TRUE.toString()); - setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + public static void doSetup() throws Exception { + conf1 = CLUSTERS.getHBaseCluster1().getConfiguration(); + conf1.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + conf2 = CLUSTERS.getHBaseCluster2().getConfiguration(); + conf2.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + standbyUri = new Path(standbyFolder.getRoot().toString()).toUri(); + fallbackUri = new Path(localFolder.getRoot().toString()).toUri(); + conf1.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf1.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); + conf1.setInt(PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, 20); + CLUSTERS.start(); + zkUrl = CLUSTERS.getZkUrl1(); + peerZkUrl = CLUSTERS.getZkUrl2(); + DriverManager.registerDriver(PhoenixDriver.INSTANCE); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + DriverManager.deregisterDriver(PhoenixDriver.INSTANCE); + CLUSTERS.close(); } @Before public void beforeTest() throws Exception { LOG.info("Starting test {}", name.getMethodName()); + haGroupName = name.getMethodName(); + haAdmin1 = CLUSTERS.getHaAdmin1(); + clientProps = HighAvailabilityTestingUtility.getHATestProperties(); + clientProps.setProperty(PHOENIX_HA_GROUP_ATTR, haGroupName); + CLUSTERS.initClusterRole(haGroupName, HighAvailabilityPolicy.FAILOVER); + haGroup = getHighAvailibilityGroup(CLUSTERS.getJdbcHAUrl(), clientProps); + LOG.info("Initialized haGroup {} with URL {}", haGroup, CLUSTERS.getJdbcHAUrl()); + + // Update the state to ACTIVE_IN_SYNC + HAGroupStoreRecord haGroupStoreRecord + = new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, + haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, + 0L, HighAvailabilityPolicy.FAILOVER.toString(), + peerZkUrl, CLUSTERS.getMasterAddress1(), CLUSTERS.getMasterAddress2(), 0L); + haAdmin1.updateHAGroupStoreRecordInZooKeeper(haGroupName, haGroupStoreRecord, -1); + logGroup = getReplicationLogGroup(); } @After public void afterTest() throws Exception { LOG.info("Starting cleanup for test {}", name.getMethodName()); - cleanupLogsFolder(standbyUri); + logGroup.close(); LOG.info("Ending cleanup for test {}", name.getMethodName()); } - /** - * Delete all the shards under the top level replication log directory - * @throws IOException - */ - private void cleanupLogsFolder(URI source) throws IOException { - FileSystem fs = FileSystem.get(config); - Path dir = new Path(source.getPath()); - FileStatus[] statuses = fs.listStatus(dir); - for (FileStatus status : statuses) { - Path shard = status.getPath(); - if (status.isDirectory()) { - fs.delete(shard, true); - } - } - } - private ReplicationLogGroup getReplicationLogGroup() throws IOException { - HRegionServer rs = getUtility().getHBaseCluster().getRegionServer(0); - return ReplicationLogGroup.get(config, rs.getServerName(), DEFAULT_HA_GROUP); + HRegionServer rs = CLUSTERS.getHBaseCluster1().getHBaseCluster().getRegionServer(0); + return ReplicationLogGroup.get(conf1, rs.getServerName(), haGroupName); } private Map> groupLogsByTable() throws Exception { - ReplicationLogGroup log = getReplicationLogGroup(); - log.getActiveWriter().closeCurrentWriter(); LogFileAnalyzer analyzer = new LogFileAnalyzer(); - analyzer.setConf(config); - String[] args = {"--check", standbyUri.getPath()}; + analyzer.setConf(conf1); + Path standByLogDir = logGroup.getStandbyShardManager().getRootDirectoryPath(); + LOG.info("Analyzing log files at {}", standByLogDir); + String[] args = {"--check", standByLogDir.toString()}; assertEquals(0, analyzer.run(args)); - return analyzer.groupLogsByTable(standbyUri.getPath()); + return analyzer.groupLogsByTable(standByLogDir.toString()); } private int getCountForTable(Map> logsByTable, @@ -129,8 +176,9 @@ private int getCountForTable(Map> logsByTable, return mutations != null ? mutations.size() : 0; } - private void verifyReplication(Connection conn, - Map expected) throws Exception { + private void verifyReplication(Map expected) throws Exception { + // first close the logGroup + logGroup.close(); Map> mutationsByTable = groupLogsByTable(); dumpTableLogCount(mutationsByTable); for (Map.Entry entry : expected.entrySet()) { @@ -147,8 +195,12 @@ private void verifyReplication(Connection conn, assertTrue("For SYSCAT", actualMutationCount >= expectedMutationCount); } } catch (AssertionError e) { - TestUtil.dumpTable(conn, TableName.valueOf(tableName)); - throw e; + // create a regular connection + try (Connection conn = + DriverManager.getConnection(CLUSTERS.getJdbcUrl1(haGroup))) { + TestUtil.dumpTable(conn, TableName.valueOf(tableName)); + throw e; + } } } } @@ -161,7 +213,7 @@ private void dumpTableLogCount(Map> mutationsByTable) { } private void moveRegionToServer(TableName tableName, ServerName sn) throws Exception { - HBaseTestingUtility util = getUtility(); + HBaseTestingUtility util = CLUSTERS.getHBaseCluster1(); try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) { String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); while (!sn.equals(locator.getAllRegionLocations().get(0).getServerName())) { @@ -178,7 +230,11 @@ private PhoenixTestBuilder.SchemaBuilder createViewHierarchy() throws Exception // 1. Table with columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP) // 2. GlobalView with columns => (ID, COL4, COL5, COL6), PK => (ID) // 3. Tenant with columns => (ZID, COL7, COL8, COL9), PK => (ZID) - final PhoenixTestBuilder.SchemaBuilder schemaBuilder = new PhoenixTestBuilder.SchemaBuilder(getUrl()); + final PhoenixTestBuilder.SchemaBuilder schemaBuilder = + new PhoenixTestBuilder.SchemaBuilder(CLUSTERS.getJdbcHAUrl()); + PhoenixTestBuilder.SchemaBuilder.ConnectOptions connectOptions = + new PhoenixTestBuilder.SchemaBuilder.ConnectOptions(); + connectOptions.setConnectProps(clientProps); PhoenixTestBuilder.SchemaBuilder.TableOptions tableOptions = PhoenixTestBuilder.SchemaBuilder.TableOptions.withDefaults(); PhoenixTestBuilder.SchemaBuilder.GlobalViewOptions @@ -187,14 +243,12 @@ private PhoenixTestBuilder.SchemaBuilder createViewHierarchy() throws Exception tenantViewWithOverrideOptions = PhoenixTestBuilder.SchemaBuilder.TenantViewOptions.withDefaults(); PhoenixTestBuilder.SchemaBuilder.TenantViewIndexOptions tenantViewIndexOverrideOptions = PhoenixTestBuilder.SchemaBuilder.TenantViewIndexOptions.withDefaults(); - - try (Connection conn = DriverManager.getConnection(getUrl())) { - schemaBuilder.withTableOptions(tableOptions) - .withGlobalViewOptions(globalViewOptions) - .withTenantViewOptions(tenantViewWithOverrideOptions) - .withTenantViewIndexOptions(tenantViewIndexOverrideOptions) - .buildWithNewTenant(); - } + schemaBuilder.withConnectOptions(connectOptions) + .withTableOptions(tableOptions) + .withGlobalViewOptions(globalViewOptions) + .withTenantViewOptions(tenantViewWithOverrideOptions) + .withTenantViewIndexOptions(tenantViewIndexOverrideOptions) + .buildWithNewTenant(); return schemaBuilder; } @@ -204,7 +258,8 @@ public void testAppendAndSync() throws Exception { final String indexName1 = "I_" + generateUniqueName(); final String indexName2 = "I_" + generateUniqueName(); final String indexName3 = "L_" + generateUniqueName(); - try (Connection conn = DriverManager.getConnection(getUrl())) { + try (FailoverPhoenixConnection conn = (FailoverPhoenixConnection) DriverManager + .getConnection(CLUSTERS.getJdbcHAUrl(), clientProps)) { String ddl = String.format("create table %s (id1 integer not null, " + "id2 integer not null, val1 varchar, val2 varchar " + "constraint pk primary key (id1, id2))", tableName); @@ -246,7 +301,8 @@ public void testAppendAndSync() throws Exception { } } // verify the correctness of the index - IndexToolIT.verifyIndexTable(tableName, indexName1, conn); + // TODO Index tool test API doesn't work with Failover connection + //IndexToolIT.verifyIndexTable(conf1, tableName, indexName1, conn); // verify replication Map expected = Maps.newHashMap(); // mutation count will be equal to row count since the atomic upsert mutations will be @@ -259,7 +315,7 @@ public void testAppendAndSync() throws Exception { // we didn't create any tenant views so no change in the syscat entries expected.put(SYSTEM_CATALOG_NAME, 0); expected.put(SYSTEM_CHILD_LINK_NAME, 0); - verifyReplication(conn, expected); + verifyReplication(expected); } } @@ -270,12 +326,13 @@ public void testAppendAndSync() throws Exception { */ @Test public void testWALRestore() throws Exception { - HBaseTestingUtility util = getUtility(); + HBaseTestingUtility util = CLUSTERS.getHBaseCluster1(); MiniHBaseCluster cluster = util.getHBaseCluster(); final String tableName = "T_" + generateUniqueName(); final String indexName = "I_" + generateUniqueName(); TableName table = TableName.valueOf(tableName); - try (Connection conn = DriverManager.getConnection(getUrl())) { + try (FailoverPhoenixConnection conn = (FailoverPhoenixConnection) DriverManager + .getConnection(CLUSTERS.getJdbcHAUrl(), clientProps)) { String ddl = String.format("create table %s (id1 integer not null, " + "id2 integer not null, val1 varchar, val2 varchar " + "constraint pk primary key (id1, id2))", tableName); @@ -294,7 +351,8 @@ public void testWALRestore() throws Exception { moveRegionToServer(TableName.valueOf(SYSTEM_CATALOG_NAME), sn2); moveRegionToServer(TableName.valueOf(SYSTEM_CHILD_LINK_NAME), sn2); int rowCount = 50; - try (Connection conn = DriverManager.getConnection(getUrl())) { + try (FailoverPhoenixConnection conn = (FailoverPhoenixConnection) DriverManager + .getConnection(CLUSTERS.getJdbcHAUrl(), clientProps)) { PreparedStatement stmt = conn.prepareStatement( "upsert into " + tableName + " VALUES(?, ?, ?, ?)"); // upsert 50 rows @@ -311,7 +369,9 @@ public void testWALRestore() throws Exception { conn.commit(); } // Create tenant views for syscat and child link replication - createViewHierarchy(); + // Mutations on SYSTEM.CATALOG and SYSTEM.CHILD_LINK are generated on the server side + // and don't have the HAGroup attribute set + //createViewHierarchy(); } finally { IndexRegionObserver.setIgnoreSyncReplicationForTesting(false); } @@ -320,21 +380,22 @@ public void testWALRestore() throws Exception { Threads.sleep(20000); // just to be sure that the kill has fully started. // Regions will be re-opened and the WAL will be replayed util.waitUntilAllRegionsAssigned(table); - try (Connection conn = DriverManager.getConnection(getUrl())) { + try (FailoverPhoenixConnection conn = (FailoverPhoenixConnection) DriverManager + .getConnection(CLUSTERS.getJdbcHAUrl(), clientProps)) { Map expected = Maps.newHashMap(); // For each row 1 Put + 1 Delete (DeleteColumn) expected.put(tableName, rowCount * 2); // unverified + verified + delete (Delete column) expected.put(indexName, rowCount * 3); // 1 tenant view was created - expected.put(SYSTEM_CHILD_LINK_NAME, 1); + //expected.put(SYSTEM_CHILD_LINK_NAME, 1); // atleast 1 log entry for syscat - expected.put(SYSTEM_CATALOG_NAME, 1); - verifyReplication(conn, expected); + //expected.put(SYSTEM_CATALOG_NAME, 1); + verifyReplication(expected); } } - @Test + @Ignore("Mutations on SYSTEM.CATALOG and SYSTEM.CHILD_LINK are generated on the server side and don't have the HAGroup attribute set") public void testSystemTables() throws Exception { createViewHierarchy(); Map> logsByTable = groupLogsByTable(); 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 index ca1171d76f6..f253bcbb075 100644 --- 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 @@ -1629,7 +1629,7 @@ public void testReplay_TriggerFailoverAfterProcessing() throws IOException { 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); + new ReplicationShardDirectoryManager(config, fileSystem, newFilesDirectory); TestableReplicationLogTracker testableReplicationLogTracker = new TestableReplicationLogTracker(config, haGroupName, fileSystem, replicationShardDirectoryManager, METRICS_REPLICATION_LOG_TRACKER); testableReplicationLogTracker.init(); return testableReplicationLogTracker; @@ -1641,7 +1641,7 @@ private TestableReplicationLogTracker createReplicationLogTracker(final Configur */ private static class TestableReplicationLogTracker extends ReplicationLogTracker { public TestableReplicationLogTracker(Configuration config, String haGroupName, FileSystem fileSystem, ReplicationShardDirectoryManager replicationShardDirectoryManager, MetricsReplicationLogTracker metrics) { - super(config, haGroupName, fileSystem, replicationShardDirectoryManager, metrics); + super(config, haGroupName, replicationShardDirectoryManager, metrics); } public Path getInProgressDirPath() { return super.getInProgressDirPath(); diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java index bf2f65f592a..48064e16b20 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java @@ -435,8 +435,9 @@ protected static String getZKClientPort(Configuration conf) { protected static boolean clusterInitialized = false; protected static HBaseTestingUtility utility; protected static final Configuration config = HBaseConfiguration.create(); - protected static final String logDir = "/PHOENIX_REPLICATION_IN"; + protected static final String logDir = "/PHOENIX_REPLICATION"; protected static URI standbyUri = new Path(logDir).toUri(); + protected static URI localUri = new Path(logDir).toUri(); protected static String getUrl() { if (!clusterInitialized) { @@ -676,6 +677,7 @@ public static Configuration setUpConfigForMiniCluster(Configuration conf, ReadOn setPhoenixRegionServerEndpoint(conf); // setup up synchronous replication conf.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, localUri.toString()); return conf; } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/PhoenixTestBuilder.java b/phoenix-core/src/test/java/org/apache/phoenix/query/PhoenixTestBuilder.java index f3c2f58f8e9..97f712d41f3 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/PhoenixTestBuilder.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/PhoenixTestBuilder.java @@ -19,6 +19,7 @@ package org.apache.phoenix.query; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.jdbc.FailoverPhoenixConnection; import org.apache.phoenix.jdbc.PhoenixStatement; import org.apache.phoenix.thirdparty.com.google.common.collect.Sets; import org.apache.phoenix.thirdparty.com.google.common.collect.Table; @@ -1117,8 +1118,14 @@ public void build() throws Exception { PTableKey tableKey = new PTableKey(null, SchemaUtil.normalizeFullTableName(entityTableName)); - setBaseTable( - globalConnection.unwrap(PhoenixConnection.class).getTable(tableKey)); + PhoenixConnection pcon; + if (globalConnection instanceof FailoverPhoenixConnection) { + pcon = globalConnection. + unwrap(FailoverPhoenixConnection.class).getWrappedConnection(); + } else { + pcon = globalConnection.unwrap(PhoenixConnection.class); + } + setBaseTable(pcon.getTable(tableKey)); } // Index on Table if (tableIndexEnabled && !tableIndexCreated) { diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java new file mode 100644 index 00000000000..9ac3bd7e6ea --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java @@ -0,0 +1,170 @@ +/* + * 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.apache.phoenix.replication.ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +import java.io.IOException; +import java.net.URI; +import java.util.Optional; + +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.ServerName; +import org.apache.phoenix.jdbc.HAGroupStoreManager; +import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.jdbc.HAGroupStoreRecord.HAGroupState; +import org.apache.phoenix.jdbc.HighAvailabilityPolicy; +import org.apache.phoenix.replication.log.LogFileWriter; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ReplicationLogBaseTest { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogBaseTest.class); + + @ClassRule + public static TemporaryFolder standbyFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder localFolder = new TemporaryFolder(); + @Rule + public TestName name = new TestName(); + + protected String haGroupName; + protected Configuration conf; + protected ServerName serverName; + protected FileSystem localFs; + protected URI standbyUri; + protected URI fallbackUri; + @Mock + protected HAGroupStoreManager haGroupStoreManager; + protected HAGroupStoreRecord storeRecord; + protected HAGroupState initialState; + protected ReplicationLogGroup logGroup; + + static final int TEST_RINGBUFFER_SIZE = 32; + static final int TEST_SYNC_TIMEOUT = 1000; + static final int TEST_ROTATION_TIME = 5000; + static final int TEST_ROTATION_SIZE_BYTES = 10 * 1024; + static final int TEST_REPLICATION_ROUND_DURATION_SECONDS = 20; + + protected ReplicationLogBaseTest() { + this(HAGroupState.ACTIVE_IN_SYNC); + } + + protected ReplicationLogBaseTest(HAGroupState initialState) { + this.initialState = initialState; + } + + @Before + public void setUpBase() throws IOException { + MockitoAnnotations.initMocks(this); + haGroupName = name.getMethodName(); + conf = HBaseConfiguration.create(); + localFs = FileSystem.getLocal(conf); + standbyUri = new Path(standbyFolder.getRoot().toString()).toUri(); + fallbackUri = new Path(localFolder.getRoot().toString()).toUri(); + serverName = ServerName.valueOf("test", 60010, EnvironmentEdgeManager.currentTimeMillis()); + conf.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); + // Small ring buffer size for testing + conf.setInt(ReplicationLogGroup.REPLICATION_LOG_RINGBUFFER_SIZE_KEY, TEST_RINGBUFFER_SIZE); + // Set a short sync timeout for testing + conf.setLong(ReplicationLogGroup.REPLICATION_LOG_SYNC_TIMEOUT_KEY, TEST_SYNC_TIMEOUT); + // Set rotation time to 10 seconds + conf.setLong(ReplicationLogGroup.REPLICATION_LOG_ROTATION_TIME_MS_KEY, TEST_ROTATION_TIME); + // Small size threshold for testing + conf.setLong(ReplicationLogGroup.REPLICATION_LOG_ROTATION_SIZE_BYTES_KEY, + TEST_ROTATION_SIZE_BYTES); + // small value of replication round duration + conf.setInt(PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, + TEST_REPLICATION_ROUND_DURATION_SECONDS); + + // initialize the group store record + storeRecord = initHAGroupStoreRecord(); + doReturn(Optional.of(storeRecord)).when(haGroupStoreManager) + .getHAGroupStoreRecord(anyString()); + + logGroup = new TestableLogGroup(conf, serverName, haGroupName, haGroupStoreManager); + logGroup.init(); + } + + @After + public void tearDown() throws Exception { + if (logGroup != null) { + logGroup.close(); + } + } + + private HAGroupStoreRecord initHAGroupStoreRecord() { + return new HAGroupStoreRecord(null, haGroupName, initialState, 0, + HighAvailabilityPolicy.FAILOVER.toString(), "peerZKUrl", "clusterUrl", + "peerClusterUrl", 0L); + } + + static class TestableLogGroup extends ReplicationLogGroup { + + public TestableLogGroup(Configuration conf, + ServerName serverName, + String haGroupName, + HAGroupStoreManager haGroupStoreManager) { + super(conf, serverName, haGroupName, haGroupStoreManager); + } + + @Override + protected ReplicationLog createStandbyLog() throws IOException { + return spy(new TestableLog(this, standbyShardManager)); + } + + @Override + protected ReplicationLog createFallbackLog() throws IOException { + return spy(new TestableLog(this, fallbackShardManager)); + } + + } + + /** + * Testable version of ReplicationLog that allows spying on the log + */ + static class TestableLog extends ReplicationLog { + + public TestableLog(ReplicationLogGroup logGroup, ReplicationShardDirectoryManager shardManager) { + super(logGroup, shardManager); + } + + @Override + protected LogFileWriter createNewWriter() throws IOException { + LogFileWriter writer = super.createNewWriter(); + return spy(writer); + } + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java new file mode 100644 index 00000000000..e32949b2e0b --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java @@ -0,0 +1,144 @@ +/* + * 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.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.doAnswer; + +import java.io.IOException; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.phoenix.jdbc.HAGroupStoreRecord.HAGroupState; +import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; +import org.apache.phoenix.replication.log.LogFileTestUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ReplicationLogDiscoveryForwarderTest extends ReplicationLogBaseTest { + private static final Logger LOG = + LoggerFactory.getLogger(ReplicationLogDiscoveryForwarderTest.class); + + public ReplicationLogDiscoveryForwarderTest() { + // we want to start in STORE_AND_FORWARD mode + super(HAGroupState.ACTIVE_NOT_IN_SYNC); + } + + @Before + public void setUp() throws IOException { + ReplicationMode mode = logGroup.getMode(); + Assert.assertTrue(mode.equals(STORE_AND_FORWARD)); + } + + @After + public void tearDown() throws IOException {} + + @Test + public void testLogForwardingAndTransitionBackToSyncMode() throws Exception { + final String tableName = "TESTTBL"; + final long count = 100L; + int roundDurationSeconds = + logGroup.getFallbackShardManager().getReplicationRoundDurationSeconds(); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) { + // explicitly set the replication mode to SYNC + logGroup.setMode(SYNC); + try { + logGroup.sync(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + } + }).when(haGroupStoreManager).setHAGroupStatusToSync(haGroupName); + + for (long id = 1; id <=count; ++id) { + Mutation put = LogFileTestUtil.newPut("row_" + id, id, 2); + logGroup.append(tableName, id, put); + } + logGroup.sync(); + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + Future future = executor.submit(new Callable() { + @Override + public Boolean call() throws Exception { + ReplicationLogTracker logTracker = + logGroup.getLogForwarder().getReplicationLogTracker(); + while (true) { + try { + if (Thread.currentThread().isInterrupted()) { + LOG.info("Task interrupted, exiting"); + return false; + } + int newFileCount = logTracker.getNewFiles().size(); + int inProgressCount = logTracker.getInProgressFiles().size(); + if (newFileCount == 0 && inProgressCount == 0) { + // wait for the mode transition to finish + Thread.sleep(2000); + LOG.info("All files processed"); + return true; + } + LOG.info("New files = {} In-progress files = {}", + newFileCount, inProgressCount); + Thread.sleep(roundDurationSeconds * 1000); + } catch (InterruptedException e) { + LOG.info("Task received InterruptedException, exiting"); + Thread.currentThread().interrupt(); // Re-interrupt the thread + return false; + } + } + }}); + try { + Boolean ret = future.get(120, TimeUnit.SECONDS); + assertTrue(ret); + // we should have switched back to the SYNC mode + assertEquals(SYNC, logGroup.getMode()); + // the log forwarder should not be running since we are in SYNC mode + assertFalse(logGroup.getLogForwarder().isRunning); + } catch (TimeoutException e) { + LOG.info("Task timed out, cancelling it"); + future.cancel(true); + fail("Task timed out"); + } catch (InterruptedException | ExecutionException e) { + LOG.error("Task failed", e); + fail("Task failed"); + } + } finally { + executor.shutdownNow(); + } + } +} 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 index 0649b551b7e..cc0ae53dd83 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryTest.java @@ -82,7 +82,7 @@ public void setUp() throws IOException { 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); + new ReplicationShardDirectoryManager(conf, localFs, newFilesDirectory); fileTracker = Mockito.spy(new TestableReplicationLogTracker(conf, haGroupName, localFs, replicationShardDirectoryManager)); fileTracker.init(); @@ -1922,7 +1922,7 @@ public void testGetNextRoundToProcess() { */ 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); + super(conf, haGroupName, replicationShardDirectoryManager, metricsLogTracker); } } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java index 9fb31112b92..455ccd20574 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java @@ -17,6 +17,8 @@ */ package org.apache.phoenix.replication; +import static java.lang.Thread.sleep; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -29,14 +31,13 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import java.io.IOException; -import java.net.URI; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -46,19 +47,9 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; -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.ServerName; -import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.replication.log.LogFileWriter; -import org.apache.phoenix.util.EnvironmentEdgeManager; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.mockito.InOrder; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -66,58 +57,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.client.Mutation; -import org.apache.phoenix.replication.ReplicationLogGroupWriter.RotationReason; +import org.apache.phoenix.replication.ReplicationLog.RotationReason; import org.apache.phoenix.replication.log.LogFile; import org.apache.phoenix.replication.log.LogFileReader; import org.apache.phoenix.replication.log.LogFileReaderContext; import org.apache.phoenix.replication.log.LogFileTestUtil; -public class ReplicationLogGroupTest { +public class ReplicationLogGroupTest extends ReplicationLogBaseTest { private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogGroupTest.class); - @ClassRule - public static TemporaryFolder testFolder = new TemporaryFolder(); - - private Configuration conf; - private ServerName serverName; - private FileSystem localFs; - private URI standbyUri; - private ReplicationLogGroup logGroup; - - static final int TEST_RINGBUFFER_SIZE = 32; - static final int TEST_SYNC_TIMEOUT = 1000; - static final int TEST_ROTATION_TIME = 5000; - static final int TEST_ROTATION_SIZE_BYTES = 10 * 1024; - - @Before - public void setUp() throws IOException { - conf = HBaseConfiguration.create(); - localFs = FileSystem.getLocal(conf); - standbyUri = new Path(testFolder.getRoot().toString()).toUri(); - serverName = ServerName.valueOf("test", 60010, EnvironmentEdgeManager.currentTimeMillis()); - conf.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); - // Small ring buffer size for testing - conf.setInt(ReplicationLogGroup.REPLICATION_LOG_RINGBUFFER_SIZE_KEY, TEST_RINGBUFFER_SIZE); - // Set a short sync timeout for testing - conf.setLong(ReplicationLogGroup.REPLICATION_LOG_SYNC_TIMEOUT_KEY, TEST_SYNC_TIMEOUT); - // Set rotation time to 10 seconds - conf.setLong(QueryServices.REPLICATION_LOG_ROTATION_TIME_MS_KEY, TEST_ROTATION_TIME); - // Small size threshold for testing - conf.setLong(ReplicationLogGroup.REPLICATION_LOG_ROTATION_SIZE_BYTES_KEY, - TEST_ROTATION_SIZE_BYTES); - - logGroup = new TestableLogGroup(conf, serverName, "testHAGroup"); - logGroup.init(); - } - - @After - public void tearDown() throws Exception { - if (logGroup != null) { - logGroup.close(); - } - } - /** * Tests basic append and sync functionality of the replication log. Verifies that mutations * are correctly appended to the log and that sync operations properly commit the changes to @@ -138,7 +87,7 @@ public void testAppendAndSync() throws Exception { final Mutation put5 = LogFileTestUtil.newPut("row5", 5, 1); // Get the inner writer - LogFileWriter writer = logGroup.getActiveWriter().getWriter(); + LogFileWriter writer = logGroup.getActiveLog().getWriter(); assertNotNull("Writer should not be null", writer); InOrder inOrder = Mockito.inOrder(writer); @@ -171,7 +120,7 @@ public void testSyncFailureAndRetry() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); // Get the inner writer - LogFileWriter writerBeforeRoll = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerBeforeRoll = logGroup.getActiveLog().getWriter(); assertNotNull("Initial writer should not be null", writerBeforeRoll); // Configure writerBeforeRoll to fail on the first sync call @@ -182,7 +131,7 @@ public void testSyncFailureAndRetry() throws Exception { logGroup.sync(); // Get the inner writer we rolled to. - LogFileWriter writerAfterRoll = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerAfterRoll = logGroup.getActiveLog().getWriter(); assertNotNull("Initial writer should not be null", writerBeforeRoll); // Verify the sequence: append, sync (fail), rotate, append (retry), sync (succeed) @@ -205,14 +154,14 @@ public void testBlockingWhenRingFull() throws Exception { long commitId = 0; // Get the inner writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Inner writer should not be null", innerWriter); // Create a slow consumer to fill up the ring buffer. doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { - Thread.sleep(50); // Simulate slow processing + sleep(50); // Simulate slow processing return invocation.callRealMethod(); } }).when(innerWriter).append(anyString(), anyLong(), any(Mutation.class)); @@ -245,7 +194,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { assertFalse("Append should be blocked when ring is full", appendFuture.isDone()); // Let some events process to free up space. - Thread.sleep(100); + sleep(100); // Now the append should complete. Any issues and we will time out here. appendFuture.get(); @@ -266,7 +215,7 @@ public void testAppendFailureAndRetry() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); // Get the inner writer - LogFileWriter writerBeforeRoll = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerBeforeRoll = logGroup.getActiveLog().getWriter(); assertNotNull("Initial writer should not be null", writerBeforeRoll); // Configure writerBeforeRoll to fail on the first append call @@ -278,7 +227,7 @@ public void testAppendFailureAndRetry() throws Exception { logGroup.sync(); // Get the inner writer we rolled to. - LogFileWriter writerAfterRoll = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerAfterRoll = logGroup.getActiveLog().getWriter(); assertNotNull("Rolled writer should not be null", writerAfterRoll); // Verify the sequence: append (fail), rotate, append (succeed), sync @@ -301,14 +250,15 @@ public void testSyncTimeout() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); // Get the inner writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Inner writer should not be null", innerWriter); doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { // Pause long enough to cause a timeout. - Thread.sleep((long)(TEST_SYNC_TIMEOUT * 1.25)); + sleep((long)(TEST_SYNC_TIMEOUT * 1.25)); + LOG.info("Waking up from sleep"); return invocation.callRealMethod(); } }).when(innerWriter).sync(); @@ -316,13 +266,15 @@ public Object answer(InvocationOnMock invocation) throws Throwable { // Append some data logGroup.append(tableName, commitId, put); - // Try to sync and expect it to timeout + // sync on the writer will timeout try { logGroup.sync(); - fail("Expected sync to timeout"); - } catch (IOException e) { + fail("Should have thrown RuntimeException because sync timed out"); + } catch (RuntimeException e) { assertTrue("Expected timeout exception", e.getCause() instanceof TimeoutException); } + // reset + doNothing().when(innerWriter).sync(); } /** @@ -339,7 +291,7 @@ public void testConcurrentProducers() throws Exception { final CountDownLatch completionLatch = new CountDownLatch(2); // Get the inner writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Inner writer should not be null", innerWriter); // Thread 1: Append mutations with even commit IDs @@ -406,7 +358,7 @@ public void testTimeBasedRotation() throws Exception { final long commitId = 1L; // Get the initial writer - LogFileWriter writerBeforeRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerBeforeRotation = logGroup.getActiveLog().getWriter(); assertNotNull("Initial writer should not be null", writerBeforeRotation); // Append some data @@ -414,14 +366,14 @@ public void testTimeBasedRotation() throws Exception { logGroup.sync(); // Wait for rotation time to elapse - Thread.sleep((long)(TEST_ROTATION_TIME * 1.25)); + sleep((long)(TEST_ROTATION_TIME * 1.25)); // Append more data to trigger rotation check logGroup.append(tableName, commitId + 1, put); logGroup.sync(); // Get the new writer after rotation - LogFileWriter writerAfterRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerAfterRotation = logGroup.getActiveLog().getWriter(); assertNotNull("New writer should not be null", writerAfterRotation); assertTrue("Writer should have been rotated", writerAfterRotation != writerBeforeRotation); @@ -447,7 +399,7 @@ public void testSizeBasedRotation() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 10); long commitId = 1L; - LogFileWriter writerBeforeRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerBeforeRotation = logGroup.getActiveLog().getWriter(); assertNotNull("Initial writer should not be null", writerBeforeRotation); // Append enough data so that we exceed the size threshold. @@ -457,7 +409,7 @@ public void testSizeBasedRotation() throws Exception { logGroup.sync(); // Should trigger a sized based rotation // Get the new writer after the expected rotation. - LogFileWriter writerAfterRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerAfterRotation = logGroup.getActiveLog().getWriter(); assertNotNull("New writer should not be null", writerAfterRotation); assertTrue("Writer should have been rotated", writerAfterRotation != writerBeforeRotation); @@ -490,7 +442,7 @@ public void testClose() throws Exception { final long commitId = 1L; // Get the inner writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Inner writer should not be null", innerWriter); // Append some data @@ -499,6 +451,9 @@ public void testClose() throws Exception { // Close the log writer logGroup.close(); + // wait for the event handler thread to exit since that calls the close + // on the underlying writer + Thread.sleep(5); // Verify the inner writer was closed verify(innerWriter, times(1)).close(); @@ -532,16 +487,16 @@ public void testRotationTask() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); long commitId = 1L; - LogFileWriter writerBeforeRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerBeforeRotation = logGroup.getActiveLog().getWriter(); assertNotNull("Initial writer should not be null", writerBeforeRotation); // Append some data and wait for the rotation time to elapse plus a small buffer. logGroup.append(tableName, commitId, put); logGroup.sync(); - Thread.sleep((long)(TEST_ROTATION_TIME * 1.25)); + sleep((long)(TEST_ROTATION_TIME * 1.25)); // Get the new writer after the rotation. - LogFileWriter writerAfterRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerAfterRotation = logGroup.getActiveLog().getWriter(); assertNotNull("New writer should not be null", writerAfterRotation); assertTrue("Writer should have been rotated", writerAfterRotation != writerBeforeRotation); @@ -573,10 +528,10 @@ public void testFailedRotation() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); long commitId = 1L; - ReplicationLogGroupWriter logGroupWriter = logGroup.getActiveWriter(); + ReplicationLog activeLog = logGroup.getActiveLog(); // Get the initial writer - LogFileWriter initialWriter = logGroupWriter.getWriter(); + LogFileWriter initialWriter = activeLog.getWriter(); assertNotNull("Initial writer should not be null", initialWriter); // Configure the log writer to fail only the first time when creating new writers. @@ -586,14 +541,14 @@ public void testFailedRotation() throws Exception { throw new IOException("Simulated failure to create new writer"); } return invocation.callRealMethod(); - }).when(logGroupWriter).createNewWriter(); + }).when(activeLog).createNewWriter(); // Append some data logGroup.append(tableName, commitId, put); logGroup.sync(); // Rotate the log. - LogFileWriter writerAfterFailedRotate = logGroupWriter.rotateLog(RotationReason.TIME); + LogFileWriter writerAfterFailedRotate = activeLog.rotateLog(RotationReason.TIME); assertEquals("Should still be using the initial writer", initialWriter, writerAfterFailedRotate); @@ -601,7 +556,7 @@ public void testFailedRotation() throws Exception { logGroup.append(tableName, commitId + 1, put); logGroup.sync(); - LogFileWriter writerAfterRotate = logGroupWriter.rotateLog(RotationReason.TIME); + LogFileWriter writerAfterRotate = activeLog.rotateLog(RotationReason.TIME); assertNotEquals("Should be using a new writer", initialWriter, writerAfterRotate); // Try to append more data. This should work with the new writer after successful rotation. @@ -631,15 +586,15 @@ public void testTooManyRotationFailures() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); long commitId = 1L; - ReplicationLogGroupWriter logGroupWriter = logGroup.getActiveWriter(); + ReplicationLog activeLog = logGroup.getActiveLog(); // Get the initial writer - LogFileWriter initialWriter = logGroupWriter.getWriter(); + LogFileWriter initialWriter = activeLog.getWriter(); assertNotNull("Initial writer should not be null", initialWriter); // Configure the log writer to always fail when creating new writers doThrow(new IOException("Simulated failure to create new writer")) - .when(logGroupWriter).createNewWriter(); + .when(activeLog).createNewWriter(); // Append some data logGroup.append(tableName, commitId, put); @@ -648,7 +603,7 @@ public void testTooManyRotationFailures() throws Exception { // Try to rotate the log multiple times until we exceed the retry limit for (int i = 0; i <= ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_RETRIES; i++) { try { - logGroupWriter.rotateLog(RotationReason.TIME); + activeLog.rotateLog(RotationReason.TIME); } catch (IOException e) { if (i < ReplicationLogGroup.DEFAULT_REPLICATION_LOG_ROTATION_RETRIES) { // Not the last attempt yet, continue @@ -662,15 +617,11 @@ public void testTooManyRotationFailures() throws Exception { } } - // Verify subsequent operations fail because the log is closed - try { - logGroup.append(tableName, commitId + 1, put); - logGroup.sync(); - fail("Expected append to fail because log is closed"); - } catch (IOException e) { - assertTrue("Expected an IOException because log is closed", - e.getMessage().contains("Closed")); - } + // Verify subsequent operations will fail because the log is closed and then trigger + // a mode switch to STORE_AND_FORWARD + logGroup.append(tableName, commitId + 1, put); + logGroup.sync(); + assertEquals(STORE_AND_FORWARD, logGroup.getMode()); } /** @@ -684,7 +635,7 @@ public void testEventProcessingException() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); // Get the inner writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Writer should not be null", innerWriter); // Configure writer to throw a RuntimeException on append @@ -695,8 +646,8 @@ public void testEventProcessingException() throws Exception { logGroup.append(tableName, commitId, put); try { logGroup.sync(); - fail("Should have thrown IOException because sync timed out"); - } catch (IOException e) { + fail("Should have thrown Runtime because sync timed out"); + } catch (RuntimeException e) { assertTrue("Expected timeout exception", e.getCause() instanceof TimeoutException); } @@ -714,19 +665,20 @@ public void testEventProcessingException() throws Exception { } /** - * Tests behavior when all sync retry attempts are exhausted. Verifies that the system properly - * handles the case where sync operations fail repeatedly and eventually timeout. + * Tests the behavior when a sync operation fails multiple times until all the + * attempts are exhausted on the remote cluster and then we switch to the STORE_AND_FORWARD + * mode and successfully complete the sync */ @Test - public void testSyncFailureAllRetriesExhausted() throws Exception { + public void testSwitchToStoreAndForwardOnSyncFailure() throws Exception { final String tableName = "TBLSAFR"; final long commitId = 1L; final Mutation put = LogFileTestUtil.newPut("row", 1, 1); - ReplicationLogGroupWriter logGroupWriter = logGroup.getActiveWriter(); + ReplicationLog activeLog = logGroup.getActiveLog(); // Get the initial writer - LogFileWriter initialWriter = logGroupWriter.getWriter(); + LogFileWriter initialWriter = activeLog.getWriter(); assertNotNull("Initial writer should not be null", initialWriter); // Configure initial writer to fail on sync @@ -734,21 +686,55 @@ public void testSyncFailureAllRetriesExhausted() throws Exception { .when(initialWriter).sync(); // createNewWriter should keep returning the bad writer - doAnswer(invocation -> initialWriter).when(logGroupWriter).createNewWriter(); + doAnswer(invocation -> initialWriter).when(activeLog).createNewWriter(); // Append data logGroup.append(tableName, commitId, put); + // Try to sync. Should fail after exhausting retries and then switch to STORE_AND_FORWARD + logGroup.sync(); + + // Each retry creates a new writer, so that is at least 1 create + 4 retries. + verify(activeLog, atLeast(5)).createNewWriter(); + assertEquals(STORE_AND_FORWARD, logGroup.getMode()); + } + + /** + * Tests the behavior when we fail to update the HAGroup store status when we switch to + * the STORE_AND_FORWARD mode and abort + */ + @Test + public void testFailToUpdateHAGroupStatusOnSwitchToStoreAndForward() throws Exception { + final String tableName = "TBLSAFR"; + final long commitId = 1L; + final Mutation put = LogFileTestUtil.newPut("row", 1, 1); + + ReplicationLog activeLog = logGroup.getActiveLog(); - // Try to sync. Should fail after exhausting retries. + // Get the initial writer + LogFileWriter initialWriter = activeLog.getWriter(); + assertNotNull("Initial writer should not be null", initialWriter); + + // Configure initial writer to fail on sync + doThrow(new IOException("Simulated sync failure")) + .when(initialWriter).sync(); + + // createNewWriter should keep returning the bad writer + doAnswer(invocation -> initialWriter).when(activeLog).createNewWriter(); + + doThrow(new IOException("Simulated failure to update HAGroupStore state")) + .when(haGroupStoreManager).setHAGroupStatusToStoreAndForward(haGroupName); + + // Append data + logGroup.append(tableName, commitId, put); + // Try to sync. Should fail after exhausting retries and then switch to STORE_AND_FORWARD try { logGroup.sync(); - fail("Expected sync to fail after exhausting retries"); - } catch (IOException e) { - assertTrue("Expected timeout exception", e.getCause() instanceof TimeoutException); + fail("Should have thrown exception because of failure to update mode"); + } catch (RuntimeException ex) { + assertTrue(ex.getMessage().contains("Simulated sync failure")); } - - // Each retry creates a new writer, so that is at least 1 create + 5 retries. - verify(logGroupWriter, atLeast(6)).createNewWriter(); + // wait for the even processor thread to clean up + Thread.sleep(3); } /** @@ -762,7 +748,7 @@ public void testRotationDuringBatch() throws Exception { long commitId = 1L; // Get the initial writer - LogFileWriter writerBeforeRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerBeforeRotation = logGroup.getActiveLog().getWriter(); assertNotNull("Initial writer should not be null", writerBeforeRotation); // Append several items to fill currentBatch but don't sync yet @@ -771,10 +757,10 @@ public void testRotationDuringBatch() throws Exception { } // Force a rotation by waiting for rotation time to elapse - Thread.sleep((long)(TEST_ROTATION_TIME * 1.25)); + sleep((long)(TEST_ROTATION_TIME * 1.25)); // Get the new writer after rotation - LogFileWriter writerAfterRotation = logGroup.getActiveWriter().getWriter(); + LogFileWriter writerAfterRotation = logGroup.getActiveLog().getWriter(); assertNotNull("New writer should not be null", writerAfterRotation); assertTrue("Writer should have been rotated", writerAfterRotation != writerBeforeRotation); @@ -813,10 +799,10 @@ public void testReadAfterWrite() throws Exception { final int NUM_RECORDS = 100; List originalRecords = new ArrayList<>(); - ReplicationLogGroupWriter logGroupWriter = logGroup.getActiveWriter(); + ReplicationLog activeLog = logGroup.getActiveLog(); // Get the path of the log file. - Path logPath = logGroupWriter.getWriter().getContext().getFilePath(); + Path logPath = activeLog.getWriter().getContext().getFilePath(); for (int i = 0; i < NUM_RECORDS; i++) { LogFile.Record record = LogFileTestUtil.newPutRecord(tableName, i, "row" + i, i, 1); @@ -827,7 +813,7 @@ public void testReadAfterWrite() throws Exception { logGroup.sync(); // Sync to commit the appends to the current writer. // Force a rotation to close the current writer. - logGroupWriter.rotateLog(RotationReason.SIZE); + activeLog.rotateLog(RotationReason.SIZE); assertTrue("Log file should exist", localFs.exists(logPath)); @@ -869,12 +855,12 @@ public void testReadAfterMultipleRotations() throws Exception { List originalRecords = new ArrayList<>(); List logPaths = new ArrayList<>(); - ReplicationLogGroupWriter logGroupWriter = logGroup.getActiveWriter(); + ReplicationLog activeLog = logGroup.getActiveLog(); // Write records across multiple rotations. for (int rotation = 0; rotation < NUM_ROTATIONS; rotation++) { // Get the path of the current log file. - Path logPath = logGroupWriter.getWriter().getContext().getFilePath(); + Path logPath = activeLog.getWriter().getContext().getFilePath(); logPaths.add(logPath); for (int i = 0; i < NUM_RECORDS_PER_ROTATION; i++) { @@ -887,7 +873,7 @@ public void testReadAfterMultipleRotations() throws Exception { } logGroup.sync(); // Sync to commit the appends to the current writer. // Force a rotation to close the current writer. - logGroupWriter.rotateLog(RotationReason.SIZE); + activeLog.rotateLog(RotationReason.SIZE); } // Verify all log files exist @@ -938,12 +924,12 @@ public void testReadAfterMultipleRotationsWithReplay() throws Exception { List originalRecords = new ArrayList<>(); List logPaths = new ArrayList<>(); - ReplicationLogGroupWriter logGroupWriter = logGroup.getActiveWriter(); + ReplicationLog activeLog = logGroup.getActiveLog(); // Write records across multiple rotations, only syncing 50% of the time. for (int rotation = 0; rotation < NUM_ROTATIONS; rotation++) { // Get the path of the current log file. - Path logPath = logGroupWriter.getWriter().getContext().getFilePath(); + Path logPath = activeLog.getWriter().getContext().getFilePath(); logPaths.add(logPath); for (int i = 0; i < NUM_RECORDS_PER_ROTATION; i++) { @@ -961,7 +947,7 @@ public void testReadAfterMultipleRotationsWithReplay() throws Exception { logGroup.sync(); // Sync to commit the appends to the current writer. } // Force a rotation to close the current writer. - logGroupWriter.rotateLog(RotationReason.SIZE); + activeLog.rotateLog(RotationReason.SIZE); } // Verify all log files exist @@ -1008,7 +994,7 @@ public void testRuntimeExceptionDuringLengthCheck() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); // Get the initial writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Inner writer should not be null", innerWriter); // Configure writer to throw RuntimeException on getLength() @@ -1019,8 +1005,8 @@ public void testRuntimeExceptionDuringLengthCheck() throws Exception { logGroup.append(tableName, commitId, put); try { logGroup.sync(); - fail("Should have thrown IOException because sync timed out"); - } catch (IOException e) { + fail("Should have thrown RuntimeException because sync timed out"); + } catch (RuntimeException e) { assertTrue("Expected timeout exception", e.getCause() instanceof TimeoutException); } @@ -1048,7 +1034,7 @@ public void testAppendAfterCloseOnError() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); // Get the inner writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Writer should not be null", innerWriter); // Configure writer to throw RuntimeException on append @@ -1059,8 +1045,8 @@ public void testAppendAfterCloseOnError() throws Exception { logGroup.append(tableName, commitId, put); try { logGroup.sync(); - fail("Should have thrown IOException because sync timed out"); - } catch (IOException e) { + fail("Should have thrown RuntimeException because sync timed out"); + } catch (RuntimeException e) { assertTrue("Expected timeout exception", e.getCause() instanceof TimeoutException); } @@ -1088,7 +1074,7 @@ public void testSyncAfterCloseOnError() throws Exception { final Mutation put = LogFileTestUtil.newPut("row", 1, 1); // Get the inner writer - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Writer should not be null", innerWriter); // Configure writer to throw RuntimeException on append @@ -1099,8 +1085,8 @@ public void testSyncAfterCloseOnError() throws Exception { logGroup.append(tableName, commitId, put); try { logGroup.sync(); - fail("Should have thrown IOException because sync timed out"); - } catch (IOException e) { + fail("Should have thrown RuntimeException because sync timed out"); + } catch (RuntimeException e) { assertTrue("Expected timeout exception", e.getCause() instanceof TimeoutException); } @@ -1133,14 +1119,14 @@ public void testSyncConsolidation() throws Exception { final Mutation put3 = LogFileTestUtil.newPut("row3", 3, 1); final long commitId3 = 3L; - LogFileWriter innerWriter = logGroup.getActiveWriter().getWriter(); + LogFileWriter innerWriter = logGroup.getActiveLog().getWriter(); assertNotNull("Inner writer should not be null", innerWriter); // Configure writer to briefly hold up the LogEventHandler upon first append. doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { - Thread.sleep(50); // Delay to allow multiple events to be posted + sleep(50); // Delay to allow multiple events to be posted return invocation.callRealMethod(); } }).when(innerWriter).append(eq(tableName), eq(commitId1), eq(put1)); @@ -1174,24 +1160,24 @@ public void testReplicationLogGroupCaching() throws Exception { final String haGroupId2 = "testHAGroup2"; // Get instances for the first HA group - ReplicationLogGroup g1_1 = ReplicationLogGroup.get(conf, serverName, haGroupId1); - ReplicationLogGroup g1_2 = ReplicationLogGroup.get(conf, serverName, haGroupId1); + ReplicationLogGroup g1_1 = ReplicationLogGroup.get(conf, serverName, haGroupId1, haGroupStoreManager); + ReplicationLogGroup g1_2 = ReplicationLogGroup.get(conf, serverName, haGroupId1, haGroupStoreManager); // Verify same instance is returned for same haGroupId assertNotNull("ReplicationLogGroup should not be null", g1_1); assertNotNull("ReplicationLogGroup should not be null", g1_2); assertTrue("Same instance should be returned for same haGroupId", g1_2 == g1_1); - assertEquals("HA Group name should match", haGroupId1, g1_1.getHaGroupName()); + assertEquals("HA Group name should match", haGroupId1, g1_1.getHAGroupName()); // Get instance for a different HA group - ReplicationLogGroup g2_1 = ReplicationLogGroup.get(conf, serverName, haGroupId2); + ReplicationLogGroup g2_1 = ReplicationLogGroup.get(conf, serverName, haGroupId2, haGroupStoreManager); assertNotNull("ReplicationLogGroup should not be null", g2_1); assertTrue("Different instance should be returned for different haGroupId", g2_1 != g1_1); - assertEquals("HA Group name should match", haGroupId2, g2_1.getHaGroupName()); + assertEquals("HA Group name should match", haGroupId2, g2_1.getHAGroupName()); // Verify multiple calls still return cached instances - ReplicationLogGroup g1_3 = ReplicationLogGroup.get(conf, serverName, haGroupId1); - ReplicationLogGroup g2_2 = ReplicationLogGroup.get(conf, serverName, haGroupId2); + ReplicationLogGroup g1_3 = ReplicationLogGroup.get(conf, serverName, haGroupId1, haGroupStoreManager); + ReplicationLogGroup g2_2 = ReplicationLogGroup.get(conf, serverName, haGroupId2, haGroupStoreManager); assertTrue("Cached instance should be returned", g1_3 == g1_1); assertTrue("Cached instance should be returned", g2_2 == g2_1); @@ -1209,12 +1195,14 @@ public void testReplicationLogGroupCacheRemovalOnClose() throws Exception { final String haGroupId = "testHAGroupCacheRemoval"; // Get initial instance - ReplicationLogGroup g1_1 = ReplicationLogGroup.get(conf, serverName, haGroupId); + ReplicationLogGroup g1_1 = ReplicationLogGroup.get(conf, + serverName, haGroupId, haGroupStoreManager); assertNotNull("ReplicationLogGroup should not be null", g1_1); assertFalse("Group should not be closed initially", g1_1.isClosed()); // Verify cached instance is returned - ReplicationLogGroup g1_2 = ReplicationLogGroup.get(conf, serverName, haGroupId); + ReplicationLogGroup g1_2 = ReplicationLogGroup.get(conf, serverName, haGroupId, + haGroupStoreManager); assertTrue("Same instance should be returned before close", g1_2 == g1_1); // Close the group @@ -1222,44 +1210,102 @@ public void testReplicationLogGroupCacheRemovalOnClose() throws Exception { assertTrue("Group should be closed", g1_1.isClosed()); // Get instance after close - should be a new instance - ReplicationLogGroup g1_3 = ReplicationLogGroup.get(conf, serverName, haGroupId); + ReplicationLogGroup g1_3 = ReplicationLogGroup.get(conf, serverName, haGroupId, + haGroupStoreManager); assertNotNull("ReplicationLogGroup should not be null after close", g1_3); assertFalse("New group should not be closed", g1_3.isClosed()); assertTrue("New instance should be created after close", g1_1 != g1_3); - assertEquals("HA Group name should match", haGroupId, g1_3.getHaGroupName()); + assertEquals("HA Group name should match", haGroupId, g1_3.getHAGroupName()); // Clean up g1_3.close(); } - static class TestableLogGroup extends ReplicationLogGroup { + @Test + public void testInFlightAppendsReplayAfterModeSwitch() throws Exception { + final String tableName = "TESTTBL"; + final long commitId1 = 1L; + final long commitId2 = 2L; + final long commitId3 = 3L; + final long commitId4 = 4L; + final long commitId5 = 5L; + final Mutation put1 = LogFileTestUtil.newPut("row1", 1, 1); + final Mutation put2 = LogFileTestUtil.newPut("row2", 2, 1); + final Mutation put3 = LogFileTestUtil.newPut("row3", 3, 1); + final Mutation put4 = LogFileTestUtil.newPut("row4", 4, 1); + final Mutation put5 = LogFileTestUtil.newPut("row5", 5, 1); - public TestableLogGroup(Configuration conf, ServerName serverName, String haGroupName) { - super(conf, serverName, haGroupName); - } + // Get the inner writer + ReplicationLog activeLog = logGroup.getActiveLog(); + LogFileWriter writer = activeLog.getWriter(); + assertNotNull("Writer should not be null", writer); + // keep returning the same writer + doAnswer(invocation -> writer).when(activeLog).createNewWriter(); - @Override - protected ReplicationLogGroupWriter createRemoteWriter() throws IOException { - ReplicationLogGroupWriter writer = spy(new TestableStandbyLogGroupWriter(this)); - writer.init(); - return writer; - } + logGroup.append(tableName, commitId1, put1); + logGroup.append(tableName, commitId2, put2); + logGroup.append(tableName, commitId3, put3); + logGroup.append(tableName, commitId4, put4); + + // configure writer to throw IOException on the 5th append + doThrow(new IOException("Simulate append failure")) + .when(writer).append(tableName, commitId5, put5); + + logGroup.append(tableName, commitId5, put5); + logGroup.sync(); + LogFileWriter storeAndForwardWriter = logGroup.getActiveLog().getWriter(); + assertTrue("After switching mode we should have a new writer", + writer != storeAndForwardWriter); + InOrder inOrder = Mockito.inOrder(storeAndForwardWriter); + + // verify that all the in-flight appends and syncs are replayed on the new store and forward writer + inOrder.verify(storeAndForwardWriter, times(1)) + .append(eq(tableName), eq(commitId1), eq(put1)); + inOrder.verify(storeAndForwardWriter, times(1)) + .append(eq(tableName), eq(commitId2), eq(put2)); + inOrder.verify(storeAndForwardWriter, times(1)) + .append(eq(tableName), eq(commitId3), eq(put3)); + inOrder.verify(storeAndForwardWriter, times(1)) + .append(eq(tableName), eq(commitId4), eq(put4)); + inOrder.verify(storeAndForwardWriter, times(1)) + .append(eq(tableName), eq(commitId5), eq(put5)); + inOrder.verify(storeAndForwardWriter, times(1)).sync(); } - /** - * Testable version of StandbyLogGroupWriter that allows spying on writers. - */ - static class TestableStandbyLogGroupWriter extends StandbyLogGroupWriter { + //@Test + public void testAppendTimeoutWhileSyncPending() throws Exception { + final String tableName = "TESTTBL"; + final long commitId1 = 1L; + final Mutation put1 = LogFileTestUtil.newPut("row1", 1, 1); - protected TestableStandbyLogGroupWriter(ReplicationLogGroup logGroup) { - super(logGroup); - } + // Get the inner writer + ReplicationLog activeLog = logGroup.getActiveLog(); + LogFileWriter writer = activeLog.getWriter(); + assertNotNull("Writer should not be null", writer); + // keep returning the same writer + //doAnswer(invocation -> writer).when(activeLog).createNewWriter(); + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + //Thread.sleep((long)(TEST_SYNC_TIMEOUT * 1.25)); // Simulate slow append processing + //throw new CallTimeoutException("Simulate append timeout"); + Object result = invocation.callRealMethod(); + sleep((long)(TEST_SYNC_TIMEOUT * 1.25)); // Simulate slow but successful append + return result; + } + }).when(writer).append(anyString(), anyLong(), any(Mutation.class)); - @Override - protected LogFileWriter createNewWriter() throws IOException { - LogFileWriter writer = super.createNewWriter(); - return spy(writer); - } + logGroup.append(tableName, commitId1, put1); + logGroup.sync(); + + LogFileWriter storeAndForwardWriter = logGroup.getActiveLog().getWriter(); + assertTrue("After switching mode we should have a new writer", + writer != storeAndForwardWriter); + InOrder inOrder = Mockito.inOrder(storeAndForwardWriter); + // verify that all the in-flight appends and syncs are replayed on the new store and forward writer + inOrder.verify(storeAndForwardWriter, times(1)) + .append(eq(tableName), eq(commitId1), eq(put1)); + inOrder.verify(storeAndForwardWriter, times(1)).sync(); } } 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 index f332ad7a8f3..e8bc60e017d 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogTrackerTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogTrackerTest.java @@ -1306,8 +1306,8 @@ private int countDirectories(FileSystem fs, Path path) throws IOException { 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); + new ReplicationShardDirectoryManager(conf, fileSystem, newFilesDirectory); + return new ReplicationLogTracker(conf, haGroupName, 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 index f26e8436e65..b5ce4548eea 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationShardDirectoryManagerTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationShardDirectoryManagerTest.java @@ -57,7 +57,7 @@ public void setUp() throws IOException { // Create manager with default configuration Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); - manager = new ReplicationShardDirectoryManager(conf, rootPath); + manager = new ReplicationShardDirectoryManager(conf, localFs, rootPath); } @After @@ -264,7 +264,8 @@ public void testGetNearestRoundStartTimestampWithCustomConfiguration() { customConf.setInt(ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, 30); Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); - ReplicationShardDirectoryManager customManager = new ReplicationShardDirectoryManager(customConf, rootPath); + ReplicationShardDirectoryManager customManager = + new ReplicationShardDirectoryManager(customConf, localFs, rootPath); long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC @@ -366,7 +367,8 @@ public void testGetReplicationRoundFromStartTimeWithCustomConfiguration() { customConf.setInt(ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, 30); Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); - ReplicationShardDirectoryManager customManager = new ReplicationShardDirectoryManager(customConf, rootPath); + ReplicationShardDirectoryManager customManager = + new ReplicationShardDirectoryManager(customConf, localFs, rootPath); long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC long roundDurationMs = 30 * 1000L; // 30 seconds in milliseconds @@ -492,7 +494,8 @@ public void testGetReplicationRoundFromEndTimeWithCustomConfiguration() { customConf.setInt(ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, 30); Path rootPath = new Path(testFolder.getRoot().getAbsolutePath()); - ReplicationShardDirectoryManager customManager = new ReplicationShardDirectoryManager(customConf, rootPath); + ReplicationShardDirectoryManager customManager = + new ReplicationShardDirectoryManager(customConf, localFs, rootPath); long dayStart = 1704067200000L; // 2024-01-01 00:00:00 UTC long roundDurationMs = 30 * 1000L; // 30 seconds in milliseconds @@ -516,7 +519,9 @@ public void testGetReplicationRoundFromEndTimeWithCustomConfiguration() { @Test public void testDefaultAndCustomConfigurationValues() { // Test with default configuration (no custom values set) - ReplicationShardDirectoryManager defaultManager = new ReplicationShardDirectoryManager(conf, new Path(testFolder.getRoot().getAbsolutePath())); + Path testPath = new Path(testFolder.getRoot().getAbsolutePath()); + ReplicationShardDirectoryManager defaultManager = + new ReplicationShardDirectoryManager(conf, localFs, testPath); // Validate default values assertEquals("numShards should use default value when not configured", @@ -532,7 +537,8 @@ public void testDefaultAndCustomConfigurationValues() { 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())); + ReplicationShardDirectoryManager customManager = + new ReplicationShardDirectoryManager(customConf, localFs, testPath); // Validate custom values assertEquals("numShards should use custom value when configured", @@ -576,7 +582,9 @@ public void testGetAllShardPathsWithCustomConfiguration() { private void testGetAllShardPathsHelper(final Configuration conf, final int expectedShardCount) { - ReplicationShardDirectoryManager replicationShardDirectoryManager = new ReplicationShardDirectoryManager(conf, new Path(testFolder.getRoot().getAbsolutePath())); + Path testPath = new Path(testFolder.getRoot().getAbsolutePath()); + ReplicationShardDirectoryManager replicationShardDirectoryManager = + new ReplicationShardDirectoryManager(conf, localFs, testPath); List shardPaths = replicationShardDirectoryManager.getAllShardPaths(); // Verify the number of shards From 54b5df8443f55f7868724c527c66e2f0b0122efb Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Mon, 10 Nov 2025 13:44:50 -0800 Subject: [PATCH 02/13] Added some more details to the ASCII diagram --- .../org/apache/phoenix/replication/ReplicationLogGroup.java | 2 ++ 1 file changed, 2 insertions(+) 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 64842eb25a3..6f03f82c512 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 @@ -104,6 +104,8 @@ * │ │ │ │ │ │ * │ │ │ ┌────────────────────────────────────────────────┐ │ │ │ * │ │ │ │ ReplicationLog │ │ │ │ + * │ │ │ │ - Log File creation,close,rotation │ │ │ │ + * │ │ │ │ - Retries │ │ │ │ * │ │ │ │ ┌──────────────────────────────────────────┐ │ │ │ │ * │ │ │ │ │ LogFileWriter │ │ │ │ │ * │ │ │ │ │ - File Management │ │ │ │ │ From 5f67242f650e8d01a0a07ae82a244a890ac53562 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Tue, 11 Nov 2025 14:52:29 -0800 Subject: [PATCH 03/13] Added Apache license files --- .../ReplicationLogDiscoveryForwarder.java | 17 +++++++++++++++++ .../replication/ReplicationModeImpl.java | 17 +++++++++++++++++ .../replication/StoreAndForwardModeImpl.java | 17 +++++++++++++++++ .../replication/SyncAndForwardModeImpl.java | 17 +++++++++++++++++ .../phoenix/replication/SyncModeImpl.java | 17 +++++++++++++++++ 5 files changed, 85 insertions(+) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java index 7f4ff3e5bd6..d6ed081c616 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java @@ -1,3 +1,20 @@ +/* + * 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.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java index 312a8cfbde2..18820a396f7 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationModeImpl.java @@ -1,3 +1,20 @@ +/* + * 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; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java index ba86f32c9a2..ae9273dfabd 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java @@ -1,3 +1,20 @@ +/* + * 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.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java index 508498409cf..6bb86a05c19 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java @@ -1,3 +1,20 @@ +/* + * 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.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java index 2d6ef349cdb..5de43fe31f2 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java @@ -1,3 +1,20 @@ +/* + * 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.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; From ac0f926d787873d6eb4b53f822cd1b585b954963 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Wed, 12 Nov 2025 14:25:29 -0800 Subject: [PATCH 04/13] Fix checkstyle issues --- .../phoenix/replication/ReplicationLog.java | 3 +- .../ReplicationLogDiscoveryForwarder.java | 38 +++++++++++------- .../ReplicationShardDirectoryManager.java | 39 +++++++------------ .../replication/StoreAndForwardModeImpl.java | 12 +++--- .../replication/SyncAndForwardModeImpl.java | 6 +-- .../phoenix/replication/SyncModeImpl.java | 6 +-- ...sReplicationLogForwarderSourceFactory.java | 23 ++++++++++- ...icsReplicationLogTrackerForwarderImpl.java | 2 +- 8 files changed, 75 insertions(+), 54 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java index 9419c967523..2c39bb227be 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLog.java @@ -77,7 +77,8 @@ public class ReplicationLog { // when the writer changes because of rotation while we are in the middle of a write operation. private long generation; - public ReplicationLog(ReplicationLogGroup logGroup, ReplicationShardDirectoryManager shardManager) { + public ReplicationLog(ReplicationLogGroup logGroup, + ReplicationShardDirectoryManager shardManager) { this.logGroup = logGroup; Configuration conf = logGroup.getConfiguration(); this.maxAttempts = conf.getInt(ReplicationLogGroup.REPLICATION_LOG_SYNC_RETRIES_KEY, diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java index d6ed081c616..6fee9f488ba 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java @@ -17,10 +17,6 @@ */ package org.apache.phoenix.replication; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; - import java.io.IOException; import org.apache.hadoop.fs.FileStatus; @@ -38,14 +34,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; + /** * ReplicationLogDiscoveryForwarder manages the forwarding of the replication log * from the fallback cluster to the remote cluster. */ public class ReplicationLogDiscoveryForwarder extends ReplicationLogDiscovery { - private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogDiscoveryForwarder.class); + private static final Logger LOG = + LoggerFactory.getLogger(ReplicationLogDiscoveryForwarder.class); + + public static final String REPLICATION_LOG_COPY_THROUGHPUT_BYTES_PER_MS_KEY = + "phoenix.replication.log.copy.throughput.bytes.per.ms"; + // TODO: come up with a better default after testing + public static final double DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS = 1; private final ReplicationLogGroup logGroup; + private double copyThroughputThresholdBytesPerMs; /** * Create a tracker for the replication logs in the fallback cluster. @@ -66,6 +73,9 @@ private static ReplicationLogTracker createLogTracker(ReplicationLogGroup logGro public ReplicationLogDiscoveryForwarder(ReplicationLogGroup logGroup) { super(createLogTracker(logGroup)); this.logGroup = logGroup; + this.copyThroughputThresholdBytesPerMs = + conf.getDouble(REPLICATION_LOG_COPY_THROUGHPUT_BYTES_PER_MS_KEY, + DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS); } @Override @@ -88,7 +98,8 @@ public void init() throws IOException { modifiedTime, clusterType, lastSyncStateTimeInMs) -> { - if (clusterType == ClusterType.LOCAL && HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC.equals(toState)) { + if (clusterType == ClusterType.LOCAL + && HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC.equals(toState)) { LOG.info("Received ACTIVE_NOT_IN_SYNC event for {}", logGroup); // If the current mode is SYNC only then switch to SYNC_AND_FORWARD mode if (logGroup.checkAndSetMode(SYNC, SYNC_AND_FORWARD)) { @@ -111,7 +122,8 @@ public void init() throws IOException { modifiedTime, clusterType, lastSyncStateTimeInMs) -> { - if (clusterType == ClusterType.LOCAL && HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC.equals(toState)) { + if (clusterType == ClusterType.LOCAL + && HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC.equals(toState)) { LOG.info("Received ACTIVE_IN_SYNC event for {}", logGroup); // Set the current mode to SYNC if (logGroup.checkAndSetMode(SYNC_AND_FORWARD, SYNC)) { @@ -145,8 +157,8 @@ protected void processFile(Path src) throws IOException { long endTime = EnvironmentEdgeManager.currentTimeMillis(); long copyTime = endTime - startTime; LOG.info("Copying file src={} dst={} size={} took {}ms", src, dst, srcStat.getLen(), copyTime); - if (logGroup.getMode() == STORE_AND_FORWARD && - isLogCopyThroughputAboveThreshold(srcStat.getLen(), copyTime)) { + if (logGroup.getMode() == STORE_AND_FORWARD + && isLogCopyThroughputAboveThreshold(srcStat.getLen(), copyTime)) { // start recovery by switching to SYNC_AND_FORWARD if (logGroup.checkAndSetMode(STORE_AND_FORWARD, SYNC_AND_FORWARD)) { // replication mode switched, notify the event handler @@ -178,13 +190,13 @@ protected void processNoMoreRoundsLeft() throws IOException { * Determine if the throughput is above the configured threshold. If it is, then we can switch * to the SYNC_AND_FORWARD mode * - * @param fileSize - * @param copyTime + * @param fileSize in bytes + * @param copyTime in ms * @return True if the throughput is good else false */ private boolean isLogCopyThroughputAboveThreshold(long fileSize, long copyTime) { - // TODO: calculate throughput and check if is above configured threshold - return true; + double actualThroughputBytesPerMs = copyTime != 0 ? fileSize/copyTime : 0; + return actualThroughputBytesPerMs >= copyThroughputThresholdBytesPerMs; } @Override 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 index 55ffea2aa21..a526817a652 100644 --- 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 @@ -44,43 +44,30 @@ */ public class ReplicationShardDirectoryManager { - /** - * The number of shards (subfolders) to maintain in the "IN" / "OUT" directory. - */ + // 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 - */ + // 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"). - */ + // 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"; - /* - * Format string for log file names. _.plog - * Example 1762470665995_localhost,54575,1762470584502.plog - */ + // Format string for log file names. _.plog + // Example 1762470665995_localhost,54575,1762470584502.plog public static final String FILE_NAME_FORMAT = "%d_%s.plog"; - /** - * Configuration key for the duration of each replication round in seconds. - */ + // 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. - */ + // 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"; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java index ae9273dfabd..00ad7302ca3 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/StoreAndForwardModeImpl.java @@ -17,10 +17,6 @@ */ package org.apache.phoenix.replication; -import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; -import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; - import java.io.IOException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -31,6 +27,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; +import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; + /** * Store and Forward mode implementation *

@@ -64,8 +64,8 @@ void onEnter() throws IOException { } private long getHAGroupStoreUpdateInterval() { - return (long)Math.ceil(logGroup.conf.getLong(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT) - * HA_GROUP_STORE_UPDATE_MULTIPLIER); + return (long) Math.ceil(logGroup.conf.getLong(ZK_SESSION_TIMEOUT, + DEFAULT_ZK_SESSION_TIMEOUT) * HA_GROUP_STORE_UPDATE_MULTIPLIER); } private void startHAGroupStoreUpdateTask() { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java index 6bb86a05c19..0cacd68ae71 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncAndForwardModeImpl.java @@ -17,15 +17,15 @@ */ package org.apache.phoenix.replication; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; - import java.io.IOException; import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; + /** * Sync and Forward mode implementation *

diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java index 5de43fe31f2..17328397a14 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/SyncModeImpl.java @@ -17,15 +17,15 @@ */ package org.apache.phoenix.replication; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; - import java.io.IOException; import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; + /** * Synchronous mode implementation *

diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java index 827751c9a11..010bb8fb861 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogForwarderSourceFactory.java @@ -1,13 +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; import java.util.concurrent.ConcurrentHashMap; - /** * Factory class for creating log forwarder metrics */ public class MetricsReplicationLogForwarderSourceFactory { + + private MetricsReplicationLogForwarderSourceFactory() { + // Utility class, no instantiation + } + /** Cache of ReplicationLogTrackerForwarderImpl instances by HA Group ID */ private static final ConcurrentHashMap TRACKER_INSTANCES = new ConcurrentHashMap<>(); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java index 030a44f964b..2c11880dc37 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/metrics/MetricsReplicationLogTrackerForwarderImpl.java @@ -7,7 +7,7 @@ * "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 + * 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, From 8eadb569fa2fc7c1ecc685f850157140d2cadb94 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Wed, 12 Nov 2025 14:26:04 -0800 Subject: [PATCH 05/13] Calling exit asynchronously on failure --- .../replication/ReplicationLogGroup.java | 115 +++++++++++------- .../replication/ReplicationLogGroupTest.java | 3 - 2 files changed, 68 insertions(+), 50 deletions(-) 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 6f03f82c512..7540f6258e1 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 @@ -17,15 +17,6 @@ */ package org.apache.phoenix.replication; -import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; -import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; -import static org.apache.phoenix.replication.ReplicationLogGroup.LogEvent.EVENT_TYPE_DATA; -import static org.apache.phoenix.replication.ReplicationLogGroup.LogEvent.EVENT_TYPE_SYNC; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.INIT; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; -import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; - import java.io.IOException; import java.io.InterruptedIOException; import java.net.URI; @@ -37,6 +28,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -70,6 +63,15 @@ import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; +import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; +import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; +import static org.apache.phoenix.replication.ReplicationLogGroup.LogEvent.EVENT_TYPE_DATA; +import static org.apache.phoenix.replication.ReplicationLogGroup.LogEvent.EVENT_TYPE_SYNC; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.INIT; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.STORE_AND_FORWARD; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC; +import static org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode.SYNC_AND_FORWARD; + /** * ReplicationLogGroup manages replication logs for a given HA Group. *

@@ -166,7 +168,7 @@ public class ReplicationLogGroup { public static final String REPLICATION_LOG_RETRY_DELAY_MS_KEY = "phoenix.replication.log.retry.delay.ms"; public static final long DEFAULT_REPLICATION_LOG_RETRY_DELAY_MS = 100L; - private static final long DEFAULT_HDFS_WRITE_RPC_TIMEOUT_MS = 30*1000; + private static final long DEFAULT_HDFS_WRITE_RPC_TIMEOUT_MS = 30 * 1000; public static final String STANDBY_DIR = "in"; public static final String FALLBACK_DIR = "out"; @@ -277,7 +279,7 @@ ReplicationMode getMode() { } } - private static final ImmutableMap> allowedTransition = + private static final ImmutableMap> VALID_TRANSITIONS = Maps.immutableEnumMap(ImmutableMap.of( INIT, EnumSet.of(SYNC, STORE_AND_FORWARD), SYNC, EnumSet.of(STORE_AND_FORWARD, SYNC_AND_FORWARD), @@ -317,6 +319,8 @@ public Record(String tableName, long commitId, Mutation mutation) { this.mutation = mutation; } } + // executor service used to do asynchronous close + protected ExecutorService disruptorExecutor; protected Disruptor disruptor; protected RingBuffer ringBuffer; protected LogEventHandler eventHandler; @@ -448,8 +452,8 @@ protected long calculateSyncTimeout() { DEFAULT_HDFS_WRITE_RPC_TIMEOUT_MS); // account for HAGroupStore update when we switch replication mode long zkTimeoutMs = conf.getLong(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT); - long totalRpcTimeout = maxAttempts*wrtiteRpcTimeout + (maxAttempts - 1)*retryDelayMs; - return 2*totalRpcTimeout + zkTimeoutMs; + long totalRpcTimeout = maxAttempts * wrtiteRpcTimeout + (maxAttempts - 1) * retryDelayMs; + return 2 * totalRpcTimeout + zkTimeoutMs; } /** @@ -468,14 +472,14 @@ protected void initializeReplicationMode() throws IOException { } else if (haGroupState.equals(HAGroupState.ACTIVE_NOT_IN_SYNC)) { setMode(STORE_AND_FORWARD); } else { - String message = String.format("HAGroup %s got an unexpected state %s while " + - "initializing mode", this, haGroupState); + String message = String.format("HAGroup %s got an unexpected state %s while " + + "initializing mode", this, haGroupState); LOG.error(message); throw new IOException(message); } } else { - String message = String.format("HAGroup %s got an empty group store record while " + - "initializing mode", this); + String message = String.format("HAGroup %s got an empty group store record while " + + "initializing mode", this); LOG.error(message); throw new IOException(message); } @@ -486,10 +490,11 @@ protected void initializeReplicationMode() throws IOException { protected void initializeDisruptor() throws IOException { int ringBufferSize = conf.getInt(REPLICATION_LOG_RINGBUFFER_SIZE_KEY, DEFAULT_REPLICATION_LOG_RINGBUFFER_SIZE); - disruptor = new Disruptor<>(LogEvent.EVENT_FACTORY, ringBufferSize, + this.disruptorExecutor = Executors.newCachedThreadPool( new ThreadFactoryBuilder() .setNameFormat("ReplicationLogGroup-" + getHAGroupName() + "-%d") - .setDaemon(true).build(), + .setDaemon(true).build()); + disruptor = new Disruptor<>(LogEvent.EVENT_FACTORY, ringBufferSize, disruptorExecutor, ProducerType.MULTI, new YieldingWaitStrategy()); eventHandler = new LogEventHandler(); eventHandler.init(); @@ -629,6 +634,7 @@ protected void closeOnError() { if (closed) { return; } + // setting closed to true prevents future producers to add events to the ring buffer closed = true; } // Directly halt the disruptor. shutdown() would wait for events to drain. We are expecting @@ -651,6 +657,7 @@ public void close() { if (closed) { return; } + // setting closed to true prevents future producers to add events to the ring buffer closed = true; // Remove from instances cache INSTANCES.remove(haGroupName); @@ -658,19 +665,33 @@ public void close() { try { syncInternal(); gracefulShutdownEventHandlerFlag.set(true); - disruptor.shutdown(); // Wait for a clean shutdown. + // waits until all the events in the disruptor have been processed + disruptor.shutdown(); } catch (IOException e) { LOG.warn("Error during final sync on close", e); gracefulShutdownEventHandlerFlag.set(false); disruptor.halt(); // Go directly to halt. } - // TODO revisit close logic and the below comment - // We must wait for the disruptor before closing the writers. + // wait for the disruptor threads to finish + shutdownDisruptorExecutor(); metrics.close(); LOG.info("HAGroup {} closed", this); } } + private void shutdownDisruptorExecutor() { + disruptorExecutor.shutdown(); + try { + if (!disruptorExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + LOG.warn("HAGroup {} shutdown of disruptor executor service timed out ", this); + disruptorExecutor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + disruptorExecutor.shutdownNow(); + } + LOG.info("HAGroup {} shutdown disruptor executor service", this); + } /** * Switch the replication mode to the new mode * @@ -678,7 +699,7 @@ public void close() { * @return previous replication mode */ protected ReplicationMode setMode(ReplicationMode newReplicationMode) { - ReplicationMode previous = mode.getAndUpdate( current -> newReplicationMode); + ReplicationMode previous = mode.getAndUpdate(current -> newReplicationMode); if (previous != newReplicationMode) { LOG.info("HAGroup {} switched from {} to {}", this, previous, newReplicationMode); } @@ -699,8 +720,8 @@ protected boolean checkAndSetMode(ReplicationMode expectedReplicationMode, LOG.info("HAGroup {} conditionally switched from {} to {}", this, expectedReplicationMode, newReplicationMode); } else { - LOG.info("HAGroup {} ignoring attempt to switch replication mode to {} " + - "because expected={} != actual={}", this, newReplicationMode, + LOG.info("HAGroup {} ignoring attempt to switch replication mode to {} " + + "because expected={} != actual={}", this, newReplicationMode, expectedReplicationMode, getMode()); } return updated; @@ -831,8 +852,7 @@ protected ReplicationLog getActiveLog() { protected void setHAGroupStatusToStoreAndForward() throws Exception { try { haGroupStoreManager.setHAGroupStatusToStoreAndForward(haGroupName); - } - catch (Exception ex) { + } catch (Exception ex) { LOG.info("HAGroup {} failed to set status to STORE_AND_FORWARD", this, ex); throw ex; } @@ -844,8 +864,7 @@ protected void setHAGroupStatusToSync() throws IOException { } catch (IOException ex) { // TODO logging throw ex; - } - catch (Exception ex) { + } catch (Exception ex) { // TODO logging throw new IOException(ex); } @@ -906,7 +925,9 @@ private void updateModeOnFailure(IOException e) throws IOException { // send the failed event to the current mode ReplicationMode newMode = currentModeImpl.onFailure(e); setMode(newMode); - currentModeImpl.onExit(true); + // on failure call the exit asynchronously + disruptorExecutor.execute(() -> + currentModeImpl.onExit(true)); initializeMode(newMode); } @@ -918,7 +939,7 @@ private void updateModeOnFailure(IOException e) throws IOException { *

  • Syncs the current writer to ensure all data is durably written.
  • *
  • Completes all pending sync futures successfully.
  • *
  • Clears the list of pending sync futures.
  • - *
  • Clears the current batch of records since they have been successfully synced.
  • + *
  • Checks if we need to switch mode
  • * * @param sequence The sequence number of the last processed event * @throws IOException if the sync operation fails @@ -935,6 +956,22 @@ private void processPendingSyncs(long sequence) throws IOException { } pendingSyncFutures.clear(); LOG.info("Sync operation completed successfully up to sequence {}", sequence); + // after a successful sync check the mode set on the replication group + // Doing the mode check on sync points makes the implementation more robust + // since we can guarantee that all unsynced appends have been flushed to the + // replication log before we switch the replication mode + ReplicationMode newMode = getMode(); + if (newMode != currentModeImpl.getMode()) { + // some other thread switched the mode on the replication group + LOG.info("Mode switched at sequence {} from {} to {}", + sequence, currentModeImpl, newMode); + // call exit on the last mode here since we can guarantee that the lastMode + // is not processing any event like append/sync because this is the only thread + // that is consuming the events from the ring buffer and handing them off to the + // mode + currentModeImpl.onExit(true); + initializeMode(newMode); + } } /** @@ -1050,22 +1087,6 @@ public void onEvent(LogEvent event, long sequence, boolean endOfBatch) throws Ex if (endOfBatch) { processPendingSyncs(sequence); } - // after a successful sync check the mode set on the replication group - // Doing the mode check on sync points makes the implementation more robust - // since we can guarantee that all unsynced appends have been flushed to the - // replication log before we switch the replication mode - ReplicationMode newMode = getMode(); - if (newMode != currentModeImpl.getMode()) { - // some other thread switched the mode on the replication group - LOG.info("Mode switched at sequence {} from {} to {}", - sequence, currentModeImpl, newMode); - // call exit on the last mode here since we can guarantee that the lastMode - // is not processing any event like append/sync because this is the only thread - // that is consuming the events from the ring buffer and handing them off to the - // mode - currentModeImpl.onExit(true); - initializeMode(newMode); - } return; default: throw new UnsupportedOperationException("Unknown event type: " diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java index 455ccd20574..76568b9bc13 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogGroupTest.java @@ -451,9 +451,6 @@ public void testClose() throws Exception { // Close the log writer logGroup.close(); - // wait for the event handler thread to exit since that calls the close - // on the underlying writer - Thread.sleep(5); // Verify the inner writer was closed verify(innerWriter, times(1)).close(); From d0e30d581b23690da457ade67017d664eca65e9e Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Wed, 12 Nov 2025 23:09:07 -0800 Subject: [PATCH 06/13] Handle the case when a RS doesn't receive any file to process --- .../replication/ReplicationLogDiscoveryForwarder.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java index 6fee9f488ba..176f0929912 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java @@ -181,6 +181,16 @@ protected void processNoMoreRoundsLeft() throws IOException { .getReplicationShardDirectoryManager() .getNextRound(getLastRoundProcessed())).isEmpty()) { LOG.info("Processed all the replication log files for {}", logGroup); + // if this RS is still in STORE_AND_FORWARD mode like when it didn't process any file + // move this RS to SYNC_AND_FORWARD + if (logGroup.checkAndSetMode(STORE_AND_FORWARD, SYNC_AND_FORWARD)) { + // replication mode switched, notify the event handler + try { + logGroup.sync(); + } catch (IOException e) { + LOG.info("Failed to send sync event to {}", logGroup); + } + } // TODO ensure the mTime on the group store record is older than the wait sync timeout logGroup.setHAGroupStatusToSync(); } From b941f01cf3100c51879c11fac66fd1b25f345345 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Wed, 19 Nov 2025 13:24:57 +0530 Subject: [PATCH 07/13] Handle waitTime returned by the setHAGroupStatusToSync API --- .../ReplicationLogDiscoveryForwarder.java | 76 ++++++++++--------- .../replication/ReplicationLogGroup.java | 11 +-- .../ReplicationLogDiscoveryForwarderTest.java | 33 +++++++- 3 files changed, 77 insertions(+), 43 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java index 176f0929912..925cfa596ea 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java @@ -27,6 +27,7 @@ import org.apache.phoenix.jdbc.HAGroupStateListener; import org.apache.phoenix.jdbc.HAGroupStoreManager; import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.replication.ReplicationLogGroup.ReplicationMode; import org.apache.phoenix.replication.metrics.MetricsReplicationLogDiscovery; import org.apache.phoenix.replication.metrics.MetricsReplicationLogForwarderSourceFactory; import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -49,10 +50,12 @@ public class ReplicationLogDiscoveryForwarder extends ReplicationLogDiscovery { public static final String REPLICATION_LOG_COPY_THROUGHPUT_BYTES_PER_MS_KEY = "phoenix.replication.log.copy.throughput.bytes.per.ms"; // TODO: come up with a better default after testing - public static final double DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS = 1; + public static final double DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS = 1.0; private final ReplicationLogGroup logGroup; - private double copyThroughputThresholdBytesPerMs; + private final double copyThroughputThresholdBytesPerMs; + // the timestamp (in future) at which we will attempt to set the HAGroup state to SYNC + private long syncUpdateTS; /** * Create a tracker for the replication logs in the fallback cluster. @@ -76,6 +79,8 @@ public ReplicationLogDiscoveryForwarder(ReplicationLogGroup logGroup) { this.copyThroughputThresholdBytesPerMs = conf.getDouble(REPLICATION_LOG_COPY_THROUGHPUT_BYTES_PER_MS_KEY, DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS); + // initialize to 0 + this.syncUpdateTS = 0; } @Override @@ -102,14 +107,7 @@ public void init() throws IOException { && HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC.equals(toState)) { LOG.info("Received ACTIVE_NOT_IN_SYNC event for {}", logGroup); // If the current mode is SYNC only then switch to SYNC_AND_FORWARD mode - if (logGroup.checkAndSetMode(SYNC, SYNC_AND_FORWARD)) { - // replication mode switched, notify the event handler - try { - logGroup.sync(); - } catch (IOException e) { - LOG.info("Failed to send sync event to {}", logGroup); - } - } + checkAndSetModeAndNotify(SYNC, SYNC_AND_FORWARD); } }; @@ -126,14 +124,7 @@ public void init() throws IOException { && HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC.equals(toState)) { LOG.info("Received ACTIVE_IN_SYNC event for {}", logGroup); // Set the current mode to SYNC - if (logGroup.checkAndSetMode(SYNC_AND_FORWARD, SYNC)) { - // replication mode switched, notify the event handler - try { - logGroup.sync(); - } catch (IOException e) { - LOG.info("Failed to send sync event to {}", logGroup); - } - } + checkAndSetModeAndNotify(SYNC_AND_FORWARD, SYNC); } }; @@ -148,7 +139,7 @@ public void init() throws IOException { protected void processFile(Path src) throws IOException { FileSystem srcFS = replicationLogTracker.getFileSystem(); FileStatus srcStat = srcFS.getFileStatus(src); - long ts = EnvironmentEdgeManager.currentTimeMillis(); + long ts = replicationLogTracker.getFileTimestamp(srcStat.getPath()); ReplicationShardDirectoryManager remoteShardManager = logGroup.getStandbyShardManager(); Path dst = remoteShardManager.getWriterPath(ts, logGroup.getServerName().getServerName()); long startTime = EnvironmentEdgeManager.currentTimeMillis(); @@ -160,14 +151,7 @@ protected void processFile(Path src) throws IOException { if (logGroup.getMode() == STORE_AND_FORWARD && isLogCopyThroughputAboveThreshold(srcStat.getLen(), copyTime)) { // start recovery by switching to SYNC_AND_FORWARD - if (logGroup.checkAndSetMode(STORE_AND_FORWARD, SYNC_AND_FORWARD)) { - // replication mode switched, notify the event handler - try { - logGroup.sync(); - } catch (IOException e) { - LOG.info("Failed to send sync event to {}", logGroup); - } - } + checkAndSetModeAndNotify(STORE_AND_FORWARD, SYNC_AND_FORWARD); } } @@ -183,16 +167,22 @@ protected void processNoMoreRoundsLeft() throws IOException { LOG.info("Processed all the replication log files for {}", logGroup); // if this RS is still in STORE_AND_FORWARD mode like when it didn't process any file // move this RS to SYNC_AND_FORWARD - if (logGroup.checkAndSetMode(STORE_AND_FORWARD, SYNC_AND_FORWARD)) { - // replication mode switched, notify the event handler + checkAndSetModeAndNotify(STORE_AND_FORWARD, SYNC_AND_FORWARD); + + if (syncUpdateTS <= EnvironmentEdgeManager.currentTimeMillis()) { try { - logGroup.sync(); - } catch (IOException e) { - LOG.info("Failed to send sync event to {}", logGroup); + long waitTime = logGroup.setHAGroupStatusToSync(); + if (waitTime != 0) { + syncUpdateTS = EnvironmentEdgeManager.currentTimeMillis() + waitTime; + LOG.info("HAGroup {} will try to update HA state to sync at {}", + logGroup, syncUpdateTS); + } else { + LOG.info("HAGroup {} updated HA state to SYNC", logGroup); + } + } catch (Exception e) { + LOG.info("Could not update status to sync for {}", logGroup, e); } } - // TODO ensure the mTime on the group store record is older than the wait sync timeout - logGroup.setHAGroupStatusToSync(); } } @@ -205,7 +195,7 @@ protected void processNoMoreRoundsLeft() throws IOException { * @return True if the throughput is good else false */ private boolean isLogCopyThroughputAboveThreshold(long fileSize, long copyTime) { - double actualThroughputBytesPerMs = copyTime != 0 ? fileSize/copyTime : 0; + double actualThroughputBytesPerMs = copyTime != 0 ? ((double) fileSize)/copyTime : 0; return actualThroughputBytesPerMs >= copyThroughputThresholdBytesPerMs; } @@ -219,4 +209,20 @@ protected MetricsReplicationLogDiscovery createMetricsSource() { protected ReplicationLogTracker getReplicationLogTracker() { return replicationLogTracker; } + + /** + * Helper API to check and set the replication mode and then notify the disruptor + */ + private boolean checkAndSetModeAndNotify(ReplicationMode expectedMode, ReplicationMode newMode) { + boolean ret = logGroup.checkAndSetMode(expectedMode, newMode); + if (ret) { + // replication mode switched, notify the event handler + try { + logGroup.sync(); + } catch (IOException e) { + LOG.info("Failed to notify event handler for {}", logGroup, e); + } + } + return ret; + } } 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 7540f6258e1..a5dd7fcc9ae 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 @@ -858,15 +858,12 @@ protected void setHAGroupStatusToStoreAndForward() throws Exception { } } - protected void setHAGroupStatusToSync() throws IOException { + protected long setHAGroupStatusToSync() throws Exception { try { - haGroupStoreManager.setHAGroupStatusToSync(haGroupName); - } catch (IOException ex) { - // TODO logging - throw ex; + return haGroupStoreManager.setHAGroupStatusToSync(haGroupName); } catch (Exception ex) { - // TODO logging - throw new IOException(ex); + LOG.info("HAGroup {} failed to set status to SYNC", this, ex); + throw ex; } } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java index e32949b2e0b..8f0100fa46c 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarderTest.java @@ -82,7 +82,7 @@ public Object answer(InvocationOnMock invocation) { } catch (IOException e) { throw new RuntimeException(e); } - return null; + return 0L; } }).when(haGroupStoreManager).setHAGroupStatusToSync(haGroupName); @@ -141,4 +141,35 @@ public Boolean call() throws Exception { executor.shutdownNow(); } } + + @Test + public void testSyncModeUpdateWaitTime() throws Exception { + final long[] waitTime = {8L}; + int roundDurationSeconds = + logGroup.getFallbackShardManager().getReplicationRoundDurationSeconds(); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) { + long ret = 0L; + if (waitTime[0] > 0) { + ret = waitTime[0]; + // reset to 0 + waitTime[0] = 0; + } else { + // explicitly set the replication mode to SYNC + logGroup.setMode(SYNC); + try { + logGroup.sync(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return ret; + } + }).when(haGroupStoreManager).setHAGroupStatusToSync(haGroupName); + Thread.sleep(roundDurationSeconds * 4 * 1000); + // we should have switched back to the SYNC mode + assertEquals(SYNC, logGroup.getMode()); + } } From 08c4aa68a4f431eb0729757cbfa9b052d9d73ea0 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Wed, 19 Nov 2025 20:17:48 +0530 Subject: [PATCH 08/13] Fix the reference used when closing the old mode asynchronously --- .../org/apache/phoenix/replication/ReplicationLogGroup.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 a5dd7fcc9ae..f6524c9c0f7 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 @@ -922,9 +922,10 @@ private void updateModeOnFailure(IOException e) throws IOException { // send the failed event to the current mode ReplicationMode newMode = currentModeImpl.onFailure(e); setMode(newMode); - // on failure call the exit asynchronously + ReplicationModeImpl oldModeImpl = currentModeImpl; + // on failure call the exit asynchronously on the old mode disruptorExecutor.execute(() -> - currentModeImpl.onExit(true)); + oldModeImpl.onExit(true)); initializeMode(newMode); } From 1d4a454560bbe2d8f2dcc09230b52a0f08d46630 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Fri, 21 Nov 2025 17:15:22 +0530 Subject: [PATCH 09/13] More checkstyle fixes and set buffer time to 0 for tests --- .../replication/ReplicationLogDiscovery.java | 2 +- .../ReplicationLogDiscoveryForwarder.java | 23 +++++++++++++++---- .../replication/ReplicationLogGroup.java | 3 ++- .../replication/ReplicationLogBaseTest.java | 2 ++ 4 files changed, 24 insertions(+), 6 deletions(-) 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 index 3d454409679..6332758bfdc 100644 --- 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 @@ -93,7 +93,7 @@ public abstract class ReplicationLogDiscovery { /** * Default buffer percentage for waiting time between processing rounds */ - private static final double DEFAULT_WAITING_BUFFER_PERCENTAGE = 15.0; + protected static final double DEFAULT_WAITING_BUFFER_PERCENTAGE = 15.0; protected final Configuration conf; protected final String haGroupName; diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java index 925cfa596ea..38dc945f6e3 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/replication/ReplicationLogDiscoveryForwarder.java @@ -52,6 +52,12 @@ public class ReplicationLogDiscoveryForwarder extends ReplicationLogDiscovery { // TODO: come up with a better default after testing public static final double DEFAULT_LOG_COPY_THROUGHPUT_BYTES_PER_MS = 1.0; + /** + * Configuration key for waiting buffer percentage + */ + public static final String REPLICATION_FORWARDER_WAITING_BUFFER_PERCENTAGE_KEY = + "phoenix.replication.forwarder.waiting.buffer.percentage"; + private final ReplicationLogGroup logGroup; private final double copyThroughputThresholdBytesPerMs; // the timestamp (in future) at which we will attempt to set the HAGroup state to SYNC @@ -130,7 +136,8 @@ public void init() throws IOException { HAGroupStoreManager haGroupStoreManager = logGroup.getHAGroupStoreManager(); haGroupStoreManager.subscribeToTargetState(logGroup.getHAGroupName(), - HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC, ClusterType.LOCAL, activeNotInSync); + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC, ClusterType.LOCAL, + activeNotInSync); haGroupStoreManager.subscribeToTargetState(logGroup.getHAGroupName(), HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, ClusterType.LOCAL, activeInSync); } @@ -147,7 +154,8 @@ protected void processFile(Path src) throws IOException { // successfully copied the file long endTime = EnvironmentEdgeManager.currentTimeMillis(); long copyTime = endTime - startTime; - LOG.info("Copying file src={} dst={} size={} took {}ms", src, dst, srcStat.getLen(), copyTime); + LOG.info("Copying file src={} dst={} size={} took {}ms", src, dst, srcStat.getLen(), + copyTime); if (logGroup.getMode() == STORE_AND_FORWARD && isLogCopyThroughputAboveThreshold(srcStat.getLen(), copyTime)) { // start recovery by switching to SYNC_AND_FORWARD @@ -195,7 +203,7 @@ protected void processNoMoreRoundsLeft() throws IOException { * @return True if the throughput is good else false */ private boolean isLogCopyThroughputAboveThreshold(long fileSize, long copyTime) { - double actualThroughputBytesPerMs = copyTime != 0 ? ((double) fileSize)/copyTime : 0; + double actualThroughputBytesPerMs = copyTime != 0 ? ((double) fileSize) / copyTime : 0; return actualThroughputBytesPerMs >= copyThroughputThresholdBytesPerMs; } @@ -213,7 +221,8 @@ protected ReplicationLogTracker getReplicationLogTracker() { /** * Helper API to check and set the replication mode and then notify the disruptor */ - private boolean checkAndSetModeAndNotify(ReplicationMode expectedMode, ReplicationMode newMode) { + private boolean checkAndSetModeAndNotify(ReplicationMode expectedMode, + ReplicationMode newMode) { boolean ret = logGroup.checkAndSetMode(expectedMode, newMode); if (ret) { // replication mode switched, notify the event handler @@ -225,4 +234,10 @@ private boolean checkAndSetModeAndNotify(ReplicationMode expectedMode, Replicati } return ret; } + + @Override + public double getWaitingBufferPercentage() { + return getConf().getDouble(REPLICATION_FORWARDER_WAITING_BUFFER_PERCENTAGE_KEY, + DEFAULT_WAITING_BUFFER_PERCENTAGE); + } } 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 f6524c9c0f7..aa0d9e903fe 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 @@ -1092,7 +1092,8 @@ public void onEvent(LogEvent event, long sequence, boolean endOfBatch) throws Ex } } catch (IOException e) { try { - LOG.info("Failed to process event at sequence {} on mode {}", sequence, currentModeImpl, e); + LOG.info("Failed to process event at sequence {} on mode {}", sequence, + currentModeImpl, e); onFailure(event, sequence, e); } catch (Exception fatalEx) { // Either we failed to switch the mode or we are in STORE_AND_FORWARD mode diff --git a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java index 9ac3bd7e6ea..6649bb220d2 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/replication/ReplicationLogBaseTest.java @@ -17,6 +17,7 @@ */ package org.apache.phoenix.replication; +import static org.apache.phoenix.replication.ReplicationLogDiscoveryForwarder.REPLICATION_FORWARDER_WAITING_BUFFER_PERCENTAGE_KEY; import static org.apache.phoenix.replication.ReplicationShardDirectoryManager.PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; @@ -108,6 +109,7 @@ public void setUpBase() throws IOException { // small value of replication round duration conf.setInt(PHOENIX_REPLICATION_ROUND_DURATION_SECONDS_KEY, TEST_REPLICATION_ROUND_DURATION_SECONDS); + conf.setDouble(REPLICATION_FORWARDER_WAITING_BUFFER_PERCENTAGE_KEY, 0.0); // initialize the group store record storeRecord = initHAGroupStoreRecord(); From ff4f47d7fb3e687f0329ed98b23e1f028a19428a Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Mon, 24 Nov 2025 22:42:31 +0530 Subject: [PATCH 10/13] Fix IndexRegionObserverMutationBlockingIT --- .../hbase/index/IndexRegionObserver.java | 55 +++++++++---------- ...IndexRegionObserverMutationBlockingIT.java | 30 +++++++++- 2 files changed, 54 insertions(+), 31 deletions(-) diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java index 8eef8d746da..4b625a5a342 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java @@ -628,38 +628,35 @@ public void preBatchMutate(ObserverContext c, throw new IOException("HAGroupStoreManager is null " + "for current cluster, check configuration"); } - Optional logGroup = Optional.empty(); - if (shouldReplicate) { - // Extract HAGroupName from the mutations - logGroup = getHAGroupFromBatch(c.getEnvironment(), miniBatchOp); - } + // Extract HAGroupName from the mutations + Optional logGroup = + getHAGroupFromBatch(c.getEnvironment(), miniBatchOp); + // We don't want to check for mutation blocking for the system ha group table - if (!dataTableName.equals(SYSTEM_HA_GROUP_NAME)) { + if (!dataTableName.equals(SYSTEM_HA_GROUP_NAME) && logGroup.isPresent()) { // Check if mutation is blocked for the HA Group - if (logGroup.isPresent()) { - String haGroupName = logGroup.get().getHAGroupName(); - //TODO: Below approach might be slow need to figure out faster way, - // slower part is getting haGroupStoreClient We can also cache - // roleRecord (I tried it and still it's slow due to haGroupStoreClient - // initialization) and caching will give us old result in case one cluster - // is unreachable instead of UNKNOWN. - - boolean isHAGroupOnClientStale = haGroupStoreManager - .isHAGroupOnClientStale(haGroupName); - if (StringUtils.isNotBlank(haGroupName) && isHAGroupOnClientStale) { - throw new StaleClusterRoleRecordException( - String.format("HAGroupStoreRecord is stale for haGroup %s on " - + "client", haGroupName)); - } + String haGroupName = logGroup.get().getHAGroupName(); + //TODO: Below approach might be slow need to figure out faster way, + // slower part is getting haGroupStoreClient We can also cache + // roleRecord (I tried it and still it's slow due to haGroupStoreClient + // initialization) and caching will give us old result in case one cluster + // is unreachable instead of UNKNOWN. + + boolean isHAGroupOnClientStale = haGroupStoreManager + .isHAGroupOnClientStale(haGroupName); + if (StringUtils.isNotBlank(haGroupName) && isHAGroupOnClientStale) { + throw new StaleClusterRoleRecordException( + String.format("HAGroupStoreRecord is stale for haGroup %s on " + + "client", haGroupName)); + } - //Check if mutation's haGroup is stale - if (StringUtils.isNotBlank(haGroupName) - && haGroupStoreManager.isMutationBlocked(haGroupName)) { - throw new MutationBlockedIOException("Blocking Mutation as Some CRRs " - + "are in ACTIVE_TO_STANDBY state and " - + "CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED is true"); - } - } + //Check if mutation's haGroup is stale + if (StringUtils.isNotBlank(haGroupName) + && haGroupStoreManager.isMutationBlocked(haGroupName)) { + throw new MutationBlockedIOException("Blocking Mutation as Some CRRs " + + "are in ACTIVE_TO_STANDBY state and " + + "CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED is true"); + } } preBatchMutateWithExceptions(c, miniBatchOp, logGroup); return; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java index e015aa4cf95..47ecc9f4dee 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java @@ -21,15 +21,19 @@ import static org.apache.phoenix.jdbc.HighAvailabilityGroup.PHOENIX_HA_GROUP_ATTR; import static org.apache.phoenix.query.BaseTest.generateUniqueName; import static org.apache.phoenix.query.QueryServices.CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED; +import static org.apache.phoenix.query.QueryServices.SYNCHRONOUS_REPLICATION_ENABLED; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.net.URI; import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.Statement; import java.util.Properties; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.exception.MutationBlockedIOException; @@ -40,11 +44,15 @@ import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility; import org.apache.phoenix.jdbc.PhoenixDriver; import org.apache.phoenix.jdbc.PhoenixHAAdmin; +import org.apache.phoenix.replication.ReplicationLogBaseTest; +import org.apache.phoenix.replication.ReplicationLogGroup; 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; /** @@ -67,10 +75,28 @@ public class IndexRegionObserverMutationBlockingIT { private Properties clientProps = new Properties(); private String haGroupName; + @ClassRule + public static TemporaryFolder standbyFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder localFolder = new TemporaryFolder(); + + private static Configuration conf1; + private static Configuration conf2; + private static URI standbyUri; + private static URI fallbackUri; + @BeforeClass public static synchronized void doSetup() throws Exception { - CLUSTERS.getHBaseCluster1().getConfiguration().setBoolean(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, true); - CLUSTERS.getHBaseCluster2().getConfiguration().setBoolean(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, true); + conf1 = CLUSTERS.getHBaseCluster1().getConfiguration(); + conf2 = CLUSTERS.getHBaseCluster2().getConfiguration(); + standbyUri = new Path(standbyFolder.getRoot().toString()).toUri(); + fallbackUri = new Path(localFolder.getRoot().toString()).toUri(); + conf1.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + conf2.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + conf1.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf1.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); + conf1.setBoolean(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, true); + conf2.setBoolean(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, true); CLUSTERS.start(); DriverManager.registerDriver(PhoenixDriver.INSTANCE); } From 7990ba698ff4e3723723553a2b24cbf2455c8359 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Tue, 25 Nov 2025 15:41:45 +0530 Subject: [PATCH 11/13] Add a common base class for IT tests --- ...IndexRegionObserverMutationBlockingIT.java | 30 +--------- .../jdbc/FailoverPhoenixConnection2IT.java | 3 +- .../jdbc/FailoverPhoenixConnectionIT.java | 3 +- .../org/apache/phoenix/jdbc/HABaseIT.java | 58 +++++++++++++++++++ 4 files changed, 62 insertions(+), 32 deletions(-) create mode 100644 phoenix-core/src/it/java/org/apache/phoenix/jdbc/HABaseIT.java diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java index 47ecc9f4dee..7a00cbdc850 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexRegionObserverMutationBlockingIT.java @@ -21,38 +21,31 @@ import static org.apache.phoenix.jdbc.HighAvailabilityGroup.PHOENIX_HA_GROUP_ATTR; import static org.apache.phoenix.query.BaseTest.generateUniqueName; import static org.apache.phoenix.query.QueryServices.CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED; -import static org.apache.phoenix.query.QueryServices.SYNCHRONOUS_REPLICATION_ENABLED; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.net.URI; import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.Statement; import java.util.Properties; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.exception.MutationBlockedIOException; import org.apache.phoenix.execute.CommitException; import org.apache.phoenix.jdbc.FailoverPhoenixConnection; +import org.apache.phoenix.jdbc.HABaseIT; import org.apache.phoenix.jdbc.HAGroupStoreRecord; import org.apache.phoenix.jdbc.HighAvailabilityPolicy; import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility; import org.apache.phoenix.jdbc.PhoenixDriver; import org.apache.phoenix.jdbc.PhoenixHAAdmin; -import org.apache.phoenix.replication.ReplicationLogBaseTest; -import org.apache.phoenix.replication.ReplicationLogGroup; 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; /** @@ -61,11 +54,10 @@ * blocking is enabled and CRRs are in ACTIVE_TO_STANDBY state. */ @Category(NeedsOwnMiniClusterTest.class) -public class IndexRegionObserverMutationBlockingIT { +public class IndexRegionObserverMutationBlockingIT extends HABaseIT { private static final Long ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS = 1000L; private PhoenixHAAdmin haAdmin; - private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); private String zkUrl; private String peerZkUrl; @@ -75,26 +67,8 @@ public class IndexRegionObserverMutationBlockingIT { private Properties clientProps = new Properties(); private String haGroupName; - @ClassRule - public static TemporaryFolder standbyFolder = new TemporaryFolder(); - @ClassRule - public static TemporaryFolder localFolder = new TemporaryFolder(); - - private static Configuration conf1; - private static Configuration conf2; - private static URI standbyUri; - private static URI fallbackUri; - @BeforeClass public static synchronized void doSetup() throws Exception { - conf1 = CLUSTERS.getHBaseCluster1().getConfiguration(); - conf2 = CLUSTERS.getHBaseCluster2().getConfiguration(); - standbyUri = new Path(standbyFolder.getRoot().toString()).toUri(); - fallbackUri = new Path(localFolder.getRoot().toString()).toUri(); - conf1.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); - conf2.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); - conf1.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); - conf1.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); conf1.setBoolean(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, true); conf2.setBoolean(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, true); CLUSTERS.start(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection2IT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection2IT.java index 69ccb7bf783..161dd6e8da8 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection2IT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection2IT.java @@ -71,9 +71,8 @@ * Test failover basics for {@link FailoverPhoenixConnection}. */ @Category(NeedsOwnMiniClusterTest.class) -public class FailoverPhoenixConnection2IT { +public class FailoverPhoenixConnection2IT extends HABaseIT { private static final Logger LOG = LoggerFactory.getLogger(FailoverPhoenixConnectionIT.class); - private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); @Rule public final TestName testName = new TestName(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnectionIT.java index ddfebbef265..999433461b1 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnectionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/FailoverPhoenixConnectionIT.java @@ -90,9 +90,8 @@ * Test failover basics for {@link FailoverPhoenixConnection}. */ @Category(NeedsOwnMiniClusterTest.class) -public class FailoverPhoenixConnectionIT { +public class FailoverPhoenixConnectionIT extends HABaseIT { private static final Logger LOG = LoggerFactory.getLogger(FailoverPhoenixConnectionIT.class); - private static final HBaseTestingUtilityPair CLUSTERS = new HBaseTestingUtilityPair(); @Rule public final TestName testName = new TestName(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HABaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HABaseIT.java new file mode 100644 index 00000000000..f2da4feed5e --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HABaseIT.java @@ -0,0 +1,58 @@ +/* + * 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.jdbc; + +import static org.apache.phoenix.query.QueryServices.SYNCHRONOUS_REPLICATION_ENABLED; + +import java.net.URI; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.phoenix.replication.ReplicationLogGroup; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; + +public class HABaseIT { + @ClassRule + public static TemporaryFolder standbyFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder localFolder = new TemporaryFolder(); + + protected static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + + protected static Configuration conf1; + protected static Configuration conf2; + protected static URI standbyUri; + protected static URI fallbackUri; + + @BeforeClass + public static synchronized void doBaseSetup() { + conf1 = CLUSTERS.getHBaseCluster1().getConfiguration(); + conf2 = CLUSTERS.getHBaseCluster2().getConfiguration(); + standbyUri = new Path(standbyFolder.getRoot().toString()).toUri(); + fallbackUri = new Path(localFolder.getRoot().toString()).toUri(); + conf1.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + conf2.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + conf1.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf1.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); + conf2.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf2.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); + } +} From 108e5c794e0fbe3783a444854619e8dc32dec8e0 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Wed, 26 Nov 2025 14:04:18 +0530 Subject: [PATCH 12/13] Fix ParallelPhoenixConnectionIT, ParallelPhoenixConnectionWorkflowIT --- .../apache/phoenix/replication/ReplicationLogGroup.java | 8 ++++++-- .../apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java | 3 +-- .../phoenix/jdbc/ParallelPhoenixConnectionWorkflowIT.java | 3 +-- 3 files changed, 8 insertions(+), 6 deletions(-) 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 aa0d9e903fe..77a52f700bb 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 @@ -467,16 +467,20 @@ protected void initializeReplicationMode() throws IOException { if (haGroupStoreRecord.isPresent()) { HAGroupStoreRecord record = haGroupStoreRecord.get(); HAGroupState haGroupState = record.getHAGroupState(); + LOG.info("HAGroup {} initializing mode from state {}", this, haGroupState); if (haGroupState.equals(HAGroupState.ACTIVE_IN_SYNC)) { setMode(SYNC); - } else if (haGroupState.equals(HAGroupState.ACTIVE_NOT_IN_SYNC)) { + } else { + setMode(STORE_AND_FORWARD); + } + /*else if (haGroupState.equals(HAGroupState.ACTIVE_NOT_IN_SYNC)) { setMode(STORE_AND_FORWARD); } else { String message = String.format("HAGroup %s got an unexpected state %s while " + "initializing mode", this, haGroupState); LOG.error(message); throw new IOException(message); - } + }*/ } else { String message = String.format("HAGroup %s got an empty group store record while " + "initializing mode", this); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java index 5d980b5abe2..9749b8df7bc 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java @@ -105,9 +105,8 @@ * Test failover basics for {@link ParallelPhoenixConnection}. */ @Category(NeedsOwnMiniClusterTest.class) -public class ParallelPhoenixConnectionIT { +public class ParallelPhoenixConnectionIT extends HABaseIT { private static final Logger LOG = LoggerFactory.getLogger(ParallelPhoenixConnectionIT.class); - private static final HBaseTestingUtilityPair CLUSTERS = new HBaseTestingUtilityPair(); private static final Properties GLOBAL_PROPERTIES = new Properties(); @Rule diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionWorkflowIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionWorkflowIT.java index ba8ba8d71d3..a5ce20f2ddf 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionWorkflowIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionWorkflowIT.java @@ -62,9 +62,8 @@ */ @Category(NeedsOwnMiniClusterTest.class) @RunWith(Parameterized.class) -public class ParallelPhoenixConnectionWorkflowIT { +public class ParallelPhoenixConnectionWorkflowIT extends HABaseIT { private static final Logger LOG = LoggerFactory.getLogger(ParallelPhoenixConnectionIT.class); - private static final HBaseTestingUtilityPair CLUSTERS = new HBaseTestingUtilityPair(); private static final Properties GLOBAL_PROPERTIES = new Properties(); private static final String tableName = generateUniqueName(); private static final String USER_CONDITION = "USER_ID=? and USER_TYPE=? and WORK_ID=?"; From 9a94380b6b240ad42e02220688b9d644ab44ae37 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Thu, 27 Nov 2025 09:38:43 +0530 Subject: [PATCH 13/13] More IT fixes --- .../jdbc/HighAvailabilityGroup2IT.java | 3 +-- .../phoenix/jdbc/HighAvailabilityGroupIT.java | 3 +-- .../jdbc/LoggingHAConnectionLimiterIT.java | 22 +++++++++++++++++++ .../ParallelPhoenixConnectionFallbackIT.java | 3 +-- 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroup2IT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroup2IT.java index 6086d38d2de..09f54ec489e 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroup2IT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroup2IT.java @@ -51,9 +51,8 @@ import static org.junit.Assert.assertTrue; @Category(NeedsOwnMiniClusterTest.class) -public class HighAvailabilityGroup2IT { +public class HighAvailabilityGroup2IT extends HABaseIT { private static final Logger LOG = LoggerFactory.getLogger(HighAvailabilityGroup2IT.class); - private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); /** Client properties to create a connection per test. */ private Properties clientProperties; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupIT.java index eb843f41908..259e85ba304 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupIT.java @@ -65,9 +65,8 @@ */ @SuppressWarnings("UnstableApiUsage") @Category(NeedsOwnMiniClusterTest.class) -public class HighAvailabilityGroupIT { +public class HighAvailabilityGroupIT extends HABaseIT { private static final Logger LOG = LoggerFactory.getLogger(HighAvailabilityGroupIT.class); - private static final HBaseTestingUtilityPair CLUSTERS = new HBaseTestingUtilityPair(); /** Client properties to create a connection per test. */ private Properties clientProperties; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingHAConnectionLimiterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingHAConnectionLimiterIT.java index 66fd160e066..02986af0602 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingHAConnectionLimiterIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingHAConnectionLimiterIT.java @@ -17,18 +17,24 @@ */ package org.apache.phoenix.jdbc; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.log.ConnectionLimiter; import org.apache.phoenix.query.ConnectionQueryServices; import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.replication.ReplicationLogGroup; import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.URI; import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; @@ -42,6 +48,7 @@ import static org.apache.phoenix.jdbc.HighAvailabilityPolicy.PARALLEL; import static org.apache.phoenix.query.QueryServices.CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED; import static org.apache.phoenix.query.QueryServices.HA_GROUP_STALE_FOR_MUTATION_CHECK_ENABLED; +import static org.apache.phoenix.query.QueryServices.SYNCHRONOUS_REPLICATION_ENABLED; @Category(NeedsOwnMiniClusterTest.class) public class LoggingHAConnectionLimiterIT extends LoggingConnectionLimiterIT { @@ -64,6 +71,11 @@ public class LoggingHAConnectionLimiterIT extends LoggingConnectionLimiterIT { */ private HighAvailabilityGroup haGroup; + @ClassRule + public static TemporaryFolder standbyFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder localFolder = new TemporaryFolder(); + @BeforeClass public static final void doSetup() throws Exception { /** @@ -83,6 +95,16 @@ public static final void doSetup() throws Exception { }}; + Configuration conf1 = CLUSTERS.getHBaseCluster1().getConfiguration(); + Configuration conf2 = CLUSTERS.getHBaseCluster2().getConfiguration(); + URI standbyUri = new Path(standbyFolder.getRoot().toString()).toUri(); + URI fallbackUri = new Path(localFolder.getRoot().toString()).toUri(); + conf1.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + conf2.setBoolean(SYNCHRONOUS_REPLICATION_ENABLED, true); + conf1.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf1.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); + conf2.set(ReplicationLogGroup.REPLICATION_STANDBY_HDFS_URL_KEY, standbyUri.toString()); + conf2.set(ReplicationLogGroup.REPLICATION_FALLBACK_HDFS_URL_KEY, fallbackUri.toString()); CLUSTERS.getHBaseCluster1().getConfiguration().setBoolean(HA_GROUP_STALE_FOR_MUTATION_CHECK_ENABLED, false); CLUSTERS.getHBaseCluster2().getConfiguration().setBoolean(HA_GROUP_STALE_FOR_MUTATION_CHECK_ENABLED, false); CLUSTERS.start(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFallbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFallbackIT.java index 42513b1dce4..aeb159e244f 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFallbackIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFallbackIT.java @@ -43,11 +43,10 @@ import org.slf4j.LoggerFactory; @Category(NeedsOwnMiniClusterTest.class) -public class ParallelPhoenixConnectionFallbackIT { +public class ParallelPhoenixConnectionFallbackIT extends HABaseIT { private static final Logger LOG = LoggerFactory.getLogger(ParallelPhoenixConnectionFallbackIT.class); - private static final HBaseTestingUtilityPair CLUSTERS = new HBaseTestingUtilityPair(); private static final Properties PROPERTIES = new Properties(); private static String jdbcUrl;