You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by su...@apache.org on 2022/11/25 07:24:06 UTC

[shardingsphere] branch master updated: Support select join using projection in ResultSetMetaData for MySQL (#22405)

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

sunnianjun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 9c04fab5244 Support select join using projection in ResultSetMetaData for MySQL (#22405)
9c04fab5244 is described below

commit 9c04fab524478ef9fd5e1873a127443cc8bd550b
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Fri Nov 25 15:23:59 2022 +0800

    Support select join using projection in ResultSetMetaData for MySQL (#22405)
    
    * Support select join using projection in ResultSetMetaData for MySQL
    
    * optimize code style
---
 .../select/projection/engine/ProjectionEngine.java | 66 ++++++++++++++++------
 .../projection/engine/ProjectionEngineTest.java    | 56 +++++++++++++++---
 .../cases/dql/dql-integration-test-cases.xml       | 25 ++++----
 3 files changed, 111 insertions(+), 36 deletions(-)

diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngine.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngine.java
index 1e2623cbd15..53ca0970982 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngine.java
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngine.java
@@ -29,10 +29,12 @@ import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.Sho
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.SubqueryProjection;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.exception.SchemaNotFoundException;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.sql.parser.sql.common.enums.AggregationType;
+import org.apache.shardingsphere.sql.parser.sql.common.enums.JoinType;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.ParameterMarkerExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.AggregationDistinctProjectionSegment;
@@ -52,7 +54,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
@@ -187,17 +188,29 @@ public final class ProjectionEngine {
             return Collections.emptyList();
         }
         JoinTableSegment joinTable = (JoinTableSegment) table;
-        Collection<Projection> projections = new LinkedList<>();
-        createProjection(joinTable.getLeft(), projectionSegment).ifPresent(projections::add);
-        createProjection(joinTable.getRight(), projectionSegment).ifPresent(projections::add);
         Collection<Projection> result = new LinkedList<>();
-        for (Projection each : projections) {
+        Collection<Projection> remainingProjections = new LinkedList<>();
+        for (Projection each : getOriginalProjections(joinTable, projectionSegment)) {
+            Collection<Projection> actualProjections = getActualProjections(Collections.singletonList(each));
             if (joinTable.getUsing().isEmpty() || (null != owner && each.getExpression().contains(owner))) {
-                result.addAll(getActualProjections(Collections.singletonList(each)));
+                result.addAll(actualProjections);
             } else {
-                result.addAll(getJoinUsingActualProjections(projections, joinTable.getUsing()));
+                remainingProjections.addAll(actualProjections);
             }
         }
+        result.addAll(getUsingActualProjections(remainingProjections, joinTable.getUsing()));
+        return result;
+    }
+    
+    private Collection<Projection> getOriginalProjections(final JoinTableSegment joinTable, final ProjectionSegment projectionSegment) {
+        Collection<Projection> result = new LinkedList<>();
+        if (databaseType instanceof MySQLDatabaseType && !joinTable.getUsing().isEmpty() && JoinType.RIGHT.name().equalsIgnoreCase(joinTable.getJoinType())) {
+            createProjection(joinTable.getRight(), projectionSegment).ifPresent(result::add);
+            createProjection(joinTable.getLeft(), projectionSegment).ifPresent(result::add);
+            return result;
+        }
+        createProjection(joinTable.getLeft(), projectionSegment).ifPresent(result::add);
+        createProjection(joinTable.getRight(), projectionSegment).ifPresent(result::add);
         return result;
     }
     
@@ -215,16 +228,22 @@ public final class ProjectionEngine {
         return result;
     }
     
-    private Collection<Projection> getJoinUsingActualProjections(final Collection<Projection> projections, final List<ColumnSegment> usingColumns) {
-        Collection<Projection> result = new LinkedList<>();
-        Collection<Projection> actualColumns = getActualProjections(projections);
+    private Collection<Projection> getUsingActualProjections(final Collection<Projection> actualProjections, final Collection<ColumnSegment> usingColumns) {
+        if (usingColumns.isEmpty()) {
+            return Collections.emptyList();
+        }
         Collection<String> usingColumnNames = getUsingColumnNames(usingColumns);
-        result.addAll(getJoinUsingColumns(actualColumns, usingColumnNames));
-        result.addAll(getRemainingColumns(actualColumns, usingColumnNames));
+        Collection<Projection> result = new LinkedList<>();
+        if (databaseType instanceof MySQLDatabaseType) {
+            result.addAll(getJoinUsingColumnsByOriginalColumnSequence(actualProjections, usingColumnNames));
+        } else {
+            result.addAll(getJoinUsingColumnsByUsingColumnSequence(actualProjections, usingColumnNames));
+        }
+        result.addAll(getRemainingColumns(actualProjections, usingColumnNames));
         return result;
     }
     
-    private Collection<String> getUsingColumnNames(final List<ColumnSegment> usingColumns) {
+    private Collection<String> getUsingColumnNames(final Collection<ColumnSegment> usingColumns) {
         Collection<String> result = new LinkedHashSet<>();
         for (ColumnSegment each : usingColumns) {
             result.add(each.getIdentifier().getValue().toLowerCase());
@@ -232,10 +251,23 @@ public final class ProjectionEngine {
         return result;
     }
     
-    private Collection<Projection> getJoinUsingColumns(final Collection<Projection> actualColumns, final Collection<String> usingColumnNames) {
+    private Collection<Projection> getJoinUsingColumnsByOriginalColumnSequence(final Collection<Projection> actualProjections, final Collection<String> usingColumnNames) {
+        Collection<Projection> result = new LinkedList<>();
+        for (Projection each : actualProjections) {
+            if (result.size() == usingColumnNames.size()) {
+                return result;
+            }
+            if (usingColumnNames.contains(each.getColumnLabel().toLowerCase())) {
+                result.add(each);
+            }
+        }
+        return result;
+    }
+    
+    private Collection<Projection> getJoinUsingColumnsByUsingColumnSequence(final Collection<Projection> actualProjections, final Collection<String> usingColumnNames) {
         Collection<Projection> result = new LinkedList<>();
         for (String each : usingColumnNames) {
-            for (Projection projection : actualColumns) {
+            for (Projection projection : actualProjections) {
                 if (each.equals(projection.getColumnLabel().toLowerCase())) {
                     result.add(projection);
                     break;
@@ -245,9 +277,9 @@ public final class ProjectionEngine {
         return result;
     }
     
-    private Collection<Projection> getRemainingColumns(final Collection<Projection> actualColumns, final Collection<String> usingColumnNames) {
+    private Collection<Projection> getRemainingColumns(final Collection<Projection> actualProjections, final Collection<String> usingColumnNames) {
         Collection<Projection> result = new LinkedList<>();
-        for (Projection each : actualColumns) {
+        for (Projection each : actualProjections) {
             if (usingColumnNames.contains(each.getColumnLabel().toLowerCase())) {
                 continue;
             }
diff --git a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngineTest.java b/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngineTest.java
index a6901ec5118..c1ed51069bb 100644
--- a/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngineTest.java
+++ b/infra/binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionEngineTest.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.Par
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ShorthandProjection;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeFactory;
 import org.apache.shardingsphere.infra.exception.SchemaNotFoundException;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
 import org.apache.shardingsphere.sql.parser.sql.common.enums.AggregationType;
@@ -240,24 +241,50 @@ public final class ProjectionEngineTest {
     }
     
     @Test
-    public void assertCreateProjectionWhenShorthandProjectionContainsJoinUsingColumn() {
+    public void assertCreateProjectionWhenShorthandProjectionContainsJoinUsingColumnForPostgreSQL() {
+        when(schema.getVisibleColumnNames("t_order")).thenReturn(Arrays.asList("order_id", "user_id", "status", "merchant_id", "remark", "creation_date"));
+        when(schema.getVisibleColumnNames("t_order_item")).thenReturn(Arrays.asList("item_id", "order_id", "user_id", "product_id", "quantity", "creation_date"));
+        Optional<Projection> actual = new ProjectionEngine("public", Collections.singletonMap("public", schema), DatabaseTypeFactory.getInstance("PostgreSQL"))
+                .createProjection(createJoinTableSegment(), new ShorthandProjectionSegment(0, 0));
+        assertTrue(actual.isPresent());
+        assertThat(actual.get(), instanceOf(ShorthandProjection.class));
+        assertThat(((ShorthandProjection) actual.get()).getActualColumns().size(), is(10));
+        assertThat(((ShorthandProjection) actual.get()).getActualColumns(), is(crateExpectedColumnsWithoutOwnerForPostgreSQL()));
+    }
+    
+    @Test
+    public void assertCreateProjectionWhenShorthandProjectionContainsJoinUsingColumnForMySQL() {
         when(schema.getVisibleColumnNames("t_order")).thenReturn(Arrays.asList("order_id", "user_id", "status", "merchant_id", "remark", "creation_date"));
         when(schema.getVisibleColumnNames("t_order_item")).thenReturn(Arrays.asList("item_id", "order_id", "user_id", "product_id", "quantity", "creation_date"));
         Optional<Projection> actual = new ProjectionEngine(DefaultDatabase.LOGIC_NAME,
-                Collections.singletonMap(DefaultDatabase.LOGIC_NAME, schema), databaseType).createProjection(createJoinTableSegment(), new ShorthandProjectionSegment(0, 0));
+                Collections.singletonMap(DefaultDatabase.LOGIC_NAME, schema), DatabaseTypeFactory.getInstance("MySQL")).createProjection(createJoinTableSegment(),
+                        new ShorthandProjectionSegment(0, 0));
         assertTrue(actual.isPresent());
         assertThat(actual.get(), instanceOf(ShorthandProjection.class));
         assertThat(((ShorthandProjection) actual.get()).getActualColumns().size(), is(10));
-        assertThat(((ShorthandProjection) actual.get()).getActualColumns(), is(crateExpectedColumnsWithoutOwner()));
+        assertThat(((ShorthandProjection) actual.get()).getActualColumns(), is(crateExpectedColumnsWithoutOwnerForMySQL()));
+    }
+    
+    @Test
+    public void assertCreateProjectionWhenShorthandProjectionContainsJoinUsingColumnAndOwnerForPostgreSQL() {
+        when(schema.getVisibleColumnNames("t_order")).thenReturn(Arrays.asList("order_id", "user_id", "status", "merchant_id", "remark", "creation_date"));
+        ShorthandProjectionSegment projectionSegment = new ShorthandProjectionSegment(0, 0);
+        projectionSegment.setOwner(new OwnerSegment(0, 0, new IdentifierValue("o")));
+        Optional<Projection> actual =
+                new ProjectionEngine("public", Collections.singletonMap("public", schema), DatabaseTypeFactory.getInstance("PostgreSQL")).createProjection(createJoinTableSegment(), projectionSegment);
+        assertTrue(actual.isPresent());
+        assertThat(actual.get(), instanceOf(ShorthandProjection.class));
+        assertThat(((ShorthandProjection) actual.get()).getActualColumns().size(), is(6));
+        assertThat(((ShorthandProjection) actual.get()).getActualColumns(), is(crateExpectedColumnsWithOwner()));
     }
     
     @Test
-    public void assertCreateProjectionWhenShorthandProjectionContainsJoinUsingColumnAndOwner() {
+    public void assertCreateProjectionWhenShorthandProjectionContainsJoinUsingColumnAndOwnerForMySQL() {
         when(schema.getVisibleColumnNames("t_order")).thenReturn(Arrays.asList("order_id", "user_id", "status", "merchant_id", "remark", "creation_date"));
         ShorthandProjectionSegment projectionSegment = new ShorthandProjectionSegment(0, 0);
         projectionSegment.setOwner(new OwnerSegment(0, 0, new IdentifierValue("o")));
         Optional<Projection> actual = new ProjectionEngine(DefaultDatabase.LOGIC_NAME,
-                Collections.singletonMap(DefaultDatabase.LOGIC_NAME, schema), databaseType).createProjection(createJoinTableSegment(), projectionSegment);
+                Collections.singletonMap(DefaultDatabase.LOGIC_NAME, schema), DatabaseTypeFactory.getInstance("MySQL")).createProjection(createJoinTableSegment(), projectionSegment);
         assertTrue(actual.isPresent());
         assertThat(actual.get(), instanceOf(ShorthandProjection.class));
         assertThat(((ShorthandProjection) actual.get()).getActualColumns().size(), is(6));
@@ -272,12 +299,12 @@ public final class ProjectionEngineTest {
         JoinTableSegment result = new JoinTableSegment();
         result.setLeft(left);
         result.setRight(right);
-        result.setJoinType(JoinType.LEFT.name());
+        result.setJoinType(JoinType.RIGHT.name());
         result.setUsing(Arrays.asList(new ColumnSegment(0, 0, new IdentifierValue("user_id")), new ColumnSegment(0, 0, new IdentifierValue("order_id"))));
         return result;
     }
     
-    private Map<String, Projection> crateExpectedColumnsWithoutOwner() {
+    private Map<String, Projection> crateExpectedColumnsWithoutOwnerForPostgreSQL() {
         Map<String, Projection> result = new LinkedHashMap<>();
         result.put("o.user_id", new ColumnProjection("o", "user_id", null));
         result.put("o.order_id", new ColumnProjection("o", "order_id", null));
@@ -292,6 +319,21 @@ public final class ProjectionEngineTest {
         return result;
     }
     
+    private Map<String, Projection> crateExpectedColumnsWithoutOwnerForMySQL() {
+        Map<String, Projection> result = new LinkedHashMap<>();
+        result.put("i.order_id", new ColumnProjection("i", "order_id", null));
+        result.put("i.user_id", new ColumnProjection("i", "user_id", null));
+        result.put("i.item_id", new ColumnProjection("i", "item_id", null));
+        result.put("i.product_id", new ColumnProjection("i", "product_id", null));
+        result.put("i.quantity", new ColumnProjection("i", "quantity", null));
+        result.put("i.creation_date", new ColumnProjection("i", "creation_date", null));
+        result.put("o.status", new ColumnProjection("o", "status", null));
+        result.put("o.merchant_id", new ColumnProjection("o", "merchant_id", null));
+        result.put("o.remark", new ColumnProjection("o", "remark", null));
+        result.put("o.creation_date", new ColumnProjection("o", "creation_date", null));
+        return result;
+    }
+    
     private Map<String, Projection> crateExpectedColumnsWithOwner() {
         Map<String, Projection> result = new LinkedHashMap<>();
         result.put("o.order_id", new ColumnProjection("o", "order_id", null));
diff --git a/test/integration-test/test-suite/src/test/resources/cases/dql/dql-integration-test-cases.xml b/test/integration-test/test-suite/src/test/resources/cases/dql/dql-integration-test-cases.xml
index 2aff53e2cb9..fdad56d324d 100644
--- a/test/integration-test/test-suite/src/test/resources/cases/dql/dql-integration-test-cases.xml
+++ b/test/integration-test/test-suite/src/test/resources/cases/dql/dql-integration-test-cases.xml
@@ -376,8 +376,7 @@
         <assertion parameters="1000:int" expected-data-source-name="read_dataset" />
     </test-case>
     
-    <!-- TODO support mysql using condition -->
-    <test-case sql="SELECT i.* FROM t_order o JOIN t_order_item i USING(order_id) WHERE o.order_id = ?" db-types="PostgreSQL" scenario-types="tbl,dbtbl_with_readwrite_splitting,readwrite_splitting">
+    <test-case sql="SELECT i.* FROM t_order o JOIN t_order_item i USING(order_id) WHERE o.order_id = ?" db-types="MySQL,PostgreSQL" scenario-types="tbl,dbtbl_with_readwrite_splitting,readwrite_splitting">
         <assertion parameters="1000:int" expected-data-source-name="read_dataset" />
     </test-case>
     
@@ -653,12 +652,11 @@
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
-    <!-- TODO support MySQL USING statement -->
-    <test-case sql="SELECT * FROM t_order o INNER JOIN t_order_item i USING(order_id) WHERE o.user_id = ? ORDER BY o.order_id, 7" db-types="PostgreSQL,openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_order o INNER JOIN t_order_item i USING(order_id) WHERE o.user_id = ? ORDER BY o.order_id, 7" db-types="MySQL,PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
-    <test-case sql="SELECT * FROM t_order o INNER JOIN t_order_item i USING(order_id) WHERE o.user_id = ? ORDER BY o.order_id, 7 LIMIT 1, 2" db-types="openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_order o INNER JOIN t_order_item i USING(order_id) WHERE o.user_id = ? ORDER BY o.order_id, 7 LIMIT 1, 2" db-types="MySQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
     
@@ -691,10 +689,11 @@
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
-    <test-case sql="SELECT * FROM t_order o LEFT JOIN t_order_item i USING(order_id) WHERE o.user_id = ? ORDER BY o.order_id, 7" db-types="PostgreSQL,openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_order o LEFT JOIN t_order_item i USING(order_id) WHERE o.user_id = ? ORDER BY o.order_id, 7" db-types="MySQL,PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
+    <!-- TODO support MySQL using statement when calcite support right join using -->
     <test-case sql="SELECT * FROM t_order o RIGHT JOIN t_order_item i USING(order_id) WHERE i.user_id = ? ORDER BY i.item_id, 7" db-types="PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
@@ -703,11 +702,11 @@
         <assertion parameters="10:int, 10:int" expected-data-source-name="read_dataset" />
     </test-case>
     
-    <test-case sql="SELECT * FROM t_order o INNER JOIN t_merchant m USING(merchant_id) WHERE o.user_id = ? ORDER BY o.order_id" db-types="PostgreSQL,openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_order o INNER JOIN t_merchant m USING(merchant_id) WHERE o.user_id = ? ORDER BY o.order_id" db-types="MySQL,PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
-    <test-case sql="SELECT * FROM t_order o INNER JOIN t_merchant m USING(merchant_id) WHERE o.user_id = ? ORDER BY o.order_id LIMIT 1, 2" db-types="openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_order o INNER JOIN t_merchant m USING(merchant_id) WHERE o.user_id = ? ORDER BY o.order_id LIMIT 1, 2" db-types="MySQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
     
@@ -739,10 +738,11 @@
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
-    <test-case sql="SELECT * FROM t_order o LEFT JOIN t_merchant m USING(merchant_id) WHERE o.user_id = ? ORDER BY o.order_id, 7" db-types="PostgreSQL,openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_order o LEFT JOIN t_merchant m USING(merchant_id) WHERE o.user_id = ? ORDER BY o.order_id, 7" db-types="MySQL,PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
+    <!-- TODO support MySQL using statement when calcite support right join using -->
     <test-case sql="SELECT * FROM t_order o RIGHT JOIN t_merchant m USING(merchant_id) WHERE m.country_id = 1 ORDER BY o.order_id, m.merchant_id, 7" db-types="PostgreSQL,openGauss" scenario-types="db">
         <assertion expected-data-source-name="read_dataset" />
     </test-case>
@@ -751,11 +751,11 @@
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
     
-    <test-case sql="SELECT * FROM t_product p INNER JOIN t_product_detail d USING(product_id) WHERE p.product_id > ? ORDER BY p.product_id DESC" db-types="PostgreSQL,openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_product p INNER JOIN t_product_detail d USING(product_id) WHERE p.product_id > ? ORDER BY p.product_id DESC" db-types="MySQL,PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
-    <test-case sql="SELECT * FROM t_product p INNER JOIN t_product_detail d USING(product_id) WHERE p.product_id > ? ORDER BY p.product_id DESC LIMIT 2, 5" db-types="openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_product p INNER JOIN t_product_detail d USING(product_id) WHERE p.product_id > ? ORDER BY p.product_id DESC LIMIT 2, 5" db-types="MySQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
@@ -779,10 +779,11 @@
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
-    <test-case sql="SELECT * FROM t_product p LEFT JOIN t_product_detail d USING(product_id) WHERE p.category_id = ? ORDER BY p.product_id, 7" db-types="PostgreSQL,openGauss" scenario-types="db">
+    <test-case sql="SELECT * FROM t_product p LEFT JOIN t_product_detail d USING(product_id) WHERE p.category_id = ? ORDER BY p.product_id, 7" db-types="MySQL,PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>
 
+    <!-- TODO support MySQL using statement when calcite support right join using -->
     <test-case sql="SELECT * FROM t_product p RIGHT JOIN t_product_detail d USING(product_id) WHERE d.detail_id = ? ORDER BY d.product_id, 7" db-types="PostgreSQL,openGauss" scenario-types="db">
         <assertion parameters="10:int" expected-data-source-name="read_dataset" />
     </test-case>