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/04/26 03:22:35 UTC

[GitHub] [shardingsphere] strongduanmu opened a new pull request #10193: fix bad rewritten result for foreign key SQL

strongduanmu opened a new pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193


   Fixes #10171.
   
   Changes proposed in this pull request:
   - fix bad rewritten result for foreign key SQL when user config single data node sharding config
   - add some test cases for single data node table rewrite


-- 
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 #10193: fix bad rewritten result for foreign key SQL

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



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sharding.route.engine.validator.ddl.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.rule.TableRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Sharding alter table statement validator.
+ */
+public final class ShardingAlterTableStatementValidator extends ShardingDDLStatementValidator<AlterTableStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        Collection<String> shardingLogicTableNames = getShardingLogicTableNames(shardingRule, sqlStatementContext);
+        if (shardingLogicTableNames.size() <= 1) {
+            return;
+        }

Review comment:
       How about `broadcast Table`or `single table`?
   Is #10172 a interim solution?




-- 
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] strongduanmu commented on a change in pull request #10193: fix bad rewritten result for foreign key SQL

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



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sharding.route.engine.validator.ddl.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.rule.TableRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Sharding alter table statement validator.
+ */
+public final class ShardingAlterTableStatementValidator extends ShardingDDLStatementValidator<AlterTableStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        Collection<String> shardingLogicTableNames = getShardingLogicTableNames(shardingRule, sqlStatementContext);
+        if (shardingLogicTableNames.size() <= 1) {
+            return;
+        }

Review comment:
       > How about `broadcast Table`or `single table`?
   > Is #10172 a interim solution?
   
   @tristaZero Sorry for late reply, I spent some time sorting out the different scenes. According to the table type, there are sharding, broadcast and single table in SS. By combining with Cartesian product, we can get nine combination types.
   
   
   case | primary table | referenced table
   -- | -- | --
   1 | sharding | sharding
   2 | sharding | broadcast
   3 | sharding | single
   4 | broadcast | sharding
   5 | broadcast | broadcast
   6 | broadcast | single
   7 | single | sharding
   8 | single | broadcast
   9 | single | single
   
   ## case 1: sharding * sharding 
   There are two situations in this scenario. The sharding tables can be binding tables or unbinding tables. 
   
   Binding table's routing logic is consistent with current logic. The only difference is that the logic is advanced to the route module for processing. Routing of the primary table is processed first, and then other binding tables are also routed with the same rule according to the binding relationship.
   
   The routing logic of the unbinding tables use Cartesian product way, and different routing results will appear for different sharding configurations.
   
   The routing logic of the unbinding tables use the Cartesian product way, but some routing results are illegal for DDL statements.
   
   Suppose we have the following sharding configuration.
   
   ```yaml
   rules:
   - !SHARDING
    tables:
      t_order:
        actualDataNodes: test_${0..1}.t_order_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_inline
        keyGenerateStrategy:
          column: order_id
          keyGeneratorName: snowflake
      t_order_item:
        actualDataNodes: test_${0..1}.t_order_item_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_item_inline
        keyGenerateStrategy:
          column: order_item_id
          keyGeneratorName: snowflake
   ```
   The table of `t_order` and `t_order_item` are not binding tables.  After routing, we will get the following results. Obviously, we can not execute the `alter table t_order_item_1 add constraint t_order_fkey ... ` statement multiple times under the same data source. Therefore, this routing result needs to throw an exception in the `postValidate` method.
   
   ```
   [INFO ] 2021-04-28 17:22:13.853 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   ```
   
   ## case 2: sharding * broadcast
   
   ## case 3: sharding * single
   
   ## case 4: broadcast * sharding
   
   ## case 5: broadcast * broadcast
   
   ## case 6: broadcast * single
   
   ## case 7: single * sharding
   
   ## case 8: single * broadcast
   
   ## case 9: single * single
   




-- 
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] strongduanmu commented on a change in pull request #10193: fix bad rewritten result for foreign key SQL

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



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sharding.route.engine.validator.ddl.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.rule.TableRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Sharding alter table statement validator.
+ */
+public final class ShardingAlterTableStatementValidator extends ShardingDDLStatementValidator<AlterTableStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        Collection<String> shardingLogicTableNames = getShardingLogicTableNames(shardingRule, sqlStatementContext);
+        if (shardingLogicTableNames.size() <= 1) {
+            return;
+        }

