lokiore commented on code in PR #2053:
URL: https://github.com/apache/phoenix/pull/2053#discussion_r1919173238


##########
phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HighAvailabilityGroup.java:
##########
@@ -195,36 +198,104 @@ private HighAvailabilityGroup(HAGroupInfo info, 
Properties properties) {
         this.state = state;
     }
 
-    public static HAGroupInfo getHAGroupInfo(String url, Properties properties)
-            throws SQLException {
-        if (url.startsWith(PhoenixRuntime.JDBC_PROTOCOL)) {
-            url = url.substring(PhoenixRuntime.JDBC_PROTOCOL.length() + 1);
-        }
-        if (!(url.contains("[") && url.contains("|") && url.contains("]"))) {
-            throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_CONNECTION_URL)
-                    .setMessage(String.format("URL %s is not a valid HA 
connection string", url))
-                    .build()
-                    .buildException();
-        }
+    /**
+     * Get an instance of HAURLInfo given the HA connecting URL (with "|") and 
client properties.
+     * Here we do parsing of url and try to extract principal and other 
additional params
+     * @throws SQLException
+     */
+    public static HAURLInfo getUrlInfo(String url, Properties properties) 
throws SQLException {

Review Comment:
   I have added tests with different principles, I think can add more with 
without principle as well



##########
phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/FailoverPhoenixConnection.java:
##########
@@ -103,13 +100,12 @@ public class FailoverPhoenixConnection implements 
PhoenixMonitoredConnection {
      */
     private Map<String, Map<MetricType, Long>> previousReadMetrics = new 
HashMap<>();
 
-    public FailoverPhoenixConnection(HighAvailabilityGroup haGroup, Properties 
properties)
+    public FailoverPhoenixConnection(FailoverPhoenixContext context)

Review Comment:
   This is created on though policy and there we create context every time, 
that's why didn't add the check.



##########
phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HAURLInfo.java:
##########
@@ -0,0 +1,87 @@
+package org.apache.phoenix.jdbc;
+
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import 
org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions;
+
+/**
+ * An HAURLInfo contains information of an HA Url with respect of HA Group 
Name.
+ * <p>
+ * It is constructed based on client input, including the JDBC connection 
string and properties.
+ * Objects of this class are used to get appropriate principal and additional 
JDBC parameters.
+ * <p>
+ * This class is immutable.
+ */
+
+@VisibleForTesting
+public class HAURLInfo {
+    private final String name;
+    private final String principal;
+    private final String additionalJDBCParams;
+
+    HAURLInfo(String name, String principal, String additionalJDBCParams) {
+        Preconditions.checkNotNull(name);
+        this.name = name;
+        this.principal = principal;
+        this.additionalJDBCParams = additionalJDBCParams;
+    }
+
+    HAURLInfo(String name, String principal) {
+        this(name, principal, null);
+    }
+
+    HAURLInfo(String name) {
+        this(name, null, null);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getPrincipal() {
+        return principal;
+    }
+
+    public String getAdditionalJDBCParams() {
+        return additionalJDBCParams;
+    }
+
+    @Override
+    public String toString() {
+        if (principal != null) {
+            return String.format("%s[%s]", name, principal);
+        }
+        return name;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (other == null) {
+            return false;
+        }
+        if (other == this) {
+            return true;
+        }
+        if (other.getClass() != getClass()) {
+            return false;
+        }
+        HAURLInfo otherInfo = (HAURLInfo) other;
+        return new EqualsBuilder()

Review Comment:
   EqualsBuilder handles the case of null in append function, will add assert 
in test for this



##########
phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HighAvailabilityGroup.java:
##########
@@ -785,26 +851,49 @@ public String getUrl2() {
             return urls.getSecond();
         }
 
-        public String getJDBCUrl(String zkUrl) {
-            Preconditions.checkArgument(zkUrl.equals(getUrl1()) || 
zkUrl.equals(getUrl2()),
+
+        //It applies only the current thread context's URL Info as we have 1:n 
mapping between
+        //HAURLInfo -> HAGroupInfo.

Review Comment:
   Forgot to remove comment it was with other approach, will update this



##########
phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/HighAvailabilityPolicy.java:
##########
@@ -66,14 +67,19 @@ private void transitStandby(HighAvailabilityGroup haGroup, 
String zkUrl)
                     zkUrl, haGroup.getGroupInfo());
             ConnectionQueryServices cqs = null;
             try {
-                cqs = PhoenixDriver.INSTANCE.getConnectionQueryServices(
-                        haGroup.getGroupInfo().getJDBCUrl(zkUrl), 
haGroup.getProperties());
-                cqs.closeAllConnections(new SQLExceptionInfo
-                        .Builder(SQLExceptionCode.HA_CLOSED_AFTER_FAILOVER)
-                        .setMessage("Phoenix connection got closed due to 
failover")
-                        .setHaGroupInfo(haGroup.getGroupInfo().toString()));
-                LOG.info("Closed all connections to cluster {} for HA group 
{}", zkUrl,
-                        haGroup.getGroupInfo());
+                //Close connections for every HAURLInfo's (different 
principal) connections for a give HAGroup
+                for (HAURLInfo haurlInfo : 
HighAvailabilityGroup.URLS.get(haGroup.getGroupInfo())) {
+                    String jdbcZKUrl = 
haGroup.getGroupInfo().getJDBCUrl(zkUrl, haurlInfo);
+                    cqs = PhoenixDriver.INSTANCE.getConnectionQueryServices(
+                            jdbcZKUrl, haGroup.getProperties());
+                    cqs.closeAllConnections(new SQLExceptionInfo
+                            .Builder(SQLExceptionCode.HA_CLOSED_AFTER_FAILOVER)
+                            .setMessage("Phoenix connection got closed due to 
failover")
+                            
.setHaGroupInfo(haGroup.getGroupInfo().toString()));
+                    LOG.info("Closed all connections to cluster {} for HA 
group {}",
+                            jdbcZKUrl, haGroup.getGroupInfo());
+                }
+

Review Comment:
   Yes Thanks for catching that



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

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

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

Reply via email to