You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/11/17 09:17:08 UTC

[shardingsphere] branch master updated: Add state pattern implement for JDBC

This is an automated email from the ASF dual-hosted git repository.

zhangyonglun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 93a3877  Add state pattern implement for JDBC
     new 99eaecb  Merge pull request #8189 from terrymanu/dev
93a3877 is described below

commit 93a387701d41421f6d0634eff628d07cecfe76c9
Author: terrymanu <te...@163.com>
AuthorDate: Tue Nov 17 17:13:03 2020 +0800

    Add state pattern implement for JDBC
---
 .../GovernanceShardingSphereDataSource.java        |  8 ++-
 .../governance/internal/state/DriverState.java     | 43 +++++++++++++++
 .../internal/state/DriverStateContext.java         | 63 ++++++++++++++++++++++
 .../state/impl/CircuitBreakDriverState.java        | 40 ++++++++++++++
 .../internal/state/impl/LockDriverState.java       | 39 ++++++++++++++
 .../internal/state/impl/OKDriverState.java         | 41 ++++++++++++++
 .../proxy/frontend/state/ProxyStateContext.java    | 10 ++--
 7 files changed, 234 insertions(+), 10 deletions(-)

diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java
index ec342b9..150c78d 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/datasource/GovernanceShardingSphereDataSource.java
@@ -19,12 +19,11 @@ package org.apache.shardingsphere.driver.governance.internal.datasource;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
-import org.apache.shardingsphere.driver.governance.internal.circuit.datasource.CircuitBreakerDataSource;
-import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
+import org.apache.shardingsphere.driver.governance.internal.state.DriverStateContext;
 import org.apache.shardingsphere.driver.jdbc.unsupported.AbstractUnsupportedOperationDataSource;
+import org.apache.shardingsphere.governance.context.metadata.GovernanceMetaDataContexts;
 import org.apache.shardingsphere.governance.core.config.ConfigCenter;
 import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
-import org.apache.shardingsphere.governance.context.metadata.GovernanceMetaDataContexts;
 import org.apache.shardingsphere.governance.repository.api.config.GovernanceConfiguration;
 import org.apache.shardingsphere.infra.auth.Authentication;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