Review comment:
       > How about `broadcast Table`or `single table`?
   > Is #10172 a interim solution?
   
   @tristaZero Sorry for late reply, I spent some time sorting out the different scenes. According to the table type, there are sharding, broadcast and single table in SS. By combining with Cartesian product, we can get nine combination types.
   
   
   case | primary table | referenced table
   -- | -- | --
   1 | sharding | sharding
   2 | sharding | broadcast
   3 | sharding | single
   4 | broadcast | sharding
   5 | broadcast | broadcast
   6 | broadcast | single
   7 | single | sharding
   8 | single | broadcast
   9 | single | single
   
   ## case 1: sharding * sharding 
   There are two situations in this scenario. The sharding tables can be binding tables or unbinding tables. 
   
   Binding table's routing logic is consistent with current logic. The only difference is that the logic is advanced to the route module for processing. Routing of the primary table is processed first, and then other binding tables are also routed with the same rule according to the binding relationship.
   
   The routing logic of the unbinding tables use the Cartesian product way, but some routing results are illegal for DDL statements.
   
   Suppose we have the following sharding configuration.
   
   ```yaml
   rules:
   - !SHARDING
    tables:
      t_order:
        actualDataNodes: test_${0..1}.t_order_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_inline
        keyGenerateStrategy:
          column: order_id
          keyGeneratorName: snowflake
      t_order_item:
        actualDataNodes: test_${0..1}.t_order_item_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_item_inline
        keyGenerateStrategy:
          column: order_item_id
          keyGeneratorName: snowflake
   ```
   The table of `t_order` and `t_order_item` are not binding tables.  After routing, we will get the following results. Obviously, we can not execute the `alter table t_order_item_1 add constraint t_order_fkey ... ` statement multiple times under the same data source. Therefore, this routing result needs to throw an exception in the `postValidate` method. BTW, this validate logic is also applicable to other scenarios where Cartesian product routing is used.
   
   ```
   [INFO ] 2021-04-28 17:22:13.853 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_1 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   ```
   
   There are also some special sharding config, such as the sharding rules that only configure a single data node.
   
   ```yaml
   rules:
   - !SHARDING
    tables:
      t_order:
         actualDataNodes: test_0.t_order_0
      t_order_item:
         actualDataNodes: test_0.t_order_item_0
   ```
   When Cartesian product routing is used in this scenario, the abnormal routing results described above will not occur, so it can be executed normally.
   
   ```
   [INFO ] 2021-04-28 17:22:13.854 [pool-21-thread-1] ShardingSphere-SQL - Actual SQL: test_0 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   ```
   
   
   ## case 2: sharding * broadcast
   
   ## case 3: sharding * single
   
   ## case 4: broadcast * sharding
   
   ## case 5: broadcast * broadcast
   
   ## case 6: broadcast * single
   
   ## case 7: single * sharding
   
   ## case 8: single * broadcast
   
   ## case 9: single * single
   




-- 
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] strongduanmu commented on a change in pull request #10193: fix bad rewritten result for foreign key SQL

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



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sharding.route.engine.validator.ddl.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.rule.TableRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Sharding alter table statement validator.
+ */
+public final class ShardingAlterTableStatementValidator extends ShardingDDLStatementValidator<AlterTableStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        Collection<String> shardingLogicTableNames = getShardingLogicTableNames(shardingRule, sqlStatementContext);
+        if (shardingLogicTableNames.size() <= 1) {
+            return;
+        }

Review comment:
       > How about `broadcast Table`or `single table`?
   > Is #10172 a interim solution?
   
   @tristaZero Sorry for late reply, I spent some time sorting out the different scenes. According to the table type, there are sharding, broadcast and single table in SS. By combining with Cartesian product, we can get nine combination types.
   
   
   




