You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2022/02/18 03:43:34 UTC

[GitHub] [shardingsphere] terrymanu commented on a change in pull request #15471: support default jdbc store for cluster mode

terrymanu commented on a change in pull request #15471:
URL: https://github.com/apache/shardingsphere/pull/15471#discussion_r809646922



##########
File path: shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-core/src/main/java/org/apache/shardingsphere/transaction/rule/TransactionRule.java
##########
@@ -34,6 +35,7 @@
     
     private final String providerType;
     
+    @Setter

Review comment:
       The final field cannot generate setter method, what is the useful of @Setter?

##########
File path: shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/MetaDataContextsBuilder.java
##########
@@ -46,13 +46,15 @@
  */
 public final class MetaDataContextsBuilder {
     
+    @Getter
     private final Map<String, SchemaConfiguration> schemaConfigMap = new LinkedHashMap<>();
     
     private final Map<String, Collection<ShardingSphereRule>> schemaRulesMap = new LinkedHashMap<>();
     
     @Getter
     private final Map<String, ShardingSphereSchema> schemaMap = new LinkedHashMap<>();
     
+    @Getter

Review comment:
       Why need getter for `globalRuleConfigs`? it can be get from `ContextManagerBuilderParameter` driectly.

##########
File path: shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/instance/ComputeNodeInstance.java
##########
@@ -32,6 +32,8 @@
     
     private InstanceDefinition instanceDefinition;
     
+    private String xaRecoveryId;

Review comment:
       Where to set XaRecoveryId? It seems always null

##########
File path: shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-type/shardingsphere-transaction-xa/shardingsphere-transaction-xa-provider/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/config/NarayanaConfigurationFileGenerator.java
##########
@@ -95,29 +107,92 @@ private NarayanaConfigEntry createEntry(final String key, final Collection<Strin
         return result;
     }
     
-    private void appendJdbcStoreConfiguration(final TransactionRule transactionRule, final NarayanaConfiguration config) {
+    private void appendUserDefinedJdbcStoreConfiguration(final TransactionRule transactionRule, final NarayanaConfiguration config) {
         String url = transactionRule.getProps().getProperty("recoveryStoreUrl");
         String user = transactionRule.getProps().getProperty("recoveryStoreUser");
         String password = String.valueOf(transactionRule.getProps().get("recoveryStorePassword"));
         String dataSourceClass = transactionRule.getProps().getProperty("recoveryStoreDataSource");
         if (null != url && null != user && null != password && null != dataSourceClass) {
-            String jdbcAccessPatten = DynamicDataSourceJDBCAccess.class.getName() + ";ClassName=%s;URL=%s;User=%s;Password=%s";
-            String jdbcAccess = String.format(jdbcAccessPatten, dataSourceClass, url, user, password);
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.tablePrefix", "Action"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.dropTable", "true"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.tablePrefix", "stateStore"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.dropTable", "true"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.tablePrefix", "Communication"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.dropTable", "true"));
+            appendJdbcStoreConfiguration(url, user, password, dataSourceClass, config);
         }
     }
     
