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 2021/01/05 11:21:17 UTC

[GitHub] [shardingsphere] JiekerTime opened a new pull request #8897: Support different Database types for Query optimization and fix some bugs

JiekerTime opened a new pull request #8897:
URL: https://github.com/apache/shardingsphere/pull/8897


   about issue https://github.com/apache/shardingsphere/issues/8880


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] JiekerTime commented on a change in pull request #8897: Support different Database types for Query optimization and fix some bugs

Posted by GitBox <gi...@apache.org>.
JiekerTime commented on a change in pull request #8897:
URL: https://github.com/apache/shardingsphere/pull/8897#discussion_r552377091



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/CalciteContextFactory.java
##########
@@ -54,68 +63,106 @@
  */
 public final class CalciteContextFactory {
 
-    private final Properties properties;
-    
+    private static final String LEX_CAMEL_NAME = CalciteConnectionProperty.LEX.camelName();
+
+    private static final String CONFORMANCE_CAMEL_NAME = CalciteConnectionProperty.CONFORMANCE.camelName();
+
+    private final Properties properties = new Properties();
+
     private final CalciteConnectionConfig connectionConfig;
-    
+
     private final Config parserConfig;
-    
+
     private final RelDataTypeFactory typeFactory;
-    
+
     private final CalciteLogicSchemaFactory factory;
-    
+
     private final RelOptCluster cluster;
-    
+
     public CalciteContextFactory(final Map<String, ShardingSphereMetaData> metaDataMap) {
-        properties = createProperties();
+        factory = new CalciteLogicSchemaFactory(metaDataMap);
+        DatabaseType databaseType = metaDataMap.values().iterator().next().getResource().getDatabaseType();
+        assert databaseType != null;
+        initProperties(databaseType);
         connectionConfig = new CalciteConnectionConfigImpl(properties);
         parserConfig = SqlParser.config()
                 .withLex(connectionConfig.lex())
                 .withIdentifierMaxLength(SqlParser.DEFAULT_IDENTIFIER_MAX_LENGTH)
                 .withConformance(connectionConfig.conformance())
                 .withParserFactory(SqlParserImpl.FACTORY);
         typeFactory = new JavaTypeFactoryImpl();
-        factory = new CalciteLogicSchemaFactory(metaDataMap);
         cluster = newCluster();
     }
-    
-    private Properties createProperties() {
-        // TODO Not only MySQL here.
-        Properties result = new Properties();
-        result.setProperty(CalciteConnectionProperty.LEX.camelName(), Lex.MYSQL.name());
-        result.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(), SqlConformanceEnum.MYSQL_5.name());
-        return result;
+
+    private void initProperties(final DatabaseType databaseType) {
+        // TODO Logic could be improved.
+        if (databaseType instanceof H2DatabaseType) {
+            // TODO No suitable type of Lex
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.DEFAULT.name());
+            return;
+        }
+        if (databaseType instanceof MariaDBDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.MYSQL_5.name());
+            return;
+        }
+        if (databaseType instanceof MySQLDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.MYSQL_5.name());
+            return;
+        }
+        if (databaseType instanceof OracleDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.ORACLE.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.ORACLE_12.name());
+            return;
+        }
+        if (databaseType instanceof PostgreSQLDatabaseType) {
+            // TODO No suitable type of Lex and conformance
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.DEFAULT.name());
+            return;
+        }
+        if (databaseType instanceof SQL92DatabaseType) {
+            // TODO No suitable type of Lex
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.STRICT_92.name());
+            return;
+        }
+        if (databaseType instanceof SQLServerDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.SQL_SERVER.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.SQL_SERVER_2008.name());
+            return;
+        }
+        throw new ShardingSphereException("No matching DatabaseType found");

Review comment:
       In order to maintain the robustness of the code and avoid some unknown errors, please allow me to keep this part....




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] JiekerTime closed pull request #8897: Support different Database types for Query optimization and fix some bugs