-- 
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] codecov-commenter edited a comment on pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-826483708


   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10193](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (999fbe9) into [master](https://codecov.io/gh/apache/shardingsphere/commit/11320534213183e9716cd867c4d4b3fda02ef4f4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1132053) will **decrease** coverage by `0.01%`.
   > The diff coverage is `60.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/shardingsphere/pull/10193/graphs/tree.svg?width=650&height=150&src=pr&token=ZvlXpWa7so&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #10193      +/-   ##
   ============================================
   - Coverage     67.96%   67.94%   -0.02%     
     Complexity      683      683              
   ============================================
     Files          1703     1704       +1     
     Lines         29201    29252      +51     
     Branches       5250     5261      +11     
   ============================================
   + Hits          19845    19875      +30     
   - Misses         7826     7844      +18     
   - Partials       1530     1533       +3     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...che/shardingsphere/sharding/rule/ShardingRule.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9zaGFyZGluZy9ydWxlL1NoYXJkaW5nUnVsZS5qYXZh) | `69.69% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...e/validator/ShardingStatementValidatorFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS92YWxpZGF0b3IvU2hhcmRpbmdTdGF0ZW1lbnRWYWxpZGF0b3JGYWN0b3J5LmphdmE=) | `13.79% <0.00%> (-1.03%)` | `0.00 <0.00> (ø)` | |
   | [...broadcast/ShardingTableBroadcastRoutingEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS90eXBlL2Jyb2FkY2FzdC9TaGFyZGluZ1RhYmxlQnJvYWRjYXN0Um91dGluZ0VuZ2luZS5qYXZh) | `72.30% <43.75%> (-27.70%)` | `0.00 <0.00> (ø)` | |
   | [.../route/engine/type/ShardingRouteEngineFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS90eXBlL1NoYXJkaW5nUm91dGVFbmdpbmVGYWN0b3J5LmphdmE=) | `68.96% <66.66%> (-0.68%)` | `0.00 <0.00> (ø)` | |
   | [...ddl/impl/ShardingAlterTableStatementValidator.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS92YWxpZGF0b3IvZGRsL2ltcGwvU2hhcmRpbmdBbHRlclRhYmxlU3RhdGVtZW50VmFsaWRhdG9yLmphdmE=) | `89.47% <89.47%> (ø)` | `0.00 <0.00> (?)` | |
   | [...sphere/sharding/rewrite/token/pojo/TableToken.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcmV3cml0ZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvc2hhcmRpbmcvcmV3cml0ZS90b2tlbi9wb2pvL1RhYmxlVG9rZW4uamF2YQ==) | `90.00% <100.00%> (+3.63%)` | `0.00 <0.00> (ø)` | |
   | [...e/type/complex/ShardingCartesianRoutingEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS90eXBlL2NvbXBsZXgvU2hhcmRpbmdDYXJ0ZXNpYW5Sb3V0aW5nRW5naW5lLmphdmE=) | `95.45% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...ardingsphere/infra/route/context/RouteContext.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL3JvdXRlL2NvbnRleHQvUm91dGVDb250ZXh0LmphdmE=) | `75.00% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...rite/token/generator/impl/TableTokenGenerator.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcmV3cml0ZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvc2hhcmRpbmcvcmV3cml0ZS90b2tlbi9nZW5lcmF0b3IvaW1wbC9UYWJsZVRva2VuR2VuZXJhdG9yLmphdmE=) | `88.88% <0.00%> (+11.11%)` | `1.00% <0.00%> (ø%)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [1132053...999fbe9](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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] strongduanmu commented on pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-829749534


   > So we can judge whether these SQLs are supported or not by `preValidate` and `postValidate`, right? `preValidate` is for table type checking, and `postVaidate` is for routed result checking?
   
   @tristaZero Yes, `preValidate` currently verifies the data source of single table, and other scenarios can handle routing well through ShardingTableBroadcastRoutingEngine.
   


-- 
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] strongduanmu commented on pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-830777676


   > Hi , so could we merge this PR for more supported cases? But the CI is still failing.
   
   @tristaZero OK, I will fix the ci exception.


-- 
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 pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-830776710


   Hi ,  so could we merge this PR for more supported cases? But the CI is still failing.


-- 
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 merged pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
tristaZero merged pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193


   


-- 
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] codecov-commenter commented on pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-826483708


   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10193](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (45f63bb) into [master](https://codecov.io/gh/apache/shardingsphere/commit/e66e4de6b2d66529754e433614361324727a265f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e66e4de) will **decrease** coverage by `0.07%`.
   > The diff coverage is `54.05%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/shardingsphere/pull/10193/graphs/tree.svg?width=650&height=150&src=pr&token=ZvlXpWa7so&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #10193      +/-   ##
   ============================================
   - Coverage     68.05%   67.98%   -0.08%     
     Complexity      680      680              
   ============================================
     Files          1696     1697       +1     
     Lines         29066    29145      +79     
     Branches       5205     5226      +21     
   ============================================
   + Hits          19782    19815      +33     
   - Misses         7768     7809      +41     
   - Partials       1516     1521       +5     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...e/validator/ShardingStatementValidatorFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS92YWxpZGF0b3IvU2hhcmRpbmdTdGF0ZW1lbnRWYWxpZGF0b3JGYWN0b3J5LmphdmE=) | `13.79% <0.00%> (-1.03%)` | `0.00 <0.00> (ø)` | |
   | [...broadcast/ShardingTableBroadcastRoutingEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS90eXBlL2Jyb2FkY2FzdC9TaGFyZGluZ1RhYmxlQnJvYWRjYXN0Um91dGluZ0VuZ2luZS5qYXZh) | `53.84% <23.52%> (-14.73%)` | `0.00 <0.00> (ø)` | |
   | [...ddl/impl/ShardingAlterTableStatementValidator.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS92YWxpZGF0b3IvZGRsL2ltcGwvU2hhcmRpbmdBbHRlclRhYmxlU3RhdGVtZW50VmFsaWRhdG9yLmphdmE=) | `87.50% <87.50%> (ø)` | `0.00 <0.00> (?)` | |
   | [...che/shardingsphere/sharding/rule/ShardingRule.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9zaGFyZGluZy9ydWxlL1NoYXJkaW5nUnVsZS5qYXZh) | `70.14% <100.00%> (+0.45%)` | `0.00 <0.00> (ø)` | |
   | [.../frontend/postgresql/PostgreSQLFrontendEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtcHJveHkvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQtcG9zdGdyZXNxbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvcHJveHkvZnJvbnRlbmQvcG9zdGdyZXNxbC9Qb3N0Z3JlU1FMRnJvbnRlbmRFbmdpbmUuamF2YQ==) | `66.66% <0.00%> (-33.34%)` | `0.00% <0.00%> (ø%)` | |
   | [...sphere/scaling/core/job/schedule/JobScheduler.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9qb2Ivc2NoZWR1bGUvSm9iU2NoZWR1bGVyLmphdmE=) | `22.22% <0.00%> (-29.63%)` | `0.00% <0.00%> (ø%)` | |
   | [...re/scaling/core/executor/engine/ExecuteEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9leGVjdXRvci9lbmdpbmUvRXhlY3V0ZUVuZ2luZS5qYXZh) | `88.88% <0.00%> (-11.12%)` | `0.00% <0.00%> (ø%)` | |
   | [...scaling/core/job/task/inventory/InventoryTask.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9qb2IvdGFzay9pbnZlbnRvcnkvSW52ZW50b3J5VGFzay5qYXZh) | `68.88% <0.00%> (-6.67%)` | `0.00% <0.00%> (ø%)` | |
   | [...ing/core/job/task/incremental/IncrementalTask.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9qb2IvdGFzay9pbmNyZW1lbnRhbC9JbmNyZW1lbnRhbFRhc2suamF2YQ==) | `72.91% <0.00%> (-2.09%)` | `0.00% <0.00%> (ø%)` | |
   | [...sphere/proxy/frontend/state/impl/OKProxyState.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtcHJveHkvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvcHJveHkvZnJvbnRlbmQvc3RhdGUvaW1wbC9PS1Byb3h5U3RhdGUuamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | ... and [10 more](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e66e4de...45f63bb](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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] strongduanmu edited a comment on pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
strongduanmu edited a comment on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-829749534


   > So we can judge whether these SQLs are supported or not by `preValidate` and `postValidate`, right? `preValidate` is for table type checking, and `postVaidate` is for routed result checking?
   
   @tristaZero Yes, `preValidate` currently verifies the data source of single table, and other scenarios can handle routing well through `ShardingTableBroadcastRoutingEngine`.
   


-- 
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] codecov-commenter commented on pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-826483708


   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10193](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (45f63bb) into [master](https://codecov.io/gh/apache/shardingsphere/commit/e66e4de6b2d66529754e433614361324727a265f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e66e4de) will **decrease** coverage by `0.07%`.
   > The diff coverage is `54.05%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/shardingsphere/pull/10193/graphs/tree.svg?width=650&height=150&src=pr&token=ZvlXpWa7so&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #10193      +/-   ##
   ============================================
   - Coverage     68.05%   67.98%   -0.08%     
     Complexity      680      680              
   ============================================
     Files          1696     1697       +1     
     Lines         29066    29145      +79     
     Branches       5205     5226      +21     
   ============================================
   + Hits          19782    19815      +33     
   - Misses         7768     7809      +41     
   - Partials       1516     1521       +5     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...e/validator/ShardingStatementValidatorFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS92YWxpZGF0b3IvU2hhcmRpbmdTdGF0ZW1lbnRWYWxpZGF0b3JGYWN0b3J5LmphdmE=) | `13.79% <0.00%> (-1.03%)` | `0.00 <0.00> (ø)` | |
   | [...broadcast/ShardingTableBroadcastRoutingEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS90eXBlL2Jyb2FkY2FzdC9TaGFyZGluZ1RhYmxlQnJvYWRjYXN0Um91dGluZ0VuZ2luZS5qYXZh) | `53.84% <23.52%> (-14.73%)` | `0.00 <0.00> (ø)` | |
   | [...ddl/impl/ShardingAlterTableStatementValidator.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctcm91dGUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NoYXJkaW5nL3JvdXRlL2VuZ2luZS92YWxpZGF0b3IvZGRsL2ltcGwvU2hhcmRpbmdBbHRlclRhYmxlU3RhdGVtZW50VmFsaWRhdG9yLmphdmE=) | `87.50% <87.50%> (ø)` | `0.00 <0.00> (?)` | |
   | [...che/shardingsphere/sharding/rule/ShardingRule.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZmVhdHVyZXMvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmcvc2hhcmRpbmdzcGhlcmUtc2hhcmRpbmctY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9zaGFyZGluZy9ydWxlL1NoYXJkaW5nUnVsZS5qYXZh) | `70.14% <100.00%> (+0.45%)` | `0.00 <0.00> (ø)` | |
   | [.../frontend/postgresql/PostgreSQLFrontendEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtcHJveHkvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQtcG9zdGdyZXNxbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvcHJveHkvZnJvbnRlbmQvcG9zdGdyZXNxbC9Qb3N0Z3JlU1FMRnJvbnRlbmRFbmdpbmUuamF2YQ==) | `66.66% <0.00%> (-33.34%)` | `0.00% <0.00%> (ø%)` | |
   | [...sphere/scaling/core/job/schedule/JobScheduler.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9qb2Ivc2NoZWR1bGUvSm9iU2NoZWR1bGVyLmphdmE=) | `22.22% <0.00%> (-29.63%)` | `0.00% <0.00%> (ø%)` | |
   | [...re/scaling/core/executor/engine/ExecuteEngine.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9leGVjdXRvci9lbmdpbmUvRXhlY3V0ZUVuZ2luZS5qYXZh) | `88.88% <0.00%> (-11.12%)` | `0.00% <0.00%> (ø%)` | |
   | [...scaling/core/job/task/inventory/InventoryTask.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9qb2IvdGFzay9pbnZlbnRvcnkvSW52ZW50b3J5VGFzay5qYXZh) | `68.88% <0.00%> (-6.67%)` | `0.00% <0.00%> (ø%)` | |
   | [...ing/core/job/task/incremental/IncrementalTask.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtc2NhbGluZy9zaGFyZGluZ3NwaGVyZS1zY2FsaW5nLWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3NjYWxpbmcvY29yZS9qb2IvdGFzay9pbmNyZW1lbnRhbC9JbmNyZW1lbnRhbFRhc2suamF2YQ==) | `72.91% <0.00%> (-2.09%)` | `0.00% <0.00%> (ø%)` | |
   | [...sphere/proxy/frontend/state/impl/OKProxyState.java](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtcHJveHkvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvcHJveHkvZnJvbnRlbmQvc3RhdGUvaW1wbC9PS1Byb3h5U3RhdGUuamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | ... and [10 more](https://codecov.io/gh/apache/shardingsphere/pull/10193/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e66e4de...45f63bb](https://codecov.io/gh/apache/shardingsphere/pull/10193?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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 pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-829741143


   @strongduanmu Hi, Your summary is much clear and 
   So we can judge whether these SQLs are supported or not by `preValidate` and `postValidate`, right?  `preValidate` is for table type checking, and `postVaidate` is for routed result checking?
   This PR does not cover all these cases above, does it? But no rush, we can process step by step.
   


-- 
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] strongduanmu commented on pull request #10193: fix bad rewritten result for foreign key SQL

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on pull request #10193:
URL: https://github.com/apache/shardingsphere/pull/10193#issuecomment-829751172


   > This PR does not cover all these cases above, does it? But no rush, we can process step by step.
   
   @tristaZero At present, all supported table combinations have been covered in the rewrite test case, and other exception scenarios have not been covered.
   


-- 
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] strongduanmu commented on a change in pull request #10193: fix bad rewritten result for foreign key SQL

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



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sharding.route.engine.validator.ddl.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.rule.TableRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Sharding alter table statement validator.
+ */
+public final class ShardingAlterTableStatementValidator extends ShardingDDLStatementValidator<AlterTableStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        Collection<String> shardingLogicTableNames = getShardingLogicTableNames(shardingRule, sqlStatementContext);
+        if (shardingLogicTableNames.size() <= 1) {
+            return;
+        }

Review comment:
       > How about `broadcast Table`or `single table`?
   > Is #10172 a interim solution?
   
   @tristaZero Sorry for late reply, I spent some time sorting out the different scenes. According to the table type, there are sharding, broadcast and single table in SS. By combining with Cartesian product, we can get nine combination types.
   
   
   case | primary table | referenced table
   -- | -- | --
   1 | sharding | sharding
   2 | sharding | broadcast
   3 | sharding | single
   4 | broadcast | sharding
   5 | broadcast | broadcast
   6 | broadcast | single
   7 | single | sharding
   8 | single | broadcast
   9 | single | single
   
   ## case 1: sharding * sharding 
   There are two situations in this scenario. The sharding tables can be binding tables or unbinding tables.
   
   




-- 
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] strongduanmu commented on a change in pull request #10193: fix bad rewritten result for foreign key SQL

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



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.sharding.route.engine.validator.ddl.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.rule.TableRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Sharding alter table statement validator.
+ */
+public final class ShardingAlterTableStatementValidator extends ShardingDDLStatementValidator<AlterTableStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        Collection<String> shardingLogicTableNames = getShardingLogicTableNames(shardingRule, sqlStatementContext);
+        if (shardingLogicTableNames.size() <= 1) {
+            return;
+        }

Review comment:
       > How about `broadcast Table`or `single table`?
   > Is #10172 a interim solution?
   
   @tristaZero Sorry for late reply, I spent some time sorting out the different scenes. According to the table type—— sharding, broadcast and single table, we can get nine combination types.
   
   ![image](https://user-images.githubusercontent.com/10829171/116523244-9cc63500-a908-11eb-9511-5c4fad57d8b7.png)
   
   The following is a detailed description of the nine types of combinations.
   
   case | primary table | referenced table | condition | support | result
   -- | -- | -- | -- | -- | --
   1 | sharding | sharding |   |   |  
   1.1 | single data node sharding | single data node sharding | binding | yes | Logic SQL: ALTER TABLE bmsql_district ADD CONSTRAINT d_warehouse_fkey FOREIGN KEY (d_w_id) REFERENCES bmsql_warehouse (w_id) Actual SQL: ds_0 ::: ALTER TABLE bmsql_district_0 ADD CONSTRAINT d_warehouse_fkey FOREIGN KEY (d_w_id) REFERENCES bmsql_warehouse_0 (w_id)
     |   |   | unbinding same datesource | yes | Logic SQL: ALTER TABLE bmsql_district ADD CONSTRAINT d_district_extend_fkey FOREIGN KEY (d_w_id) REFERENCES bmsql_district_extend (d_w_id) Actual SQL: ds_0 ::: ALTER TABLE bmsql_district_0 ADD CONSTRAINT d_district_extend_fkey FOREIGN KEY (d_w_id) REFERENCES bmsql_district_extend_0 (d_w_id)
     |   |   | unbinding different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   1.2 | single data node sharding | multi data node sharding | exist same datesource | no | [58000] ERROR: ALTER TABLE ... statement can not support unbinding sharding tables route to multiple same data nodes.
     |   |   | all different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   1.3 | multi data node sharding | single data node sharding | exist same datesource | yes | Logic SQL: ALTER TABLE t_order ADD CONSTRAINT d_district_fkey FOREIGN KEY (order_id) REFERENCES bmsql_district (d_w_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_0 ADD CONSTRAINT d_district_fkey FOREIGN KEY (order_id) REFERENCES bmsql_district_0 (d_w_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_1 ADD CONSTRAINT d_district_fkey FOREIGN KEY (order_id) REFERENCES bmsql_district_0 (d_w_id)
     |   |   | all different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   1.4 | multi data node sharding | multi data node sharding | binding | yes | Logic SQL: ALTER TABLE t_order_item ADD CONSTRAINT t_order_fkey FOREIGN KEY (order_id) REFERENCES t_order (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_item_0 ADD CONSTRAINT t_order_fkey FOREIGN KEY (order_id) REFERENCES t_order_0 (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_item_1 ADD CONSTRAINT t_order_fkey FOREIGN KEY (order_id) REFERENCES t_order_1 (order_id) Actual SQL: ds_1 ::: ALTER TABLE t_order_item_0 ADD CONSTRAINT t_order_fkey FOREIGN KEY (order_id) REFERENCES t_order_0 (order_id) Actual SQL: ds_1 ::: ALTER TABLE t_order_item_1 ADD CONSTRAINT t_order_fkey FOREIGN KEY (order_id) REFERENCES t_order_1 (order_id)
     |   |   | unbinding exist same datesource | no | [58000] ERROR: ALTER TABLE ... statement can not support unbinding sharding tables route to multiple same data nodes.
     |   |   | unbinding all different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   2 | sharding | broadcast |   |   |  
   2.1 | single data node sharding | broadcast |   | yes | Logic SQL: ALTER TABLE bmsql_district ADD CONSTRAINT d_config_fkey FOREIGN KEY (d_w_id) REFERENCES t_config (order_id) Actual SQL: ds_0 ::: ALTER TABLE bmsql_district_0 ADD CONSTRAINT d_config_fkey FOREIGN KEY (d_w_id) REFERENCES t_config (order_id)
   2.2 | multi data node sharding | broadcast |   | yes | Logic SQL: ALTER TABLE t_order ADD CONSTRAINT d_config_fkey FOREIGN KEY (order_id) REFERENCES t_config (order_id) Actual SQL: ds_1 ::: ALTER TABLE t_order_0 ADD CONSTRAINT d_config_fkey FOREIGN KEY (order_id) REFERENCES t_config (order_id) Actual SQL: ds_1 ::: ALTER TABLE t_order_1 ADD CONSTRAINT d_config_fkey FOREIGN KEY (order_id) REFERENCES t_config (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_0 ADD CONSTRAINT d_config_fkey FOREIGN KEY (order_id) REFERENCES t_config (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_1 ADD CONSTRAINT d_config_fkey FOREIGN KEY (order_id) REFERENCES t_config (order_id)
   3 | sharding | single |   |   |  
   3.1 | single data node sharding | single | same datesource | yes | Logic SQL: ALTER TABLE bmsql_district ADD CONSTRAINT d_single_fkey FOREIGN KEY (d_w_id) REFERENCES t_single (order_id) Actual SQL: ds_0 ::: ALTER TABLE bmsql_district_0 ADD CONSTRAINT d_single_fkey FOREIGN KEY (d_w_id) REFERENCES t_single (order_id)
     |   |   | different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   3.2 | multi data node sharding | single | exist same datesource | yes | Logic SQL: ALTER TABLE t_order ADD CONSTRAINT d_single_fkey FOREIGN KEY (order_id) REFERENCES t_single (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_0 ADD CONSTRAINT d_single_fkey FOREIGN KEY (order_id) REFERENCES t_single (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_order_1 ADD CONSTRAINT d_single_fkey FOREIGN KEY (order_id) REFERENCES t_single (order_id)
     |   |   | all different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   4 | broadcast | sharding |   |   |  
   4.1 | broadcast | single data node sharding |   | yes | Logic SQL: ALTER TABLE t_config ADD CONSTRAINT d_district_fkey FOREIGN KEY (order_id) REFERENCES bmsql_district (d_w_id) Actual SQL: ds_0 ::: ALTER TABLE t_config ADD CONSTRAINT d_district_fkey FOREIGN KEY (order_id) REFERENCES bmsql_district_0 (d_w_id)
   4.2 | broadcast | multi data node sharding |   | no | [58000] ERROR: ALTER TABLE ... statement can not support unbinding sharding tables route to multiple same data nodes.
   5 | broadcast | broadcast |   | yes | Logic SQL: ALTER TABLE t_config ADD CONSTRAINT d_order_type_fkey FOREIGN KEY (order_id) REFERENCES t_order_type (order_id) Actual SQL: ds_2 ::: ALTER TABLE t_config ADD CONSTRAINT d_order_type_fkey FOREIGN KEY (order_id) REFERENCES t_order_type (order_id) Actual SQL: ds_1 ::: ALTER TABLE t_config ADD CONSTRAINT d_order_type_fkey FOREIGN KEY (order_id) REFERENCES t_order_type (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_config ADD CONSTRAINT d_order_type_fkey FOREIGN KEY (order_id) REFERENCES t_order_type (order_id)
   6 | broadcast | single |   | yes | Logic SQL: ALTER TABLE t_config ADD CONSTRAINT d_single_fkey FOREIGN KEY (order_id) REFERENCES t_single (order_id) Actual SQL: ds_2 ::: ALTER TABLE t_config ADD CONSTRAINT d_single_fkey FOREIGN KEY (order_id) REFERENCES t_single (order_id)
   7 | single | sharding |   |   |  
   7.1 | single | single data node sharding | same datesource | yes | Logic SQL: ALTER TABLE t_single ADD CONSTRAINT d_district_fkey FOREIGN KEY (order_id) REFERENCES bmsql_district (d_w_id) Actual SQL: ds_0 ::: ALTER TABLE t_single ADD CONSTRAINT d_district_fkey FOREIGN KEY (order_id) REFERENCES bmsql_district_0 (d_w_id)
     |   |   | different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   7.2 | single | multi data node sharding | exist same datesource | no | [58000] ERROR: ALTER TABLE ... statement can not support unbinding sharding tables route to multiple same data nodes.
     |   |   | all different datesource | no | [58000] ERROR: Can not get route result, please check your sharding table config.
   8 | single | broadcast |   | yes | Logic SQL: ALTER TABLE t_single ADD CONSTRAINT d_order_fkey FOREIGN KEY (order_id) REFERENCES t_config (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_single ADD CONSTRAINT d_order_fkey FOREIGN KEY (order_id) REFERENCES t_config (order_id)
   9 | single | single | same datesource | yes | Logic SQL: ALTER TABLE t_single ADD CONSTRAINT d_single_extend_fkey FOREIGN KEY (order_id) REFERENCES t_single_extend (order_id) Actual SQL: ds_0 ::: ALTER TABLE t_single ADD CONSTRAINT d_single_extend_fkey FOREIGN KEY (order_id) REFERENCES t_single_extend (order_id)
     |   |   | different datesource | no | [58000] ERROR: Single tables must be in the same datasource.
   
   The above test case uses the following sharding configuration.
   
   ```yaml
   rules:
   - !SHARDING
    tables:
      bmsql_district:
         actualDataNodes: ds_0.bmsql_district_0
      bmsql_warehouse:
         actualDataNodes: ds_0.bmsql_warehouse_0
      bmsql_district_extend:
         actualDataNodes: ds_0.bmsql_district_extend_0
      bmsql_warehouse_extend:
         actualDataNodes: ds_2.bmsql_warehouse_extend_0
      t_order:
        actualDataNodes: ds_${0..1}.t_order_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_inline
        keyGenerateStrategy:
          column: order_id
          keyGeneratorName: snowflake
      t_order_extend:
        actualDataNodes: ds_${1..2}.t_order_extend_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_inline
        keyGenerateStrategy:
          column: order_id
          keyGeneratorName: snowflake
      t_order_item:
        actualDataNodes: ds_${0..1}.t_order_item_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_item_inline
        keyGenerateStrategy:
          column: order_item_id
          keyGeneratorName: snowflake
      t_order_item_extend:
        actualDataNodes: ds_2.t_order_item_extend_${0..1}
        tableStrategy:
          standard:
            shardingColumn: order_id
            shardingAlgorithmName: t_order_item_inline
        keyGenerateStrategy:
          column: order_item_id
          keyGeneratorName: snowflake
    bindingTables:
      - t_order,t_order_item
      - bmsql_district,bmsql_warehouse
    broadcastTables:
      - t_config
      - t_order_type
    defaultDatabaseStrategy:
      standard:
        shardingColumn: order_id
        shardingAlgorithmName: database_inline
    defaultTableStrategy:
      none:
   ```
   
   The combination of `sharding table` and `sharding table` is special, and there are two different scenarios——binding table and unbinding table.
   
   Binding table's routing logic is consistent with current logic. The only difference is that the logic is advanced to the route module for processing. Primary table is processed first, and then other binding tables are also routed with the same rule according to the binding relationship.
   
   Unbinding tables and other cases use the Cartesian product engine, and then check the routing result in the `postValidate` method. If there is no route result, it means that primary table and referenced tables do not have the same data source. On the contrary, when the routing result is obtained, we will verify the legitimacy of the routing result. The logic in the `postValidate` method, take the following SQL as an example(case 1.4 unbinding exist same datesource), and the table of `t_order` and `t_order_item` are sharding tables, but not binding tables.
   
   ```sql
   Logic SQL: alter table t_order_item add constraint t_order_fkey foreign key (order_id) references t_order (order_id)
   Actual SQL: test_0 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   Actual SQL: test_0 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   Actual SQL: test_0 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   Actual SQL: test_0 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   Actual SQL: test_1 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   Actual SQL: test_1 ::: alter table t_order_item_1 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   Actual SQL: test_1 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_0 (order_id)
   Actual SQL: test_1 ::: alter table t_order_item_0 add constraint t_order_fkey foreign key (order_id) references t_order_1 (order_id)
   ```
   
   Obviously, we can not execute the `alter table t_order_item_1 add constraint t_order_fkey ... ` statement multiple times under the same data source. Therefore, this routing result needs to throw an exception in the `postValidate` method.




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