+    private void appendJdbcStoreConfiguration(final String jdbcUrl, final String user, final String password, final String dataSourceClassName, final NarayanaConfiguration config) {
+        String jdbcAccessPatten = DynamicDataSourceJDBCAccess.class.getName() + ";ClassName=%s;URL=%s;User=%s;Password=%s";
+        String jdbcAccess = String.format(jdbcAccessPatten, dataSourceClassName, jdbcUrl, user, password);
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.tablePrefix", "Action"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.dropTable", "true"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.tablePrefix", "stateStore"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.dropTable", "true"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.tablePrefix", "Communication"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.dropTable", "true"));
+    }
+    
+    @Override
+    public Properties getTransactionProps(final TransactionRuleConfiguration transactionRuleConfiguration, final SchemaConfiguration schemaConfiguration) {
+        Properties result = new Properties();
+        if (!transactionRuleConfiguration.getProps().isEmpty()) {
+            getUserDefinedJdbcStoreConfiguration(transactionRuleConfiguration, result);
+        } else {
+            getDefaultJdbcStoreConfiguration(schemaConfiguration, result);
+        }
+        return result;
+    }
+    
+    private void getUserDefinedJdbcStoreConfiguration(final TransactionRuleConfiguration transactionRuleConfiguration, final Properties props) {

Review comment:
       The method name is `getXXX`, but why return type is void?

##########
File path: shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-type/shardingsphere-transaction-xa/shardingsphere-transaction-xa-provider/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/config/NarayanaConfigurationFileGenerator.java
##########
@@ -95,29 +107,92 @@ private NarayanaConfigEntry createEntry(final String key, final Collection<Strin
         return result;
     }
     
-    private void appendJdbcStoreConfiguration(final TransactionRule transactionRule, final NarayanaConfiguration config) {
+    private void appendUserDefinedJdbcStoreConfiguration(final TransactionRule transactionRule, final NarayanaConfiguration config) {
         String url = transactionRule.getProps().getProperty("recoveryStoreUrl");
         String user = transactionRule.getProps().getProperty("recoveryStoreUser");
         String password = String.valueOf(transactionRule.getProps().get("recoveryStorePassword"));
         String dataSourceClass = transactionRule.getProps().getProperty("recoveryStoreDataSource");
         if (null != url && null != user && null != password && null != dataSourceClass) {
-            String jdbcAccessPatten = DynamicDataSourceJDBCAccess.class.getName() + ";ClassName=%s;URL=%s;User=%s;Password=%s";
-            String jdbcAccess = String.format(jdbcAccessPatten, dataSourceClass, url, user, password);
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.tablePrefix", "Action"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.dropTable", "true"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.tablePrefix", "stateStore"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.dropTable", "true"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.tablePrefix", "Communication"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.dropTable", "true"));
+            appendJdbcStoreConfiguration(url, user, password, dataSourceClass, config);
         }
     }
     