Posted by GitBox <gi...@apache.org>.
JiekerTime closed pull request #8897:
URL: https://github.com/apache/shardingsphere/pull/8897


   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on a change in pull request #8897: Support different Database types for Query optimization and fix some bugs

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #8897:
URL: https://github.com/apache/shardingsphere/pull/8897#discussion_r552332495



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/context/CalciteContextFactoryTest.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.infra.optimize.context;
+
+import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
+import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.optimize.schema.CalciteLogicSchema;
+import org.apache.shardingsphere.infra.optimize.schema.row.CalciteRowExecutor;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public class CalciteContextFactoryTest {

Review comment:
       Define it as final class plz

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/context/CalciteContextFactoryTest.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.infra.optimize.context;
+
+import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
+import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.optimize.schema.CalciteLogicSchema;
+import org.apache.shardingsphere.infra.optimize.schema.row.CalciteRowExecutor;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public class CalciteContextFactoryTest {
+
+    @Test
+    public void createTest() {
+        Map<String, ShardingSphereMetaData> metaDataMap = new HashMap<>();
+        ShardingSphereResource shardingSphereResource = new ShardingSphereResource(null, null, null, new H2DatabaseType());
+        TableMetaData tableMetaData = new TableMetaData();
+        ShardingSphereSchema schema = new ShardingSphereSchema();
+        schema.put("tab_user", tableMetaData);
+        ShardingSphereRuleMetaData metaData = new ShardingSphereRuleMetaData(new ArrayList<>(), new ArrayList<>());
+        ShardingSphereMetaData shardingSphereMetaData = new ShardingSphereMetaData("logic_db", shardingSphereResource, metaData, schema);
+        metaDataMap.put("logic_db", shardingSphereMetaData);
+        CalciteContextFactory calciteContextFactory = new CalciteContextFactory(metaDataMap);
+        assertNotNull(calciteContextFactory);
+        CalciteContext logicDb = calciteContextFactory.create("logic_db", new CalciteRowExecutor(new ArrayList<>(), 0, null, new JDBCExecutor(null, true), null, null));
+        assertNotNull(logicDb);
+        Properties properties = logicDb.getConnectionProperties();
+        assertNotNull(properties);
+        assertThat(properties.getProperty("lex"), is("MYSQL"));
+        assertThat(properties.getProperty("conformance"), is("DEFAULT"));
+        CalciteLogicSchema calciteLogicSchema = logicDb.getCalciteLogicSchema();
+        assertNotNull(calciteLogicSchema);
+        assertThat(calciteLogicSchema.getName(), is("logic_db"));
+    }
+

Review comment:
       Please remove this redundant blank line.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/schema/CalciteLogicSchemaFactory.java
##########
@@ -44,7 +44,7 @@ public CalciteLogicSchemaFactory(final Map<String, ShardingSphereMetaData> metaD
             }
         }
     }
-    
+

Review comment:
       Please check style for these new blank lines.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/CalciteContextFactory.java
##########
@@ -54,68 +63,106 @@
  */
 public final class CalciteContextFactory {
 
-    private final Properties properties;
-    
+    private static final String LEX_CAMEL_NAME = CalciteConnectionProperty.LEX.camelName();
+
+    private static final String CONFORMANCE_CAMEL_NAME = CalciteConnectionProperty.CONFORMANCE.camelName();
+
+    private final Properties properties = new Properties();
+
     private final CalciteConnectionConfig connectionConfig;
-    
+
     private final Config parserConfig;
-    
+
     private final RelDataTypeFactory typeFactory;
-    
+
     private final CalciteLogicSchemaFactory factory;
-    
+
     private final RelOptCluster cluster;
-    
+
     public CalciteContextFactory(final Map<String, ShardingSphereMetaData> metaDataMap) {
-        properties = createProperties();
+        factory = new CalciteLogicSchemaFactory(metaDataMap);
+        DatabaseType databaseType = metaDataMap.values().iterator().next().getResource().getDatabaseType();
+        assert databaseType != null;
+        initProperties(databaseType);
         connectionConfig = new CalciteConnectionConfigImpl(properties);
         parserConfig = SqlParser.config()
                 .withLex(connectionConfig.lex())
                 .withIdentifierMaxLength(SqlParser.DEFAULT_IDENTIFIER_MAX_LENGTH)
                 .withConformance(connectionConfig.conformance())
                 .withParserFactory(SqlParserImpl.FACTORY);
         typeFactory = new JavaTypeFactoryImpl();
-        factory = new CalciteLogicSchemaFactory(metaDataMap);
         cluster = newCluster();
     }
-    
-    private Properties createProperties() {
-        // TODO Not only MySQL here.
-        Properties result = new Properties();
-        result.setProperty(CalciteConnectionProperty.LEX.camelName(), Lex.MYSQL.name());
-        result.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(), SqlConformanceEnum.MYSQL_5.name());
-        return result;
+
+    private void initProperties(final DatabaseType databaseType) {
+        // TODO Logic could be improved.
+        if (databaseType instanceof H2DatabaseType) {
+            // TODO No suitable type of Lex
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.DEFAULT.name());
+            return;
+        }
+        if (databaseType instanceof MariaDBDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.MYSQL_5.name());
+            return;
+        }
+        if (databaseType instanceof MySQLDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.MYSQL_5.name());
+            return;
+        }
+        if (databaseType instanceof OracleDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.ORACLE.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.ORACLE_12.name());
+            return;
+        }
+        if (databaseType instanceof PostgreSQLDatabaseType) {
+            // TODO No suitable type of Lex and conformance
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.DEFAULT.name());
+            return;
+        }
+        if (databaseType instanceof SQL92DatabaseType) {
+            // TODO No suitable type of Lex
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.STRICT_92.name());
+            return;
+        }
+        if (databaseType instanceof SQLServerDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.SQL_SERVER.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.SQL_SERVER_2008.name());
+            return;
+        }
+        throw new ShardingSphereException("No matching DatabaseType found");
     }
