dbwong commented on code in PR #1430:
URL: https://github.com/apache/phoenix/pull/1430#discussion_r903009823


##########
phoenix-core/src/main/java/org/apache/phoenix/jdbc/ParallelPhoenixConnection.java:
##########
@@ -0,0 +1,653 @@
+/*
+ * 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 org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.jdbc.ParallelPhoenixUtil.FutureResult;
+import org.apache.phoenix.monitoring.MetricType;
+import 
org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import static 
org.apache.phoenix.exception.SQLExceptionCode.CLASS_NOT_UNWRAPPABLE;
+
+public class ParallelPhoenixConnection implements PhoenixMonitoredConnection {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(ParallelPhoenixConnection.class);
+
+    private final ParallelPhoenixContext context;
+    CompletableFuture<PhoenixConnection> futureConnection1;
+    CompletableFuture<PhoenixConnection> futureConnection2;
+
+    public ParallelPhoenixConnection(ParallelPhoenixContext context) throws 
SQLException {
+        this.context = context;
+        LOG.debug("First Url: {} Second Url: {}", 
context.getHaGroup().getGroupInfo().getUrl1(),
+                context.getHaGroup().getGroupInfo().getUrl2());
+        futureConnection1 = context.chainOnConn1(() -> 
getConnection(context.getHaGroup(),
+                context.getHaGroup().getGroupInfo().getUrl1(),
+                context.getProperties()));
+        futureConnection2 = context.chainOnConn2(() -> 
getConnection(context.getHaGroup(),
+                context.getHaGroup().getGroupInfo().getUrl2(),
+                context.getProperties()));
+    }
+
+    @VisibleForTesting
+    ParallelPhoenixConnection(ParallelPhoenixContext context, 
CompletableFuture<PhoenixConnection> futureConnection1, 
CompletableFuture<PhoenixConnection> futureConnection2) {
+        this.context = context;
+        this.futureConnection1 = futureConnection1;
+        this.futureConnection2 = futureConnection2;
+    }
+
+    private static PhoenixConnection getConnection(HighAvailabilityGroup 
haGroup, String url, Properties properties) {
+        try {
+            return haGroup.connectToOneCluster(url, properties);
+        } catch (SQLException exception) {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace(String.format("Failed to get a connection for 
haGroup %s to %s", haGroup.toString(), url), exception);
+            }
+            throw new CompletionException(exception);
+        }
+    }
+
+    public CompletableFuture<PhoenixConnection> getFutureConnection1() {
+        return futureConnection1;
+    }
+
+    public CompletableFuture<PhoenixConnection> getFutureConnection2() {
+        return futureConnection2;
+    }
+
+    @VisibleForTesting
+    ParallelPhoenixContext getContext() {
+        return this.context;
+    }
+
+    Object runOnConnections(Function<PhoenixConnection, ?> function, boolean 
useMetrics) throws SQLException {
+        return ParallelPhoenixUtil.INSTANCE.runFutures(function, 
futureConnection1, futureConnection2, context, useMetrics);
+    }
+
+    PairOfSameType<Object> runOnConnectionsGetAll(Function<PhoenixConnection, 
?> function, boolean useMetrics) throws SQLException {
+        return ParallelPhoenixUtil.INSTANCE.runOnFuturesGetAll(function, 
futureConnection1, futureConnection2, context, useMetrics);
+    }
+
+    @Override
+    public ParallelPhoenixStatement createStatement() throws SQLException {
+        context.checkOpen();
+
+        Function<PhoenixConnection, PhoenixMonitoredStatement> function = (T) 
-> {
+            try {
+                return (PhoenixStatement) T.createStatement();
+            } catch (SQLException exception) {
+                throw new CompletionException(exception);
+            }
+        };
+
+        List<CompletableFuture<PhoenixMonitoredStatement>> futures = 
ParallelPhoenixUtil.INSTANCE.applyFunctionToFutures(function, futureConnection1,
+                futureConnection2, context, true);
+
+        Preconditions.checkState(futures.size() == 2);
+        CompletableFuture<PhoenixMonitoredStatement> statement1 = 
futures.get(0);
+        CompletableFuture<PhoenixMonitoredStatement> statement2 = 
futures.get(1);
+
+        //Ensure one statement is successful before returning
+        ParallelPhoenixUtil.INSTANCE.runFutures(futures, context, true);
+
+        return new ParallelPhoenixStatement(context, statement1, statement2);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql) throws SQLException {
+        context.checkOpen();
+
+        Function<PhoenixConnection, PhoenixMonitoredPreparedStatement> 
function = (T) -> {
+            try {
+                return (PhoenixMonitoredPreparedStatement) 
T.prepareStatement(sql);
+            } catch (SQLException exception) {
+                throw new CompletionException(exception);
+            }
+        };
+
+        List<CompletableFuture<PhoenixMonitoredPreparedStatement>> futures = 
ParallelPhoenixUtil.INSTANCE.applyFunctionToFutures(function, futureConnection1,
+                futureConnection2, context, true);
+
+        Preconditions.checkState(futures.size() == 2);
+        CompletableFuture<PhoenixMonitoredPreparedStatement> statement1 = 
futures.get(0);
+        CompletableFuture<PhoenixMonitoredPreparedStatement> statement2 = 
futures.get(1);
+
+        //Ensure one statement is successful before returning
+        ParallelPhoenixUtil.INSTANCE.runFutures(futures, context, true);
+
+        return new ParallelPhoenixPreparedStatement(this.context, statement1, 
statement2);
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public String nativeSQL(String sql) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public boolean getAutoCommit() throws SQLException {
+        Function<PhoenixConnection, Boolean> function = (T) -> {
+            try {
+                return T.getAutoCommit();
+            } catch (SQLException exception) {
+                throw new CompletionException(exception);
+            }
+        };
+
+        return (boolean) runOnConnections(function, true);
+    }
+
+    @Override
+    public void setAutoCommit(boolean autoCommit) throws SQLException {
+        Function<PhoenixConnection, Void> function = (T) -> {
+            try {
+                T.setAutoCommit(autoCommit);
+                return null;
+            } catch (SQLException exception) {
+                throw new CompletionException(exception);
+            }
+        };
+
+        runOnConnections(function, true);
+    }
+
+    @Override
+    public void commit() throws SQLException {
+        Function<PhoenixConnection, Void> function = (T) -> {
+            try {
+                T.commit();
+                return null;
+            } catch (SQLException exception) {
+                throw new CompletionException(exception);
+            }
+        };
+
+        runOnConnections(function, true);
+    }
+
+    @Override
+    public void rollback() throws SQLException {
+
+    }
+
+    /**
+     * Close the underlying connections. Returns after any one of the 
underlying connections have
+     * closed successfully
+     *
+     * @throws SQLException if trying to close both the underlying connections 
encounters errors
+     */
+    @Override
+    public void close() throws SQLException {
+        context.close();
+        SQLException closeExp = null;
+
+        Supplier<Boolean> closeSupplier1 = getCloseSupplier(futureConnection1);
+        Supplier<Boolean> closeSupplier2 = getCloseSupplier(futureConnection2);
+
+        // We can have errors on the chain, we still need to close the 
underlying connections
+        // irrespective. Do a close when we're at the end of the chain and 
wait for any 1 to be
+        // successful. We need to track which future completed hence use 
FutureResult
+        // Enqueue a close operation at the end of the chain in the common 
ForkJoin Pool
+        CompletableFuture<FutureResult<CompletableFuture<Boolean>>> 
closeFuture1 =
+                context.getChainOnConn1().handle((obj, e) -> {
+                    return CompletableFuture.supplyAsync(closeSupplier1);
+                }).thenApply(t -> new FutureResult<>(t, 0));
+        CompletableFuture<FutureResult<CompletableFuture<Boolean>>> 
closeFuture2 =
+                context.getChainOnConn2().handle((obj, e) -> {
+                    return CompletableFuture.supplyAsync(closeSupplier2);
+                }).thenApply(t -> new FutureResult<>(t, 1));
+
+        FutureResult<CompletableFuture<Boolean>> result =
+                (FutureResult<CompletableFuture<Boolean>>) 
ParallelPhoenixUtil.INSTANCE
+                        .getAnyOfNonExceptionally(Arrays.asList(closeFuture1, 
closeFuture2), context);
+
+        try {
+            ParallelPhoenixUtil.INSTANCE.getFutureNoRetry(result.getResult(), 
context);
+            return;
+        } catch (Exception e) {
+            closeExp = new SQLException(e);
+        }
+        // The previous close encountered an exception try the other one
+        CompletableFuture<FutureResult<CompletableFuture<Boolean>>> 
otherFuture =
+                (result.getIndex() == 0) ? closeFuture2 : closeFuture1;
+
+        try {
+            FutureResult<CompletableFuture<Boolean>> otherResult =

Review Comment:
   I don't think that case is possible as we queue them both but we can improve 
the safety a bit by doing the setup for each individually.  I don't think those 
methods should throw but can add some more safety.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to