+    private void appendJdbcStoreConfiguration(final String jdbcUrl, final String user, final String password, final String dataSourceClassName, final NarayanaConfiguration config) {
+        String jdbcAccessPatten = DynamicDataSourceJDBCAccess.class.getName() + ";ClassName=%s;URL=%s;User=%s;Password=%s";
+        String jdbcAccess = String.format(jdbcAccessPatten, dataSourceClassName, jdbcUrl, user, password);
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.tablePrefix", "Action"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.dropTable", "true"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.tablePrefix", "stateStore"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.dropTable", "true"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.tablePrefix", "Communication"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.dropTable", "true"));
+    }
+    
+    @Override
+    public Properties getTransactionProps(final TransactionRuleConfiguration transactionRuleConfiguration, final SchemaConfiguration schemaConfiguration) {
+        Properties result = new Properties();
+        if (!transactionRuleConfiguration.getProps().isEmpty()) {
+            getUserDefinedJdbcStoreConfiguration(transactionRuleConfiguration, result);
+        } else {
+            getDefaultJdbcStoreConfiguration(schemaConfiguration, result);
+        }
+        return result;
+    }
+    
+    private void getUserDefinedJdbcStoreConfiguration(final TransactionRuleConfiguration transactionRuleConfiguration, final Properties props) {
+        String url = transactionRuleConfiguration.getProps().getProperty("recoveryStoreUrl");
+        String user = transactionRuleConfiguration.getProps().getProperty("recoveryStoreUser");
+        String password = String.valueOf(transactionRuleConfiguration.getProps().get("recoveryStorePassword"));
+        String dataSourceClass = transactionRuleConfiguration.getProps().getProperty("recoveryStoreDataSource");
+        generateTransactionProps(url, user, password, dataSourceClass, props);
+    }
+    
+    private void getDefaultJdbcStoreConfiguration(final SchemaConfiguration schemaConfiguration, final Properties props) {
+        Map<String, DataSource> datasourceMap = schemaConfiguration.getDataSources();
+        Optional<DataSource> dataSource = datasourceMap.values().stream().findFirst();
+        if (dataSource.isPresent()) {
+            HikariDataSource hikariDataSource = (HikariDataSource) dataSource.get();
+            int endIndex = hikariDataSource.getJdbcUrl().indexOf("?");
+            String jdbcUrl = hikariDataSource.getJdbcUrl().substring(0, endIndex);
+            String user = hikariDataSource.getUsername();
+            String password = hikariDataSource.getPassword();
+            String dataSourceClassName = getDataSourceClassNameByJdbcUrl(jdbcUrl);
+            generateTransactionProps(jdbcUrl, user, password, dataSourceClassName, props);
+        }
+    }
+    
+    private String getDataSourceClassNameByJdbcUrl(final String jdbcUrl) {
+        DatabaseType type = DatabaseTypeRegistry.getDatabaseTypeByURL(jdbcUrl);
+        return getDataSourceClassNameByDatabaseType(type);
+    }
+    
+    private String getDataSourceClassNameByDatabaseType(final DatabaseType databaseType) {
+        switch (databaseType.getName()) {
+            case "MySQL":
+                return "com.mysql.jdbc.jdbc2.optional.MysqlDataSource";
+            case "PostgreSQL":
+                return "org.postgresql.ds.PGSimpleDataSource";
+            case "openGauss":
+                return "org.opengauss.ds.PGSimpleDataSource";
+            default:
+                throw new UnsupportedOperationException(String.format("Cannot support database type: `%s` as narayana recovery store", databaseType));

Review comment:
       Could we use SPI to decouple here?

##########
File path: shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-type/shardingsphere-transaction-xa/shardingsphere-transaction-xa-provider/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/config/NarayanaConfigurationFileGenerator.java
##########
@@ -25,34 +25,46 @@
 import com.arjuna.ats.internal.jta.recovery.arjunacore.JTANodeNameXAResourceOrphanFilter;
 import com.arjuna.ats.internal.jta.recovery.arjunacore.JTATransactionLogXAResourceOrphanFilter;
 import com.arjuna.ats.internal.jta.recovery.arjunacore.XARecoveryModule;
+import com.zaxxer.hikari.HikariDataSource;
+import org.apache.shardingsphere.infra.config.schema.SchemaConfiguration;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.transaction.config.TransactionRuleConfiguration;
 import org.apache.shardingsphere.transaction.rule.TransactionRule;
 import org.apache.shardingsphere.transaction.spi.TransactionConfigurationFileGenerator;
 
+import javax.sql.DataSource;
 import javax.xml.bind.JAXB;
 import java.io.File;
 import java.util.Collection;
 import java.util.LinkedList;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
 
 /**
  * Narayana transaction configuration file generator.
  */
 public final class NarayanaConfigurationFileGenerator implements TransactionConfigurationFileGenerator {
     
     @Override
-    public void generateFile(final TransactionRule transactionRule, final String instanceId) {
-        NarayanaConfiguration config = createDefaultConfiguration(instanceId);
+    public void generateFile(final TransactionRule transactionRule, final InstanceContext instanceContext) {
+        String instanceId = instanceContext.getInstance().getInstanceDefinition().getInstanceId().getId();
+        String recoveryId = null == instanceContext.getInstance().getXaRecoveryId() ? instanceId : instanceContext.getInstance().getXaRecoveryId();

Review comment:
       Where is the setter of XaRecoveryId? Can `XaRecoveryId` same with `instanceId`?

##########
File path: shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
##########
@@ -64,9 +67,9 @@ public ContextManager build(final ContextManagerBuilderParameter parameter) thro
         persistConfigurations(metaDataPersistService, parameter);
         RegistryCenter registryCenter = new RegistryCenter(repository);
         MetaDataContextsBuilder metaDataContextsBuilder = createMetaDataContextsBuilder(metaDataPersistService, parameter);
+        persistTransactionConfiguration(metaDataContextsBuilder.getSchemaConfigMap(), metaDataContextsBuilder.getGlobalRuleConfigs(), metaDataPersistService);

Review comment:
       Why need getter for `globalRuleConfigs`? it can be get from `ContextManagerBuilderParameter` driectly.

##########
File path: shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-type/shardingsphere-transaction-xa/shardingsphere-transaction-xa-provider/shardingsphere-transaction-xa-narayana/src/main/java/org/apache/shardingsphere/transaction/xa/narayana/config/NarayanaConfigurationFileGenerator.java
##########
@@ -95,29 +107,92 @@ private NarayanaConfigEntry createEntry(final String key, final Collection<Strin
         return result;
     }
     
-    private void appendJdbcStoreConfiguration(final TransactionRule transactionRule, final NarayanaConfiguration config) {
+    private void appendUserDefinedJdbcStoreConfiguration(final TransactionRule transactionRule, final NarayanaConfiguration config) {
         String url = transactionRule.getProps().getProperty("recoveryStoreUrl");
         String user = transactionRule.getProps().getProperty("recoveryStoreUser");
         String password = String.valueOf(transactionRule.getProps().get("recoveryStorePassword"));
         String dataSourceClass = transactionRule.getProps().getProperty("recoveryStoreDataSource");
         if (null != url && null != user && null != password && null != dataSourceClass) {
-            String jdbcAccessPatten = DynamicDataSourceJDBCAccess.class.getName() + ";ClassName=%s;URL=%s;User=%s;Password=%s";
-            String jdbcAccess = String.format(jdbcAccessPatten, dataSourceClass, url, user, password);
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.tablePrefix", "Action"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.dropTable", "true"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.tablePrefix", "stateStore"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.dropTable", "true"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.objectStoreType", JDBCStore.class.getName()));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.jdbcAccess", jdbcAccess));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.tablePrefix", "Communication"));
-            config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.dropTable", "true"));
+            appendJdbcStoreConfiguration(url, user, password, dataSourceClass, config);
         }
     }
     
+    private void appendJdbcStoreConfiguration(final String jdbcUrl, final String user, final String password, final String dataSourceClassName, final NarayanaConfiguration config) {
+        String jdbcAccessPatten = DynamicDataSourceJDBCAccess.class.getName() + ";ClassName=%s;URL=%s;User=%s;Password=%s";
+        String jdbcAccess = String.format(jdbcAccessPatten, dataSourceClassName, jdbcUrl, user, password);
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.tablePrefix", "Action"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.dropTable", "true"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.tablePrefix", "stateStore"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.stateStore.dropTable", "true"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.objectStoreType", JDBCStore.class.getName()));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.jdbcAccess", jdbcAccess));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.tablePrefix", "Communication"));
+        config.getEntries().add(createEntry("ObjectStoreEnvironmentBean.communicationStore.dropTable", "true"));
+    }
+    
+    @Override
+    public Properties getTransactionProps(final TransactionRuleConfiguration transactionRuleConfiguration, final SchemaConfiguration schemaConfiguration) {
+        Properties result = new Properties();
+        if (!transactionRuleConfiguration.getProps().isEmpty()) {
+            getUserDefinedJdbcStoreConfiguration(transactionRuleConfiguration, result);
+        } else {
+            getDefaultJdbcStoreConfiguration(schemaConfiguration, result);
+        }
+        return result;
+    }
+    
+    private void getUserDefinedJdbcStoreConfiguration(final TransactionRuleConfiguration transactionRuleConfiguration, final Properties props) {
+        String url = transactionRuleConfiguration.getProps().getProperty("recoveryStoreUrl");
+        String user = transactionRuleConfiguration.getProps().getProperty("recoveryStoreUser");
+        String password = String.valueOf(transactionRuleConfiguration.getProps().get("recoveryStorePassword"));
+        String dataSourceClass = transactionRuleConfiguration.getProps().getProperty("recoveryStoreDataSource");
+        generateTransactionProps(url, user, password, dataSourceClass, props);
+    }
+    
+    private void getDefaultJdbcStoreConfiguration(final SchemaConfiguration schemaConfiguration, final Properties props) {

Review comment:
       The method name is `getXXX`, but why return type is void?




-- 
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: notifications-unsubscribe@shardingsphere.apache.org

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