-    
+
     private RelOptCluster newCluster() {
         RelOptPlanner planner = new VolcanoPlanner();
         PlannerInitializer.init(planner);
         return RelOptCluster.create(planner, new RexBuilder(typeFactory));
     }
-    
+
     /**
      * Create.
      *
-     * @param schema schema
+     * @param schema  schema

Review comment:
       Why do you add a space?

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/CalciteContextFactory.java
##########
@@ -54,68 +63,106 @@
  */
 public final class CalciteContextFactory {
 
-    private final Properties properties;
-    
+    private static final String LEX_CAMEL_NAME = CalciteConnectionProperty.LEX.camelName();
+
+    private static final String CONFORMANCE_CAMEL_NAME = CalciteConnectionProperty.CONFORMANCE.camelName();
+
+    private final Properties properties = new Properties();
+
     private final CalciteConnectionConfig connectionConfig;
-    
+
     private final Config parserConfig;
-    
+
     private final RelDataTypeFactory typeFactory;
-    
+
     private final CalciteLogicSchemaFactory factory;
-    
+
     private final RelOptCluster cluster;
-    
+
     public CalciteContextFactory(final Map<String, ShardingSphereMetaData> metaDataMap) {
-        properties = createProperties();
+        factory = new CalciteLogicSchemaFactory(metaDataMap);
+        DatabaseType databaseType = metaDataMap.values().iterator().next().getResource().getDatabaseType();
+        assert databaseType != null;

Review comment:
       `Preconditions.checkArgument` is recommended.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/CalciteContextFactory.java
##########
@@ -54,68 +63,106 @@
  */
 public final class CalciteContextFactory {
 
-    private final Properties properties;
-    
+    private static final String LEX_CAMEL_NAME = CalciteConnectionProperty.LEX.camelName();
+
+    private static final String CONFORMANCE_CAMEL_NAME = CalciteConnectionProperty.CONFORMANCE.camelName();
+
+    private final Properties properties = new Properties();
+
     private final CalciteConnectionConfig connectionConfig;
-    
+
     private final Config parserConfig;
-    
+
     private final RelDataTypeFactory typeFactory;
-    
+
     private final CalciteLogicSchemaFactory factory;
-    
+
     private final RelOptCluster cluster;
-    
+
     public CalciteContextFactory(final Map<String, ShardingSphereMetaData> metaDataMap) {
-        properties = createProperties();
+        factory = new CalciteLogicSchemaFactory(metaDataMap);
+        DatabaseType databaseType = metaDataMap.values().iterator().next().getResource().getDatabaseType();
+        assert databaseType != null;
+        initProperties(databaseType);
         connectionConfig = new CalciteConnectionConfigImpl(properties);
         parserConfig = SqlParser.config()
                 .withLex(connectionConfig.lex())
                 .withIdentifierMaxLength(SqlParser.DEFAULT_IDENTIFIER_MAX_LENGTH)
                 .withConformance(connectionConfig.conformance())
                 .withParserFactory(SqlParserImpl.FACTORY);
         typeFactory = new JavaTypeFactoryImpl();
-        factory = new CalciteLogicSchemaFactory(metaDataMap);
         cluster = newCluster();
     }
-    
-    private Properties createProperties() {
-        // TODO Not only MySQL here.
-        Properties result = new Properties();
-        result.setProperty(CalciteConnectionProperty.LEX.camelName(), Lex.MYSQL.name());
-        result.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(), SqlConformanceEnum.MYSQL_5.name());
-        return result;
+
+    private void initProperties(final DatabaseType databaseType) {
+        // TODO Logic could be improved.
+        if (databaseType instanceof H2DatabaseType) {
+            // TODO No suitable type of Lex
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.DEFAULT.name());
+            return;
+        }
+        if (databaseType instanceof MariaDBDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.MYSQL_5.name());
+            return;
+        }
+        if (databaseType instanceof MySQLDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.MYSQL_5.name());
+            return;
+        }
+        if (databaseType instanceof OracleDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.ORACLE.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.ORACLE_12.name());
+            return;
+        }
+        if (databaseType instanceof PostgreSQLDatabaseType) {
+            // TODO No suitable type of Lex and conformance
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.DEFAULT.name());
+            return;
+        }
+        if (databaseType instanceof SQL92DatabaseType) {
+            // TODO No suitable type of Lex
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.MYSQL.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.STRICT_92.name());
+            return;
+        }
+        if (databaseType instanceof SQLServerDatabaseType) {
+            this.properties.setProperty(LEX_CAMEL_NAME, Lex.SQL_SERVER.name());
+            this.properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.SQL_SERVER_2008.name());
+            return;
+        }
+        throw new ShardingSphereException("No matching DatabaseType found");

Review comment:
       I guess there is no any chance running here, right? Please consider removing it.




----------------------------------------------------------------
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.

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