You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/05/12 07:22:17 UTC

[shardingsphere] branch master updated: Impl for ShardingSphere-JDBC driver (#17602)

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

duanzhengqiang 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 3ac9a6e5a28 Impl for ShardingSphere-JDBC driver (#17602)
3ac9a6e5a28 is described below

commit 3ac9a6e5a284b92fc99d60a20269093783e1f12d
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Thu May 12 15:22:09 2022 +0800

    Impl for ShardingSphere-JDBC driver (#17602)
    
    * Fixes #17498
    
    * For checkstyle
    
    * For checkstyle
---
 .../driver/ShardingSphereDriver.java               | 84 ++++++++++++++++++++++
 .../jdbc/core/driver/DriverDataSourceCache.java    | 66 +++++++++++++++++
 .../jdbc/core/driver/ShardingSphereDriverURL.java  | 53 ++++++++++++++
 .../driver/ShardingSphereDriverTest.java           | 43 +++++++++++
 .../core/driver/DriverDataSourceCacheTest.java     | 46 ++++++++++++
 .../core/driver/ShardingSphereDriverURLTest.java   | 46 ++++++++++++
 .../resources/META-INF/services/java.sql.Driver    | 18 +++++
 .../config/driver/bar-driver-fixture.yaml          | 18 +++++
 .../config/driver/foo-driver-fixture.yaml          | 18 +++++
 9 files changed, 392 insertions(+)

diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/ShardingSphereDriver.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/ShardingSphereDriver.java
new file mode 100644
index 00000000000..b386ab88174
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/ShardingSphereDriver.java
@@ -0,0 +1,84 @@
+/*
+ * 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;
+
+import org.apache.shardingsphere.driver.jdbc.core.driver.DriverDataSourceCache;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.logging.Logger;
+
+/**
+ * ShardingSphere driver.
+ */
+public final class ShardingSphereDriver implements Driver {
+    
+    private static final int MAJOR_DRIVER_VERSION = 5;
+    
+    private static final int MINOR_DRIVER_VERSION = 1;
+    
+    private final DriverDataSourceCache dataSourceCache = new DriverDataSourceCache();
+    
+    static {
+        try {
+            DriverManager.registerDriver(new ShardingSphereDriver());
+        } catch (final SQLException ex) {
+            throw new ShardingSphereException("Can not register driver.", ex);
+        }
+    }
+    
+    @Override
+    public Connection connect(final String url, final Properties info) throws SQLException {
+        return acceptsURL(url) ? dataSourceCache.get(url).getConnection() : null;
+    }
+    
+    @Override
+    public boolean acceptsURL(final String url) {
+        return null != url && url.startsWith("jdbc:shardingsphere:");
+    }
+    
+    @Override
+    public DriverPropertyInfo[] getPropertyInfo(final String url, final Properties info) {
+        return new DriverPropertyInfo[0];
+    }
+    
+    @Override
+    public int getMajorVersion() {
+        return MAJOR_DRIVER_VERSION;
+    }
+    
+    @Override
+    public int getMinorVersion() {
+        return MINOR_DRIVER_VERSION;
+    }
+    
+    @Override
+    public boolean jdbcCompliant() {
+        return false;
+    }
+    
+    @Override
+    public Logger getParentLogger() {
+        return Logger.getLogger(Logger.GLOBAL_LOGGER_NAME);
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/driver/DriverDataSourceCache.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/driver/DriverDataSourceCache.java
new file mode 100644
index 00000000000..7c043ab6069
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/driver/DriverDataSourceCache.java
@@ -0,0 +1,66 @@
+/*
+ * 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.jdbc.core.driver;
+
+import org.apache.shardingsphere.driver.api.yaml.YamlShardingSphereDataSourceFactory;
+import org.apache.shardingsphere.driver.jdbc.core.datasource.ShardingSphereDataSource;
+
+import javax.sql.DataSource;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Driver data source cache.
+ */
+public final class DriverDataSourceCache {
+    
+    private final Map<String, DataSource> dataSourceMap = new ConcurrentHashMap<>();
+    
+    /**
+     * Get data source.
+     * 
+     * @param url URL
+     * @return got data source
+     * @throws SQLException SQL exception
+     */
+    public DataSource get(final String url) throws SQLException {
+        if (dataSourceMap.containsKey(url)) {
+            return dataSourceMap.get(url);
+        }
+        DataSource dataSource;
+        try {
+            dataSource = YamlShardingSphereDataSourceFactory.createDataSource(new ShardingSphereDriverURL(url).toConfigurationFile());
+        } catch (final IOException ex) {
+            throw new SQLException(ex);
+        }
+        DataSource previousDataSource = dataSourceMap.putIfAbsent(url, dataSource);
+        if (null == previousDataSource) {
+            return dataSource;
+        }
+        try {
+            ((ShardingSphereDataSource) dataSource).close();
+            // CHECKSTYLE:OFF
+        } catch (Exception ex) {
+            // CHECKSTYLE:ON
+            throw new SQLException(ex);
+        }
+        return previousDataSource;
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/driver/ShardingSphereDriverURL.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/driver/ShardingSphereDriverURL.java
new file mode 100644
index 00000000000..1299271d9e2
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/driver/ShardingSphereDriverURL.java
@@ -0,0 +1,53 @@
+/*
+ * 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.jdbc.core.driver;
+
+import java.io.File;
+import java.util.Objects;
+
+/**
+ * ShardingSphere driver URL.
+ */
+public final class ShardingSphereDriverURL {
+    
+    private static final String CLASSPATH_TYPE = "classpath:";
+    
+    private final String file;
+    
+    private final boolean inClasspath;
+    
+    public ShardingSphereDriverURL(final String url) {
+        String configuredFile = url.substring("jdbc:shardingsphere:".length(), url.contains("?") ? url.indexOf("?") : url.length());
+        if (configuredFile.startsWith(CLASSPATH_TYPE)) {
+            file = configuredFile.substring(CLASSPATH_TYPE.length());
+            inClasspath = true;
+        } else {
+            file = configuredFile;
+            inClasspath = false;
+        }
+    }
+    
+    /**
+     * Generate to configuration file.
+     * 
+     * @return generated configuration file
+     */
+    public File toConfigurationFile() {
+        return new File(inClasspath ? Objects.requireNonNull(ShardingSphereDriverURL.class.getResource("/" + file), String.format("Can not find configuration file `%s`", file)).getFile() : file);
+    } 
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/ShardingSphereDriverTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/ShardingSphereDriverTest.java
new file mode 100644
index 00000000000..9f542534f9a
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/ShardingSphereDriverTest.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;
+
+import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertThat;
+
+public final class ShardingSphereDriverTest {
+    
+    @Test(expected = SQLException.class)
+    public void assertConnectWithInvalidURL() throws SQLException {
+        DriverManager.getConnection("jdbc:invalid:xxx");
+    }
+    
+    @Test
+    public void assertConnect() throws SQLException {
+        try (Connection actual = DriverManager.getConnection("jdbc:shardingsphere:classpath:config/driver/foo-driver-fixture.yaml")) {
+            assertThat(actual, instanceOf(ShardingSphereConnection.class));
+        }
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/driver/DriverDataSourceCacheTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/driver/DriverDataSourceCacheTest.java
new file mode 100644
index 00000000000..fb4d70fb861
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/driver/DriverDataSourceCacheTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.jdbc.core.driver;
+
+import org.junit.Test;
+
+import javax.sql.DataSource;
+import java.sql.SQLException;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertThat;
+
+public class DriverDataSourceCacheTest {
+    
+    @Test
+    public void assertGetNewDataSource() throws SQLException {
+        DriverDataSourceCache dataSourceCache = new DriverDataSourceCache();
+        DataSource fooDataSource = dataSourceCache.get("jdbc:shardingsphere:classpath:config/driver/foo-driver-fixture.yaml");
+        DataSource barDataSource = dataSourceCache.get("jdbc:shardingsphere:classpath:config/driver/bar-driver-fixture.yaml");
+        assertThat(fooDataSource, not(barDataSource));
+    }
+    
+    @Test
+    public void assertGetExistedDataSource() throws SQLException {
+        DriverDataSourceCache dataSourceCache = new DriverDataSourceCache();
+        DataSource dataSource1 = dataSourceCache.get("jdbc:shardingsphere:classpath:config/driver/foo-driver-fixture.yaml");
+        DataSource dataSource2 = dataSourceCache.get("jdbc:shardingsphere:classpath:config/driver/foo-driver-fixture.yaml");
+        assertThat(dataSource1, is(dataSource2));
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/driver/ShardingSphereDriverURLTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/driver/ShardingSphereDriverURLTest.java
new file mode 100644
index 00000000000..84a43c56407
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/driver/ShardingSphereDriverURLTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.jdbc.core.driver;
+
+import org.junit.Test;
+
+import java.io.File;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public final class ShardingSphereDriverURLTest {
+    
+    @Test
+    public void assertToConfigurationFile() {
+        ShardingSphereDriverURL actual = new ShardingSphereDriverURL("jdbc:shardingsphere:config/driver/foo-driver-fixture.yaml");
+        assertThat(actual.toConfigurationFile(), is(new File("config/driver/foo-driver-fixture.yaml")));
+    }
+    
+    @Test
+    public void assertToClasspathConfigurationFile() {
+        ShardingSphereDriverURL actual = new ShardingSphereDriverURL("jdbc:shardingsphere:classpath:config/driver/foo-driver-fixture.yaml");
+        assertThat(actual.toConfigurationFile().getName(), is("foo-driver-fixture.yaml"));
+    }
+    
+    @Test
+    public void assertToConfigurationFileWithOtherParameters() {
+        ShardingSphereDriverURL actual = new ShardingSphereDriverURL("jdbc:shardingsphere:config/driver/foo-driver-fixture.yaml?xxx=xxx&yyy=yyy");
+        assertThat(actual.toConfigurationFile(), is(new File("config/driver/foo-driver-fixture.yaml")));
+    }
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/META-INF/services/java.sql.Driver b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/META-INF/services/java.sql.Driver
new file mode 100644
index 00000000000..e0e769f57cb
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/META-INF/services/java.sql.Driver
@@ -0,0 +1,18 @@
+#
+# 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.driver.ShardingSphereDriver
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/driver/bar-driver-fixture.yaml b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/driver/bar-driver-fixture.yaml
new file mode 100644
index 00000000000..c98845eeb05
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/driver/bar-driver-fixture.yaml
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+schemaName: bar_driver_fixture_db
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/driver/foo-driver-fixture.yaml b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/driver/foo-driver-fixture.yaml
new file mode 100644
index 00000000000..862f18955e7
--- /dev/null
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/resources/config/driver/foo-driver-fixture.yaml
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+schemaName: foo_driver_fixture_db