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/12/11 13:19:52 UTC

[shardingsphere] branch master updated: XA transaction manager SPI (#8582)

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

zhangliang 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 c85b9f4  XA transaction manager SPI (#8582)
c85b9f4 is described below

commit c85b9f4afb38c6c70cdd2b1cce166d62f13d114b
Author: xiaoyu <54...@qq.com>
AuthorDate: Fri Dec 11 21:19:27 2020 +0800

    XA transaction manager SPI (#8582)
---
 .../properties/ConfigurationPropertyKey.java       | 20 ++++++++
 .../shardingsphere-proxy-bootstrap/pom.xml         | 20 ++++++++
 .../impl/AbstractBootstrapInitializer.java         |  4 ++
 .../manager/AtomikosTransactionManager.java        |  3 +-
 ...gsphere.transaction.xa.spi.XATransactionManager | 19 ++++++++
 .../manager/BitronixXATransactionManager.java      |  3 +-
 .../shardingsphere-transaction-xa-core/pom.xml     | 10 ++++
 .../xa/manager/XATransactionManagerLoader.java     | 17 ++++---
 .../manager/NarayanaXATransactionManager.java      | 12 +++--
 .../transaction/core/XATransactionManagerType.java | 48 +++++++++++++++++++
 .../core/XATransactionManagerTypeHolder.java       | 55 ++++++++++++++++++++++
 11 files changed, 197 insertions(+), 14 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/properties/ConfigurationPropertyKey.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/properties/ConfigurationPropertyKey.java
index f687789..3055a15 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/properties/ConfigurationPropertyKey.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/config/properties/ConfigurationPropertyKey.java
@@ -94,6 +94,26 @@ public enum ConfigurationPropertyKey implements TypedPropertyKey {
     PROXY_TRANSACTION_TYPE("proxy-transaction-type", "LOCAL", String.class),
     
     /**
+     * XA transaction manager type of proxy.
+     *
+     * <p>
+     * atomikos:
+     * ShardingSphere-Proxy will run with XA transaction with atomikos.
+     * </p>
+     *
+     * <p>
+     * narayana:
+     * ShardingSphere-Proxy will run with XA transaction with narayana.
+     * </p>
+     *
+     * <p>
+     * bitronix:
+     * ShardingSphere-Proxy will run with XA transaction with bitronix.
+     * </p>
+     */
+    PROXY_XA_TRANSACTION_MANAGER_TYPE("proxy-xa-transaction-manager-type", "atomikos", String.class),
+    
+    /**
      * Whether enable opentracing for ShardingSphere-Proxy.
      */
     PROXY_OPENTRACING_ENABLED("proxy-opentracing-enabled", String.valueOf(Boolean.FALSE), boolean.class),
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml
index 1811386..2b51cfd 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml
@@ -156,5 +156,25 @@
             <artifactId>logback-classic</artifactId>
             <scope>runtime</scope>
         </dependency>
+        <dependency>
+            <groupId>org.jboss.narayana.jta</groupId>
+            <artifactId>jta</artifactId>
+            <scope>runtime</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.jboss.narayana.jts</groupId>
+            <artifactId>narayana-jts-integration</artifactId>
+            <scope>runtime</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.jboss</groupId>
+            <artifactId>jboss-transaction-spi</artifactId>
+            <scope>runtime</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.jboss.logging</groupId>
+            <artifactId>jboss-logging</artifactId>
+            <scope>runtime</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java
index 8bac5a5..09d701b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/AbstractBootstrapInitializer.java
@@ -48,6 +48,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
 import java.util.stream.Collectors;
+import org.apache.shardingsphere.transaction.core.XATransactionManagerType;
+import org.apache.shardingsphere.transaction.core.XATransactionManagerTypeHolder;
 
 /**
  * Abstract bootstrap initializer.
@@ -100,6 +102,8 @@ public abstract class AbstractBootstrapInitializer implements BootstrapInitializ
     
     private TransactionContexts createTransactionContexts(final MetaDataContexts metaDataContexts) {
         Map<String, ShardingTransactionManagerEngine> transactionManagerEngines = new HashMap<>(metaDataContexts.getMetaDataMap().size(), 1);
+        XATransactionManagerType transactionType = XATransactionManagerType.valueFrom(metaDataContexts.getProps().getValue(ConfigurationPropertyKey.PROXY_XA_TRANSACTION_MANAGER_TYPE));
+        XATransactionManagerTypeHolder.set(transactionType);
         for (Entry<String, ShardingSphereMetaData> entry : metaDataContexts.getMetaDataMap().entrySet()) {
             ShardingTransactionManagerEngine engine = new ShardingTransactionManagerEngine();
             engine.init(metaDataContexts.getDatabaseType(), entry.getValue().getResource().getDataSources());
diff --git a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-atomikos/src/main/java/org/apache/shardingsphere/transaction/xa/atomikos/manager/AtomikosTransactionManager.java b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-atomikos/src/main/java/org/apache/shardingsphere/transaction/xa/atomikos/manager/AtomikosTransactionManager.java
index a2aab89..738a28f 100644
--- a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-atomikos/src/main/java/org/apache/shardingsphere/transaction/xa/atomikos/manager/AtomikosTransactionManager.java
+++ b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-atomikos/src/main/java/org/apache/shardingsphere/transaction/xa/atomikos/manager/AtomikosTransactionManager.java
@@ -21,6 +21,7 @@ import com.atomikos.icatch.config.UserTransactionService;
 import com.atomikos.icatch.config.UserTransactionServiceImp;
 import com.atomikos.icatch.jta.UserTransactionManager;
 import lombok.SneakyThrows;
+import org.apache.shardingsphere.transaction.core.XATransactionManagerType;
 import org.apache.shardingsphere.transaction.xa.spi.SingleXAResource;
 import org.apache.shardingsphere.transaction.xa.spi.XATransactionManager;
 
@@ -71,6 +72,6 @@ public final class AtomikosTransactionManager implements XATransactionManager {
     
     @Override
     public String getType() {
-        return "atomikos";
+        return XATransactionManagerType.ATOMIKOS.getType();
     }
 }
diff --git a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-atomikos/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.spi.XATransactionManager b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-atomikos/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.spi.XATransactionManager
new file mode 100644
index 0000000..50b47fc
--- /dev/null
+++ b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-atomikos/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.spi.XATransactionManager
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.transaction.xa.atomikos.manager.AtomikosTransactionManager
+
diff --git a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-bitronix/src/main/java/org/apache/shardingsphere/transaction/xa/bitronix/manager/BitronixXATransactionManager.java b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-bitronix/src/main/java/org/apache/shardingsphere/transaction/xa/bitronix/manager/BitronixXATransactionManager.java
index ed688fd..dac3df9 100644
--- a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-bitronix/src/main/java/org/apache/shardingsphere/transaction/xa/bitronix/manager/BitronixXATransactionManager.java
+++ b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-bitronix/src/main/java/org/apache/shardingsphere/transaction/xa/bitronix/manager/BitronixXATransactionManager.java
@@ -22,6 +22,7 @@ import bitronix.tm.TransactionManagerServices;
 import bitronix.tm.recovery.RecoveryException;
 import bitronix.tm.resource.ResourceRegistrar;
 import lombok.SneakyThrows;
+import org.apache.shardingsphere.transaction.core.XATransactionManagerType;
 import org.apache.shardingsphere.transaction.xa.spi.SingleXAResource;
 import org.apache.shardingsphere.transaction.xa.spi.XATransactionManager;
 
@@ -70,6 +71,6 @@ public final class BitronixXATransactionManager implements XATransactionManager
     
     @Override
     public String getType() {
-        return "bitronix";
+        return XATransactionManagerType.BITRONIX.getType();
     }
 }
diff --git a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/pom.xml b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/pom.xml
index fbad390..583a33c 100644
--- a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/pom.xml
+++ b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/pom.xml
@@ -37,6 +37,16 @@
             <artifactId>shardingsphere-transaction-xa-atomikos</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-transaction-xa-narayana</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-transaction-xa-bitronix</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         
         <dependency>
             <groupId>com.atomikos</groupId>
diff --git a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/manager/XATransactionManagerLoader.java b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/manager/XATransactionManagerLoader.java
index ba7d907..6c81011 100644
--- a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/manager/XATransactionManagerLoader.java
+++ b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/manager/XATransactionManagerLoader.java
@@ -19,10 +19,11 @@ package org.apache.shardingsphere.transaction.xa.manager;
 
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.transaction.core.XATransactionManagerType;
+import org.apache.shardingsphere.transaction.core.XATransactionManagerTypeHolder;
 import org.apache.shardingsphere.transaction.xa.atomikos.manager.AtomikosTransactionManager;
 import org.apache.shardingsphere.transaction.xa.spi.XATransactionManager;
 
-import java.util.Iterator;
 import java.util.ServiceLoader;
 
 /**
@@ -41,15 +42,13 @@ public final class XATransactionManagerLoader {
     }
     
     private XATransactionManager load() {
-        Iterator<XATransactionManager> xaTransactionManagers = ServiceLoader.load(XATransactionManager.class).iterator();
-        if (!xaTransactionManagers.hasNext()) {
-            return new AtomikosTransactionManager();
+        XATransactionManagerType xaTransactionManagerType = XATransactionManagerTypeHolder.get();
+        for (XATransactionManager each : ServiceLoader.load(XATransactionManager.class)) {
+            if (null != xaTransactionManagerType && each.getType().equalsIgnoreCase(xaTransactionManagerType.getType())) {
+                return each;
+            }
         }
-        XATransactionManager result = xaTransactionManagers.next();
-        if (xaTransactionManagers.hasNext()) {
-            log.warn("There are more than one transaction mangers existing, chosen first one by default.");
-        }
-        return result;
+        return new AtomikosTransactionManager();
     }
     
     /**
diff --git a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/manager/NarayanaXATransactionManager.java b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/manager/NarayanaXATransactionManager.java
index b7e3a1d..c21b6ad 100644
--- a/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/manager/NarayanaXATransactionManager.java
+++ b/shardingsphere-transaction/shardingsphere-transaction-2pc/shardingsphere-transaction-xa/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/manager/NarayanaXATransactionManager.java
@@ -21,7 +21,9 @@ import com.arjuna.ats.arjuna.recovery.RecoveryManager;
 import com.arjuna.ats.internal.jta.recovery.arjunacore.XARecoveryModule;
 import com.arjuna.ats.jbossatx.jta.RecoveryManagerService;
 import com.arjuna.ats.jta.common.jtaPropertyManager;
+import java.util.Objects;
 import lombok.SneakyThrows;
+import org.apache.shardingsphere.transaction.core.XATransactionManagerType;
 import org.apache.shardingsphere.transaction.xa.spi.SingleXAResource;
 import org.apache.shardingsphere.transaction.xa.spi.XATransactionManager;
 
@@ -50,12 +52,16 @@ public final class NarayanaXATransactionManager implements XATransactionManager
     
     @Override
     public void registerRecoveryResource(final String dataSourceName, final XADataSource xaDataSource) {
-        xaRecoveryModule.addXAResourceRecoveryHelper(new DataSourceXAResourceRecoveryHelper(xaDataSource));
+        if (Objects.nonNull(xaRecoveryModule)) {
+            xaRecoveryModule.addXAResourceRecoveryHelper(new DataSourceXAResourceRecoveryHelper(xaDataSource));
+        }
     }
     
     @Override
     public void removeRecoveryResource(final String dataSourceName, final XADataSource xaDataSource) {
-        xaRecoveryModule.removeXAResourceRecoveryHelper(new DataSourceXAResourceRecoveryHelper(xaDataSource));
+        if (Objects.nonNull(xaRecoveryModule)) {
+            xaRecoveryModule.removeXAResourceRecoveryHelper(new DataSourceXAResourceRecoveryHelper(xaDataSource));
+        }
     }
     
     @SneakyThrows({SystemException.class, RollbackException.class})
@@ -77,6 +83,6 @@ public final class NarayanaXATransactionManager implements XATransactionManager
     
     @Override
     public String getType() {
-        return "narayana";
+        return XATransactionManagerType.NARAYANA.getType();
     }
 }
diff --git a/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/core/XATransactionManagerType.java b/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/core/XATransactionManagerType.java
new file mode 100644
index 0000000..f5c55b1
--- /dev/null
+++ b/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/core/XATransactionManagerType.java
@@ -0,0 +1,48 @@
+/*
+ * 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.transaction.core;
+
+import java.util.Arrays;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+/**
+ * XA transaction manager type.
+ */
+@RequiredArgsConstructor
+@Getter
+public enum XATransactionManagerType {
+    
+    ATOMIKOS("atomikos"),
+    
+    NARAYANA("narayana"),
+    
+    BITRONIX("bitronix");
+
+    private final String type;
+    
+    /**
+     * Value from xa transaction manager type.
+     *
+     * @param type value to be xa transaction manager type
+     * @return value from xa transaction manager type
+     */
+    public static XATransactionManagerType valueFrom(final String type) {
+        return Arrays.stream(values()).filter(each -> each.type.equalsIgnoreCase(type)).findFirst().orElse(XATransactionManagerType.ATOMIKOS);
+    }
+}
diff --git a/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/core/XATransactionManagerTypeHolder.java b/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/core/XATransactionManagerTypeHolder.java
new file mode 100644
index 0000000..c86ea26
--- /dev/null
+++ b/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/core/XATransactionManagerTypeHolder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.transaction.core;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+
+/**
+ * Hold xa transaction manager type for current thread.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class XATransactionManagerTypeHolder {
+    
+    private static final ThreadLocal<XATransactionManagerType> CONTEXT = ThreadLocal.withInitial(() -> XATransactionManagerType.ATOMIKOS);
+    
+    /**
+     * Get xa transaction manager type for current thread.
+     *
+     * @return xa transaction manager type
+     */
+    public static XATransactionManagerType get() {
+        return CONTEXT.get();
+    }
+    
+    /**
+     * Set xa transaction manager type for current thread.
+     *
+     * @param xaTransactionManagerType xa transaction manager type
+     */
+    public static void set(final XATransactionManagerType xaTransactionManagerType) {
+        CONTEXT.set(xaTransactionManagerType);
+    }
+    
+    /**
+     * Clear xa transaction manger type for current thread.
+     */
+    public static void clear() {
+        CONTEXT.remove();
+    }
+}