@@ -127,8 +126,7 @@ public final class GovernanceShardingSphereDataSource extends AbstractUnsupporte
     
     @Override
     public Connection getConnection() {
-        return metaDataContexts.isCircuitBreak() ? new CircuitBreakerDataSource().getConnection()
-                : new ShardingSphereConnection(getDataSourceMap(), metaDataContexts, transactionContexts, TransactionTypeHolder.get());
+        return DriverStateContext.getConnection(getDataSourceMap(), metaDataContexts, transactionContexts, TransactionTypeHolder.get());
     }
     
     @Override
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/DriverState.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/DriverState.java
new file mode 100644
index 0000000..7d62de0
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/DriverState.java
@@ -0,0 +1,43 @@
+/*
+ * 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.shardingsphere.driver.governance.internal.state;
+
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.apache.shardingsphere.transaction.core.TransactionType;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.util.Map;
+
+/**
+ * Driver state.
+ */
+public interface DriverState {
+    
+    /**
+     * Get connection.
+     * 
+     * @param dataSourceMap data source map
+     * @param metaDataContexts meta data contexts
+     * @param transactionContexts transaction contexts
+     * @param transactionType transaction type
+     * @return connection
+     */
+    Connection getConnection(Map<String, DataSource> dataSourceMap, MetaDataContexts metaDataContexts, TransactionContexts transactionContexts, TransactionType transactionType);
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/DriverStateContext.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/DriverStateContext.java
new file mode 100644
index 0000000..3b45648
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/DriverStateContext.java
@@ -0,0 +1,63 @@
+/*
+ * 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.shardingsphere.driver.governance.internal.state;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.driver.governance.internal.state.impl.CircuitBreakDriverState;
+import org.apache.shardingsphere.driver.governance.internal.state.impl.LockDriverState;
+import org.apache.shardingsphere.driver.governance.internal.state.impl.OKDriverState;
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.infra.state.StateContext;
+import org.apache.shardingsphere.infra.state.StateType;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.apache.shardingsphere.transaction.core.TransactionType;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Driver state context.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class DriverStateContext {
+    
+    private static final Map<StateType, DriverState> STATE_MAP = new ConcurrentHashMap<>(3, 1);
+    
+    static {
+        STATE_MAP.put(StateType.OK, new OKDriverState());
+        STATE_MAP.put(StateType.LOCK, new LockDriverState());
+        STATE_MAP.put(StateType.CIRCUIT_BREAK, new CircuitBreakDriverState());
+    }
+    
+    /**
+     * Get connection.
+     *
+     * @param dataSourceMap data source map
+     * @param metaDataContexts meta data contexts
+     * @param transactionContexts transaction contexts
+     * @param transactionType transaction type
+     * @return connection
+     */
+    public static Connection getConnection(final Map<String, DataSource> dataSourceMap, 
+                                           final MetaDataContexts metaDataContexts, final TransactionContexts transactionContexts, final TransactionType transactionType) {
+        return STATE_MAP.get(StateContext.getCurrentState()).getConnection(dataSourceMap, metaDataContexts, transactionContexts, transactionType);
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/CircuitBreakDriverState.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/CircuitBreakDriverState.java
new file mode 100644
index 0000000..7578363
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/CircuitBreakDriverState.java
@@ -0,0 +1,40 @@
+/*
+ * 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.shardingsphere.driver.governance.internal.state.impl;
+
+import org.apache.shardingsphere.driver.governance.internal.circuit.datasource.CircuitBreakerDataSource;
+import org.apache.shardingsphere.driver.governance.internal.state.DriverState;
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.apache.shardingsphere.transaction.core.TransactionType;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.util.Map;
+
+/**
+ * Circuit break driver state.
+ */
+public final class CircuitBreakDriverState implements DriverState {
+    
+    @Override
+    public Connection getConnection(final Map<String, DataSource> dataSourceMap, 
+                                    final MetaDataContexts metaDataContexts, final TransactionContexts transactionContexts, final TransactionType transactionType) {
+        return new CircuitBreakerDataSource().getConnection();
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/LockDriverState.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/LockDriverState.java
new file mode 100644
index 0000000..13064e6
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/LockDriverState.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.driver.governance.internal.state.impl;
+
+import org.apache.shardingsphere.driver.governance.internal.state.DriverState;
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.apache.shardingsphere.transaction.core.TransactionType;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.util.Map;
+
+/**
+ * Lock driver state.
+ */
+public final class LockDriverState implements DriverState {
+    
+    @Override
+    public Connection getConnection(final Map<String, DataSource> dataSourceMap, 
+                                    final MetaDataContexts metaDataContexts, final TransactionContexts transactionContexts, final TransactionType transactionType) {
+        throw new UnsupportedOperationException("LockProxyState");
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/OKDriverState.java b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/OKDriverState.java
new file mode 100644
index 0000000..87244d6
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-governance/src/main/java/org/apache/shardingsphere/driver/governance/internal/state/impl/OKDriverState.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.shardingsphere.driver.governance.internal.state.impl;
+
+import org.apache.shardingsphere.driver.governance.internal.state.DriverState;
+import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.apache.shardingsphere.transaction.core.TransactionType;
+import org.apache.shardingsphere.transaction.core.TransactionTypeHolder;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.util.Map;
+
+/**
+ * OK driver state.
+ */
+public final class OKDriverState implements DriverState {
+    
+    @Override
+    public Connection getConnection(final Map<String, DataSource> dataSourceMap, 
+                                    final MetaDataContexts metaDataContexts, final TransactionContexts transactionContexts, final TransactionType transactionType) {
+        return new ShardingSphereConnection(dataSourceMap, metaDataContexts, transactionContexts, TransactionTypeHolder.get());
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java
index edeec76..87c2ec8 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-core/src/main/java/org/apache/shardingsphere/proxy/frontend/state/ProxyStateContext.java
@@ -37,12 +37,12 @@ import java.util.concurrent.ConcurrentHashMap;
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class ProxyStateContext {
     
-    private static final Map<StateType, ProxyState> PROXY_STATE_MAP = new ConcurrentHashMap<>(3, 1);
+    private static final Map<StateType, ProxyState> STATE_MAP = new ConcurrentHashMap<>(3, 1);
     
     static {
-        PROXY_STATE_MAP.put(StateType.OK, new OKProxyState());
-        PROXY_STATE_MAP.put(StateType.LOCK, new LockProxyState());
-        PROXY_STATE_MAP.put(StateType.CIRCUIT_BREAK, new CircuitBreakProxyState());
+        STATE_MAP.put(StateType.OK, new OKProxyState());
+        STATE_MAP.put(StateType.LOCK, new LockProxyState());
+        STATE_MAP.put(StateType.CIRCUIT_BREAK, new CircuitBreakProxyState());
     }
     
     /**
@@ -55,6 +55,6 @@ public final class ProxyStateContext {
      */
     public static void execute(final ChannelHandlerContext context, final Object message, 
                                final DatabaseProtocolFrontendEngine databaseProtocolFrontendEngine, final BackendConnection backendConnection) {
-        PROXY_STATE_MAP.get(StateContext.getCurrentState()).execute(context, message, databaseProtocolFrontendEngine, backendConnection);
+        STATE_MAP.get(StateContext.getCurrentState()).execute(context, message, databaseProtocolFrontendEngine, backendConnection);
     }
 }