diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java index f24f51ddc4d..919fc040bdb 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/BaseScannerRegionObserverConstants.java @@ -199,4 +199,9 @@ public static long getMaxLookbackInMillis(Configuration conf) { /** Exposed for testing */ public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server"; + + /** + * Attribute name used to pass HAGroupName in Mutations. + */ + public static final String HA_GROUP_NAME_ATTRIB = "_HAGroupName"; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/exception/InvalidClusterRoleTransitionException.java b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/InvalidClusterRoleTransitionException.java new file mode 100644 index 00000000000..86de67c4572 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/InvalidClusterRoleTransitionException.java @@ -0,0 +1,41 @@ +/* + * 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.exception; + +/** + * Exception thrown when attempting an invalid cluster role transition. + */ +public class InvalidClusterRoleTransitionException extends Exception { + private static final long serialVersionUID = 1L; + + /** + * @param msg reason for the exception + */ + public InvalidClusterRoleTransitionException(String msg) { + super(msg); + } + + /** + * @param msg reason for the exception + * @param cause the underlying cause + */ + public InvalidClusterRoleTransitionException(String msg, Throwable cause) { + super(msg, cause); + } + +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/exception/MutationBlockedIOException.java b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/MutationBlockedIOException.java index 4a28be6e39c..c1030680441 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/exception/MutationBlockedIOException.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/MutationBlockedIOException.java @@ -32,5 +32,4 @@ public class MutationBlockedIOException extends IOException { public MutationBlockedIOException(String msg) { super(msg); } - } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/exception/StaleHAGroupStoreRecordVersionException.java b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/StaleHAGroupStoreRecordVersionException.java new file mode 100644 index 00000000000..857ed0a98f3 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/StaleHAGroupStoreRecordVersionException.java @@ -0,0 +1,42 @@ +/* + * 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.exception; + +/** + * Exception thrown when attempting to update HAGroupStoreRecord with a stale ZK stat version, + * indicating that the record has been modified by another process. + */ +public class StaleHAGroupStoreRecordVersionException extends Exception { + private static final long serialVersionUID = 1L; + + /** + * @param msg reason for the exception + */ + public StaleHAGroupStoreRecordVersionException(String msg) { + super(msg); + } + + /** + * @param msg reason for the exception + * @param cause the underlying cause + */ + public StaleHAGroupStoreRecordVersionException(String msg, Throwable cause) { + super(msg, cause); + } + +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java index fb82692be53..7e84129e284 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java @@ -55,6 +55,7 @@ import java.util.Set; import javax.annotation.Nonnull; import javax.annotation.concurrent.Immutable; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; @@ -433,7 +434,7 @@ public boolean isTransactionStarted() { public long getInitialWritePointer() { return phoenixTransactionContext.getTransactionId(); // First write pointer - won't change with - // checkpointing + // checkpointing } // For testing @@ -905,8 +906,11 @@ private void annotateMutationWithMetadata(PTable table, Mutation mutation) { table.getExternalSchemaId() != null ? Bytes.toBytes(table.getExternalSchemaId()) : null; byte[] lastDDLTimestamp = table.getLastDDLTimestamp() != null ? Bytes.toBytes(table.getLastDDLTimestamp()) : null; + byte[] haGroupName = StringUtils.isNotBlank(connection.getHAGroupName()) + ? Bytes.toBytes(connection.getHAGroupName()) + : null; WALAnnotationUtil.annotateMutation(mutation, tenantId, schemaName, tableName, tableType, - lastDDLTimestamp); + lastDDLTimestamp, haGroupName); } /** @@ -1867,7 +1871,7 @@ static MutationMetric getCommittedMutationsMetric(MutationBytes totalMutationByt return new MutationMetric(numMutations, committedUpsertMutationBytes, committedDeleteMutationBytes, upsertMutationCommitTime, atomicUpsertMutationCommitTime, deleteMutationCommitTime, 0, // num failed mutations have been counted already in - // updateMutationBatchFailureMetrics() + // updateMutationBatchFailureMetrics() committedUpsertMutationCounter, committedDeleteMutationCounter, committedTotalMutationBytes, numFailedPhase3Mutations, 0, 0, 0, 0, mutationBatchCounter, mutationQueryParsingTimeMS); } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/ClusterRoleRecord.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/ClusterRoleRecord.java index 65a323e4346..e9a94db0464 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/ClusterRoleRecord.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/ClusterRoleRecord.java @@ -59,11 +59,13 @@ public enum ClusterRole { STANDBY, OFFLINE, UNKNOWN, - ACTIVE_TO_STANDBY; + ACTIVE_TO_STANDBY, + STANDBY_TO_ACTIVE; /** Returns true if a cluster with this role can be connected, otherwise false */ public boolean canConnect() { - return this == ACTIVE || this == STANDBY || this == ACTIVE_TO_STANDBY; + return this == ACTIVE || this == STANDBY || this == ACTIVE_TO_STANDBY + || this == STANDBY_TO_ACTIVE; } public static ClusterRole from(byte[] bytes) { @@ -71,6 +73,30 @@ public static ClusterRole from(byte[] bytes) { return Arrays.stream(ClusterRole.values()).filter(r -> r.name().equalsIgnoreCase(value)) .findFirst().orElse(UNKNOWN); } + + public boolean isMutationBlocked() { + return this == ACTIVE_TO_STANDBY; + } + + /** Returns the default HAGroupState for this ClusterRole */ + @JsonIgnore + public HAGroupStoreRecord.HAGroupState getDefaultHAGroupState() { + switch (this) { + case ACTIVE: + return HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC; + case STANDBY: + return HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY; + case OFFLINE: + return HAGroupStoreRecord.HAGroupState.OFFLINE; + case ACTIVE_TO_STANDBY: + return HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY; + case STANDBY_TO_ACTIVE: + return HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE; + case UNKNOWN: + default: + return HAGroupStoreRecord.HAGroupState.UNKNOWN; + } + } } /** diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreClient.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreClient.java index 93102d79ac4..64d76b9e6d7 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreClient.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreClient.java @@ -17,63 +17,171 @@ */ package org.apache.phoenix.jdbc; +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.jdbc.PhoenixDatabaseMetaData.CLUSTER_ROLE_1; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLUSTER_ROLE_2; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLUSTER_URL_1; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLUSTER_URL_2; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.HA_GROUP_NAME; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.POLICY; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_HA_GROUP_NAME; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VERSION; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ZK_URL_1; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ZK_URL_2; import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.toPath; +import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR; +import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_ZK; import java.io.Closeable; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; import org.apache.hadoop.conf.Configuration; +import org.apache.phoenix.exception.InvalidClusterRoleTransitionException; +import org.apache.phoenix.exception.StaleHAGroupStoreRecordVersionException; +import org.apache.phoenix.jdbc.ClusterRoleRecord.ClusterRole; +import org.apache.phoenix.jdbc.ClusterRoleRecord.RegistryType; +import org.apache.phoenix.jdbc.HAGroupStoreRecord.HAGroupState; +import org.apache.phoenix.util.JDBCUtil; +import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions; /** - * Write-through cache for HAGroupStore. Uses {@link PathChildrenCache} from + * Main implementation of HAGroupStoreClient with peer support. Write-through cache for HAGroupStore + * based on {@link HAGroupStoreRecord}. Uses {@link PathChildrenCache} from * {@link org.apache.curator.framework.CuratorFramework}. */ public class HAGroupStoreClient implements Closeable { + public static final String ZK_CONSISTENT_HA_NAMESPACE = + "phoenix" + ZKPaths.PATH_SEPARATOR + "consistentHA"; private static final long HA_GROUP_STORE_CLIENT_INITIALIZATION_TIMEOUT_MS = 30000L; + // Multiplier for ZK session timeout to account for time it will take for HMaster to abort + // the region server in case ZK connection is lost from the region server. + private static final double ZK_SESSION_TIMEOUT_MULTIPLIER = 1.1; + private static final String CACHE_TYPE_LOCAL = "LOCAL"; + private static final String CACHE_TYPE_PEER = "PEER"; private PhoenixHAAdmin phoenixHaAdmin; + private PhoenixHAAdmin peerPhoenixHaAdmin; private static final Logger LOGGER = LoggerFactory.getLogger(HAGroupStoreClient.class); - // Map contains > - private final ConcurrentHashMap> clusterRoleToCRRMap = new ConcurrentHashMap<>(); - private PathChildrenCache pathChildrenCache; - private volatile boolean isHealthy; - private static final Map instances = new ConcurrentHashMap<>(); + // Map of > + private static final Map> instances = + new ConcurrentHashMap<>(); + // HAGroupName for this instance + private final String haGroupName; + // PathChildrenCache for current cluster and HAGroupName + private PathChildrenCache pathChildrenCache = null; + // PathChildrenCache for peer cluster and HAGroupName + private PathChildrenCache peerPathChildrenCache = null; + // Whether the client is healthy + private volatile boolean isHealthy = false; + // Configuration + private final Configuration conf; + // ZK URL for the current cluster and HAGroupName + private String zkUrl; + // Peer ZK URL for peer cluster and HAGroupName + private String peerZKUrl = null; + // Peer Custom Event Listener + private final PathChildrenCacheListener peerCustomPathChildrenCacheListener; + // Wait time for sync mode + private final long waitTimeForSyncModeInMs; + // Policy for the HA group + private HighAvailabilityPolicy policy; + private ClusterRole clusterRole; + private ClusterRole peerClusterRole; + private String clusterUrl; + private String peerClusterUrl; + private long clusterRoleRecordVersion; + + public static HAGroupStoreClient getInstance(Configuration conf, String haGroupName) + throws SQLException { + return getInstanceForZkUrl(conf, haGroupName, null); + } /** * Creates/gets an instance of HAGroupStoreClient. Can return null instance if unable to * initialize. - * @param conf configuration + * @param conf configuration + * @param haGroupName name of the HA group. Only specified HA group is tracked. + * @param zkUrl zkUrl to use for the client. Prefer providing this parameter to avoid the + * overhead of getting the local zkUrl from the configuration. * @return HAGroupStoreClient instance */ - public static HAGroupStoreClient getInstance(Configuration conf) { - final String zkUrl = getLocalZkUrl(conf); - HAGroupStoreClient result = instances.get(zkUrl); + public static HAGroupStoreClient getInstanceForZkUrl(Configuration conf, String haGroupName, + String zkUrl) throws SQLException { + Preconditions.checkNotNull(haGroupName, "haGroupName cannot be null"); + String localZkUrl = Objects.toString(zkUrl, getLocalZkUrl(conf)); + Preconditions.checkNotNull(localZkUrl, "zkUrl cannot be null"); + HAGroupStoreClient result = + instances.getOrDefault(localZkUrl, new ConcurrentHashMap<>()).getOrDefault(haGroupName, null); if (result == null || !result.isHealthy) { synchronized (HAGroupStoreClient.class) { - result = instances.get(zkUrl); + result = instances.getOrDefault(localZkUrl, new ConcurrentHashMap<>()) + .getOrDefault(haGroupName, null); if (result == null || !result.isHealthy) { - result = new HAGroupStoreClient(conf, null); + result = new HAGroupStoreClient(conf, null, null, haGroupName, zkUrl); if (!result.isHealthy) { result.close(); result = null; + } else { + instances.putIfAbsent(localZkUrl, new ConcurrentHashMap<>()); + instances.get(localZkUrl).put(haGroupName, result); } - instances.put(zkUrl, result); + } + } + } + return result; + } + + /** + * Get the list of HAGroupNames from system table. We can also get the list of HAGroupNames from + * the system table by providing the zkUrl in where clause but we need to match the formatted + * zkUrl with the zkUrl in the system table so that matching is done correctly. + * @param zkUrl for connecting to Table + * @return the list of HAGroupNames + * @throws SQLException in case of unexpected error + */ + public static List getHAGroupNames(String zkUrl) throws SQLException { + List result = new ArrayList<>(); + String queryString = String.format("SELECT %s,%s,%s FROM %s", HA_GROUP_NAME, ZK_URL_1, ZK_URL_2, + SYSTEM_HA_GROUP_NAME); + try ( + PhoenixConnection conn = (PhoenixConnection) DriverManager + .getConnection(JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + zkUrl); + Statement stmt = conn.createStatement(); ResultSet rs = stmt.executeQuery(queryString)) { + while (rs.next()) { + String zkUrl1 = rs.getString(ZK_URL_1); + String zkUrl2 = rs.getString(ZK_URL_2); + String formattedZkUrl1 = JDBCUtil.formatUrl(zkUrl1, RegistryType.ZK); + String formattedZkUrl2 = JDBCUtil.formatUrl(zkUrl2, RegistryType.ZK); + String formattedZkUrl = JDBCUtil.formatUrl(zkUrl, RegistryType.ZK); + if ( + StringUtils.equals(formattedZkUrl1, formattedZkUrl) + || StringUtils.equals(formattedZkUrl2, formattedZkUrl) + ) { + result.add(rs.getString(HA_GROUP_NAME)); } } } @@ -82,152 +190,427 @@ public static HAGroupStoreClient getInstance(Configuration conf) { @VisibleForTesting HAGroupStoreClient(final Configuration conf, - final PathChildrenCacheListener pathChildrenCacheListener) { + final PathChildrenCacheListener pathChildrenCacheListener, + final PathChildrenCacheListener peerPathChildrenCacheListener, final String haGroupName, + final String zkUrl) { + this.conf = conf; + this.haGroupName = haGroupName; + this.zkUrl = zkUrl; + this.waitTimeForSyncModeInMs = (long) Math.ceil( + conf.getLong(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT) * ZK_SESSION_TIMEOUT_MULTIPLIER); + // Custom Event Listener + this.peerCustomPathChildrenCacheListener = peerPathChildrenCacheListener; try { - this.phoenixHaAdmin = new PhoenixHAAdmin(conf); - final PathChildrenCache pathChildrenCache; - pathChildrenCache = - new PathChildrenCache(phoenixHaAdmin.getCurator(), ZKPaths.PATH_SEPARATOR, true); - final CountDownLatch latch = new CountDownLatch(1); - if (pathChildrenCacheListener != null) { - pathChildrenCache.getListenable().addListener(pathChildrenCacheListener); - } else { - pathChildrenCache.getListenable().addListener((client, event) -> { - LOGGER.info("HAGroupStoreClient PathChildrenCache Received event for type {}", - event.getType()); - final ChildData childData = event.getData(); - ClusterRoleRecord eventCRR = extractCRROrNull(childData); - switch (event.getType()) { - case CHILD_ADDED: - case CHILD_UPDATED: - if (eventCRR != null && eventCRR.getHaGroupName() != null) { - updateClusterRoleRecordMap(eventCRR); - } - break; - case CHILD_REMOVED: - // In case of CHILD_REMOVED, we get the old version of data that was just deleted in - // event. - if ( - eventCRR != null && eventCRR.getHaGroupName() != null - && !eventCRR.getHaGroupName().isEmpty() - && eventCRR.getRole(phoenixHaAdmin.getZkUrl()) != null - ) { - LOGGER.info( - "Received CHILD_REMOVED event, Removing CRR {} from existing CRR Map {}", - eventCRR, clusterRoleToCRRMap); - final ClusterRoleRecord.ClusterRole role = - eventCRR.getRole(phoenixHaAdmin.getZkUrl()); - clusterRoleToCRRMap.putIfAbsent(role, new ConcurrentHashMap<>()); - clusterRoleToCRRMap.get(role).remove(eventCRR.getHaGroupName()); - } - break; - case INITIALIZED: - latch.countDown(); - break; - case CONNECTION_SUSPENDED: - LOGGER.warn("Received CONNECTION_SUSPENDED event, will wait until either " - + "ZK connection is restored or session timeout occurs"); - break; - case CONNECTION_LOST: - LOGGER.error("Received CONNECTION_LOST event, marking cache as unhealthy"); - isHealthy = false; - break; - case CONNECTION_RECONNECTED: - LOGGER.info("Received CONNECTION_RECONNECTED event, marking cache as healthy"); - isHealthy = true; - break; - default: - LOGGER.warn("Unexpected event type {}, complete event {}", event.getType(), event); - } - }); + // Initialize HAGroupStoreClient attributes + initializeHAGroupStoreClientAttributes(haGroupName); + // Initialize Phoenix HA Admin + this.phoenixHaAdmin = new PhoenixHAAdmin(this.zkUrl, conf, ZK_CONSISTENT_HA_NAMESPACE); + // Initialize local cache + this.pathChildrenCache = + initializePathChildrenCache(phoenixHaAdmin, pathChildrenCacheListener, CACHE_TYPE_LOCAL); + // Initialize ZNode if not present in ZK + initializeZNodeIfNeeded(); + if (this.pathChildrenCache != null) { + this.isHealthy = true; + // Initialize peer cache + maybeInitializePeerPathChildrenCache(); } - pathChildrenCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); - this.pathChildrenCache = pathChildrenCache; - isHealthy = - latch.await(HA_GROUP_STORE_CLIENT_INITIALIZATION_TIMEOUT_MS, TimeUnit.MILLISECONDS); - buildClusterRoleToCRRMap(); + } catch (Exception e) { - isHealthy = false; - LOGGER.error( - "Unexpected error occurred while initializing HAGroupStoreClient, marking cache as unhealthy", - e); + this.isHealthy = false; + close(); + LOGGER.error("Unexpected error occurred while initializing HAGroupStoreClient, " + + "marking cache as unhealthy", e); } } - private ClusterRoleRecord extractCRROrNull(final ChildData childData) { - if (childData != null) { - byte[] data = childData.getData(); - return ClusterRoleRecord.fromJson(data).orElse(null); + /** + * Rebuilds the internal cache by querying for all needed data. This is a blocking operation that + * does not generate events to listeners. + * @throws Exception if rebuild fails or client is not healthy + */ + public void rebuild() throws Exception { + if (!isHealthy) { + throw new IOException("HAGroupStoreClient is not healthy"); } - return null; + LOGGER.info("Rebuilding HAGroupStoreClient for HA group {}", haGroupName); + initializeHAGroupStoreClientAttributes(haGroupName); + initializeZNodeIfNeeded(); + maybeInitializePeerPathChildrenCache(); + + // NOTE: this is a BLOCKING method. + // Completely rebuild the internal cache by querying for all needed data + // WITHOUT generating any events to send to listeners. + if (pathChildrenCache != null) { + pathChildrenCache.rebuild(); + } + if (peerPathChildrenCache != null) { + peerPathChildrenCache.rebuild(); + } + LOGGER.info("Rebuild Complete for HAGroupStoreClient for HA group {}", haGroupName); } - private void updateClusterRoleRecordMap(final ClusterRoleRecord crr) { + /** + * Get HAGroupStoreRecord from local quorum. + * @return HAGroupStoreRecord for the specified HA group name, or null if not found + * @throws IOException if the client is not healthy + */ + public HAGroupStoreRecord getHAGroupStoreRecord() throws IOException { + if (!isHealthy) { + throw new IOException("HAGroupStoreClient is not healthy"); + } + return fetchCacheRecord(this.pathChildrenCache, CACHE_TYPE_LOCAL).getLeft(); + } + + /** + * Set the HA group status for the specified HA group name. Checks if the status is needed to be + * updated based on logic in isUpdateNeeded function. + * @param haGroupState the HA group state to set + * @throws IOException if the client is not healthy or the operation + * fails + * @throws StaleHAGroupStoreRecordVersionException if the version is stale + * @throws InvalidClusterRoleTransitionException when transition is not valid + */ + public void setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState haGroupState) + throws IOException, StaleHAGroupStoreRecordVersionException, + InvalidClusterRoleTransitionException { + Preconditions.checkNotNull(haGroupState, "haGroupState cannot be null"); + if (!isHealthy) { + throw new IOException("HAGroupStoreClient is not healthy"); + } + Pair cacheRecord = + fetchCacheRecord(this.pathChildrenCache, CACHE_TYPE_LOCAL); + HAGroupStoreRecord currentHAGroupStoreRecord = cacheRecord.getLeft(); + Stat currentHAGroupStoreRecordStat = cacheRecord.getRight(); + if (currentHAGroupStoreRecord == null) { + throw new IOException("Current HAGroupStoreRecordStat in cache is null, " + + "cannot update HAGroupStoreRecord, the record should be initialized " + + "in System Table first" + haGroupName); + } if ( - crr != null && crr.getHaGroupName() != null && crr.getRole(phoenixHaAdmin.getZkUrl()) != null + isUpdateNeeded(currentHAGroupStoreRecord.getHAGroupState(), + currentHAGroupStoreRecordStat.getMtime(), haGroupState) ) { - ClusterRoleRecord.ClusterRole role = crr.getRole(phoenixHaAdmin.getZkUrl()); - LOGGER.info("Updating Existing CRR Map {} with new CRR {}", clusterRoleToCRRMap, crr); - clusterRoleToCRRMap.putIfAbsent(role, new ConcurrentHashMap<>()); - clusterRoleToCRRMap.get(role).put(crr.getHaGroupName(), crr); - LOGGER.info("Added new CRR {} to CRR Map", crr); - // Remove any pre-existing mapping with any other role for this HAGroupName - for (ClusterRoleRecord.ClusterRole mapRole : clusterRoleToCRRMap.keySet()) { - if (mapRole != role) { - ConcurrentHashMap roleWiseMap = - clusterRoleToCRRMap.get(mapRole); - if (roleWiseMap.containsKey(crr.getHaGroupName())) { - LOGGER.info("Removing any pre-existing mapping with role {} for HAGroupName {}", - mapRole, crr.getHaGroupName()); - roleWiseMap.remove(crr.getHaGroupName()); - } + HAGroupStoreRecord newHAGroupStoreRecord = + new HAGroupStoreRecord(currentHAGroupStoreRecord.getProtocolVersion(), + currentHAGroupStoreRecord.getHaGroupName(), haGroupState); + // TODO: Check if cluster role is changing, if so, we need to update + // the system table first + // Lock the row in System Table and make sure update is reflected + // in all regionservers + // It should automatically update the ZK record as well. + phoenixHaAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, newHAGroupStoreRecord, + currentHAGroupStoreRecordStat.getVersion()); + } else { + LOGGER.info("Not updating HAGroupStoreRecord for HA group {} with state {}", haGroupName, + haGroupState); + } + } + + /** + * Returns the ClusterRoleRecord for the cluster pair. Information in System Table for peer + * cluster might be stale, so we need to get the latest information from ZK. + * @return ClusterRoleRecord for the cluster pair + */ + public ClusterRoleRecord getClusterRoleRecord() throws IOException { + HAGroupStoreRecord peerHAGroupStoreRecord = getHAGroupStoreRecordFromPeer(); + ClusterRoleRecord.ClusterRole peerClusterRole = peerHAGroupStoreRecord != null + ? peerHAGroupStoreRecord.getClusterRole() + : ClusterRole.UNKNOWN; + return new ClusterRoleRecord(this.haGroupName, this.policy, this.clusterUrl, this.clusterRole, + this.peerClusterUrl, peerClusterRole, this.clusterRoleRecordVersion); + } + + /** + * Get HAGroupStoreRecord from peer cluster. + * @return HAGroupStoreRecord for the specified HA group name, or null if not found + * @throws IOException if the client is not healthy + */ + private HAGroupStoreRecord getHAGroupStoreRecordFromPeer() throws IOException { + if (!isHealthy) { + throw new IOException("HAGroupStoreClient is not healthy"); + } + return fetchCacheRecord(this.peerPathChildrenCache, CACHE_TYPE_PEER).getLeft(); + } + + private void initializeZNodeIfNeeded() + throws IOException, StaleHAGroupStoreRecordVersionException { + // Sometimes the ZNode might not be available in local cache yet, so better to check + // in ZK directly if we need to initialize + Pair cacheRecordFromZK = + phoenixHaAdmin.getHAGroupStoreRecordInZooKeeper(this.haGroupName); + HAGroupStoreRecord haGroupStoreRecord = cacheRecordFromZK.getLeft(); + HAGroupStoreRecord newHAGroupStoreRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + this.clusterRole.getDefaultHAGroupState()); + // Only update current ZNode if it doesn't have the same role as present in System Table. + // If not exists, then create ZNode + if ( + haGroupStoreRecord != null && !haGroupStoreRecord.getClusterRole().equals(this.clusterRole) + ) { + phoenixHaAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, newHAGroupStoreRecord, + cacheRecordFromZK.getRight().getVersion()); + } else if (haGroupStoreRecord == null) { + phoenixHaAdmin.createHAGroupStoreRecordInZooKeeper(newHAGroupStoreRecord); + } + } + + private void initializeHAGroupStoreClientAttributes(String haGroupName) throws SQLException { + String queryString = String.format("SELECT * FROM %s WHERE %s = '%s'", SYSTEM_HA_GROUP_NAME, + HA_GROUP_NAME, haGroupName); + try ( + PhoenixConnection conn = (PhoenixConnection) DriverManager + .getConnection(JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + zkUrl); + Statement stmt = conn.createStatement(); ResultSet rs = stmt.executeQuery(queryString)) { + if (rs.next()) { + this.policy = HighAvailabilityPolicy.valueOf(rs.getString(POLICY)); + String zkUrl1 = rs.getString(ZK_URL_1); + String zkUrl2 = rs.getString(ZK_URL_2); + String clusterRole1 = rs.getString(CLUSTER_ROLE_1); + String clusterRole2 = rs.getString(CLUSTER_ROLE_2); + String clusterUrl1 = rs.getString(CLUSTER_URL_1); + String clusterUrl2 = rs.getString(CLUSTER_URL_2); + this.clusterRoleRecordVersion = rs.getLong(VERSION); + Preconditions.checkNotNull(zkUrl1, "ZK_URL_1 in System Table cannot be null"); + Preconditions.checkNotNull(zkUrl2, "ZK_URL_2 in System Table cannot be null"); + String formattedZkUrl1 = JDBCUtil.formatUrl(zkUrl1, RegistryType.ZK); + String formattedZkUrl2 = JDBCUtil.formatUrl(zkUrl2, RegistryType.ZK); + String formattedZkUrl = JDBCUtil.formatUrl(this.zkUrl, RegistryType.ZK); + if (StringUtils.equals(formattedZkUrl1, formattedZkUrl)) { + this.peerZKUrl = formattedZkUrl2; + this.clusterRole = + ClusterRoleRecord.ClusterRole.from(clusterRole1.getBytes(StandardCharsets.UTF_8)); + this.peerClusterRole = + ClusterRoleRecord.ClusterRole.from(clusterRole2.getBytes(StandardCharsets.UTF_8)); + this.clusterUrl = clusterUrl1; + this.peerClusterUrl = clusterUrl2; + } else if (StringUtils.equals(formattedZkUrl2, formattedZkUrl)) { + this.peerZKUrl = JDBCUtil.formatUrl(zkUrl1, RegistryType.ZK); + this.clusterRole = + ClusterRoleRecord.ClusterRole.from(clusterRole2.getBytes(StandardCharsets.UTF_8)); + this.peerClusterRole = + ClusterRoleRecord.ClusterRole.from(clusterRole1.getBytes(StandardCharsets.UTF_8)); + this.clusterUrl = clusterUrl2; + this.peerClusterUrl = clusterUrl1; } + } else { + throw new SQLException("HAGroupStoreRecord not found for HA group name: " + haGroupName + + " in System Table " + SYSTEM_HA_GROUP_NAME); } - LOGGER.info("Final Updated CRR Map {}", clusterRoleToCRRMap); } + Preconditions.checkNotNull(this.clusterRole, "Cluster role in System Table cannot be null"); + Preconditions.checkNotNull(this.peerClusterRole, + "Peer cluster role in System Table cannot be null"); + Preconditions.checkNotNull(this.clusterUrl, "Cluster URL in System Table cannot be null"); + Preconditions.checkNotNull(this.peerZKUrl, "Peer ZK URL in System Table cannot be null"); + Preconditions.checkNotNull(this.peerClusterUrl, + "Peer Cluster URL in System Table cannot be null"); + Preconditions.checkNotNull(this.clusterRoleRecordVersion, + "Cluster role record version in System Table cannot be null"); } - private void buildClusterRoleToCRRMap() { - List clusterRoleRecords = pathChildrenCache.getCurrentData().stream() - .map(this::extractCRROrNull).filter(Objects::nonNull).collect(Collectors.toList()); - for (ClusterRoleRecord crr : clusterRoleRecords) { - updateClusterRoleRecordMap(crr); + private void maybeInitializePeerPathChildrenCache() { + if (StringUtils.isNotBlank(this.peerZKUrl)) { + try { + // Setup peer connection if needed (first time or ZK Url changed) + if ( + peerPathChildrenCache == null || (peerPhoenixHaAdmin != null + && !StringUtils.equals(this.peerZKUrl, peerPhoenixHaAdmin.getZkUrl())) + ) { + // Clean up existing peer connection if it exists + closePeerConnection(); + // Setup new peer connection + this.peerPhoenixHaAdmin = + new PhoenixHAAdmin(this.peerZKUrl, conf, ZK_CONSISTENT_HA_NAMESPACE); + // Create new PeerPathChildrenCache + this.peerPathChildrenCache = initializePathChildrenCache(peerPhoenixHaAdmin, + this.peerCustomPathChildrenCacheListener, CACHE_TYPE_PEER); + } + } catch (Exception e) { + closePeerConnection(); + LOGGER.error("Unable to initialize PeerPathChildrenCache for HAGroupStoreClient", e); + // Don't think we should mark HAGroupStoreClient as unhealthy if + // peerCache is unhealthy, if needed we can introduce a config to control behavior. + } + } else { + // Close Peer Cache for this HAGroupName if currentClusterRecord is null + // or peerZKUrl is blank + closePeerConnection(); + LOGGER.error("Not initializing PeerPathChildrenCache for HAGroupStoreClient " + + "with HAGroupName {} as peerZKUrl is blank", haGroupName); } } - public void rebuild() throws Exception { - if (!isHealthy) { - throw new IOException("HAGroupStoreClient is not healthy"); + private PathChildrenCache initializePathChildrenCache(PhoenixHAAdmin admin, + PathChildrenCacheListener customListener, String cacheType) { + LOGGER.info("Initializing {} PathChildrenCache with URL {}", cacheType, admin.getZkUrl()); + PathChildrenCache newPathChildrenCache = null; + try { + newPathChildrenCache = + new PathChildrenCache(admin.getCurator(), ZKPaths.PATH_SEPARATOR, true); + final CountDownLatch latch = new CountDownLatch(1); + newPathChildrenCache.getListenable().addListener( + customListener != null ? customListener : createCacheListener(latch, cacheType)); + newPathChildrenCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); + boolean initialized = + latch.await(HA_GROUP_STORE_CLIENT_INITIALIZATION_TIMEOUT_MS, TimeUnit.MILLISECONDS); + return initialized ? newPathChildrenCache : null; + } catch (Exception e) { + if (newPathChildrenCache != null) { + try { + newPathChildrenCache.close(); + } catch (IOException ioe) { + LOGGER.error("Failed to close {} PathChildrenCache with ZKUrl", cacheType, ioe); + } + } + LOGGER.error("Failed to initialize {} PathChildrenCache", cacheType, e); + return null; + } + } + + private PathChildrenCacheListener createCacheListener(CountDownLatch latch, String cacheType) { + return (client, event) -> { + final ChildData childData = event.getData(); + HAGroupStoreRecord eventRecord = extractHAGroupStoreRecordOrNull(childData); + LOGGER.info("HAGroupStoreClient Cache {} received event {} type {} at {}", cacheType, + eventRecord, event.getType(), System.currentTimeMillis()); + switch (event.getType()) { + // TODO: Add support for event watcher for HAGroupStoreRecord + // case CHILD_ADDED: + // case CHILD_UPDATED: + // case CHILD_REMOVED: + case INITIALIZED: + latch.countDown(); + break; + case CONNECTION_LOST: + case CONNECTION_SUSPENDED: + if (CACHE_TYPE_LOCAL.equals(cacheType)) { + isHealthy = false; + } + LOGGER.warn("{} HAGroupStoreClient cache connection lost/suspended", cacheType); + break; + case CONNECTION_RECONNECTED: + if (CACHE_TYPE_LOCAL.equals(cacheType)) { + isHealthy = true; + } + LOGGER.info("{} HAGroupStoreClient cache connection reconnected", cacheType); + break; + default: + LOGGER.warn("Unexpected {} event type {}, complete event {}", cacheType, event.getType(), + event); + } + }; + } + + private Pair fetchCacheRecord(PathChildrenCache cache, + String cacheType) { + if (cache == null) { + LOGGER.warn("{} HAGroupStoreClient cache is null, returning null", cacheType); + return Pair.of(null, null); + } + + String targetPath = toPath(this.haGroupName); + // Try to get record from current cache data + Pair result = extractRecordAndStat(cache, targetPath, cacheType); + if (result.getLeft() != null) { + return result; + } + + if (cacheType.equals(CACHE_TYPE_PEER)) { + return Pair.of(null, null); + } + // If no record found, try to rebuild and fetch again + LOGGER.info("No record found at path {} for {} cluster, trying to initialize ZNode " + + "from System Table in case it might have been deleted", targetPath, cacheType); + try { + rebuild(); + return extractRecordAndStat(cache, targetPath, cacheType); + } catch (Exception e) { + LOGGER.error( + "Failed to initialize ZNode from System Table, giving up " + "and returning null", e); + return Pair.of(null, null); + } + } + + private Pair extractRecordAndStat(PathChildrenCache cache, + String targetPath, String cacheType) { + ChildData childData = cache.getCurrentData(targetPath); + if (childData != null) { + HAGroupStoreRecord record = extractHAGroupStoreRecordOrNull(childData); + Stat currentStat = childData.getStat(); + LOGGER.info("Built {} cluster record: {}", cacheType, record); + return Pair.of(record, currentStat); + } + return Pair.of(null, null); + } + + private HAGroupStoreRecord extractHAGroupStoreRecordOrNull(final ChildData childData) { + if (childData != null) { + byte[] data = childData.getData(); + return HAGroupStoreRecord.fromJson(data).orElse(null); + } + return null; + } + + /** + * Closes the peer connection and cleans up peer-related resources. + */ + private void closePeerConnection() { + try { + if (peerPathChildrenCache != null) { + peerPathChildrenCache.close(); + peerPathChildrenCache = null; + } + if (peerPhoenixHaAdmin != null) { + peerPhoenixHaAdmin.close(); + peerPhoenixHaAdmin = null; + } + } catch (Exception e) { + LOGGER.warn("Failed to close peer connection", e); } - LOGGER.info("Rebuilding HAGroupStoreClient for HA groups"); - // NOTE: this is a BLOCKING method. - // Completely rebuild the internal cache by querying for all needed data - // WITHOUT generating any events to send to listeners. - pathChildrenCache.rebuild(); - buildClusterRoleToCRRMap(); - LOGGER.info("Rebuild Complete for HAGroupStoreClient"); } @Override public void close() { try { LOGGER.info("Closing HAGroupStoreClient"); - clusterRoleToCRRMap.clear(); if (pathChildrenCache != null) { pathChildrenCache.close(); + pathChildrenCache = null; } + closePeerConnection(); LOGGER.info("Closed HAGroupStoreClient"); } catch (IOException e) { LOGGER.error("Exception closing HAGroupStoreClient", e); } } - public List getCRRsByClusterRole(ClusterRoleRecord.ClusterRole clusterRole) - throws IOException { - if (!isHealthy) { - throw new IOException("HAGroupStoreClient is not healthy"); + /** + * Checks if the HAGroupStoreRecord needs to be updated. If the cluster role is allowed to + * transition to the new state and the status refresh interval has expired, the HAGroupStoreRecord + * needs to be updated. If the transition is not allowed, an exception is thrown. + * @param currentHAGroupState the current HAGroupState of the HAGroupStoreRecord + * @param currentHAGroupStoreRecordMtime the last modified time of the current HAGroupStoreRecord + * @param newHAGroupState the cluster state to check + * @return true if the HAGroupStoreRecord needs to be updated, false otherwise + * @throws InvalidClusterRoleTransitionException if the cluster role transition is invalid + */ + private boolean isUpdateNeeded(HAGroupStoreRecord.HAGroupState currentHAGroupState, + long currentHAGroupStoreRecordMtime, HAGroupStoreRecord.HAGroupState newHAGroupState) + throws InvalidClusterRoleTransitionException { + long waitTime = 0L; + if (currentHAGroupState.isTransitionAllowed(newHAGroupState)) { + if ( + currentHAGroupState == HAGroupState.ACTIVE_NOT_IN_SYNC + && newHAGroupState == HAGroupState.ACTIVE_IN_SYNC + ) { + waitTime = waitTimeForSyncModeInMs; + } + } else { + throw new InvalidClusterRoleTransitionException( + "Cannot transition from " + currentHAGroupState + " to " + newHAGroupState); } - return ImmutableList - .copyOf(clusterRoleToCRRMap.getOrDefault(clusterRole, new ConcurrentHashMap<>()).values()); + return ((System.currentTimeMillis() - currentHAGroupStoreRecordMtime) > waitTime); } + } 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 d50eae4f851..1a7fe0985bc 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 @@ -22,51 +22,91 @@ import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED; import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; +import org.apache.phoenix.exception.InvalidClusterRoleTransitionException; +import org.apache.phoenix.exception.StaleHAGroupStoreRecordVersionException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * Implementation of HAGroupStoreManager that uses HAGroupStoreClient. Manages all + * HAGroupStoreClient instances. + */ public class HAGroupStoreManager { + private static volatile HAGroupStoreManager haGroupStoreManagerInstance; + private static final Logger LOGGER = LoggerFactory.getLogger(HAGroupStoreManager.class); private final boolean mutationBlockEnabled; + private final String zkUrl; private final Configuration conf; - private static volatile Map INSTANCES = new ConcurrentHashMap<>(); + // Map of for different MiniClusters + // Can revert this but will fail for tests with one cluster down + private static Map instances = new ConcurrentHashMap<>(); /** - * Creates/gets an instance of HAGroupStoreManager. Provides unique instance for each ZK URL + * Creates/gets an instance of HAGroupStoreManager. Use the ZK URL from the configuration to + * determine the instance. * @param conf configuration * @return HAGroupStoreManager instance */ - public static HAGroupStoreManager getInstance(Configuration conf) { - final String zkUrl = getLocalZkUrl(conf); - HAGroupStoreManager result = INSTANCES.get(zkUrl); - if (result == null) { - synchronized (HAGroupStoreManager.class) { - result = INSTANCES.get(zkUrl); - if (result == null) { - result = new HAGroupStoreManager(conf); - INSTANCES.put(zkUrl, result); - } - } - } - return result; + public static HAGroupStoreManager getInstance(final Configuration conf) { + return getInstanceForZkUrl(conf, null); } - private HAGroupStoreManager(final Configuration conf) { + /** + * Creates/gets an instance of HAGroupStoreManager for a specific ZK URL. This allows different + * region servers to have their own instances. + * @param conf configuration + * @param zkUrl specific ZK URL to use, null to use local ZK URL from config + * @return HAGroupStoreManager instance for the specified ZK URL + */ + public static HAGroupStoreManager getInstanceForZkUrl(final Configuration conf, String zkUrl) { + String localZkUrl = Objects.toString(zkUrl, getLocalZkUrl(conf)); + Objects.requireNonNull(localZkUrl, "zkUrl cannot be null"); + + return instances.computeIfAbsent(localZkUrl, url -> { + LOGGER.info("Creating new HAGroupStoreManager instance for ZK URL: {}", url); + return new HAGroupStoreManager(conf, url); + }); + } + + private HAGroupStoreManager(final Configuration conf, final String zkUrl) { this.mutationBlockEnabled = conf.getBoolean(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, DEFAULT_CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED); + this.zkUrl = zkUrl; this.conf = conf; + LOGGER.info("Started HAGroupStoreManager with ZK URL: {}", zkUrl); + } + + /** + * Returns the list of all HA group names. + * @return list of all HA group names. + * @throws SQLException if there is an error with querying the table. + */ + public List getHAGroupNames() throws SQLException { + return HAGroupStoreClient.getHAGroupNames(this.zkUrl); } /** - * Checks whether mutation is blocked or not. + * Checks whether mutation is blocked or not for a specific HA group. + * @param haGroupName name of the HA group, null for default HA group which tracks all HA groups. + * @return true if mutation is blocked, false otherwise. * @throws IOException when HAGroupStoreClient is not healthy. */ - public boolean isMutationBlocked() throws IOException { + public boolean isMutationBlocked(String haGroupName) throws IOException, SQLException { if (mutationBlockEnabled) { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(conf); + HAGroupStoreClient haGroupStoreClient = + HAGroupStoreClient.getInstanceForZkUrl(conf, haGroupName, zkUrl); if (haGroupStoreClient != null) { - return !haGroupStoreClient - .getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY).isEmpty(); + return haGroupStoreClient.getHAGroupStoreRecord() != null + && haGroupStoreClient.getHAGroupStoreRecord().getClusterRole() != null + && haGroupStoreClient.getHAGroupStoreRecord().getClusterRole().isMutationBlocked(); } throw new IOException("HAGroupStoreClient is not initialized"); } @@ -74,14 +114,111 @@ public boolean isMutationBlocked() throws IOException { } /** - * Force rebuilds the HAGroupStoreClient + * Force rebuilds the HAGroupStoreClient instance for all HA groups. If any HAGroupStoreClient + * instance is not created, it will be created. + * @param broadcastUpdate if true, the update will be broadcasted to all regionserver endpoints. + * @throws Exception in case of an error with dependencies or table. + */ + public void invalidateHAGroupStoreClient(boolean broadcastUpdate) throws Exception { + List haGroupNames = HAGroupStoreClient.getHAGroupNames(this.zkUrl); + List failedHAGroupNames = new ArrayList<>(); + for (String haGroupName : haGroupNames) { + try { + invalidateHAGroupStoreClient(haGroupName, broadcastUpdate); + } catch (Exception e) { + failedHAGroupNames.add(haGroupName); + LOGGER.error("Failed to invalidate HAGroupStoreClient for " + haGroupName, e); + } + } + if (!failedHAGroupNames.isEmpty()) { + throw new IOException("Failed to invalidate HAGroupStoreClient for " + failedHAGroupNames + + "successfully invalidated HAGroupStoreClient instance for " + + haGroupNames.removeAll(failedHAGroupNames) + " HA groups"); + } + } + + /** + * Force rebuilds the HAGroupStoreClient for a specific HA group. + * @param haGroupName name of the HA group, null for default HA group and tracks all HA + * groups. + * @param broadcastUpdate if true, the update will be broadcasted to all regionserver endpoints. + * @throws Exception in case of an error with dependencies or table. */ - public void invalidateHAGroupStoreClient() throws Exception { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(conf); + public void invalidateHAGroupStoreClient(final String haGroupName, boolean broadcastUpdate) + throws Exception { + HAGroupStoreClient haGroupStoreClient = + HAGroupStoreClient.getInstanceForZkUrl(conf, haGroupName, zkUrl); if (haGroupStoreClient != null) { haGroupStoreClient.rebuild(); } else { throw new IOException("HAGroupStoreClient is not initialized"); } } + + /** + * Returns the HAGroupStoreRecord for a specific HA group. + * @param haGroupName name of the HA group + * @return Optional HAGroupStoreRecord for the HA group, can be empty if the HA group is not + * found. + * @throws IOException when HAGroupStoreClient is not healthy. + */ + public Optional getHAGroupStoreRecord(final String haGroupName) + throws IOException, SQLException { + HAGroupStoreClient haGroupStoreClient = + HAGroupStoreClient.getInstanceForZkUrl(conf, haGroupName, zkUrl); + if (haGroupStoreClient != null) { + return Optional.ofNullable(haGroupStoreClient.getHAGroupStoreRecord()); + } + throw new IOException("HAGroupStoreClient is not initialized"); + } + + /** + * Sets the HAGroupStoreRecord to StoreAndForward mode in local cluster. + * @param haGroupName name of the HA group + * @throws IOException when HAGroupStoreClient is not healthy. + */ + public void setHAGroupStatusToStoreAndForward(final String haGroupName) throws IOException, + StaleHAGroupStoreRecordVersionException, InvalidClusterRoleTransitionException, SQLException { + HAGroupStoreClient haGroupStoreClient = + HAGroupStoreClient.getInstanceForZkUrl(conf, haGroupName, zkUrl); + if (haGroupStoreClient != null) { + haGroupStoreClient + .setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC); + } else { + throw new IOException("HAGroupStoreClient is not initialized"); + } + } + + /** + * Sets the HAGroupStoreRecord to Sync mode in local cluster. + * @param haGroupName name of the HA group + * @throws IOException when HAGroupStoreClient is not healthy. + */ + public void setHAGroupStatusRecordToSync(final String haGroupName) throws IOException, + StaleHAGroupStoreRecordVersionException, InvalidClusterRoleTransitionException, SQLException { + HAGroupStoreClient haGroupStoreClient = + HAGroupStoreClient.getInstanceForZkUrl(conf, haGroupName, zkUrl); + if (haGroupStoreClient != null) { + haGroupStoreClient.setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + } else { + throw new IOException("HAGroupStoreClient is not initialized"); + } + } + + /** + * Returns the ClusterRoleRecord for the cluster pair. If the peer cluster is not connected or + * peer cluster is not configured, it will return UNKNOWN for peer cluster. Only implemented by + * HAGroupStoreManagerImpl. + * @return ClusterRoleRecord for the cluster pair + * @throws IOException when HAGroupStoreClient is not healthy. + */ + public ClusterRoleRecord getClusterRoleRecord(String haGroupName) + throws IOException, SQLException { + HAGroupStoreClient haGroupStoreClient = + HAGroupStoreClient.getInstanceForZkUrl(conf, haGroupName, zkUrl); + if (haGroupStoreClient != null) { + return haGroupStoreClient.getClusterRoleRecord(); + } + throw new IOException("HAGroupStoreClient is not initialized"); + } } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreRecord.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreRecord.java new file mode 100644 index 00000000000..e3e1a4a628b --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAGroupStoreRecord.java @@ -0,0 +1,243 @@ +/* + * 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 com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.phoenix.util.JacksonUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions; +import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableSet; + +/** + * Immutable class representing an HA group store record with simplified fields. This is a + * simplified version of ClusterRoleRecord that contains essential information about an HA group + * only for a single cluster. + */ +public class HAGroupStoreRecord { + + private static final Logger LOG = LoggerFactory.getLogger(HAGroupStoreRecord.class); + public static final String DEFAULT_PROTOCOL_VERSION = "1.0"; + + /** + * Enum representing the HA group state with each state having a corresponding ClusterRole. + */ + public enum HAGroupState { + ABORT_TO_ACTIVE_IN_SYNC, + ABORT_TO_ACTIVE_NOT_IN_SYNC, + ABORT_TO_STANDBY, + ACTIVE_IN_SYNC, + ACTIVE_NOT_IN_SYNC, + ACTIVE_NOT_IN_SYNC_TO_STANDBY, + ACTIVE_NOT_IN_SYNC_WITH_OFFLINE_PEER, + ACTIVE_IN_SYNC_TO_STANDBY, + ACTIVE_WITH_OFFLINE_PEER, + DEGRADED_STANDBY, + DEGRADED_STANDBY_FOR_READER, + DEGRADED_STANDBY_FOR_WRITER, + OFFLINE, + STANDBY, + STANDBY_TO_ACTIVE, + UNKNOWN; + + private Set allowedTransitions; + + /** + * Gets the corresponding ClusterRole for this HAGroupState. + * @return the corresponding ClusterRole + */ + public ClusterRoleRecord.ClusterRole getClusterRole() { + switch (this) { + case ABORT_TO_ACTIVE_IN_SYNC: + case ABORT_TO_ACTIVE_NOT_IN_SYNC: + case ACTIVE_IN_SYNC: + case ACTIVE_NOT_IN_SYNC: + case ACTIVE_NOT_IN_SYNC_WITH_OFFLINE_PEER: + case ACTIVE_WITH_OFFLINE_PEER: + return ClusterRoleRecord.ClusterRole.ACTIVE; + case ACTIVE_IN_SYNC_TO_STANDBY: + case ACTIVE_NOT_IN_SYNC_TO_STANDBY: + return ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY; + case ABORT_TO_STANDBY: + case DEGRADED_STANDBY: + case DEGRADED_STANDBY_FOR_READER: + case DEGRADED_STANDBY_FOR_WRITER: + case STANDBY: + return ClusterRoleRecord.ClusterRole.STANDBY; + case STANDBY_TO_ACTIVE: + return ClusterRoleRecord.ClusterRole.STANDBY_TO_ACTIVE; + case OFFLINE: + return ClusterRoleRecord.ClusterRole.OFFLINE; + case UNKNOWN: + default: + return ClusterRoleRecord.ClusterRole.UNKNOWN; + } + } + + static { + // Initialize allowed transitions + ACTIVE_NOT_IN_SYNC.allowedTransitions = ImmutableSet.of(ACTIVE_NOT_IN_SYNC, ACTIVE_IN_SYNC, + ACTIVE_NOT_IN_SYNC_TO_STANDBY, ACTIVE_NOT_IN_SYNC_WITH_OFFLINE_PEER); + + ACTIVE_IN_SYNC.allowedTransitions = + ImmutableSet.of(ACTIVE_NOT_IN_SYNC, ACTIVE_WITH_OFFLINE_PEER, ACTIVE_IN_SYNC_TO_STANDBY); + + STANDBY.allowedTransitions = ImmutableSet.of(STANDBY_TO_ACTIVE, DEGRADED_STANDBY_FOR_READER, + DEGRADED_STANDBY_FOR_WRITER); + // This needs to be manually recovered by operator + OFFLINE.allowedTransitions = ImmutableSet.of(); + // This needs to be manually recovered by operator + UNKNOWN.allowedTransitions = ImmutableSet.of(); + ACTIVE_NOT_IN_SYNC_TO_STANDBY.allowedTransitions = + ImmutableSet.of(ABORT_TO_ACTIVE_NOT_IN_SYNC, ACTIVE_IN_SYNC_TO_STANDBY); + ACTIVE_IN_SYNC_TO_STANDBY.allowedTransitions = + ImmutableSet.of(ABORT_TO_ACTIVE_IN_SYNC, STANDBY); + STANDBY_TO_ACTIVE.allowedTransitions = ImmutableSet.of(ABORT_TO_STANDBY, ACTIVE_IN_SYNC); + DEGRADED_STANDBY.allowedTransitions = + ImmutableSet.of(DEGRADED_STANDBY_FOR_READER, DEGRADED_STANDBY_FOR_WRITER); + DEGRADED_STANDBY_FOR_WRITER.allowedTransitions = ImmutableSet.of(STANDBY, DEGRADED_STANDBY); + DEGRADED_STANDBY_FOR_READER.allowedTransitions = ImmutableSet.of(STANDBY, DEGRADED_STANDBY); + ACTIVE_WITH_OFFLINE_PEER.allowedTransitions = ImmutableSet.of(ACTIVE_NOT_IN_SYNC); + ABORT_TO_ACTIVE_IN_SYNC.allowedTransitions = ImmutableSet.of(ACTIVE_IN_SYNC); + ABORT_TO_ACTIVE_NOT_IN_SYNC.allowedTransitions = ImmutableSet.of(ACTIVE_NOT_IN_SYNC); + ABORT_TO_STANDBY.allowedTransitions = ImmutableSet.of(STANDBY); + ACTIVE_NOT_IN_SYNC_WITH_OFFLINE_PEER.allowedTransitions = ImmutableSet.of(ACTIVE_NOT_IN_SYNC); + } + + /** + * Checks if the transition from this state to the target state is allowed. + * @param targetState the state to transition to + * @return true if the transition is allowed, false otherwise + */ + public boolean isTransitionAllowed(HAGroupState targetState) { + return allowedTransitions.contains(targetState); + } + + public static HAGroupState from(byte[] bytes) { + if (bytes == null) { + return UNKNOWN; + } + String value = new String(bytes, StandardCharsets.UTF_8); + return Arrays.stream(HAGroupState.values()).filter(r -> r.name().equalsIgnoreCase(value)) + .findFirst().orElse(UNKNOWN); + } + } + + private final String protocolVersion; + private final String haGroupName; + private final HAGroupState haGroupState; + + @JsonCreator + public HAGroupStoreRecord(@JsonProperty("protocolVersion") String protocolVersion, + @JsonProperty("haGroupName") String haGroupName, + @JsonProperty("haGroupState") HAGroupState haGroupState) { + Preconditions.checkNotNull(haGroupName, "HA group name cannot be null!"); + Preconditions.checkNotNull(haGroupState, "HA group state cannot be null!"); + + this.protocolVersion = Objects.toString(protocolVersion, DEFAULT_PROTOCOL_VERSION); + this.haGroupName = haGroupName; + this.haGroupState = haGroupState; + } + + public static Optional fromJson(byte[] bytes) { + if (bytes == null) { + return Optional.empty(); + } + try { + return Optional.of(JacksonUtil.getObjectReader(HAGroupStoreRecord.class).readValue(bytes)); + } catch (Exception e) { + LOG.error("Fail to deserialize data to an HA group store record", e); + return Optional.empty(); + } + } + + public static byte[] toJson(HAGroupStoreRecord record) throws IOException { + return JacksonUtil.getObjectWriter().withoutAttribute("clusterRole").writeValueAsBytes(record); + } + + public boolean hasSameInfo(HAGroupStoreRecord other) { + return haGroupName.equals(other.haGroupName) && haGroupState.equals(other.haGroupState) + && protocolVersion.equals(other.protocolVersion); + } + + public String getProtocolVersion() { + return protocolVersion; + } + + public String getHaGroupName() { + return haGroupName; + } + + @JsonProperty("haGroupState") + public HAGroupState getHAGroupState() { + return haGroupState; + } + + @JsonIgnore + public ClusterRoleRecord.ClusterRole getClusterRole() { + return haGroupState.getClusterRole(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder().append(protocolVersion).append(haGroupName).append(haGroupState) + .hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } else if (other == null) { + return false; + } else if (!(other instanceof HAGroupStoreRecord)) { + return false; + } else { + HAGroupStoreRecord otherRecord = (HAGroupStoreRecord) other; + return new EqualsBuilder().append(protocolVersion, otherRecord.protocolVersion) + .append(haGroupName, otherRecord.haGroupName).append(haGroupState, otherRecord.haGroupState) + .isEquals(); + } + } + + @Override + public String toString() { + return "HAGroupStoreRecord{" + "protocolVersion='" + protocolVersion + '\'' + ", haGroupName='" + + haGroupName + '\'' + ", haGroupState=" + haGroupState + '}'; + } + + public String toPrettyString() { + try { + return JacksonUtil.getObjectWriterPretty().writeValueAsString(this); + } catch (Exception e) { + LOG.error("Fail to wrap this object as JSON, returning the oneliner " + "using toString", e); + return toString(); + } + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HighAvailabilityGroup.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HighAvailabilityGroup.java index 5e522db88a5..3064717a497 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HighAvailabilityGroup.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HighAvailabilityGroup.java @@ -136,17 +136,15 @@ public class HighAvailabilityGroup { */ @VisibleForTesting static final Map GROUPS = new ConcurrentHashMap<>(); + static final Map> URLS = new ConcurrentHashMap<>(); @VisibleForTesting - public static final Map> URLS = new ConcurrentHashMap<>(); - @VisibleForTesting - public static final Cache MISSING_CRR_GROUPS_CACHE = - CacheBuilder.newBuilder() - .expireAfterWrite(PHOENIX_HA_TRANSITION_TIMEOUT_MS_DEFAULT, TimeUnit.MILLISECONDS).build(); + static final Cache MISSING_CRR_GROUPS_CACHE = CacheBuilder.newBuilder() + .expireAfterWrite(PHOENIX_HA_TRANSITION_TIMEOUT_MS_DEFAULT, TimeUnit.MILLISECONDS).build(); /** - * The Curator client cache, one client instance per cluster. + * The Curator client cache, one client instance per cluster and namespace combination. */ @VisibleForTesting - public static final Cache CURATOR_CACHE = CacheBuilder.newBuilder() .expireAfterAccess(DEFAULT_CLIENT_CONNECTION_CACHE_MAX_DURATION, TimeUnit.MILLISECONDS) .removalListener( @@ -342,8 +340,10 @@ public static Optional get(String url, Properties propert GROUPS.remove(info); haGroup.close(); try { - CuratorFramework curator1 = CURATOR_CACHE.getIfPresent(info.getUrl1()); - CuratorFramework curator2 = CURATOR_CACHE.getIfPresent(info.getUrl2()); + CuratorFramework curator1 = + CURATOR_CACHE.getIfPresent(generateCacheKey(info.getUrl1(), null)); + CuratorFramework curator2 = + CURATOR_CACHE.getIfPresent(generateCacheKey(info.getUrl2(), null)); if (curator1 != null && curator2 != null) { Stat node1 = curator1.checkExists().forPath(info.getZkPath()); Stat node2 = curator2.checkExists().forPath(info.getZkPath()); @@ -411,6 +411,19 @@ static Optional getFallbackCluster(String url, Properties properties) return Optional.of(fallbackCluster); } + /** + * Generate cache key for curator cache based on URL and namespace. + * @param jdbcUrl the ZK endpoint host:port or the JDBC connection String host:port:/hbase + * @param namespace the ZooKeeper namespace, uses default if null + * @return cache key string + */ + @VisibleForTesting + static String generateCacheKey(String jdbcUrl, String namespace) { + String effectiveNamespace = + namespace != null ? namespace : PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE; + return jdbcUrl + ":" + effectiveNamespace; + } + /** * Get an active curator ZK client for the given properties and ZK endpoint. *

@@ -422,9 +435,29 @@ static Optional getFallbackCluster(String url, Properties properties) @SuppressWarnings("UnstableApiUsage") public static CuratorFramework getCurator(String jdbcUrl, Properties properties) throws IOException { + return getCurator(jdbcUrl, properties, PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE); + } + + /** + * Get an active curator ZK client for the given properties, ZK endpoint and namespace. + *

+ * This can be from cached object since Curator should be shared per cluster. + * @param jdbcUrl the ZK endpoint host:port or the JDBC connection String host:port:/hbase + * @param properties the properties defining time out values and retry count + * @param namespace the ZooKeeper namespace to use, defaults to + * PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE if null + * @return a new Curator framework client + */ + @SuppressWarnings("UnstableApiUsage") + public static CuratorFramework getCurator(String jdbcUrl, Properties properties, String namespace) + throws IOException { + // Use namespace as part of cache key to avoid conflicts between different namespaces + String effectiveNamespace = + namespace != null ? namespace : PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE; + String cacheKey = generateCacheKey(jdbcUrl, namespace); try { - return CURATOR_CACHE.get(jdbcUrl, () -> { - CuratorFramework curator = createCurator(jdbcUrl, properties); + return CURATOR_CACHE.get(cacheKey, () -> { + CuratorFramework curator = createCurator(jdbcUrl, properties, effectiveNamespace); try { if ( !curator.blockUntilConnected(PHOENIX_HA_ZK_CONNECTION_TIMEOUT_MS_DEFAULT, @@ -447,17 +480,23 @@ public static CuratorFramework getCurator(String jdbcUrl, Properties properties) } catch (Exception e) { LOG.error("Fail to get an active curator for url {}", jdbcUrl, e); // invalidate the cache when getting/creating throws exception - CURATOR_CACHE.invalidate(jdbcUrl); + CURATOR_CACHE.invalidate(cacheKey); throw new IOException(e); } } /** - * Create a curator ZK client for the given properties and ZK endpoint. + * Create a curator ZK client for the given properties, ZK endpoint and namespace. *

* Unless caller needs a new curator, it should use {@link #getCurator(String, Properties)}. + * @param jdbcUrl the ZK endpoint host:port or the JDBC connection String host:port:/hbase + * @param properties the properties defining time out values and retry count + * @param namespace the ZooKeeper namespace to use, defaults to + * PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE if null + * @return a new Curator framework client */ - private static CuratorFramework createCurator(String jdbcUrl, Properties properties) { + private static CuratorFramework createCurator(String jdbcUrl, Properties properties, + String namespace) { // Get the ZK endpoint in host:port format by removing JDBC protocol and HBase root node final String zkUrl; if (jdbcUrl.startsWith(PhoenixRuntime.JDBC_PROTOCOL)) { @@ -485,8 +524,9 @@ private static CuratorFramework createCurator(String jdbcUrl, Properties propert final RetryPolicy retryPolicy = createRetryPolicy(properties); CuratorFramework curator = CuratorFrameworkFactory.builder().connectString(zkUrl) - .namespace(PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE).connectionTimeoutMs(connectionTimeoutMs) - .sessionTimeoutMs(sessionTimeoutMs).retryPolicy(retryPolicy).canBeReadOnly(true).build(); + .namespace(namespace != null ? namespace : PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE) + .connectionTimeoutMs(connectionTimeoutMs).sessionTimeoutMs(sessionTimeoutMs) + .retryPolicy(retryPolicy).canBeReadOnly(true).build(); curator.start(); return curator; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java index 7f2612fa7f8..cd2b9a44b87 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java @@ -185,6 +185,7 @@ public class PhoenixConnection private LogLevel auditLogLevel; private Double logSamplingRate; private String sourceOfOperation; + private String haGroupName; private volatile SQLException reasonForClose; private static final String[] CONNECTION_PROPERTIES; @@ -1166,6 +1167,18 @@ public void setConsistency(Consistency val) { this.consistency = val; } + /** + * This is temporary method to set HAGroupName for the connection. This will be removed once we + * have a proper way to set HAGroupName in the connection. + */ + public void setHAGroupName(String haGroupName) { + this.haGroupName = haGroupName; + } + + public String getHAGroupName() { + return this.haGroupName; + } + @Override public void setCatalog(String catalog) throws SQLException { checkOpen(); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java index 3bc68c5ddb7..0fc4803720c 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java @@ -449,6 +449,23 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData { public static final String INDEX_WHERE = "INDEX_WHERE"; public static final byte[] INDEX_WHERE_BYTES = Bytes.toBytes(INDEX_WHERE); + public static final String SYSTEM_HA_GROUP_TABLE = "HA_GROUP"; + public static final String SYSTEM_HA_GROUP_NAME = + SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_HA_GROUP_TABLE); + public static final byte[] SYSTEM_HA_GROUP_NAME_BYTES = Bytes.toBytes(SYSTEM_HA_GROUP_NAME); + public static final TableName SYSTEM_HA_GROUP_HBASE_TABLE_NAME = + TableName.valueOf(SYSTEM_HA_GROUP_NAME); + + public static final String HA_GROUP_NAME = "HA_GROUP_NAME"; + public static final String POLICY = "POLICY"; + public static final String VERSION = "VERSION"; + public static final String ZK_URL_1 = "ZK_URL_1"; + public static final String ZK_URL_2 = "ZK_URL_2"; + public static final String CLUSTER_URL_1 = "CLUSTER_URL_1"; + public static final String CLUSTER_URL_2 = "CLUSTER_URL_2"; + public static final String CLUSTER_ROLE_1 = "CLUSTER_ROLE_1"; + public static final String CLUSTER_ROLE_2 = "CLUSTER_ROLE_2"; + public static final String SYSTEM_CHILD_LINK_TABLE = "CHILD_LINK"; public static final String SYSTEM_CHILD_LINK_NAME = SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CHILD_LINK_TABLE); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixHAAdmin.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixHAAdmin.java index e1db43ae9f4..dca3c6a4373 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixHAAdmin.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixHAAdmin.java @@ -17,6 +17,8 @@ */ package org.apache.phoenix.jdbc; +import static org.apache.phoenix.jdbc.HighAvailabilityGroup.PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE; + import java.io.Closeable; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -24,9 +26,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Properties; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.atomic.AtomicValue; @@ -35,9 +39,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.util.PairOfSameType; +import org.apache.phoenix.exception.StaleHAGroupStoreRecordVersionException; import org.apache.phoenix.util.JDBCUtil; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,6 +71,14 @@ public static class HighAvailibilityCuratorProvider { public CuratorFramework getCurator(String zkUrl, Properties properties) throws IOException { return HighAvailabilityGroup.getCurator(zkUrl, properties); } + + /** + * Gets curator with specific namespace blocking if necessary to create it + */ + public CuratorFramework getCurator(String zkUrl, Properties properties, String namespace) + throws IOException { + return HighAvailabilityGroup.getCurator(zkUrl, properties, namespace); + } } private static final Logger LOG = LoggerFactory.getLogger(PhoenixHAAdmin.class); @@ -77,17 +91,28 @@ public CuratorFramework getCurator(String zkUrl, Properties properties) throws I private final Properties properties = new Properties(); /** Curator Provider **/ private final HighAvailibilityCuratorProvider highAvailibilityCuratorProvider; + /** Namespace for Curator **/ + private final String namespace; public PhoenixHAAdmin(Configuration conf) { - this(getLocalZkUrl(conf), conf, HighAvailibilityCuratorProvider.INSTANCE); + this(getLocalZkUrl(conf), conf, HighAvailibilityCuratorProvider.INSTANCE, null); + } + + public PhoenixHAAdmin(Configuration conf, String namespace) { + this(getLocalZkUrl(conf), conf, HighAvailibilityCuratorProvider.INSTANCE, namespace); } - public PhoenixHAAdmin(String zkUrl, Configuration conf) { - this(zkUrl, conf, HighAvailibilityCuratorProvider.INSTANCE); + public PhoenixHAAdmin(String zkUrl, Configuration conf, String namespace) { + this(zkUrl, conf, HighAvailibilityCuratorProvider.INSTANCE, namespace); } public PhoenixHAAdmin(String zkUrl, Configuration conf, HighAvailibilityCuratorProvider highAvailibilityCuratorProvider) { + this(zkUrl, conf, highAvailibilityCuratorProvider, null); + } + + public PhoenixHAAdmin(String zkUrl, Configuration conf, + HighAvailibilityCuratorProvider highAvailibilityCuratorProvider, String namespace) { Preconditions.checkNotNull(zkUrl); Preconditions.checkNotNull(conf); Preconditions.checkNotNull(highAvailibilityCuratorProvider); @@ -95,6 +120,7 @@ public PhoenixHAAdmin(String zkUrl, Configuration conf, this.conf = conf; conf.iterator().forEachRemaining(k -> properties.setProperty(k.getKey(), k.getValue())); this.highAvailibilityCuratorProvider = highAvailibilityCuratorProvider; + this.namespace = Objects.toString(namespace, PHOENIX_HA_ZOOKEEPER_ZNODE_NAMESPACE); } /** @@ -130,7 +156,7 @@ public static String getLocalZkUrl(Configuration conf) { * Gets curator from the cache if available otherwise calls into getCurator to make it. */ public CuratorFramework getCurator() throws IOException { - return highAvailibilityCuratorProvider.getCurator(zkUrl, properties); + return highAvailibilityCuratorProvider.getCurator(zkUrl, properties, namespace); } /** @@ -158,20 +184,22 @@ boolean isCurrentActiveCluster(String haGroupName) { } } - /** - * This lists all cluster role records stored in the zookeeper nodes. This read-only operation and - * hence no side effect on the ZK cluster. - */ - public List listAllClusterRoleRecordsOnZookeeper() throws IOException { - List haGroupNames; + public List getHAGroupNames() throws IOException { try { - haGroupNames = getCurator().getChildren().forPath(ZKPaths.PATH_SEPARATOR); + return getCurator().getChildren().forPath(ZKPaths.PATH_SEPARATOR); } catch (Exception e) { String msg = String.format("Got exception when listing all HA groups in %s", zkUrl); LOG.error(msg); throw new IOException(msg, e); } + } + /** + * This lists all cluster role records stored in the zookeeper nodes. This read-only operation and + * hence no side effect on the ZK cluster. + */ + public List listAllClusterRoleRecordsOnZookeeper() throws IOException { + List haGroupNames = getHAGroupNames(); List records = new ArrayList<>(); List failedHaGroups = new ArrayList<>(); for (String haGroupName : haGroupNames) { @@ -445,6 +473,87 @@ public static String toPath(String haGroupName) { return ZKPaths.PATH_SEPARATOR + haGroupName; } + /** + * Creates a new HAGroupStoreRecord in ZooKeeper. + * @param newHAGroupStoreRecord the new record to create + * @throws IOException if any error occurs during the creation + */ + public void createHAGroupStoreRecordInZooKeeper(HAGroupStoreRecord newHAGroupStoreRecord) + throws IOException { + try { + getCurator().create().creatingParentsIfNeeded().forPath( + toPath(newHAGroupStoreRecord.getHaGroupName()), + HAGroupStoreRecord.toJson(newHAGroupStoreRecord)); + } catch (Exception e) { + throw new IOException("Failed to create HAGroupStoreRecord for HA group " + + newHAGroupStoreRecord.getHaGroupName(), e); + } + } + + /** + * Updates the HAGroupStoreRecord in ZooKeeper with version checking. + * @param haGroupName the HA group name + * @param newHAGroupStoreRecord the new record to store + * @param currentStatVersion the expected stat version for optimistic locking + * @throws StaleHAGroupStoreRecordVersionException if the version is stale + * @throws IOException if any other error occurs during the update + */ + public void updateHAGroupStoreRecordInZooKeeper(String haGroupName, + HAGroupStoreRecord newHAGroupStoreRecord, int currentStatVersion) + throws IOException, StaleHAGroupStoreRecordVersionException { + try { + getCurator().setData().withVersion(currentStatVersion).forPath(toPath(haGroupName), + HAGroupStoreRecord.toJson(newHAGroupStoreRecord)); + } catch (KeeperException.BadVersionException e) { + LOG.error("Failed to set HAGroupStoreRecord for HA group {}, stale stat version", haGroupName, + e); + throw new StaleHAGroupStoreRecordVersionException( + "Failed to set HAGroupStoreRecord for HA group " + haGroupName + + " with cached stat version " + currentStatVersion, + e); + } catch (Exception e) { + LOG.error("Failed to set HAGroupStoreRecord for HA group {}, unexpected error", haGroupName, + e); + throw new IOException("Failed to set HAGroupStoreRecord for HA group " + haGroupName, e); + } + } + + /** + * Gets the HAGroupStoreRecord and Stat from ZooKeeper. + * @param haGroupName the HA group name + * @return a pair of HAGroupStoreRecord and Stat + * @throws IOException if any error occurs during the retrieval + */ + public Pair getHAGroupStoreRecordInZooKeeper(String haGroupName) + throws IOException { + try { + byte[] data = getCurator().getData().forPath(toPath(haGroupName)); + HAGroupStoreRecord record = HAGroupStoreRecord.fromJson(data).orElse(null); + Stat stat = getCurator().checkExists().forPath(toPath(haGroupName)); + return Pair.of(record, stat); + } catch (KeeperException.NoNodeException nne) { + LOG.warn("No HAGroupStoreRecord for HA group {} in ZK", haGroupName, nne); + return Pair.of(null, null); + } catch (Exception e) { + LOG.error("Failed to get HAGroupStoreRecord for HA group {}", haGroupName, e); + throw new IOException("Failed to get HAGroupStoreRecord for HA group " + haGroupName, e); + } + } + + /** + * Deletes the HAGroupStoreRecord in ZooKeeper. + * @param haGroupName the HA group name + * @throws IOException if any error occurs during the deletion + */ + public void deleteHAGroupStoreRecordInZooKeeper(String haGroupName) throws IOException { + try { + getCurator().delete().quietly().deletingChildrenIfNeeded().forPath(toPath(haGroupName)); + } catch (Exception e) { + LOG.error("Failed to delete HAGroupStoreRecord for HA group {}", haGroupName, e); + throw new IOException("Failed to delete HAGroupStoreRecord for HA group " + haGroupName, e); + } + } + public String getZkUrl() { return zkUrl; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java index cd141e1182d..60cbe51707e 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java @@ -4101,6 +4101,10 @@ protected String getCDCStreamDDL() { return ddl + ",TTL='" + ttlExpression + "'"; } + protected String getHAGroupDDL() { + return setSystemDDLProperties(QueryConstants.CREATE_HA_GROUP_METADATA); + } + private String setSystemDDLProperties(String ddl) { return String.format(ddl, props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, @@ -4797,6 +4801,15 @@ protected PhoenixConnection upgradeSystemCatalogIfRequired(PhoenixConnection met return metaConnection; } + private PhoenixConnection upgradeSystemHAGroup(PhoenixConnection metaConnection) + throws SQLException { + try (Statement stmt = metaConnection.createStatement()) { + stmt.executeUpdate(getHAGroupDDL()); + } catch (TableAlreadyExistsException ignored) { + } + return metaConnection; + } + /** * There is no other locking needed here since only one connection (on the same or different JVM) * will be able to acquire the upgrade mutex via {@link #acquireUpgradeMutex(long)} . @@ -5006,6 +5019,7 @@ private PhoenixConnection upgradeOtherSystemTablesIfRequired(PhoenixConnection m metaConnection = upgradeSystemMutex(metaConnection); metaConnection = upgradeSystemCDCStreamStatus(metaConnection); metaConnection = upgradeSystemCDCStream(metaConnection); + metaConnection = upgradeSystemHAGroup(metaConnection); // As this is where the most time will be spent during an upgrade, // especially when there are large number of views. diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java index ad5df1e39e2..c613db2ed28 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java @@ -22,6 +22,7 @@ import java.sql.Connection; import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -219,6 +220,10 @@ protected String getCDCStreamDDL() { return setSystemDDLProperties(QueryConstants.CREATE_CDC_STREAM_METADATA); } + protected String getHAGroupDDL() { + return setSystemDDLProperties(QueryConstants.CREATE_HA_GROUP_METADATA); + } + private String setSystemDDLProperties(String ddl) { return String.format(ddl, props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, @@ -496,6 +501,10 @@ public void init(String url, Properties props) throws SQLException { metaConnection.createStatement().executeUpdate(getCDCStreamDDL()); } catch (TableAlreadyExistsException ignore) { } + try (Statement stmt = metaConnection.createStatement()) { + stmt.executeUpdate(getHAGroupDDL()); + } catch (TableAlreadyExistsException ignore) { + } } catch (SQLException e) { sqlE = e; } finally { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java index fe9ea6ef8bf..a78e99a8648 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryConstants.java @@ -30,6 +30,10 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLIENT_IP; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLUSTER_ROLE_1; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLUSTER_ROLE_2; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLUSTER_URL_1; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLUSTER_URL_2; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY; @@ -54,6 +58,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.HA_GROUP_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY; @@ -100,6 +105,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_TABLE_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.POLICY; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.QUERY; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.QUERY_ID; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.QUERY_STATUS; @@ -133,6 +139,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_HA_GROUP_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE; @@ -168,11 +175,14 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USER; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VERSION; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA_TYPE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ZK_URL_1; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ZK_URL_2; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; @@ -551,7 +561,7 @@ enum JoinType { // Non-PK columns NEW_PHYS_TABLE_NAME + " VARCHAR,\n" + TRANSFORM_TYPE + " INTEGER," + LOGICAL_PARENT_NAME + " VARCHAR NULL,\n" + // If this is an index, Logical_Parent_Name is the data table name. - // Index name is not unique. + // Index name is not unique. TRANSFORM_STATUS + " VARCHAR NULL," + TRANSFORM_JOB_ID + " VARCHAR NULL," + TRANSFORM_RETRY_COUNT + " INTEGER NULL," + TRANSFORM_START_TS + " TIMESTAMP NULL," + TRANSFORM_LAST_STATE_TS + " TIMESTAMP NULL," + OLD_METADATA + " VARBINARY NULL,\n" @@ -586,4 +596,26 @@ enum JoinType { + PARENT_PARTITION_ID + "))\n" + HConstants.VERSIONS + "=%s,\n" + ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + TRANSACTIONAL + "=" + Boolean.FALSE + ",\n" + UPDATE_CACHE_FREQUENCY + "=" + "7200000"; + + String CREATE_HA_GROUP_METADATA = + "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_HA_GROUP_TABLE + "\"(\n" + + // PK column + HA_GROUP_NAME + " VARCHAR NOT NULL," + // This is the unique identifier for the HA group + // Non-PK columns + ZK_URL_1 + " VARCHAR," + // This will be used to associate the ClusterRole and URL with local + // or peer cluster + ZK_URL_2 + " VARCHAR," + // This will be used to associate the ClusterRole and URL with local + // or peer cluster + CLUSTER_URL_1 + " VARCHAR," + // This will be returned to the client + CLUSTER_URL_2 + " VARCHAR," + // This will be returned to the client + CLUSTER_ROLE_1 + " VARCHAR," + // Role for peer cluster might not be accurate, we will use + // only local role for recovery if needed + CLUSTER_ROLE_2 + " VARCHAR," + // Role for peer cluster might not be accurate, we will use + // only local role for recovery if needed + POLICY + " VARCHAR," + // There should be only one policy for an HA group + VERSION + " BIGINT," + // Version should be incremented for Admin Updates, only for + // CLUSTER_URLs and REGISTRY_TYPE + "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + HA_GROUP_NAME + "))\n" + + HConstants.VERSIONS + "=%s,\n" + ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + + TRANSACTIONAL + "=" + Boolean.FALSE + ",\n" + UPDATE_CACHE_FREQUENCY + "=" + "7200000"; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java index 00d2608b9bc..596f9efa4b1 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java @@ -148,7 +148,6 @@ public interface QueryServices extends SQLCloseable { "phoenix.coprocessor.maxMetaDataCacheSize"; public static final String MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB = "phoenix.client.maxMetaDataCacheSize"; - public static final String HA_GROUP_NAME_ATTRIB = "phoenix.ha.group"; public static final String AUTO_UPGRADE_WHITELIST_ATTRIB = "phoenix.client.autoUpgradeWhiteList"; // Mainly for testing to force spilling public static final String MAX_MEMORY_SIZE_ATTRIB = "phoenix.query.maxGlobalMemorySize"; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/WALAnnotationUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/WALAnnotationUtil.java index 0c8b434f60e..bf1c4d00384 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/WALAnnotationUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/WALAnnotationUtil.java @@ -19,6 +19,7 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Mutation; +import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; import org.apache.phoenix.execute.MutationState; /** @@ -44,7 +45,7 @@ public static void annotateMutation(Mutation m, byte[] externalSchemaId) { } public static void annotateMutation(Mutation m, byte[] tenantId, byte[] schemaName, - byte[] logicalTableName, byte[] tableType, byte[] ddlTimestamp) { + byte[] logicalTableName, byte[] tableType, byte[] ddlTimestamp, byte[] haGroupName) { if (!m.getDurability().equals(Durability.SKIP_WAL)) { if (tenantId != null) { m.setAttribute(MutationState.MutationMetadataType.TENANT_ID.toString(), tenantId); @@ -54,6 +55,9 @@ public static void annotateMutation(Mutation m, byte[] tenantId, byte[] schemaNa logicalTableName); m.setAttribute(MutationState.MutationMetadataType.TABLE_TYPE.toString(), tableType); m.setAttribute(MutationState.MutationMetadataType.TIMESTAMP.toString(), ddlTimestamp); + if (haGroupName != null) { + m.setAttribute(BaseScannerRegionObserverConstants.HA_GROUP_NAME_ATTRIB, haGroupName); + } } } diff --git a/phoenix-core-client/src/main/protobuf/RegionServerEndpointService.proto b/phoenix-core-client/src/main/protobuf/RegionServerEndpointService.proto index 725f3fc7632..3df27aebaff 100644 --- a/phoenix-core-client/src/main/protobuf/RegionServerEndpointService.proto +++ b/phoenix-core-client/src/main/protobuf/RegionServerEndpointService.proto @@ -51,11 +51,28 @@ message InvalidateServerMetadataCacheRequest { } message InvalidateHAGroupStoreClientRequest { + required bytes haGroupName = 1; + required bool broadcastUpdate = 2; } message InvalidateHAGroupStoreClientResponse { } +message GetClusterRoleRecordRequest { + required bytes haGroupName = 1; +} + +message GetClusterRoleRecordResponse { + // In case of single cluster configuration, one of the urls and roles can be empty + required bytes haGroupName = 1; + required bytes policy = 2; + optional bytes url1 = 3; + optional bytes role1 = 4; + optional bytes url2 = 5; + optional bytes role2 = 6; + optional int64 version = 7; +} + service RegionServerEndpointService { rpc validateLastDDLTimestamp(ValidateLastDDLTimestampRequest) returns (ValidateLastDDLTimestampResponse); @@ -65,4 +82,7 @@ service RegionServerEndpointService { rpc invalidateHAGroupStoreClient(InvalidateHAGroupStoreClientRequest) returns (InvalidateHAGroupStoreClientResponse); + + rpc getClusterRoleRecord(GetClusterRoleRecordRequest) + returns (GetClusterRoleRecordResponse); } \ No newline at end of file diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java index 44441185838..b1e3e974e83 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java @@ -17,6 +17,9 @@ */ package org.apache.phoenix.coprocessor; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; + +import com.google.protobuf.ByteString; import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; import com.google.protobuf.Service; @@ -35,6 +38,7 @@ import org.apache.phoenix.hbase.index.parallel.ThreadPoolBuilder; import org.apache.phoenix.hbase.index.parallel.ThreadPoolManager; import org.apache.phoenix.hbase.index.parallel.WaitForCompletionTaskRunner; +import org.apache.phoenix.jdbc.ClusterRoleRecord; import org.apache.phoenix.jdbc.HAGroupStoreManager; import org.apache.phoenix.protobuf.ProtobufUtil; import org.apache.phoenix.query.QueryServices; @@ -53,6 +57,7 @@ public class PhoenixRegionServerEndpoint extends private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixRegionServerEndpoint.class); private MetricsMetadataCachingSource metricsSource; protected Configuration conf; + private String zkUrl; // regionserver level thread pool used by Uncovered Indexes to scan data table rows private static TaskRunner uncoveredIndexThreadPool; @@ -63,6 +68,7 @@ public void start(CoprocessorEnvironment env) throws IOException { this.metricsSource = MetricsPhoenixCoprocessorSourceFactory.getInstance().getMetadataCachingSource(); initUncoveredIndexThreadPool(this.conf); + this.zkUrl = getLocalZkUrl(conf); } @Override @@ -132,10 +138,15 @@ public void invalidateHAGroupStoreClient(RpcController controller, RegionServerEndpointProtos.InvalidateHAGroupStoreClientRequest request, RpcCallback done) { LOGGER.info("PhoenixRegionServerEndpoint invalidating HAGroupStoreClient"); - HAGroupStoreManager haGroupStoreManager; try { - haGroupStoreManager = HAGroupStoreManager.getInstance(conf); - haGroupStoreManager.invalidateHAGroupStoreClient(); + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(conf); + if (haGroupStoreManager != null) { + haGroupStoreManager.invalidateHAGroupStoreClient(request.getHaGroupName().toStringUtf8(), + request.getBroadcastUpdate()); + } else { + throw new IOException( + "HAGroupStoreManager is null for " + "current cluster, check configuration"); + } } catch (Throwable t) { String errorMsg = "Invalidating HAGroupStoreClient FAILED, check exception for " + "specific details"; @@ -145,6 +156,38 @@ public void invalidateHAGroupStoreClient(RpcController controller, } } + @Override + public void getClusterRoleRecord(RpcController controller, + RegionServerEndpointProtos.GetClusterRoleRecordRequest request, + RpcCallback done) { + try { + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(conf); + if (haGroupStoreManager != null) { + ClusterRoleRecord clusterRoleRecord = + haGroupStoreManager.getClusterRoleRecord(request.getHaGroupName().toStringUtf8()); + RegionServerEndpointProtos.GetClusterRoleRecordResponse.Builder responseBuilder = + RegionServerEndpointProtos.GetClusterRoleRecordResponse.newBuilder(); + responseBuilder.setHaGroupName(request.getHaGroupName()); + responseBuilder.setPolicy(ByteString.copyFromUtf8(clusterRoleRecord.getPolicy().name())); + responseBuilder.setUrl1(ByteString.copyFromUtf8(clusterRoleRecord.getUrl1())); + responseBuilder.setRole1(ByteString.copyFromUtf8(clusterRoleRecord.getRole1().name())); + responseBuilder.setUrl2(ByteString.copyFromUtf8(clusterRoleRecord.getUrl2())); + responseBuilder.setRole2(ByteString.copyFromUtf8(clusterRoleRecord.getRole2().name())); + responseBuilder.setVersion(clusterRoleRecord.getVersion()); + done.run(responseBuilder.build()); + } else { + throw new IOException( + "HAGroupStoreManager is null for " + "current cluster, check configuration"); + } + } catch (Throwable t) { + String errorMsg = + "Getting ClusterRoleRecord FAILED, check exception for " + "specific details"; + LOGGER.error(errorMsg, t); + IOException ioe = ClientUtil.createIOException(errorMsg, t); + ProtobufUtil.setControllerException(controller, ioe); + } + } + @Override public Iterable getServices() { return Collections.singletonList(this); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java index d85f8005405..493add1fa94 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java @@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.controller.InterRegionServerIndexRpcControllerFactory; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -86,7 +85,6 @@ import org.apache.phoenix.index.PhoenixIndexFailurePolicyHelper; import org.apache.phoenix.index.PhoenixIndexFailurePolicyHelper.MutateCommand; import org.apache.phoenix.index.PhoenixIndexMetaData; -import org.apache.phoenix.jdbc.HAGroupStoreManager; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; import org.apache.phoenix.join.HashJoinInfo; @@ -1037,19 +1035,4 @@ public void preClose(ObserverContext c, boolean ab protected boolean isRegionObserverFor(Scan scan) { return scan.getAttribute(BaseScannerRegionObserverConstants.UNGROUPED_AGG) != null; } - - @Override - public void preBatchMutate(ObserverContext c, - MiniBatchOperationInProgress miniBatchOp) throws IOException { - final Configuration conf = c.getEnvironment().getConfiguration(); - try { - final HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(conf); - if (haGroupStoreManager.isMutationBlocked()) { - throw new IOException("Blocking Mutation as Some CRRs are in ACTIVE_TO_STANDBY " - + "state and CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED is true"); - } - } catch (Exception e) { - throw new IOException(e); - } - } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java index 1a4794e86e6..f9daa5694e9 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java @@ -819,9 +819,11 @@ private void annotateDataMutations(UngroupedAggregateRegionObserver.MutationList byte[] tableType = scan.getAttribute(MutationState.MutationMetadataType.TABLE_TYPE.toString()); byte[] ddlTimestamp = scan.getAttribute(MutationState.MutationMetadataType.TIMESTAMP.toString()); + byte[] haGroupName = scan.getAttribute(BaseScannerRegionObserverConstants.HA_GROUP_NAME_ATTRIB); for (Mutation m : mutationsList) { - annotateMutation(m, tenantId, schemaName, logicalTableName, tableType, ddlTimestamp); + annotateMutation(m, tenantId, schemaName, logicalTableName, tableType, ddlTimestamp, + haGroupName); } } 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 5c12fc719e5..f08ff0bcd7d 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 @@ -33,6 +33,7 @@ import java.io.DataInputStream; import java.io.EOFException; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -50,6 +51,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.Predicate; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; @@ -636,10 +638,20 @@ public void preBatchMutate(ObserverContext c, try { final Configuration conf = c.getEnvironment().getConfiguration(); final HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(conf); - if (haGroupStoreManager.isMutationBlocked()) { - throw new MutationBlockedIOException( - "Blocking Mutation as some CRRs " + "are in ACTIVE_TO_STANDBY state and " - + "CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED is true"); + if (haGroupStoreManager == null) { + throw new IOException( + "HAGroupStoreManager is null " + "for current cluster, check configuration"); + } + // Extract HAGroupName from the mutations + final Set haGroupNames = extractHAGroupNameAttribute(miniBatchOp); + // Check if mutation is blocked for any of the HAGroupNames + for (String haGroupName : haGroupNames) { + 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); return; @@ -650,6 +662,19 @@ public void preBatchMutate(ObserverContext c, "Somehow didn't return an index update but also didn't propagate the failure to the client!"); } + private Set + extractHAGroupNameAttribute(MiniBatchOperationInProgress miniBatchOp) { + Set haGroupNames = new HashSet<>(); + 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) { + haGroupNames.add(new String(haGroupName, StandardCharsets.UTF_8)); + } + } + return haGroupNames; + } + @Override public void preWALRestore( org.apache.hadoop.hbase.coprocessor.ObserverContext ctx, @@ -1970,7 +1995,7 @@ public void postBatchMutateIndispensably(ObserverContext postIndexFuture = CompletableFuture.runAsync(() -> doPost(c, context)); long start = EnvironmentEdgeManager.currentTimeMillis(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java index 9109e837f52..624f4047606 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java @@ -66,11 +66,11 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest { private static final Set PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList("SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION", "SYSTEM.MUTEX", "SYSTEM.LOG", "SYSTEM.CHILD_LINK", "SYSTEM.TASK", - "SYSTEM.TRANSFORM", "SYSTEM.CDC_STREAM_STATUS", "SYSTEM.CDC_STREAM")); + "SYSTEM.TRANSFORM", "SYSTEM.CDC_STREAM_STATUS", "SYSTEM.CDC_STREAM", "SYSTEM.HA_GROUP")); private static final Set PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION", "SYSTEM:MUTEX", "SYSTEM:LOG", "SYSTEM:CHILD_LINK", "SYSTEM:TASK", - "SYSTEM:TRANSFORM", "SYSTEM:CDC_STREAM_STATUS", "SYSTEM:CDC_STREAM")); + "SYSTEM:TRANSFORM", "SYSTEM:CDC_STREAM_STATUS", "SYSTEM:CDC_STREAM", "SYSTEM:HA_GROUP")); private static final String SCHEMA_NAME = "MIGRATETEST"; private static final String TABLE_NAME = SCHEMA_NAME + "." + MigrateSystemTablesToSystemNamespaceIT.class.getSimpleName().toUpperCase(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointIT.java index 9e3645efd81..8c6acdc39ca 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointIT.java @@ -156,18 +156,6 @@ public void testValidateLastDDLTimestampWithTenantID() throws SQLException { } } - @Test - public void testInvalidateHAGroupStoreClient() { - HRegionServer regionServer = utility.getMiniHBaseCluster().getRegionServer(0); - PhoenixRegionServerEndpoint coprocessor = getPhoenixRegionServerEndpoint(regionServer); - assertNotNull(coprocessor); - ServerRpcController controller = new ServerRpcController(); - RegionServerEndpointProtos.InvalidateHAGroupStoreClientRequest request = - RegionServerEndpointProtos.InvalidateHAGroupStoreClientRequest.newBuilder().build(); - coprocessor.invalidateHAGroupStoreClient(controller, request, null); - assertFalse(controller.failed()); - } - private String getCreateTableStmt(String tableName) { return "CREATE TABLE " + tableName + " (a_string varchar not null, col1 integer" + " CONSTRAINT pk PRIMARY KEY (a_string)) "; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointITWithConsistentFailover.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointITWithConsistentFailover.java new file mode 100644 index 00000000000..e34513b135b --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointITWithConsistentFailover.java @@ -0,0 +1,172 @@ +/* + * 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.end2end; + +import static org.apache.phoenix.jdbc.HAGroupStoreClient.ZK_CONSISTENT_HA_NAMESPACE; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; + +import com.google.protobuf.RpcCallback; +import java.util.Map; +import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.coprocessor.PhoenixRegionServerEndpoint; +import org.apache.phoenix.coprocessor.generated.RegionServerEndpointProtos; +import org.apache.phoenix.jdbc.ClusterRoleRecord; +import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.jdbc.HAGroupStoreRecord.HAGroupState; +import org.apache.phoenix.jdbc.HighAvailabilityPolicy; +import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility; +import org.apache.phoenix.jdbc.PhoenixHAAdmin; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.util.HAGroupStoreTestUtil; +import org.apache.phoenix.util.ReadOnlyProps; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; + +@Category({ NeedsOwnMiniClusterTest.class }) +public class PhoenixRegionServerEndpointITWithConsistentFailover extends BaseTest { + + private static final Long ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS = 5000L; + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + private String zkUrl; + private String peerZkUrl; + + @Rule + public TestName testName = new TestName(); + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(1); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + CLUSTERS.start(); + } + + @Before + public void setUp() throws Exception { + zkUrl = getLocalZkUrl(config); + peerZkUrl = CLUSTERS.getZkUrl2(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(testName.getMethodName(), zkUrl, + peerZkUrl, ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + } + + @Test + public void testGetClusterRoleRecordAndInvalidate() throws Exception { + String haGroupName = testName.getMethodName(); + HRegionServer regionServer = utility.getMiniHBaseCluster().getRegionServer(0); + PhoenixRegionServerEndpoint coprocessor = getPhoenixRegionServerEndpoint(regionServer); + assertNotNull(coprocessor); + ServerRpcController controller = new ServerRpcController(); + + try (PhoenixHAAdmin peerHAAdmin = new PhoenixHAAdmin( + CLUSTERS.getHBaseCluster2().getConfiguration(), ZK_CONSISTENT_HA_NAMESPACE)) { + HAGroupStoreRecord peerHAGroupStoreRecord = new HAGroupStoreRecord( + HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, HAGroupState.STANDBY); + peerHAAdmin.createHAGroupStoreRecordInZooKeeper(peerHAGroupStoreRecord); + } + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // First getClusterRoleRecord to check if HAGroupStoreClient is working as expected + ClusterRoleRecord expectedRecord = buildExpectedClusterRoleRecord(haGroupName, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY); + executeGetClusterRoleRecordAndVerify(coprocessor, controller, haGroupName, expectedRecord); + + // Change the role of local cluster to ACTIVE_TO_STANDBY in System Table + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(testName.getMethodName(), zkUrl, + peerZkUrl, ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, + ClusterRoleRecord.ClusterRole.STANDBY, null); + + // Cluster Role will still be same as before as cache is not invalidated yet + executeGetClusterRoleRecordAndVerify(coprocessor, controller, haGroupName, expectedRecord); + + // Now Invalidate the Cache + coprocessor.invalidateHAGroupStoreClient(controller, + getInvalidateHAGroupStoreClientRequest(haGroupName), null); + assertFalse(controller.failed()); + + // Local Cluster Role will be updated to ACTIVE_TO_STANDBY as cache is invalidated + ClusterRoleRecord expectedRecordAfterInvalidation = buildExpectedClusterRoleRecord(haGroupName, + ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, ClusterRoleRecord.ClusterRole.STANDBY); + executeGetClusterRoleRecordAndVerify(coprocessor, controller, haGroupName, + expectedRecordAfterInvalidation); + } + + private ClusterRoleRecord buildExpectedClusterRoleRecord(String haGroupName, + ClusterRoleRecord.ClusterRole localRole, ClusterRoleRecord.ClusterRole peerRole) { + return new ClusterRoleRecord(haGroupName, HighAvailabilityPolicy.FAILOVER, zkUrl, localRole, + peerZkUrl, peerRole, 1); + } + + private void executeGetClusterRoleRecordAndVerify(PhoenixRegionServerEndpoint coprocessor, + ServerRpcController controller, String haGroupName, ClusterRoleRecord expectedRecord) { + RpcCallback rpcCallback = + createValidationCallback(haGroupName, expectedRecord); + coprocessor.getClusterRoleRecord(controller, getClusterRoleRecordRequest(haGroupName), + rpcCallback); + assertFalse(controller.failed()); + } + + private RpcCallback + createValidationCallback(String haGroupName, ClusterRoleRecord expectedRecord) { + return response -> { + assertNotNull(response); + ClusterRoleRecord actual = new ClusterRoleRecord(haGroupName, + HighAvailabilityPolicy.valueOf(response.getPolicy().toStringUtf8()), + response.getUrl1().toStringUtf8(), + ClusterRoleRecord.ClusterRole.valueOf(response.getRole1().toStringUtf8()), + response.getUrl2().toStringUtf8(), + ClusterRoleRecord.ClusterRole.valueOf(response.getRole2().toStringUtf8()), + response.getVersion()); + assertEquals(actual, expectedRecord); + }; + } + + private PhoenixRegionServerEndpoint getPhoenixRegionServerEndpoint(HRegionServer regionServer) { + PhoenixRegionServerEndpoint coproc = regionServer.getRegionServerCoprocessorHost() + .findCoprocessor(PhoenixRegionServerEndpoint.class); + return coproc; + } + + private RegionServerEndpointProtos.GetClusterRoleRecordRequest + getClusterRoleRecordRequest(String haGroupName) { + RegionServerEndpointProtos.GetClusterRoleRecordRequest.Builder requestBuilder = + RegionServerEndpointProtos.GetClusterRoleRecordRequest.newBuilder(); + requestBuilder.setHaGroupName(ByteStringer.wrap(Bytes.toBytes(haGroupName))); + return requestBuilder.build(); + } + + private RegionServerEndpointProtos.InvalidateHAGroupStoreClientRequest + getInvalidateHAGroupStoreClientRequest(String haGroupName) { + RegionServerEndpointProtos.InvalidateHAGroupStoreClientRequest.Builder requestBuilder = + RegionServerEndpointProtos.InvalidateHAGroupStoreClientRequest.newBuilder(); + requestBuilder.setBroadcastUpdate(false); + requestBuilder.setHaGroupName(ByteStringer.wrap(Bytes.toBytes(haGroupName))); + return requestBuilder.build(); + } +} diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java index 7d839f4c16c..791f9a476f0 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java @@ -189,6 +189,10 @@ public void testTableMetadataScan() throws SQLException { assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE")); assertTrue(rs.next()); assertEquals(SYSTEM_CATALOG_SCHEMA, rs.getString("TABLE_SCHEM")); + assertEquals(PhoenixDatabaseMetaData.SYSTEM_HA_GROUP_TABLE, rs.getString("TABLE_NAME")); + assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE")); + assertTrue(rs.next()); + assertEquals(SYSTEM_CATALOG_SCHEMA, rs.getString("TABLE_SCHEM")); assertEquals(PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE, rs.getString("TABLE_NAME")); assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE")); assertTrue(rs.next()); @@ -425,7 +429,7 @@ public void testShowTables() throws SQLException { tables.add(rs.getString("TABLE_NAME")); assertEquals("SYSTEM", rs.getString("TABLE_SCHEM")); } - assertEquals(11, tables.size()); + assertEquals(12, tables.size()); assertTrue(tables.contains("CATALOG")); assertTrue(tables.contains("FUNCTION")); @@ -725,10 +729,10 @@ public void testPrimaryKeyMetadataScan() throws SQLException { assertEquals(SchemaUtil.normalizeIdentifier("organization_id"), rs.getString("COLUMN_NAME")); assertEquals(1, rs.getInt("KEY_SEQ")); assertEquals(SchemaUtil.normalizeIdentifier("pk"), rs.getString("PK_NAME")); // TODO: - // this is - // on the - // table - // row + // this is + // on the + // table + // row assertFalse(rs.next()); rs.close(); @@ -893,14 +897,14 @@ public void testCreateOnExistingTable() throws Exception { assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED, cdA.getKeepDeletedCells()); assertNotEquals(ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKSIZE, cdA.getBlocksize()); assertEquals(DataBlockEncoding.NONE, cdA.getDataBlockEncoding()); // Overriden using - // WITH + // WITH assertEquals(1, cdA.getMaxVersions());// Overriden using WITH ColumnFamilyDescriptor cdB = descriptor.getColumnFamily(cfB); // Allow KEEP_DELETED_CELLS to be false for VIEW assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED, cdB.getKeepDeletedCells()); assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKSIZE, cdB.getBlocksize()); assertEquals(DataBlockEncoding.NONE, cdB.getDataBlockEncoding()); // Should keep the - // original value. + // original value. // CF c should stay the same since it's not a Phoenix cf. ColumnFamilyDescriptor cdC = descriptor.getColumnFamily(cfC); assertNotNull("Column family not found", cdC); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java index 819feebcaa4..dfa7fe9bfaf 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java @@ -100,16 +100,15 @@ public class SystemTablesCreationOnConnectionIT { private static final String CREATE_INDEX_STMT = "CREATE INDEX DUMMYIDX ON %s (K1) INCLUDE (K2)"; private static final String UPSERT_STMT = "UPSERT INTO %s VALUES ('A', 'B')"; private static final String QUERY_SYSTEM_CATALOG = "SELECT * FROM SYSTEM.CATALOG LIMIT 1"; - private static final Set PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList("SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION", "SYSTEM.MUTEX", "SYSTEM.LOG", "SYSTEM.CHILD_LINK", "SYSTEM.TASK", - "SYSTEM.TRANSFORM", "SYSTEM.CDC_STREAM_STATUS", "SYSTEM.CDC_STREAM")); + "SYSTEM.TRANSFORM", "SYSTEM.CDC_STREAM_STATUS", "SYSTEM.CDC_STREAM", "SYSTEM.HA_GROUP")); private static final Set PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION", "SYSTEM:MUTEX", "SYSTEM:LOG", "SYSTEM:CHILD_LINK", "SYSTEM:TASK", - "SYSTEM:TRANSFORM", "SYSTEM:CDC_STREAM_STATUS", "SYSTEM:CDC_STREAM")); + "SYSTEM:TRANSFORM", "SYSTEM:CDC_STREAM_STATUS", "SYSTEM:CDC_STREAM", "SYSTEM:HA_GROUP")); private static class PhoenixSysCatCreationServices extends ConnectionQueryServicesImpl { diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java index bf396d25d9a..6ee7fa43584 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java @@ -578,6 +578,9 @@ public void testTableMetadataScan() throws Exception { assertTrue(rs.next()); assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, SYSTEM_FUNCTION_TABLE, SYSTEM); assertTrue(rs.next()); + assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, + PhoenixDatabaseMetaData.SYSTEM_HA_GROUP_TABLE, PTableType.SYSTEM); + assertTrue(rs.next()); assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE, PTableType.SYSTEM); assertTrue(rs.next()); 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 82f49645358..c24e15b2ab7 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 @@ -17,31 +17,33 @@ */ package org.apache.phoenix.end2end.index; -import static org.apache.phoenix.jdbc.PhoenixHAAdmin.toPath; +import static org.apache.phoenix.jdbc.HAGroupStoreClient.ZK_CONSISTENT_HA_NAMESPACE; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; import static org.apache.phoenix.query.QueryServices.CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.sql.Connection; import java.sql.DriverManager; -import java.util.List; import java.util.Map; 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.ClusterRoleRecord; -import org.apache.phoenix.jdbc.HAGroupStoreManager; -import org.apache.phoenix.jdbc.HighAvailabilityPolicy; +import org.apache.phoenix.jdbc.HAGroupStoreRecord; +import org.apache.phoenix.jdbc.HighAvailabilityTestingUtility; +import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixHAAdmin; import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.util.HAGroupStoreTestUtil; import org.apache.phoenix.util.ReadOnlyProps; -import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; @@ -55,7 +57,11 @@ public class IndexRegionObserverMutationBlockingIT extends BaseTest { private static final Long ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS = 1000L; private PhoenixHAAdmin haAdmin; - private HAGroupStoreManager haGroupStoreManager; + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + + @Rule + public TestName testName = new TestName(); @BeforeClass public static synchronized void doSetup() throws Exception { @@ -65,38 +71,27 @@ public static synchronized void doSetup() throws Exception { // No retries so that tests fail faster. props.put("hbase.client.retries.number", "0"); setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + CLUSTERS.start(); } @Before public void setUp() throws Exception { - haAdmin = new PhoenixHAAdmin(config); - haGroupStoreManager = HAGroupStoreManager.getInstance(config); - - // Clean up all existing CRRs before each test - List crrs = haAdmin.listAllClusterRoleRecordsOnZookeeper(); - for (ClusterRoleRecord crr : crrs) { - haAdmin.getCurator().delete().forPath(toPath(crr.getHaGroupName())); - } + haAdmin = new PhoenixHAAdmin(config, ZK_CONSISTENT_HA_NAMESPACE); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - } - - @After - public void tearDown() throws Exception { - // Clean up CRRs after each test - if (haAdmin != null) { - List crrs = haAdmin.listAllClusterRoleRecordsOnZookeeper(); - for (ClusterRoleRecord crr : crrs) { - haAdmin.getCurator().delete().forPath(toPath(crr.getHaGroupName())); - } - } + String zkUrl = getLocalZkUrl(config); + String peerZkUrl = CLUSTERS.getZkUrl2(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(testName.getMethodName(), zkUrl, + peerZkUrl, ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); } @Test public void testMutationBlockedOnDataTableWithIndex() throws Exception { String dataTableName = generateUniqueName(); String indexName = generateUniqueName(); + String haGroupName = testName.getMethodName(); - try (Connection conn = DriverManager.getConnection(getUrl())) { + try (PhoenixConnection conn = (PhoenixConnection) DriverManager.getConnection(getUrl())) { + conn.setHAGroupName(haGroupName); // Create data table and index conn.createStatement().execute( "CREATE TABLE " + dataTableName + " (id VARCHAR PRIMARY KEY, name VARCHAR, age INTEGER)"); @@ -107,19 +102,15 @@ public void testMutationBlockedOnDataTableWithIndex() throws Exception { conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES ('1', 'John', 25)"); conn.commit(); - // Set up CRR that will block mutations (ACTIVE_TO_STANDBY state) - ClusterRoleRecord blockingCrr = - new ClusterRoleRecord("failover_test", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "standby-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(blockingCrr); + // Set up HAGroupStoreRecord that will block mutations (ACTIVE_TO_STANDBY state) + HAGroupStoreRecord haGroupStoreRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, haGroupStoreRecord, -1); // Wait for the event to propagate Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - // Verify that mutations are now blocked - assertTrue("Mutations should be blocked", haGroupStoreManager.isMutationBlocked()); - // Test that UPSERT throws MutationBlockedIOException try { conn.createStatement() @@ -139,21 +130,14 @@ public void testMutationAllowedWhenNotBlocked() throws Exception { String dataTableName = generateUniqueName(); String indexName = generateUniqueName(); - try (Connection conn = DriverManager.getConnection(getUrl())) { + try (PhoenixConnection conn = (PhoenixConnection) DriverManager.getConnection(getUrl())) { + conn.setHAGroupName(testName.getMethodName()); // Create data table and index conn.createStatement().execute( "CREATE TABLE " + dataTableName + " (id VARCHAR PRIMARY KEY, name VARCHAR, age INTEGER)"); conn.createStatement() .execute("CREATE INDEX " + indexName + " ON " + dataTableName + "(name)"); - // Set up CRR in ACTIVE state (should not block) - ClusterRoleRecord nonBlockingCrr = new ClusterRoleRecord("active_test", - HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, - "standby-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(nonBlockingCrr); - - Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - // Mutations should work fine conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES ('1', 'Bob', 35)"); conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES ('2', 'Carol', 27)"); @@ -172,30 +156,24 @@ public void testMutationAllowedWhenNotBlocked() throws Exception { public void testMutationBlockingTransition() throws Exception { String dataTableName = generateUniqueName(); String indexName = generateUniqueName(); + String haGroupName = testName.getMethodName(); - try (Connection conn = DriverManager.getConnection(getUrl())) { + try (PhoenixConnection conn = (PhoenixConnection) DriverManager.getConnection(getUrl())) { + conn.setHAGroupName(testName.getMethodName()); // Create data table and index conn.createStatement().execute( "CREATE TABLE " + dataTableName + " (id VARCHAR PRIMARY KEY, name VARCHAR, age INTEGER)"); conn.createStatement() .execute("CREATE INDEX " + indexName + " ON " + dataTableName + "(name)"); - // Initially set up non-blocking CRR - ClusterRoleRecord crr = new ClusterRoleRecord("transition_test", - HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, - "standby-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr); - Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - // Mutation should work conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES ('1', 'David', 40)"); conn.commit(); - - // Transition to ACTIVE_TO_STANDBY (blocking state) - crr = new ClusterRoleRecord("transition_test", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "standby-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 2L); - haAdmin.createOrUpdateDataOnZookeeper(crr); + // Set up HAGroupStoreRecord that will block mutations (ACTIVE_TO_STANDBY state) + HAGroupStoreRecord haGroupStoreRecord = + new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, haGroupStoreRecord, -1); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); // Now mutations should be blocked @@ -209,10 +187,10 @@ public void testMutationBlockingTransition() throws Exception { } // Transition back to ACTIVE (non-blocking state) and peer cluster is in ATS state - crr = new ClusterRoleRecord("transition_test", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "standby-zk-url", - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, 3L); - haAdmin.createOrUpdateDataOnZookeeper(crr); + // Set up HAGroupStoreRecord that will block mutations (ACTIVE_TO_STANDBY state) + haGroupStoreRecord = new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, + haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, haGroupStoreRecord, -1); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); // Mutations should work again diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreClientIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreClientIT.java index d5b1406c1a0..f5df507c7bf 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreClientIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreClientIT.java @@ -17,12 +17,27 @@ */ package org.apache.phoenix.jdbc; -import static org.apache.phoenix.jdbc.HighAvailabilityGroup.PHOENIX_HA_ZK_SESSION_TIMEOUT_MS_DEFAULT; +import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; +import static org.apache.phoenix.jdbc.HAGroupStoreClient.ZK_CONSISTENT_HA_NAMESPACE; import static org.apache.phoenix.jdbc.HighAvailabilityGroup.PHOENIX_HA_ZK_SESSION_TIMEOUT_MS_KEY; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_HA_GROUP_NAME; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; import static org.apache.phoenix.jdbc.PhoenixHAAdmin.toPath; +import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR; +import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_ZK; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.IOException; +import java.lang.reflect.Field; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -32,15 +47,22 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; -import org.apache.curator.utils.ZKPaths; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.exception.InvalidClusterRoleTransitionException; import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.util.HAGroupStoreTestUtil; import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.zookeeper.data.Stat; +import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; @@ -50,185 +72,334 @@ @Category(NeedsOwnMiniClusterTest.class) public class HAGroupStoreClientIT extends BaseTest { - private final PhoenixHAAdmin haAdmin = new PhoenixHAAdmin(config); private static final Long ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS = 5000L; + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + private PhoenixHAAdmin haAdmin; + private PhoenixHAAdmin peerHaAdmin; + private String zkUrl; + private String peerZKUrl; + + @Rule + public TestName testName = new TestName(); @BeforeClass public static synchronized void doSetup() throws Exception { Map props = Maps.newHashMapWithExpectedSize(1); setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + CLUSTERS.start(); } @Before public void before() throws Exception { - // Clean up all the existing CRRs - List crrs = haAdmin.listAllClusterRoleRecordsOnZookeeper(); - for (ClusterRoleRecord crr : crrs) { - haAdmin.getCurator().delete().forPath(toPath(crr.getHaGroupName())); + haAdmin = new PhoenixHAAdmin(CLUSTERS.getHBaseCluster1().getConfiguration(), + ZK_CONSISTENT_HA_NAMESPACE); + peerHaAdmin = new PhoenixHAAdmin(CLUSTERS.getHBaseCluster2().getConfiguration(), + ZK_CONSISTENT_HA_NAMESPACE); + haAdmin.getCurator().delete().quietly().forPath(toPath(testName.getMethodName())); + peerHaAdmin.getCurator().delete().quietly().forPath(toPath(testName.getMethodName())); + zkUrl = getLocalZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration()); + // Clean existing records in system table + List haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + for (String groupName : haGroupNames) { + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(groupName, zkUrl); } + // Insert a HAGroupStoreRecord into the system table + String haGroupName = testName.getMethodName(); + this.peerZKUrl = CLUSTERS.getZkUrl2(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + } + + @After + public void after() throws Exception { + haAdmin.getCurator().delete().quietly().forPath(toPath(testName.getMethodName())); + peerHaAdmin.getCurator().delete().quietly().forPath(toPath(testName.getMethodName())); + haAdmin.close(); + peerHaAdmin.close(); + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(testName.getMethodName(), zkUrl); + } @Test - public void testHAGroupStoreClientWithSingleCRR() throws Exception { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(config); - ClusterRoleRecord crr = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr); + public void testHAGroupStoreClientWithBothNullZKUrl() throws Exception { + String haGroupName = testName.getMethodName(); + // Clean existing record + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName, zkUrl); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, null, null, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, this.zkUrl); + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + assertNull(haGroupStoreClient); + } + + @Test + public void testHAGroupStoreClientChangingPeerZKUrlToNullUrlToValidUrlToInvalidUrl() + throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreRecord record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null + && currentRecord.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + + // Base case: Check that peerPathChildrenCache is not null in HAGroupStoreClient via reflection + Field peerPathChildrenCache = + HAGroupStoreClient.class.getDeclaredField("peerPathChildrenCache"); + peerPathChildrenCache.setAccessible(true); + assertNotNull(peerPathChildrenCache.get(haGroupStoreClient)); + + // Clean existing record + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName, zkUrl); + // Now update peerZKUrl to null and rebuild + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, null, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + try { + haGroupStoreClient.rebuild(); + fail("Should have thrown NullPointerException"); + } catch (NullPointerException npe) { + // This exception is expected here. + } + + // Now update System table to contain valid peer ZK URL and also change local cluster role to + // STANDBY + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.STANDBY, ClusterRoleRecord.ClusterRole.ACTIVE, null); + haGroupStoreClient.rebuild(); + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertNotNull(currentRecord); + assertEquals(HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY, currentRecord.getHAGroupState()); + + // Check that peerPathChildrenCache is not null now in HAGroupStoreClient via reflection + assertNotNull(peerPathChildrenCache.get(haGroupStoreClient)); + + // Now update local HAGroupStoreRecord to STANDBY to verify that HAGroupStoreClient is working + // as normal + record = new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .isEmpty(); - // Now Update CRR so that current cluster has state ACTIVE_TO_STANDBY - crr = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 2L); - haAdmin.createOrUpdateDataOnZookeeper(crr); + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertNotNull(currentRecord); + assertEquals(HAGroupStoreRecord.HAGroupState.STANDBY, currentRecord.getHAGroupState()); + + // Now update peerZKUrl to invalid but non-null url and rebuild + // This URL can also be considered unreachable url due to a connectivity issue. + String invalidUrl = "invalidURL"; + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, invalidUrl, + ClusterRoleRecord.ClusterRole.STANDBY, ClusterRoleRecord.ClusterRole.ACTIVE, null); + haGroupStoreClient.rebuild(); + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertNotNull(currentRecord); + assertEquals(HAGroupStoreRecord.HAGroupState.STANDBY, currentRecord.getHAGroupState()); + + // Check CRR, the role for peer cluster with invalid url should be UNKNOWN. + ClusterRoleRecord clusterRoleRecord = haGroupStoreClient.getClusterRoleRecord(); + assertNotNull(clusterRoleRecord); + ClusterRoleRecord expected = new ClusterRoleRecord(haGroupName, HighAvailabilityPolicy.FAILOVER, + this.zkUrl, ClusterRoleRecord.ClusterRole.STANDBY, invalidUrl, + ClusterRoleRecord.ClusterRole.UNKNOWN, 1); + assertEquals(expected, clusterRoleRecord); + + // Check that peerPathChildrenCache is null now in HAGroupStoreClient via reflection + assertNull(peerPathChildrenCache.get(haGroupStoreClient)); + + } + + @Test + public void testHAGroupStoreClientWithoutPeerZK() throws Exception { + String haGroupName = testName.getMethodName(); + // Clean existing record + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName, this.zkUrl); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, null, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + HAGroupStoreRecord record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); + assertNull(HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl)); + } + + @Test + public void testHAGroupStoreClientWithSingleHAGroupStoreRecord() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + + // Create and store HAGroupStoreRecord with ACTIVE state + HAGroupStoreRecord record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null + && currentRecord.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + + // Now Update HAGroupStoreRecord so that current cluster has state ACTIVE_TO_STANDBY + record = new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); // Check that now the cluster should be in ActiveToStandby - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).isEmpty(); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .size() == 1; + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null && currentRecord.getHAGroupState() + == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY; // Change it back to ACTIVE so that cluster is not in ACTIVE_TO_STANDBY state - crr = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 3L); - haAdmin.createOrUpdateDataOnZookeeper(crr); + record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .isEmpty(); + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null + && currentRecord.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; // Change it again to ACTIVE_TO_STANDBY so that we can validate watcher works repeatedly - crr = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 4L); - haAdmin.createOrUpdateDataOnZookeeper(crr); + record = new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).isEmpty(); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .size() == 1; + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null && currentRecord.getHAGroupState() + == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY; - // Change peer cluster to ACTIVE_TO_STANDBY so that we can still process mutation on this - // cluster - crr = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, 5L); - haAdmin.createOrUpdateDataOnZookeeper(crr); + // Change it back to ACTIVE to verify transition works + record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .isEmpty(); + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null + && currentRecord.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + } + + /** + * Helper method to create or update HAGroupStoreRecord on ZooKeeper + */ + private void createOrUpdateHAGroupStoreRecordOnZookeeper(PhoenixHAAdmin haAdmin, + String haGroupName, HAGroupStoreRecord record) throws Exception { + String path = toPath(haGroupName); + if (haAdmin.getCurator().checkExists().forPath(path) == null) { + haAdmin.createHAGroupStoreRecordInZooKeeper(record); + } else { + final Pair currentRecord = + haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + if (currentRecord.getRight() != null && currentRecord.getLeft() != null) { + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, record, + currentRecord.getRight().getVersion()); + } else { + throw new IOException( + "Current HAGroupStoreRecord in ZK is null, cannot update HAGroupStoreRecord " + + haGroupName); + } + } } @Test - public void testHAGroupStoreClientWithMultipleCRRs() throws Exception { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(config); - // Setup initial CRRs - ClusterRoleRecord crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - ClusterRoleRecord crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + public void testHAGroupStoreClientWithMultipleHAGroupStoreRecords() throws Exception { + String haGroupName1 = testName.getMethodName() + "1"; + String haGroupName2 = testName.getMethodName() + "2"; + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName1, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName2, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + // Setup initial HAGroupStoreRecords + HAGroupStoreRecord record1 = + new HAGroupStoreRecord("v1.0", haGroupName1, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + HAGroupStoreRecord record2 = + new HAGroupStoreRecord("v1.0", haGroupName2, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName1, record1); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName2, record2); + + HAGroupStoreClient haGroupStoreClient1 = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName1, zkUrl); + HAGroupStoreClient haGroupStoreClient2 = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName2, zkUrl); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 2; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .isEmpty(); - - // Now Update CRR so that current cluster has state ACTIVE_TO_STANDBY for only 1 crr - crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 2L); - crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 2L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + HAGroupStoreRecord currentRecord1 = haGroupStoreClient1.getHAGroupStoreRecord(); + HAGroupStoreRecord currentRecord2 = haGroupStoreClient2.getHAGroupStoreRecord(); + assert currentRecord1 != null + && currentRecord1.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + assert currentRecord2 != null + && currentRecord2.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + + // Now Update HAGroupStoreRecord so that current cluster has state ACTIVE_TO_STANDBY for only 1 + // record + record1 = new HAGroupStoreRecord("v1.0", haGroupName1, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + record2 = + new HAGroupStoreRecord("v1.0", haGroupName2, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName1, record1); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName2, record2); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); // Check that now the cluster should be in ActiveToStandby - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .size() == 1; + currentRecord1 = haGroupStoreClient1.getHAGroupStoreRecord(); + currentRecord2 = haGroupStoreClient2.getHAGroupStoreRecord(); + assert currentRecord1 != null && currentRecord1.getHAGroupState() + == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY; + assert currentRecord2 != null + && currentRecord2.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; // Change it back to ACTIVE so that cluster is not in ACTIVE_TO_STANDBY state - crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 3L); - crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 3L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + record1 = + new HAGroupStoreRecord("v1.0", haGroupName1, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + record2 = + new HAGroupStoreRecord("v1.0", haGroupName2, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); - Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 2; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .isEmpty(); - - // Change other crr to ACTIVE_TO_STANDBY and one in ACTIVE state so that we can validate watcher - // works repeatedly - crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 4L); - crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 4L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName1, record1); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName2, record2); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .size() == 1; - - // Change peer cluster to ACTIVE_TO_STANDBY so that we can still process mutation on this - // cluster - crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, 5L); - crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, 5L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + currentRecord1 = haGroupStoreClient1.getHAGroupStoreRecord(); + currentRecord2 = haGroupStoreClient2.getHAGroupStoreRecord(); + assert currentRecord1 != null + && currentRecord1.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + assert currentRecord2 != null + && currentRecord2.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + + // Change other record to ACTIVE_TO_STANDBY and one in ACTIVE state so that we can validate + // watcher works repeatedly + record1 = + new HAGroupStoreRecord("v1.0", haGroupName1, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + record2 = new HAGroupStoreRecord("v1.0", haGroupName2, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName1, record1); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName2, record2); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 2; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .isEmpty(); + currentRecord1 = haGroupStoreClient1.getHAGroupStoreRecord(); + currentRecord2 = haGroupStoreClient2.getHAGroupStoreRecord(); + assert currentRecord1 != null + && currentRecord1.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + assert currentRecord2 != null && currentRecord2.getHAGroupState() + == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY; } @Test public void testMultiThreadedAccessToHACache() throws Exception { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(config); - // Setup initial CRRs - ClusterRoleRecord crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - ClusterRoleRecord crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + String haGroupName = testName.getMethodName(); + + // Setup initial HAGroupStoreRecord + HAGroupStoreRecord record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); @@ -238,10 +409,9 @@ public void testMultiThreadedAccessToHACache() throws Exception { for (int i = 0; i < threadCount; i++) { executor.submit(() -> { try { - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE) - .size() == 2; - assert haGroupStoreClient - .getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY).isEmpty(); + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null + && currentRecord.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; latch.countDown(); } catch (Exception e) { throw new RuntimeException(e); @@ -250,15 +420,11 @@ public void testMultiThreadedAccessToHACache() throws Exception { } assert latch.await(10, TimeUnit.SECONDS); - // Update CRRs - crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 2L); - crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 2L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + // Update HAGroupStoreRecord + record = new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); @@ -267,10 +433,9 @@ public void testMultiThreadedAccessToHACache() throws Exception { for (int i = 0; i < threadCount; i++) { executor.submit(() -> { try { - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE) - .size() == 1; - assert haGroupStoreClient - .getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY).size() == 1; + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null && currentRecord.getHAGroupState() + == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY; latch2.countDown(); } catch (Exception e) { throw new RuntimeException(e); @@ -282,114 +447,277 @@ public void testMultiThreadedAccessToHACache() throws Exception { @Test public void testHAGroupStoreClientWithRootPathDeletion() throws Exception { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(config); - ClusterRoleRecord crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - ClusterRoleRecord crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + String haGroupName = testName.getMethodName(); + + HAGroupStoreRecord record1 = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record1); + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .size() == 1; + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertNotNull(currentRecord); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, currentRecord.getHAGroupState()); - haAdmin.getCurator().delete().deletingChildrenIfNeeded().forPath(ZKPaths.PATH_SEPARATOR); + haAdmin.getCurator().delete().deletingChildrenIfNeeded().forPath(toPath(StringUtils.EMPTY)); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).isEmpty(); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .isEmpty(); - - crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 2L); - crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 2L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + + // The record should be automatically rebuilt from System Table as it is not in ZK + assertNotNull(currentRecord); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC, + currentRecord.getHAGroupState()); + + record1 = new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC_TO_STANDBY); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record1); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .size() == 1; + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertNotNull(currentRecord); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC_TO_STANDBY, + currentRecord.getHAGroupState()); } @Test public void testThrowsExceptionWithZKDisconnectionAndThenConnection() throws Exception { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(config); - // Setup initial CRRs - ClusterRoleRecord crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - ClusterRoleRecord crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + String haGroupName = testName.getMethodName(); + + // Setup initial HAGroupStoreRecord + HAGroupStoreRecord record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 2; + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null + && currentRecord.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; // Shutdown the ZK Cluster to simulate CONNECTION_SUSPENDED event - utility.shutdownMiniZKCluster(); + CLUSTERS.getHBaseCluster1().shutdownMiniZKCluster(); // Check that immediately after ZK is down, the connection state // should be SUSPENDED (and not LOST), so no exception should be thrown - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 2; + assertNotNull(haGroupStoreClient.getHAGroupStoreRecord()); long sessionTimeout = config.getLong(PHOENIX_HA_ZK_SESSION_TIMEOUT_MS_KEY, - PHOENIX_HA_ZK_SESSION_TIMEOUT_MS_DEFAULT); + HighAvailabilityGroup.PHOENIX_HA_ZK_SESSION_TIMEOUT_MS_DEFAULT); Thread.sleep(sessionTimeout + ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); // Check that HAGroupStoreClient instance is not healthy and throws IOException - assertThrows(IOException.class, - () -> haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE)); + assertThrows(IOException.class, () -> haGroupStoreClient.getHAGroupStoreRecord()); + // Check that the HAGroupStoreClient instance is not healthy via reflection + Field isHealthyField = HAGroupStoreClient.class.getDeclaredField("isHealthy"); + isHealthyField.setAccessible(true); + assertFalse((boolean) isHealthyField.get(haGroupStoreClient)); // Start ZK on the same port to simulate CONNECTION_RECONNECTED event - utility.startMiniZKCluster(1, Integer.parseInt(getZKClientPort(config))); + CLUSTERS.getHBaseCluster1().startMiniZKCluster(1, Integer.parseInt( + CLUSTERS.getHBaseCluster1().getConfiguration().get("hbase.zookeeper.property.clientPort"))); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); // Check that HAGroupStoreClient instance is back to healthy and provides correct response - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 2; + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assert currentRecord != null + && currentRecord.getHAGroupState() == HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC; + // Check that the HAGroupStoreClient instance is healthy via reflection + assertTrue((boolean) isHealthyField.get(haGroupStoreClient)); } + // Tests for setHAGroupStatusIfNeeded method @Test - public void testHAGroupStoreClientWithDifferentZKURLFormats() throws Exception { - HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient.getInstance(config); - final String zkClientPort = getZKClientPort(config); - // Setup initial CRRs - final String format1 = "127.0.0.1\\:" + zkClientPort + "::/hbase"; // 127.0.0.1\:53228::/hbase - final String format2 = "127.0.0.1:" + zkClientPort + "::/hbase"; // 127.0.0.1:53228::/hbase - final String format3 = "127.0.0.1\\:" + zkClientPort + ":/hbase"; // 127.0.0.1\:53228:/hbase - - ClusterRoleRecord crr1 = new ClusterRoleRecord("parallel1", HighAvailabilityPolicy.PARALLEL, - format1, ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - - ClusterRoleRecord crr2 = new ClusterRoleRecord("parallel2", HighAvailabilityPolicy.PARALLEL, - format2, ClusterRoleRecord.ClusterRole.STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr2); - - ClusterRoleRecord crr3 = new ClusterRoleRecord("parallel3", HighAvailabilityPolicy.PARALLEL, - format3, ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr3); + public void testSetHAGroupStatusIfNeededDeleteZKAndSystemTableRecord() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create HAGroupStoreClient without any existing record + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + // Delete the record from ZK + haAdmin.getCurator().delete().deletingChildrenIfNeeded().forPath(toPath(haGroupName)); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE).size() - == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY) - .size() == 1; - assert haGroupStoreClient.getCRRsByClusterRole(ClusterRoleRecord.ClusterRole.STANDBY).size() - == 1; + + // Delete the record from System Table + try ( + PhoenixConnection conn = (PhoenixConnection) DriverManager + .getConnection(JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + zkUrl); + Statement stmt = conn.createStatement()) { + stmt.execute( + "DELETE FROM " + SYSTEM_HA_GROUP_NAME + " WHERE HA_GROUP_NAME = '" + haGroupName + "'"); + conn.commit(); + } + + // This should fail because no record exists in either ZK or System Table + try { + haGroupStoreClient.setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + fail("Expected IOException for missing system table record"); + } catch (IOException e) { + assertTrue("Exception should mention system table", + e.getMessage().contains("Current HAGroupStoreRecordStat in cache is null, " + + "cannot update HAGroupStoreRecord, the record should be initialized in System Table first")); + } + } + + @Test + public void testSetHAGroupStatusIfNeededUpdateExistingRecord() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create initial record + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, initialRecord); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Verify initial state + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, currentRecord.getHAGroupState()); + + // Update to STANDBY (this should succeed as it's a valid transition) + haGroupStoreClient + .setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Verify the record was updated + currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY, + currentRecord.getHAGroupState()); + } + + @Test + public void testSetHAGroupStatusIfNeededNoUpdateWhenNotNeeded() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create initial record with current timestamp + HAGroupStoreRecord initialRecord = new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, initialRecord); + Stat initialRecordInZKStat = haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName).getRight(); + int initialRecordVersion = initialRecordInZKStat.getVersion(); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + + // Get the current record + HAGroupStoreRecord currentRecord = haGroupStoreClient.getHAGroupStoreRecord(); + + // Try to set to ACTIVE_IN_SYNC immediately (should not update due to timing) + haGroupStoreClient.setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + // Add sleep if due to any bug the update might have gone through and we can assert below this. + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + // Verify no update occurred + HAGroupStoreRecord afterRecord = haGroupStoreClient.getHAGroupStoreRecord(); + Stat afterRecordInZKStat = haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName).getRight(); + int afterRecordVersion = afterRecordInZKStat.getVersion(); + + assertEquals(initialRecordVersion, afterRecordVersion); + assertEquals("State should not change", HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC, + afterRecord.getHAGroupState()); + } + + @Test + public void testSetHAGroupStatusIfNeededWithTimingLogic() throws Exception { + String haGroupName = testName.getMethodName(); + // Create initial record + HAGroupStoreRecord initialRecord = new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, initialRecord); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS + DEFAULT_ZK_SESSION_TIMEOUT); + + haGroupStoreClient.setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Verify the record was updated + HAGroupStoreRecord updatedRecord = haGroupStoreClient.getHAGroupStoreRecord(); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, updatedRecord.getHAGroupState()); + } + + @Test + public void testSetHAGroupStatusIfNeededWithInvalidTransition() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create initial record with ACTIVE state + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, initialRecord); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Try to transition to STANDBY (invalid transition from ACTIVE) + try { + haGroupStoreClient.setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.STANDBY); + fail("Expected InvalidClusterRoleTransitionException"); + } catch (InvalidClusterRoleTransitionException e) { + // Should get InvalidClusterRoleTransitionException (might be wrapped) + assertTrue("Exception should be about invalid transition", + e.getMessage().contains("Cannot transition from ACTIVE_IN_SYNC to STANDBY") + || e.getCause() != null && e.getCause().getMessage() + .contains("Cannot transition from ACTIVE_IN_SYNC to STANDBY")); + } + } + + @Test + public void testSetHAGroupStatusIfNeededWithUnhealthyClient() throws Exception { + String haGroupName = testName.getMethodName(); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + + // Make client unhealthy by accessing private field + Field isHealthyField = HAGroupStoreClient.class.getDeclaredField("isHealthy"); + isHealthyField.setAccessible(true); + isHealthyField.set(haGroupStoreClient, false); + + // Try to set status on unhealthy client + try { + haGroupStoreClient.setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + fail("Expected IOException for unhealthy client"); + } catch (IOException e) { + assertTrue("Exception should mention unhealthy client", + e.getMessage().contains("not healthy")); + } + } + + @Test + public void testSetHAGroupStatusIfNeededMultipleTransitions() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create initial record with old timestamp + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, initialRecord); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // First transition: ACTIVE -> ACTIVE_TO_STANDBY + haGroupStoreClient + .setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + HAGroupStoreRecord afterFirst = haGroupStoreClient.getHAGroupStoreRecord(); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY, + afterFirst.getHAGroupState()); + + // Wait and make another transition: ACTIVE_TO_STANDBY -> STANDBY + Thread.sleep(100); // Small delay to ensure timestamp difference + haGroupStoreClient.setHAGroupStatusIfNeeded(HAGroupStoreRecord.HAGroupState.STANDBY); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + HAGroupStoreRecord afterSecond = haGroupStoreClient.getHAGroupStoreRecord(); + assertEquals(HAGroupStoreRecord.HAGroupState.STANDBY, afterSecond.getHAGroupState()); } /** @@ -398,25 +726,30 @@ public void testHAGroupStoreClientWithDifferentZKURLFormats() throws Exception { */ @Test public void testHAGroupStoreClientWithMultiThreadedUpdates() throws Exception { + String haGroupName = testName.getMethodName(); + // Number of threads to execute int threadCount = 5; - // Capture versions of crr in a list(crrEventVersions) in order they are received. - List crrEventVersions = new ArrayList<>(); + // Capture versions of records in a list(recordEventVersions) in order they are received. + List recordEventVersions = new ArrayList<>(); CountDownLatch eventsLatch = new CountDownLatch(threadCount); PathChildrenCacheListener pathChildrenCacheListener = (client, event) -> { if ( event.getData() != null && event.getData().getData() != null - && ClusterRoleRecord.fromJson(event.getData().getData()).isPresent() + && HAGroupStoreRecord.fromJson(event.getData().getData()).isPresent() ) { - ClusterRoleRecord crr = ClusterRoleRecord.fromJson(event.getData().getData()).get(); - crrEventVersions.add((int) crr.getVersion()); - eventsLatch.countDown(); + HAGroupStoreRecord record = HAGroupStoreRecord.fromJson(event.getData().getData()).get(); + if (record.getHaGroupName().equals(haGroupName)) { + recordEventVersions.add(event.getData().getStat().getVersion()); + eventsLatch.countDown(); + } } }; - // Start a new HAGroupStoreClient. - new HAGroupStoreClient(config, pathChildrenCacheListener); + // Start a new HAGroupStoreClient with custom listener. + new HAGroupStoreClient(CLUSTERS.getHBaseCluster1().getConfiguration(), + pathChildrenCacheListener, null, haGroupName, zkUrl); // Create multiple threads for update to ZK. final CountDownLatch updateLatch = new CountDownLatch(threadCount); @@ -426,10 +759,10 @@ public void testHAGroupStoreClientWithMultiThreadedUpdates() throws Exception { List updateList = new ArrayList<>(); // Create a queue which can be polled to send updates to ZK. - ConcurrentLinkedQueue updateQueue = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue updateQueue = new ConcurrentLinkedQueue<>(); for (int i = 0; i < threadCount; i++) { - updateQueue.add(createCRR(i + 1)); - updateList.add(i + 1); + updateQueue.add(createHAGroupStoreRecord(haGroupName)); + updateList.add(i); } // Submit updates to ZK. @@ -437,7 +770,8 @@ public void testHAGroupStoreClientWithMultiThreadedUpdates() throws Exception { executor.submit(() -> { try { synchronized (HAGroupStoreClientIT.class) { - haAdmin.createOrUpdateDataOnZookeeper(Objects.requireNonNull(updateQueue.poll())); + HAGroupStoreRecord record = Objects.requireNonNull(updateQueue.poll()); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, record); } updateLatch.countDown(); } catch (Exception e) { @@ -453,13 +787,207 @@ public void testHAGroupStoreClientWithMultiThreadedUpdates() throws Exception { TimeUnit.MILLISECONDS); // Assert that the order of updates is same as order of events. - assert updateList.equals(crrEventVersions); + assert updateList.equals(recordEventVersions); + } + + @Test + public void testGetClusterRoleRecordNormalCase() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + // Create HAGroupStoreRecord for local cluster + HAGroupStoreRecord localRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, localRecord); + + // Create HAGroupStoreRecord for peer cluster + HAGroupStoreRecord peerRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY); + createOrUpdateHAGroupStoreRecordOnZookeeper(peerHaAdmin, haGroupName, peerRecord); + + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + assertNotNull(haGroupStoreClient); + + // Test getClusterRoleRecord + ClusterRoleRecord clusterRoleRecord = haGroupStoreClient.getClusterRoleRecord(); + assertNotNull(clusterRoleRecord); + ClusterRoleRecord expectedClusterRoleRecord = + new ClusterRoleRecord(haGroupName, HighAvailabilityPolicy.FAILOVER, zkUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, peerZKUrl, ClusterRoleRecord.ClusterRole.STANDBY, 1); + assertEquals(expectedClusterRoleRecord, clusterRoleRecord); } - private ClusterRoleRecord createCRR(Integer version) { - return new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - version); + @Test + public void testGetClusterRoleRecordWithValidPeerZKUrlButNoPeerRecord() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + // Create HAGroupStoreRecord for local cluster only (no peer record) + HAGroupStoreRecord localRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + createOrUpdateHAGroupStoreRecordOnZookeeper(haAdmin, haGroupName, localRecord); + + // Explicitly ensure no peer record exists + String peerPath = toPath(haGroupName); + if (peerHaAdmin.getCurator().checkExists().forPath(peerPath) != null) { + peerHaAdmin.getCurator().delete().quietly().forPath(peerPath); + } + + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + HAGroupStoreClient haGroupStoreClient = HAGroupStoreClient + .getInstanceForZkUrl(CLUSTERS.getHBaseCluster1().getConfiguration(), haGroupName, zkUrl); + assertNotNull(haGroupStoreClient); + + // Test getClusterRoleRecord when peer ZK URL is valid but no peer HAGroupStoreRecord exists in + // peer ZK + ClusterRoleRecord clusterRoleRecord = haGroupStoreClient.getClusterRoleRecord(); + assertNotNull(clusterRoleRecord); + ClusterRoleRecord expectedClusterRoleRecord = new ClusterRoleRecord(haGroupName, + HighAvailabilityPolicy.FAILOVER, zkUrl, ClusterRoleRecord.ClusterRole.ACTIVE, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.UNKNOWN, 1); + assertEquals(expectedClusterRoleRecord, clusterRoleRecord); + } + + private HAGroupStoreRecord createHAGroupStoreRecord(String haGroupName) { + return new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + } + + // Tests for getHAGroupNames static method + @Test + public void testGetHAGroupNamesWithSingleGroup() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + + List haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + + assertNotNull("HA group names list should not be null", haGroupNames); + assertTrue("HA group names list should contain the test group", + haGroupNames.contains(haGroupName)); + + // Clean up + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName, zkUrl); + } + + @Test + public void testGetHAGroupNamesWithMultipleGroups() throws Exception { + // Delete old HA entry created by Before method + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(testName.getMethodName(), this.zkUrl); + // Verify that there are no groups returned + assertEquals(0, HAGroupStoreClient.getHAGroupNames(zkUrl).size()); + + String haGroupName1 = testName.getMethodName() + "_1"; + String haGroupName2 = testName.getMethodName() + "_2"; + String haGroupName3 = testName.getMethodName() + "_3"; + String haGroupName4 = testName.getMethodName() + "_4"; + + // Insert multiple HA group records + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName1, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName2, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.STANDBY, ClusterRoleRecord.ClusterRole.ACTIVE, null); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName3, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName4, "bad_zk_url", + this.peerZKUrl, ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, + this.zkUrl); + + List haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + + assertNotNull("HA group names list should not be null", haGroupNames); + assertTrue("HA group names list should contain haGroupName1", + haGroupNames.contains(haGroupName1)); + assertTrue("HA group names list should contain haGroupName2", + haGroupNames.contains(haGroupName2)); + assertTrue("HA group names list should contain haGroupName3", + haGroupNames.contains(haGroupName3)); + assertTrue("HA group names list should not contain haGroupName4", + !haGroupNames.contains(haGroupName4)); + assertEquals(3, haGroupNames.size()); + + // Clean up + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName1, zkUrl); + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName2, zkUrl); + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName3, zkUrl); + } + + @Test + public void testGetHAGroupNamesWithEmptyTable() throws Exception { + // First, delete any existing records that might interfere + List existingGroups = HAGroupStoreClient.getHAGroupNames(zkUrl); + for (String groupName : existingGroups) { + if (groupName.startsWith("test")) { // Only clean up test groups + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(groupName, zkUrl); + } + } + + // Verify the table is empty of test records + List haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + long testGroupCount = haGroupNames.stream().filter(name -> name.startsWith("test")).count(); + assertEquals("Should have no test groups", 0, testGroupCount); + } + + @Test + public void testGetHAGroupNamesAfterDeletingGroups() throws Exception { + String haGroupName1 = testName.getMethodName() + "_delete_1"; + String haGroupName2 = testName.getMethodName() + "_delete_2"; + + // Insert HA group records + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName1, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName2, this.zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.STANDBY, ClusterRoleRecord.ClusterRole.ACTIVE, null); + + // Verify both groups exist + List haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + assertTrue("Should contain haGroupName1", haGroupNames.contains(haGroupName1)); + assertTrue("Should contain haGroupName2", haGroupNames.contains(haGroupName2)); + + // Delete one group + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName1, zkUrl); + + // Verify only one group remains + haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + assertFalse("Should not contain deleted haGroupName1", haGroupNames.contains(haGroupName1)); + assertTrue("Should still contain haGroupName2", haGroupNames.contains(haGroupName2)); + + // Clean up remaining group + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName2, zkUrl); + + // Verify the group is gone + haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + assertFalse("Should not contain deleted haGroupName2", haGroupNames.contains(haGroupName2)); + } + + @Test + public void testGetHAGroupNamesWithInvalidZkUrl() throws Exception { + String invalidZkUrl = "invalid:2181"; + + try { + HAGroupStoreClient.getHAGroupNames(invalidZkUrl); + fail("Expected SQLException for invalid ZK URL"); + } catch (SQLException e) { + // Expected - the connection should fail with invalid ZK URL + assertNotNull("SQLException should have a message", e.getMessage()); + } + } + + @Test + public void testGetHAGroupNamesWithNullZkUrl() throws Exception { + try { + HAGroupStoreClient.getHAGroupNames(null); + fail("Expected SQLException for null ZK URL"); + } catch (SQLException e) { + // Expected - the connection should fail with null ZK URL + assertNotNull("SQLException should have a message", e.getMessage()); + } } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreManagerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreManagerIT.java index 778e97f4e1b..a4ae0aeb255 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreManagerIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HAGroupStoreManagerIT.java @@ -17,71 +17,343 @@ */ package org.apache.phoenix.jdbc; +import static org.apache.phoenix.jdbc.HAGroupStoreClient.ZK_CONSISTENT_HA_NAMESPACE; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.getLocalZkUrl; import static org.apache.phoenix.jdbc.PhoenixHAAdmin.toPath; import static org.apache.phoenix.query.QueryServices.CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.util.HAGroupStoreTestUtil; import org.apache.phoenix.util.ReadOnlyProps; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; +/** + * Integration tests for {@link HAGroupStoreManager}. + */ @Category(NeedsOwnMiniClusterTest.class) public class HAGroupStoreManagerIT extends BaseTest { - private final PhoenixHAAdmin haAdmin = new PhoenixHAAdmin(config); - private static final Long ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS = 1000L; + + @Rule + public TestName testName = new TestName(); + + private PhoenixHAAdmin haAdmin; + private static final Long ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS = 2000L; + private String zkUrl; + private String peerZKUrl; + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); @BeforeClass public static synchronized void doSetup() throws Exception { - Map props = Maps.newHashMapWithExpectedSize(1); + Map props = Maps.newHashMapWithExpectedSize(2); props.put(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, "true"); setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + CLUSTERS.start(); } @Before public void before() throws Exception { - // Clean up all the existing CRRs - List crrs = haAdmin.listAllClusterRoleRecordsOnZookeeper(); - for (ClusterRoleRecord crr : crrs) { - haAdmin.getCurator().delete().forPath(toPath(crr.getHaGroupName())); + haAdmin = new PhoenixHAAdmin(config, ZK_CONSISTENT_HA_NAMESPACE); + zkUrl = getLocalZkUrl(config); + this.peerZKUrl = CLUSTERS.getZkUrl2(); + + // Clean up existing HAGroupStoreRecords + try { + List haGroupNames = HAGroupStoreClient.getHAGroupNames(zkUrl); + for (String haGroupName : haGroupNames) { + haAdmin.getCurator().delete().quietly().forPath(toPath(haGroupName)); + } + } catch (Exception e) { + // Ignore cleanup errors } + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(testName.getMethodName(), zkUrl, + peerZKUrl, ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); } @Test - public void testHAGroupStoreManagerWithSingleCRR() throws Exception { + public void testMutationBlockingWithSingleHAGroup() throws Exception { + String haGroupName = testName.getMethodName(); HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); - // Setup initial CRRs - ClusterRoleRecord crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - ClusterRoleRecord crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, - haAdmin.getZkUrl(), ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 1L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + // Initially no mutation should be blocked + assertFalse(haGroupStoreManager.isMutationBlocked(haGroupName)); + + // Update to ACTIVE_TO_STANDBY role (should block mutations) + HAGroupStoreRecord transitionRecord = new HAGroupStoreRecord("1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, transitionRecord, 0); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assertFalse(haGroupStoreManager.isMutationBlocked()); + // Now mutations should be blocked + assertTrue(haGroupStoreManager.isMutationBlocked(haGroupName)); + } + + @Test + public void testMutationBlockingWithMultipleHAGroups() throws Exception { + String haGroupName1 = testName.getMethodName() + "_1"; + String haGroupName2 = testName.getMethodName() + "_2"; + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); + + // Create two HA groups with ACTIVE and ACTIVE_NOT_IN_SYNC roles + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName1, zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.ACTIVE, null); + HAGroupStoreRecord activeRecord1 = + new HAGroupStoreRecord("1.0", haGroupName1, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + haAdmin.createHAGroupStoreRecordInZooKeeper(activeRecord1); + + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName2, zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.ACTIVE, null); + + // No mutations should be blocked + assertFalse(haGroupStoreManager.isMutationBlocked(haGroupName1)); + assertFalse(haGroupStoreManager.isMutationBlocked(haGroupName2)); + + // Update only second group to ACTIVE_NOT_IN_SYNC_TO_STANDBY + HAGroupStoreRecord transitionRecord2 = new HAGroupStoreRecord("1.0", haGroupName2, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC_TO_STANDBY); + + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName2, transitionRecord2, 0); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Global mutations should be blocked due to second group + assertFalse(haGroupStoreManager.isMutationBlocked(haGroupName1)); + assertTrue(haGroupStoreManager.isMutationBlocked(haGroupName2)); + } + + @Test + public void testGetHAGroupStoreRecord() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); + + // Get record from HAGroupStoreManager + Optional recordOpt = haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + // Should be present + assertTrue(recordOpt.isPresent()); + + // Delete record from System Table + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupName, zkUrl); + // Delete record from ZK + haAdmin.deleteHAGroupStoreRecordInZooKeeper(haGroupName); + // Sleep for propagation time so that it is now reflected in cache. + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + // Get record from HAGroupStoreManager + recordOpt = haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + // Should not be present + assertFalse(recordOpt.isPresent()); + + // Create record again in System Table + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, zkUrl, this.peerZKUrl, + ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, null); + // Now it should be present + Optional retrievedOpt = + haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + assertTrue(retrievedOpt.isPresent()); + + // Record for comparison + HAGroupStoreRecord record = new HAGroupStoreRecord(HAGroupStoreRecord.DEFAULT_PROTOCOL_VERSION, + haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC); + + // Complete object comparison instead of field-by-field + assertEquals(record, retrievedOpt.get()); + } + + @Test + public void testInvalidateHAGroupStoreClient() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); + + // Create a HAGroupStoreRecord first + HAGroupStoreRecord record = + new HAGroupStoreRecord("1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(record); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Ensure we can get the record + Optional recordOpt = haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + assertTrue(recordOpt.isPresent()); + + // Invalidate the specific HA group client + haGroupStoreManager.invalidateHAGroupStoreClient(haGroupName, false); + + // Should still be able to get the record after invalidation + recordOpt = haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + assertTrue(recordOpt.isPresent()); + + // Test global invalidation + haGroupStoreManager.invalidateHAGroupStoreClient(false); + + // Should still be able to get the record after global invalidation + recordOpt = haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + assertTrue(recordOpt.isPresent()); + } + + @Test + public void testMutationBlockDisabled() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create configuration with mutation block disabled + Configuration conf = new Configuration(); + conf.set(CLUSTER_ROLE_BASED_MUTATION_BLOCK_ENABLED, "false"); + conf.set(HConstants.ZOOKEEPER_QUORUM, getLocalZkUrl(config)); + + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); - crr1 = new ClusterRoleRecord("failover", HighAvailabilityPolicy.FAILOVER, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE, "random-zk-url", ClusterRoleRecord.ClusterRole.STANDBY, - 2L); - crr2 = new ClusterRoleRecord("parallel", HighAvailabilityPolicy.PARALLEL, haAdmin.getZkUrl(), - ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, "random-zk-url", - ClusterRoleRecord.ClusterRole.STANDBY, 2L); - haAdmin.createOrUpdateDataOnZookeeper(crr1); - haAdmin.createOrUpdateDataOnZookeeper(crr2); + // Create HAGroupStoreRecord with ACTIVE_TO_STANDBY role + HAGroupStoreRecord transitionRecord = new HAGroupStoreRecord("1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + haAdmin.createHAGroupStoreRecordInZooKeeper(transitionRecord); Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); - assert haGroupStoreManager.isMutationBlocked(); + // Mutations should not be blocked even with ACTIVE_TO_STANDBY role + assertFalse(haGroupStoreManager.isMutationBlocked(haGroupName)); } + + @Test + public void testSetHAGroupStatusToStoreAndForward() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); + + // Create an initial HAGroupStoreRecord with ACTIVE status + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Set the HA group status to store and forward (ACTIVE_NOT_IN_SYNC) + haGroupStoreManager.setHAGroupStatusToStoreAndForward(haGroupName); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Verify the status was updated to ACTIVE_NOT_IN_SYNC + Optional updatedRecordOpt = + haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + assertTrue(updatedRecordOpt.isPresent()); + HAGroupStoreRecord updatedRecord = updatedRecordOpt.get(); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC, + updatedRecord.getHAGroupState()); + } + + @Test + public void testSetHAGroupStatusRecordToSync() throws Exception { + String haGroupName = testName.getMethodName(); + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); + + // Create an initial HAGroupStoreRecord with ACTIVE_NOT_IN_SYNC status + HAGroupStoreRecord initialRecord = new HAGroupStoreRecord("1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Set the HA group status to sync (ACTIVE) + haGroupStoreManager.setHAGroupStatusRecordToSync(haGroupName); + Thread.sleep(ZK_CURATOR_EVENT_PROPAGATION_TIMEOUT_MS); + + // Verify the status was updated to ACTIVE + Optional updatedRecordOpt = + haGroupStoreManager.getHAGroupStoreRecord(haGroupName); + assertTrue(updatedRecordOpt.isPresent()); + HAGroupStoreRecord updatedRecord = updatedRecordOpt.get(); + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE, updatedRecord.getClusterRole()); + } + + @Test + public void testGetHAGroupNamesFiltersCorrectlyByZkUrl() throws Exception { + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); + + List initialHAGroupNames = haGroupStoreManager.getHAGroupNames(); + + // Create HA groups with current zkUrl as ZK_URL_1 + String haGroupWithCurrentZkUrl = testName.getMethodName() + "_current_zk"; + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupWithCurrentZkUrl, zkUrl, + this.peerZKUrl, ClusterRoleRecord.ClusterRole.ACTIVE, ClusterRoleRecord.ClusterRole.STANDBY, + null); + + // Create HA group with current zkUrl as ZK_URL_2 (swapped) + String haGroupWithCurrentZkUrlAsPeer = testName.getMethodName() + "_current_as_peer"; + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupWithCurrentZkUrlAsPeer, + this.peerZKUrl, zkUrl, ClusterRoleRecord.ClusterRole.STANDBY, + ClusterRoleRecord.ClusterRole.ACTIVE, zkUrl); + + // Create HA group with different zkUrl (should not appear in results) + String differentZkUrl = "localhost:2182:/different"; + String haGroupWithDifferentZkUrl = testName.getMethodName() + "_different_zk"; + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupWithDifferentZkUrl, differentZkUrl, + "localhost:2183:/other", ClusterRoleRecord.ClusterRole.ACTIVE, + ClusterRoleRecord.ClusterRole.STANDBY, zkUrl); + + // Get HA group names - should only return groups where current zkUrl matches ZK_URL_1 or + // ZK_URL_2 + List filteredHAGroupNames = haGroupStoreManager.getHAGroupNames(); + + // Extract only new groups from filteredHAGroupNames + List newHAGroupNames = filteredHAGroupNames.stream() + .filter(name -> !initialHAGroupNames.contains(name)).collect(Collectors.toList()); + + // Check size of filteredHAGroupNames + assertEquals(2, newHAGroupNames.size()); + + // Should contain groups where current zkUrl is involved + assertTrue("Should contain HA group with current zkUrl as ZK_URL_1", + newHAGroupNames.contains(haGroupWithCurrentZkUrl)); + assertTrue("Should contain HA group with current zkUrl as ZK_URL_2", + newHAGroupNames.contains(haGroupWithCurrentZkUrlAsPeer)); + // Should NOT contain HA group with different zkUrl + assertFalse("Should NOT contain HA group with different zkUrl", + newHAGroupNames.contains(haGroupWithDifferentZkUrl)); + + // Clean up + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupWithCurrentZkUrl, zkUrl); + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupWithCurrentZkUrlAsPeer, zkUrl); + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupWithDifferentZkUrl, zkUrl); + } + + @Test + public void testGetHAGroupNamesWhenNoMatchingZkUrl() throws Exception { + HAGroupStoreManager haGroupStoreManager = HAGroupStoreManager.getInstance(config); + + // Clean up existing HA group created in before() + String testHAGroupName = testName.getMethodName(); + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(testHAGroupName, zkUrl); + + // Create HA groups with completely different zkUrls + String differentZkUrl1 = "localhost:2182:/different1"; + String differentZkUrl2 = "localhost:2183:/different2"; + String haGroupWithDifferentZkUrls = testName.getMethodName() + "_different"; + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupWithDifferentZkUrls, + differentZkUrl1, differentZkUrl2, ClusterRoleRecord.ClusterRole.ACTIVE, + ClusterRoleRecord.ClusterRole.STANDBY, zkUrl); + + // Get HA group names - should not contain the group with different zkUrls + List filteredHAGroupNames = haGroupStoreManager.getHAGroupNames(); + + // Should NOT contain the HA group with different zkUrls + assertFalse("Should NOT contain HA group with different zkUrls", + filteredHAGroupNames.contains(haGroupWithDifferentZkUrls)); + + // Clean up + HAGroupStoreTestUtil.deleteHAGroupRecordInSystemTable(haGroupWithDifferentZkUrls, zkUrl); + + } + } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupTestIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupTestIT.java index e35ea8e5d23..cec15ffaab9 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupTestIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityGroupTestIT.java @@ -299,8 +299,11 @@ public void testNegativeCacheWhenMissingClusterRoleRecords() throws Exception { // Make ZK connectable and the cluster role record be missing CuratorFramework curator = mock(CuratorFramework.class); when(curator.blockUntilConnected(anyInt(), any(TimeUnit.class))).thenReturn(true); - HighAvailabilityGroup.CURATOR_CACHE.put(ZK2, curator); - HighAvailabilityGroup.CURATOR_CACHE.put(ZK1, curator); + // Use proper cache keys with default namespace + HighAvailabilityGroup.CURATOR_CACHE.put(HighAvailabilityGroup.generateCacheKey(ZK2, null), + curator); + HighAvailabilityGroup.CURATOR_CACHE.put(HighAvailabilityGroup.generateCacheKey(ZK1, null), + curator); ExistsBuilder eb = mock(ExistsBuilder.class); when(eb.forPath(anyString())).thenReturn(null); 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 655cbbcfaab..6f59ef3f9d8 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 @@ -86,6 +86,7 @@ import org.apache.phoenix.query.ConnectionQueryServices; import org.apache.phoenix.query.ConnectionQueryServicesImpl; import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.util.HAGroupStoreTestUtil; import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.QueryUtil; import org.junit.AfterClass; @@ -134,8 +135,8 @@ public static void setUpBeforeClass() throws Exception { GLOBAL_PROPERTIES.setProperty(QueryServices.COLLECT_REQUEST_LEVEL_METRICS, String.valueOf(true)); GLOBAL_PROPERTIES.setProperty(QueryServices.LOG_LEVEL, LogLevel.DEBUG.name()); // Need logging - // for query - // metrics + // for query + // metrics GLOBAL_PROPERTIES.setProperty(CQSI_THREAD_POOL_ENABLED, String.valueOf(true)); GLOBAL_PROPERTIES.setProperty(CQSI_THREAD_POOL_KEEP_ALIVE_SECONDS, String.valueOf(13)); GLOBAL_PROPERTIES.setProperty(CQSI_THREAD_POOL_CORE_POOL_SIZE, String.valueOf(17)); @@ -380,9 +381,18 @@ public void testCluster1OfflineRole() throws Exception { */ @Test public void testBothClusterATSRole() throws Exception { - CLUSTERS.transitClusterRole(haGroup, ClusterRole.ACTIVE_TO_STANDBY, - ClusterRole.ACTIVE_TO_STANDBY); - try (Connection conn = getParallelConnection()) { + String zkUrl1 = CLUSTERS.getZkUrl1(); + String zkUrl2 = CLUSTERS.getZkUrl2(); + String haGroupName = testName.getMethodName(); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, zkUrl1, zkUrl2, + ClusterRole.ACTIVE_TO_STANDBY, ClusterRole.ACTIVE_TO_STANDBY, null); + HAGroupStoreTestUtil.upsertHAGroupRecordInSystemTable(haGroupName, zkUrl2, zkUrl1, + ClusterRole.ACTIVE_TO_STANDBY, ClusterRole.ACTIVE_TO_STANDBY, null); + try (ParallelPhoenixConnection conn = (ParallelPhoenixConnection) getParallelConnection()) { + PhoenixConnection conn1 = conn.futureConnection1.get(); + PhoenixConnection conn2 = conn.futureConnection2.get(); + conn1.setHAGroupName(haGroupName); + conn2.setHAGroupName(haGroupName); doTestBasicOperationsWithConnection(conn, tableName, haGroupName); fail("Expected MutationBlockedIOException to be thrown"); } catch (SQLException e) { diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/PhoenixHAAdminIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/PhoenixHAAdminIT.java new file mode 100644 index 00000000000..79ae44203f8 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/PhoenixHAAdminIT.java @@ -0,0 +1,444 @@ +/* + * 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.jdbc.HAGroupStoreClient.ZK_CONSISTENT_HA_NAMESPACE; +import static org.apache.phoenix.jdbc.PhoenixHAAdmin.toPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.exception.StaleHAGroupStoreRecordVersionException; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; + +/** + * Integration tests for {@link PhoenixHAAdmin} HAGroupStoreRecord operations + */ +@Category(NeedsOwnMiniClusterTest.class) +public class PhoenixHAAdminIT extends BaseTest { + + private static final HighAvailabilityTestingUtility.HBaseTestingUtilityPair CLUSTERS = + new HighAvailabilityTestingUtility.HBaseTestingUtilityPair(); + private PhoenixHAAdmin haAdmin; + private PhoenixHAAdmin peerHaAdmin; + + @Rule + public TestName testName = new TestName(); + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(1); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + CLUSTERS.start(); + } + + @Before + public void before() throws Exception { + haAdmin = new PhoenixHAAdmin(CLUSTERS.getHBaseCluster1().getConfiguration(), + ZK_CONSISTENT_HA_NAMESPACE); + peerHaAdmin = new PhoenixHAAdmin(CLUSTERS.getHBaseCluster2().getConfiguration(), + ZK_CONSISTENT_HA_NAMESPACE); + cleanupTestZnodes(); + } + + @After + public void after() throws Exception { + cleanupTestZnodes(); + if (haAdmin != null) { + haAdmin.close(); + } + if (peerHaAdmin != null) { + peerHaAdmin.close(); + } + } + + private void cleanupTestZnodes() throws Exception { + haAdmin.getCurator().delete().quietly().forPath(toPath(testName.getMethodName())); + peerHaAdmin.getCurator().delete().quietly().forPath(toPath(testName.getMethodName())); + } + + @Test + public void testCreateHAGroupStoreRecordInZooKeeper() throws Exception { + String haGroupName = testName.getMethodName(); + String peerZKUrl = CLUSTERS.getZkUrl2(); + + HAGroupStoreRecord record = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + // Create the record in ZooKeeper + haAdmin.createHAGroupStoreRecordInZooKeeper(record); + + // Verify the record was created by reading it back + byte[] data = haAdmin.getCurator().getData().forPath(toPath(haGroupName)); + HAGroupStoreRecord savedRecord = HAGroupStoreRecord.fromJson(data).get(); + + assertEquals(record.getHaGroupName(), savedRecord.getHaGroupName()); + assertEquals(record.getClusterRole(), savedRecord.getClusterRole()); + } + + @Test + public void testCreateHAGroupStoreRecordInZooKeeperWithExistingNode() throws Exception { + String haGroupName = testName.getMethodName(); + + HAGroupStoreRecord record1 = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + // Create the first record + haAdmin.createHAGroupStoreRecordInZooKeeper(record1); + + // Try to create again with different data + HAGroupStoreRecord record2 = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY); + + // This should throw an exception due to NodeExistsException handling + try { + haAdmin.createHAGroupStoreRecordInZooKeeper(record2); + fail("Expected NodeExistsException"); + } catch (IOException e) { + // Expected exception + assertTrue(e.getCause() instanceof KeeperException.NodeExistsException); + assertTrue(e.getMessage().contains("Failed to create HAGroupStoreRecord for HA group")); + } + + // Verify the original record is still there (not overwritten) + byte[] data = haAdmin.getCurator().getData().forPath(toPath(haGroupName)); + HAGroupStoreRecord savedRecord = HAGroupStoreRecord.fromJson(data).get(); + + assertEquals(record1.getHAGroupState(), savedRecord.getHAGroupState()); + assertEquals(record1.getProtocolVersion(), savedRecord.getProtocolVersion()); + assertEquals(record1.getHaGroupName(), savedRecord.getHaGroupName()); + } + + @Test + public void testUpdateHAGroupStoreRecordInZooKeeper() throws Exception { + String haGroupName = testName.getMethodName(); + String peerZKUrl = CLUSTERS.getZkUrl2(); + + // Create initial record + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + + // Get the current stat for version checking + Stat stat = haAdmin.getCurator().checkExists().forPath(toPath(haGroupName)); + assertNotNull(stat); + int currentVersion = stat.getVersion(); + + // Update the record + HAGroupStoreRecord updatedRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY); + + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, updatedRecord, currentVersion); + + // Verify the record was updated + byte[] data = haAdmin.getCurator().getData().forPath(toPath(haGroupName)); + HAGroupStoreRecord savedRecord = HAGroupStoreRecord.fromJson(data).get(); + + assertEquals(updatedRecord.getClusterRole(), savedRecord.getClusterRole()); + assertEquals(updatedRecord.getHAGroupState(), savedRecord.getHAGroupState()); + assertEquals(updatedRecord.getProtocolVersion(), savedRecord.getProtocolVersion()); + assertEquals(updatedRecord.getHaGroupName(), savedRecord.getHaGroupName()); + } + + @Test + public void testUpdateHAGroupStoreRecordInZooKeeperWithStaleVersion() throws Exception { + String haGroupName = testName.getMethodName(); + String peerZKUrl = CLUSTERS.getZkUrl2(); + + // Create initial record + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + + // Get the current stat for version checking + Stat stat = haAdmin.getCurator().checkExists().forPath(toPath(haGroupName)); + assertNotNull(stat); + int currentVersion = stat.getVersion(); + + // Update the record with current version (should succeed) + HAGroupStoreRecord updatedRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY); + + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, updatedRecord, currentVersion); + + // Try to update again with the same (now stale) version - should fail + HAGroupStoreRecord anotherUpdate = new HAGroupStoreRecord("v1.0", haGroupName, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY); + + try { + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, anotherUpdate, currentVersion); + fail("Expected StaleHAGroupStoreRecordVersionException"); + } catch (StaleHAGroupStoreRecordVersionException e) { + // Expected exception + assertTrue(e.getMessage().contains("with cached stat version")); + } + } + + @Test + public void testGetHAGroupStoreRecordInZooKeeper() throws Exception { + String haGroupName = testName.getMethodName(); + String peerZKUrl = CLUSTERS.getZkUrl2(); + + // Create initial record + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + + // Get the record and stat + Pair result = haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + + assertNotNull(result); + assertNotNull(result.getLeft()); + assertNotNull(result.getRight()); + + HAGroupStoreRecord retrievedRecord = result.getLeft(); + Stat stat = result.getRight(); + + assertEquals(initialRecord.getHaGroupName(), retrievedRecord.getHaGroupName()); + assertEquals(initialRecord.getClusterRole(), retrievedRecord.getClusterRole()); + assertEquals(initialRecord.getHAGroupState(), retrievedRecord.getHAGroupState()); + assertEquals(initialRecord.getProtocolVersion(), retrievedRecord.getProtocolVersion()); + + // Verify stat is valid + assertTrue(stat.getVersion() >= 0); + assertTrue(stat.getCtime() > 0); + assertTrue(stat.getMtime() > 0); + } + + @Test + public void testGetHAGroupStoreRecordInZooKeeperNonExistentNode() throws Exception { + assertNull(haAdmin.getHAGroupStoreRecordInZooKeeper(testName.getMethodName()).getLeft()); + assertNull(haAdmin.getHAGroupStoreRecordInZooKeeper(testName.getMethodName()).getLeft()); + } + + @Test + public void testCompleteWorkflowCreateUpdateGet() throws Exception { + String haGroupName = testName.getMethodName(); + String peerZKUrl = CLUSTERS.getZkUrl2(); + + // Step 1: Create initial record + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + + // Step 2: Get the record + Pair result = haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + HAGroupStoreRecord retrievedRecord = result.getLeft(); + Stat stat = result.getRight(); + + assertEquals(initialRecord.getHaGroupName(), retrievedRecord.getHaGroupName()); + assertEquals(initialRecord.getClusterRole(), retrievedRecord.getClusterRole()); + assertEquals(initialRecord.getHAGroupState(), retrievedRecord.getHAGroupState()); + assertEquals(initialRecord.getProtocolVersion(), retrievedRecord.getProtocolVersion()); + + // Step 3: Update the record + HAGroupStoreRecord updatedRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY); + + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, updatedRecord, stat.getVersion()); + + // Step 4: Get the updated record + Pair updatedResult = + haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + HAGroupStoreRecord finalRecord = updatedResult.getLeft(); + Stat finalStat = updatedResult.getRight(); + + assertEquals(updatedRecord.getHaGroupName(), finalRecord.getHaGroupName()); + assertEquals(updatedRecord.getClusterRole(), finalRecord.getClusterRole()); + assertEquals(updatedRecord.getHAGroupState(), finalRecord.getHAGroupState()); + assertEquals(updatedRecord.getProtocolVersion(), finalRecord.getProtocolVersion()); + + // Verify stat version increased + assertTrue(finalStat.getVersion() > stat.getVersion()); + } + + @Test + public void testMultiThreadedUpdatesConcurrentVersionConflict() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create initial record + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + + // Get the current stat for version checking + Pair result = haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + int currentVersion = result.getRight().getVersion(); + + // Number of threads to run concurrently + int threadCount = 5; + ExecutorService executorService = Executors.newFixedThreadPool(threadCount); + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch finishLatch = new CountDownLatch(threadCount); + + // Counters for tracking results + AtomicInteger successCount = new AtomicInteger(0); + AtomicInteger staleVersionExceptionCount = new AtomicInteger(0); + AtomicInteger otherExceptionCount = new AtomicInteger(0); + + // Submit multiple threads that will all try to update with the same version + for (int i = 0; i < threadCount; i++) { + executorService.submit(() -> { + try { + // Wait for all threads to be ready + startLatch.await(); + + // Create a unique update record for this thread + HAGroupStoreRecord updatedRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.STANDBY); + + // All threads use the same currentVersion, causing conflicts + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, updatedRecord, currentVersion); + successCount.incrementAndGet(); + + } catch (StaleHAGroupStoreRecordVersionException e) { + staleVersionExceptionCount.incrementAndGet(); + } catch (Exception e) { + otherExceptionCount.incrementAndGet(); + } finally { + finishLatch.countDown(); + } + }); + } + + // Start all threads at the same time + startLatch.countDown(); + + // Wait for all threads to complete + assertTrue("Threads did not complete within timeout", finishLatch.await(10, TimeUnit.SECONDS)); + + executorService.shutdown(); + assertTrue("ExecutorService did not shutdown within timeout", + executorService.awaitTermination(5, TimeUnit.SECONDS)); + + // Verify results + assertEquals("No other exceptions should occur", 0, otherExceptionCount.get()); + assertEquals("Exactly one thread should succeed", 1, successCount.get()); + assertEquals("All other threads should get stale version exception", threadCount - 1, + staleVersionExceptionCount.get()); + + // Verify the final state - should contain the update from the successful thread + Pair finalResult = + haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + HAGroupStoreRecord finalRecord = finalResult.getLeft(); + Stat finalStat = finalResult.getRight(); + + // The successful update should have changed the role to STANDBY + assertEquals(ClusterRoleRecord.ClusterRole.STANDBY, finalRecord.getClusterRole()); + + // The ZooKeeper version should have increased + assertTrue("ZooKeeper version should have increased", finalStat.getVersion() > currentVersion); + } + + @Test + public void testMultiThreadedUpdatesWithDifferentVersions() throws Exception { + String haGroupName = testName.getMethodName(); + + // Create initial record + HAGroupStoreRecord initialRecord = + new HAGroupStoreRecord("v1.0", haGroupName, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + haAdmin.createHAGroupStoreRecordInZooKeeper(initialRecord); + + // Number of threads to run sequentially (each gets the latest version) + int threadCount = 10; + ExecutorService executorService = Executors.newFixedThreadPool(1); // Single thread pool for + // sequential execution + CountDownLatch finishLatch = new CountDownLatch(threadCount); + + // Counter for tracking results + AtomicInteger successCount = new AtomicInteger(0); + AtomicInteger failureCount = new AtomicInteger(0); + + // Submit multiple threads that will update sequentially + for (int i = 0; i < threadCount; i++) { + final int threadId = i; + executorService.submit(() -> { + try { + // Get the current version for this thread + Pair currentResult = + haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + int currentVersion = currentResult.getRight().getVersion(); + + // Create update record for this thread + HAGroupStoreRecord updatedRecord = new HAGroupStoreRecord("v1.0", haGroupName, + threadId % 2 == 0 + ? HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC + : HAGroupStoreRecord.HAGroupState.STANDBY); + + // Update with the current version + haAdmin.updateHAGroupStoreRecordInZooKeeper(haGroupName, updatedRecord, currentVersion); + successCount.incrementAndGet(); + + } catch (Exception e) { + failureCount.incrementAndGet(); + } finally { + finishLatch.countDown(); + } + }); + } + + // Wait for all threads to complete + assertTrue("Threads did not complete within timeout", finishLatch.await(15, TimeUnit.SECONDS)); + + executorService.shutdown(); + assertTrue("ExecutorService did not shutdown within timeout", + executorService.awaitTermination(5, TimeUnit.SECONDS)); + + // Verify results - all should succeed since they each get the latest version + assertEquals("All threads should succeed when using correct versions", threadCount, + successCount.get()); + assertEquals("No threads should fail", 0, failureCount.get()); + + // Verify the final state + Pair finalResult = + haAdmin.getHAGroupStoreRecordInZooKeeper(haGroupName); + + // The final record should have the version from the last update + assertEquals(threadCount, finalResult.getRight().getVersion()); + } + +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ClusterRoleRecordTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ClusterRoleRecordTest.java index 2fc0f0f8011..9702acd4df3 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ClusterRoleRecordTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ClusterRoleRecordTest.java @@ -65,8 +65,8 @@ public ClusterRoleRecordTest(ClusterRoleRecord.RegistryType registryType) { public static Collection data() { return Arrays.asList(new Object[] { ClusterRoleRecord.RegistryType.ZK, ClusterRoleRecord.RegistryType.MASTER, ClusterRoleRecord.RegistryType.RPC, null // For - // Backward - // Compatibility + // Backward + // Compatibility }); } @@ -251,6 +251,15 @@ public void testToPrettyString() { assertNotEquals(record.toString(), record.toPrettyString()); } + @Test + public void testClusterRoleFromInvalidBytes() { + ClusterRole role = ClusterRole.from(new byte[0]); + assertEquals(ClusterRole.UNKNOWN, role); + + role = ClusterRole.from("random".getBytes()); + assertEquals(ClusterRole.UNKNOWN, role); + } + // Private Helper Methods private ClusterRoleRecord getClusterRoleRecord(String name, HighAvailabilityPolicy policy, diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/HAGroupStoreRecordTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/HAGroupStoreRecordTest.java new file mode 100644 index 00000000000..c474c0eeb16 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/HAGroupStoreRecordTest.java @@ -0,0 +1,359 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.Optional; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.phoenix.util.JacksonUtil; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Unit test for {@link HAGroupStoreRecord}. + */ +public class HAGroupStoreRecordTest { + private static final Logger LOG = LoggerFactory.getLogger(HAGroupStoreRecordTest.class); + private static final String PROTOCOL_VERSION = "1.0"; + + @Rule + public final TestName testName = new TestName(); + + /** + * Helper method to create a temp JSON file with the given array of HA group store records. + */ + public static String createJsonFileWithRecords(HAGroupStoreRecord record) throws IOException { + File file = File.createTempFile("phoenix.ha.group.store.records", ".test.json"); + file.deleteOnExit(); + JacksonUtil.getObjectWriterPretty().writeValue(file, record); + LOG.info("Prepared the JSON file for testing, file:{}, content:\n{}", file, + FileUtils.readFileToString(file, "UTF-8")); + return file.getPath(); + } + + @Test + public void testReadWriteJsonToFile() throws IOException { + HAGroupStoreRecord record = getHAGroupStoreRecord(testName.getMethodName(), PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + String fileName = createJsonFileWithRecords(record); + String fileContent = FileUtils.readFileToString(new File(fileName), "UTF-8"); + assertTrue(fileContent.contains(record.getHaGroupName())); + assertTrue(fileContent.contains(record.getProtocolVersion())); + assertTrue(fileContent.contains(record.getHAGroupState().toString())); + // Create a new record from file + Optional record2 = HAGroupStoreRecord.fromJson(fileContent.getBytes()); + assertTrue(record2.isPresent()); + // Check if same info + assertTrue(record.hasSameInfo(record2.get())); + } + + @Test + public void testToAndFromJson() throws IOException { + HAGroupStoreRecord record = getHAGroupStoreRecord(testName.getMethodName(), PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + byte[] bytes = HAGroupStoreRecord.toJson(record); + Optional record2 = HAGroupStoreRecord.fromJson(bytes); + assertTrue(record2.isPresent()); + assertEquals(record, record2.get()); + } + + @Test + public void testFromJsonWithNullBytes() { + Optional record = HAGroupStoreRecord.fromJson(null); + assertFalse(record.isPresent()); + } + + @Test + public void testFromJsonWithInvalidJson() { + byte[] invalidJson = "invalid json".getBytes(); + Optional record = HAGroupStoreRecord.fromJson(invalidJson); + assertFalse(record.isPresent()); + } + + @Test + public void testHasSameInfo() { + String haGroupName = testName.getMethodName(); + HAGroupStoreRecord record1 = getHAGroupStoreRecord(haGroupName, PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + HAGroupStoreRecord record2 = getHAGroupStoreRecord(haGroupName, PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + assertTrue(record1.hasSameInfo(record2)); // Same core info despite different state + assertTrue(record1.hasSameInfo(record1)); // reflexive + assertTrue(record2.hasSameInfo(record1)); // symmetric + } + + @Test + public void testHasSameInfoNegative() { + String haGroupName = testName.getMethodName(); + HAGroupStoreRecord record = getHAGroupStoreRecord(haGroupName, PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + + // Different protocol version + HAGroupStoreRecord recordDifferentProtocol = + getHAGroupStoreRecord(haGroupName, "2.0", HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + assertFalse(record.hasSameInfo(recordDifferentProtocol)); + assertFalse(recordDifferentProtocol.hasSameInfo(record)); + + // Different HA group name + String haGroupName2 = haGroupName + RandomStringUtils.randomAlphabetic(2); + HAGroupStoreRecord record2 = getHAGroupStoreRecord(haGroupName2, PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + assertFalse(record.hasSameInfo(record2)); + assertFalse(record2.hasSameInfo(record)); + + // Different HA group state + HAGroupStoreRecord recordDifferentState = + getHAGroupStoreRecord(haGroupName, PROTOCOL_VERSION, HAGroupStoreRecord.HAGroupState.STANDBY); + assertFalse(record.hasSameInfo(recordDifferentState)); + assertFalse(recordDifferentState.hasSameInfo(record)); + } + + @Test + public void testGetters() { + String haGroupName = testName.getMethodName(); + String protocolVersion = "1.5"; + HAGroupStoreRecord.HAGroupState haGroupState = HAGroupStoreRecord.HAGroupState.STANDBY; + + HAGroupStoreRecord record = getHAGroupStoreRecord(haGroupName, protocolVersion, haGroupState); + + assertEquals(haGroupName, record.getHaGroupName()); + assertEquals(protocolVersion, record.getProtocolVersion()); + assertEquals(haGroupState, record.getHAGroupState()); + assertEquals(haGroupState.getClusterRole(), record.getClusterRole()); + } + + @Test + public void testEqualsAndHashCode() { + String haGroupName = testName.getMethodName(); + HAGroupStoreRecord record1 = getHAGroupStoreRecord(haGroupName, PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + HAGroupStoreRecord record2 = getHAGroupStoreRecord(haGroupName, PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + HAGroupStoreRecord record3 = + getHAGroupStoreRecord(haGroupName, PROTOCOL_VERSION, HAGroupStoreRecord.HAGroupState.STANDBY); // Different + // state + + // Test equals + assertEquals(record1, record2); // symmetric + assertEquals(record2, record1); // symmetric + assertNotEquals(record1, record3); // different state + assertNotEquals(null, record1); // null comparison + assertNotEquals("not a record", record1); // different type + + // Test hashCode + assertEquals(record1.hashCode(), record2.hashCode()); // equal objects have same hash + assertNotEquals(record1.hashCode(), record3.hashCode()); // different objects likely have + // different hash + } + + @Test + public void testToString() { + HAGroupStoreRecord record = getHAGroupStoreRecord(testName.getMethodName(), PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + String toString = record.toString(); + + // Verify all fields are present in toString + assertTrue(toString.contains(record.getHaGroupName())); + assertTrue(toString.contains(record.getProtocolVersion())); + assertTrue(toString.contains(record.getHAGroupState().toString())); + } + + @Test + public void testToPrettyString() { + HAGroupStoreRecord record = getHAGroupStoreRecord(testName.getMethodName(), PROTOCOL_VERSION, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + LOG.info("toString(): {}", record.toString()); + LOG.info("toPrettyString:\n{}", record.toPrettyString()); + assertNotEquals(record.toString(), record.toPrettyString()); + assertTrue(record.toPrettyString().contains(record.getHaGroupName())); + } + + @Test(expected = NullPointerException.class) + public void testConstructorWithNullHaGroupName() { + getHAGroupStoreRecord(null, PROTOCOL_VERSION, HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC); + } + + @Test(expected = NullPointerException.class) + public void testConstructorWithNullHAGroupState() { + getHAGroupStoreRecord(testName.getMethodName(), PROTOCOL_VERSION, null); + } + + // Tests for HAGroupState enum + @Test + public void testHAGroupStateGetClusterRole() { + // Test that each HAGroupState maps to the correct ClusterRole + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE, + HAGroupStoreRecord.HAGroupState.ABORT_TO_ACTIVE_IN_SYNC.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.STANDBY, + HAGroupStoreRecord.HAGroupState.ABORT_TO_STANDBY.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC_TO_STANDBY.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE, + HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC_WITH_OFFLINE_PEER.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE_TO_STANDBY, + HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.ACTIVE, + HAGroupStoreRecord.HAGroupState.ACTIVE_WITH_OFFLINE_PEER.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.STANDBY, + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.STANDBY, + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY_FOR_READER.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.STANDBY, + HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY_FOR_WRITER.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.OFFLINE, + HAGroupStoreRecord.HAGroupState.OFFLINE.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.STANDBY, + HAGroupStoreRecord.HAGroupState.STANDBY.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.STANDBY_TO_ACTIVE, + HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE.getClusterRole()); + assertEquals(ClusterRoleRecord.ClusterRole.UNKNOWN, + HAGroupStoreRecord.HAGroupState.UNKNOWN.getClusterRole()); + } + + @Test + public void testHAGroupStateValidTransitions() { + // Test valid transitions for ACTIVE_NOT_IN_SYNC + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC)); + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC)); + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC_TO_STANDBY)); + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC_WITH_OFFLINE_PEER)); + + // Test valid transitions for ACTIVE + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC)); + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_WITH_OFFLINE_PEER)); + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY)); + + // Test valid transitions for STANDBY + assertTrue(HAGroupStoreRecord.HAGroupState.STANDBY + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE)); + assertTrue(HAGroupStoreRecord.HAGroupState.STANDBY + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY_FOR_READER)); + assertTrue(HAGroupStoreRecord.HAGroupState.STANDBY + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY_FOR_WRITER)); + + // Test valid transitions for ACTIVE_TO_STANDBY + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ABORT_TO_ACTIVE_IN_SYNC)); + assertTrue(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC_TO_STANDBY + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.STANDBY)); + + // Test valid transitions for STANDBY_TO_ACTIVE + assertTrue(HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ABORT_TO_STANDBY)); + assertTrue(HAGroupStoreRecord.HAGroupState.STANDBY_TO_ACTIVE + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC)); + } + + @Test + public void testHAGroupStateInvalidTransitions() { + // Test invalid transitions - ACTIVE cannot directly go to STANDBY + assertFalse(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.STANDBY)); + + // Test invalid transitions - STANDBY cannot directly go to ACTIVE + assertFalse(HAGroupStoreRecord.HAGroupState.STANDBY + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC)); + + // Test invalid transitions - OFFLINE has no allowed transitions (manual recovery needed) + assertFalse(HAGroupStoreRecord.HAGroupState.OFFLINE + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC)); + assertFalse(HAGroupStoreRecord.HAGroupState.OFFLINE + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.STANDBY)); + assertFalse(HAGroupStoreRecord.HAGroupState.OFFLINE + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.UNKNOWN)); + + // Test invalid transitions - UNKNOWN has no allowed transitions (manual recovery needed) + assertFalse(HAGroupStoreRecord.HAGroupState.UNKNOWN + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC)); + assertFalse(HAGroupStoreRecord.HAGroupState.UNKNOWN + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.STANDBY)); + assertFalse(HAGroupStoreRecord.HAGroupState.UNKNOWN + .isTransitionAllowed(HAGroupStoreRecord.HAGroupState.OFFLINE)); + } + + @Test + public void testHAGroupStateFromBytesValidValues() { + // Test valid enum names (case insensitive) + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, + HAGroupStoreRecord.HAGroupState.from("ACTIVE_IN_SYNC".getBytes())); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, + HAGroupStoreRecord.HAGroupState.from("active_in_sync".getBytes())); + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_IN_SYNC, + HAGroupStoreRecord.HAGroupState.from("Active_in_Sync".getBytes())); + + assertEquals(HAGroupStoreRecord.HAGroupState.STANDBY, + HAGroupStoreRecord.HAGroupState.from("STANDBY".getBytes())); + assertEquals(HAGroupStoreRecord.HAGroupState.STANDBY, + HAGroupStoreRecord.HAGroupState.from("standby".getBytes())); + + assertEquals(HAGroupStoreRecord.HAGroupState.OFFLINE, + HAGroupStoreRecord.HAGroupState.from("OFFLINE".getBytes())); + + assertEquals(HAGroupStoreRecord.HAGroupState.ACTIVE_NOT_IN_SYNC, + HAGroupStoreRecord.HAGroupState.from("ACTIVE_NOT_IN_SYNC".getBytes())); + + assertEquals(HAGroupStoreRecord.HAGroupState.DEGRADED_STANDBY_FOR_READER, + HAGroupStoreRecord.HAGroupState.from("DEGRADED_STANDBY_FOR_READER".getBytes())); + } + + @Test + public void testHAGroupStateFromBytesInvalidValues() { + // Test invalid enum names return UNKNOWN + assertEquals(HAGroupStoreRecord.HAGroupState.UNKNOWN, + HAGroupStoreRecord.HAGroupState.from("INVALID_STATE".getBytes())); + assertEquals(HAGroupStoreRecord.HAGroupState.UNKNOWN, + HAGroupStoreRecord.HAGroupState.from("".getBytes())); + assertEquals(HAGroupStoreRecord.HAGroupState.UNKNOWN, + HAGroupStoreRecord.HAGroupState.from("null".getBytes())); + assertEquals(HAGroupStoreRecord.HAGroupState.UNKNOWN, + HAGroupStoreRecord.HAGroupState.from("BLAH".getBytes())); // typo + assertEquals(HAGroupStoreRecord.HAGroupState.UNKNOWN, + HAGroupStoreRecord.HAGroupState.from("ACTIVE_EXTRA".getBytes())); // extra text + assertEquals(HAGroupStoreRecord.HAGroupState.UNKNOWN, + HAGroupStoreRecord.HAGroupState.from(null)); + + } + + // Private Helper Methods + private HAGroupStoreRecord getHAGroupStoreRecord(String haGroupName, String protocolVersion, + HAGroupStoreRecord.HAGroupState haGroupState) { + return new HAGroupStoreRecord(protocolVersion, haGroupName, haGroupState); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixHAAdminToolTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixHAAdminToolTest.java index 32f4b8ad0c5..5fa3c24b4f8 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixHAAdminToolTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixHAAdminToolTest.java @@ -86,8 +86,8 @@ public class PhoenixHAAdminToolTest { @Before public void setup() throws Exception { - when(mockHighAvailibilityCuratorProvider.getCurator(Mockito.anyString(), any(Properties.class))) - .thenReturn(curator); + when(mockHighAvailibilityCuratorProvider.getCurator(anyString(), any(Properties.class), + anyString())).thenReturn(curator); haGroupName = testName.getMethodName(); recordV1 = new ClusterRoleRecord(haGroupName, HighAvailabilityPolicy.FAILOVER, ZK1, ClusterRole.ACTIVE, ZK2, ClusterRole.STANDBY, 1); @@ -185,7 +185,7 @@ public void testUpdate() throws Exception { saveRecordV1ToZk(); ClusterRoleRecord recordV2 = new ClusterRoleRecord(haGroupName, HighAvailabilityPolicy.FAILOVER, ZK1, ClusterRole.STANDBY, ZK2, ClusterRole.STANDBY, 2); // higher version than recordV1 so - // update should be tried + // update should be tried try { result = admin.createOrUpdateDataOnZookeeper(recordV2); } catch (Exception e) { diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/HAGroupStoreTestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/HAGroupStoreTestUtil.java new file mode 100644 index 00000000000..b25c6c77a05 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/util/HAGroupStoreTestUtil.java @@ -0,0 +1,117 @@ +/* + * 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.util; + +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_HA_GROUP_NAME; +import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR; +import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_ZK; + +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import org.apache.phoenix.jdbc.ClusterRoleRecord; +import org.apache.phoenix.jdbc.PhoenixConnection; + +/** + * Utility class for HA Group Store testing operations. + */ +public class HAGroupStoreTestUtil { + + /** + * Upserts an HA group record into the system table for testing purposes. + * @param haGroupName the HA group name + * @param zkUrl the ZooKeeper URL for the local cluster + * @param peerZKUrl the ZooKeeper URL for the peer cluster + * @param localClusterRole the role of the local cluster + * @param peerClusterRole the role of the peer cluster + * @param overrideConnZkUrl optional override for the connection ZK URL + * @throws SQLException if the database operation fails + */ + public static void upsertHAGroupRecordInSystemTable(String haGroupName, String zkUrl, + String peerZKUrl, ClusterRoleRecord.ClusterRole localClusterRole, + ClusterRoleRecord.ClusterRole peerClusterRole, String overrideConnZkUrl) throws SQLException { + try ( + PhoenixConnection conn = (PhoenixConnection) DriverManager.getConnection(JDBC_PROTOCOL_ZK + + JDBC_PROTOCOL_SEPARATOR + (overrideConnZkUrl != null ? overrideConnZkUrl : zkUrl)); + Statement stmt = conn.createStatement()) { + // Only insert values that are not null + StringBuilder queryBuilder = + new StringBuilder("UPSERT INTO " + SYSTEM_HA_GROUP_NAME + " (HA_GROUP_NAME, "); + if (zkUrl != null) { + queryBuilder.append("ZK_URL_1, "); + } + if (peerZKUrl != null) { + queryBuilder.append("ZK_URL_2, "); + } + if (localClusterRole != null) { + queryBuilder.append("CLUSTER_ROLE_1, "); + } + if (peerClusterRole != null) { + queryBuilder.append("CLUSTER_ROLE_2, "); + } + if (zkUrl != null) { + queryBuilder.append("CLUSTER_URL_1, "); + } + if (peerZKUrl != null) { + queryBuilder.append("CLUSTER_URL_2, "); + } + queryBuilder.append("POLICY, VERSION) "); + queryBuilder.append("VALUES ('" + haGroupName + "', "); + if (zkUrl != null) { + queryBuilder.append("'" + zkUrl + "', "); + } + if (peerZKUrl != null) { + queryBuilder.append("'" + peerZKUrl + "', "); + } + if (localClusterRole != null) { + queryBuilder.append("'" + localClusterRole + "', "); + } + if (peerClusterRole != null) { + queryBuilder.append("'" + peerClusterRole + "', "); + } + if (zkUrl != null) { + queryBuilder.append("'" + zkUrl + "', "); + } + if (peerZKUrl != null) { + queryBuilder.append("'" + peerZKUrl + "', "); + } + queryBuilder.append("'FAILOVER', 1)"); + stmt.executeUpdate(queryBuilder.toString()); + conn.commit(); + } + } + + /** + * Deletes an HA group record from the system table for testing purposes. + * @param haGroupName the HA group name to delete + * @param zkUrl the ZooKeeper URL to connect to + * @throws SQLException if the database operation fails + */ + public static void deleteHAGroupRecordInSystemTable(String haGroupName, String zkUrl) + throws SQLException { + // Delete the record from System Table + try ( + PhoenixConnection conn = (PhoenixConnection) DriverManager + .getConnection(JDBC_PROTOCOL_ZK + JDBC_PROTOCOL_SEPARATOR + zkUrl); + Statement stmt = conn.createStatement()) { + stmt.execute( + "DELETE FROM " + SYSTEM_HA_GROUP_NAME + " WHERE HA_GROUP_NAME = '" + haGroupName + "'"); + conn.commit(); + } + } +}