You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2023/06/08 07:58:41 UTC

[shardingsphere] branch master updated: Fix unknown column exception when execute subquery contains join statement with multi encrypt table columns (#26167)

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

panjuan 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 19f030808dc Fix unknown column exception when execute subquery contains join statement with multi encrypt table columns (#26167)
19f030808dc is described below

commit 19f030808dcf199c0b4b1d53e2db0cdc140a7d2c
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Thu Jun 8 15:58:32 2023 +0800

    Fix unknown column exception when execute subquery contains join statement with multi encrypt table columns (#26167)
    
    * Fix unknown column exception when execute subquery contains join statement with multi encrypt table columns
    
    * revert it-env.properties
---
 .../merge/dql/EncryptAlgorithmMetaData.java        |  2 +-
 .../generator/EncryptProjectionTokenGenerator.java | 18 +++++----
 .../select/subquery/SubqueryTableContext.java      |  5 ++-
 .../engine/SubqueryTableContextEngine.java         | 46 ++++++++++++++++------
 .../infra/binder/segment/table/TablesContext.java  |  8 ++--
 .../cases/dql/dql-integration-select-sub-query.xml | 12 +++++-
 6 files changed, 65 insertions(+), 26 deletions(-)

diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java
index e910b7537b4..293a5784eaf 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java
@@ -18,10 +18,10 @@
 package org.apache.shardingsphere.encrypt.merge.dql;
 
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.encrypt.api.context.EncryptContext;
 import org.apache.shardingsphere.encrypt.api.encrypt.standard.StandardEncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.context.EncryptContextBuilder;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
-import org.apache.shardingsphere.encrypt.api.context.EncryptContext;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.Projection;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ColumnProjection;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.SubqueryProjection;
diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java
index 14c015bed3a..915ba0aa714 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java
@@ -82,18 +82,18 @@ public final class EncryptProjectionTokenGenerator implements CollectionSQLToken
     
     private void addGenerateSQLTokens(final Collection<SQLToken> result, final SelectStatementContext selectStatementContext) {
         Map<String, String> columnTableNames = getColumnTableNames(selectStatementContext);
-        for (ProjectionSegment projection : selectStatementContext.getSqlStatement().getProjections().getProjections()) {
+        for (ProjectionSegment each : selectStatementContext.getSqlStatement().getProjections().getProjections()) {
             SubqueryType subqueryType = selectStatementContext.getSubqueryType();
-            if (projection instanceof ColumnProjectionSegment) {
-                ColumnProjectionSegment columnSegment = (ColumnProjectionSegment) projection;
+            if (each instanceof ColumnProjectionSegment) {
+                ColumnProjectionSegment columnSegment = (ColumnProjectionSegment) each;
                 ColumnProjection columnProjection = buildColumnProjection(columnSegment);
                 String tableName = columnTableNames.get(columnProjection.getExpression());
-                if (null != tableName && encryptRule.findEncryptColumn(tableName, columnProjection.getName()).isPresent()) {
+                if (isEncryptColumn(tableName, columnProjection.getName())) {
                     result.add(generateSQLToken(tableName, columnSegment, columnProjection, subqueryType));
                 }
             }
-            if (projection instanceof ShorthandProjectionSegment) {
-                ShorthandProjectionSegment shorthandSegment = (ShorthandProjectionSegment) projection;
+            if (each instanceof ShorthandProjectionSegment) {
+                ShorthandProjectionSegment shorthandSegment = (ShorthandProjectionSegment) each;
                 Collection<Projection> actualColumns = getShorthandProjection(shorthandSegment, selectStatementContext.getProjectionsContext()).getActualColumns();
                 if (!actualColumns.isEmpty()) {
                     result.add(generateSQLToken(shorthandSegment, actualColumns, selectStatementContext, subqueryType, columnTableNames));
@@ -102,6 +102,10 @@ public final class EncryptProjectionTokenGenerator implements CollectionSQLToken
         }
     }
     
+    private boolean isEncryptColumn(final String tableName, final String columnName) {
+        return null != tableName && encryptRule.findEncryptColumn(tableName, columnName).isPresent();
+    }
+    
     private SubstitutableColumnNameToken generateSQLToken(final String tableName, final ColumnProjectionSegment columnSegment,
                                                           final ColumnProjection columnProjection, final SubqueryType subqueryType) {
         Collection<Projection> projections = generateProjections(tableName, columnProjection, subqueryType, false, null);
@@ -115,7 +119,7 @@ public final class EncryptProjectionTokenGenerator implements CollectionSQLToken
         List<Projection> projections = new LinkedList<>();
         for (Projection each : actualColumns) {
             String tableName = columnTableNames.get(each.getExpression());
-            if (null == tableName || !encryptRule.findStandardEncryptor(tableName, each.getColumnLabel()).isPresent()) {
+            if (!isEncryptColumn(tableName, each.getColumnLabel())) {
                 projections.add(each.getAlias().map(optional -> (Projection) new ColumnProjection(null, optional, null)).orElse(each));
             } else if (each instanceof ColumnProjection) {
                 projections.addAll(generateProjections(tableName, (ColumnProjection) each, subqueryType, true, segment));
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/SubqueryTableContext.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/SubqueryTableContext.java
index 81793496d01..3ff6e40b582 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/SubqueryTableContext.java
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/SubqueryTableContext.java
@@ -21,6 +21,7 @@ import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 
 import java.util.Collection;
+import java.util.LinkedList;
 
 /**
  * Subquery table context.
@@ -31,7 +32,7 @@ public final class SubqueryTableContext {
     
     private final String tableName;
     
-    private final String alias;
+    private final String aliasName;
     
-    private final Collection<String> columnNames;
+    private final Collection<String> columnNames = new LinkedList<>();
 }
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/engine/SubqueryTableContextEngine.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/engine/SubqueryTableContextEngine.java
index 1651d4f8378..3899af3f9aa 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/engine/SubqueryTableContextEngine.java
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/select/subquery/engine/SubqueryTableContextEngine.java
@@ -17,14 +17,18 @@
 
 package org.apache.shardingsphere.infra.binder.segment.select.subquery.engine;
 
+import org.apache.shardingsphere.infra.binder.segment.select.projection.Projection;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ColumnProjection;
 import org.apache.shardingsphere.infra.binder.segment.select.subquery.SubqueryTableContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.JoinTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableSegment;
 
 import java.util.Collection;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.stream.Collectors;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
 
 /**
  * Subquery table context engine.
@@ -35,16 +39,36 @@ public final class SubqueryTableContextEngine {
      * Create subquery table contexts.
      *
      * @param subqueryContext subquery context
-     * @param alias subquery alias
-     * @return subquery table context collection
+     * @param aliasName subquery alias name
+     * @return subquery table context map
      */
-    public Collection<SubqueryTableContext> createSubqueryTableContexts(final SelectStatementContext subqueryContext, final String alias) {
-        Collection<SubqueryTableContext> result = new LinkedList<>();
-        List<String> columnNames = subqueryContext.getProjectionsContext().getExpandProjections().stream()
-                .filter(ColumnProjection.class::isInstance).map(each -> ((ColumnProjection) each).getName()).collect(Collectors.toList());
-        for (String each : subqueryContext.getTablesContext().getTableNames()) {
-            result.add(new SubqueryTableContext(each, alias, columnNames));
+    public Map<String, SubqueryTableContext> createSubqueryTableContexts(final SelectStatementContext subqueryContext, final String aliasName) {
+        Map<String, SubqueryTableContext> result = new LinkedHashMap<>();
+        TableSegment tableSegment = subqueryContext.getSqlStatement().getFrom();
+        for (Projection each : subqueryContext.getProjectionsContext().getExpandProjections()) {
+            if (!(each instanceof ColumnProjection)) {
+                continue;
+            }
+            String columnName = ((ColumnProjection) each).getName();
+            if (tableSegment instanceof SimpleTableSegment) {
+                String tableName = ((SimpleTableSegment) tableSegment).getTableName().getIdentifier().getValue();
+                result.computeIfAbsent(tableName.toLowerCase(), unused -> new SubqueryTableContext(tableName, aliasName)).getColumnNames().add(columnName);
+            }
+            if (tableSegment instanceof JoinTableSegment && null != ((ColumnProjection) each).getOwner()) {
+                Optional<String> tableName = getTableNameByOwner(subqueryContext.getTablesContext().getSimpleTableSegments(), ((ColumnProjection) each).getOwner());
+                tableName.ifPresent(optional -> result.computeIfAbsent(optional.toLowerCase(), unused -> new SubqueryTableContext(optional, aliasName)).getColumnNames().add(columnName));
+            }
         }
         return result;
     }
+    
+    private Optional<String> getTableNameByOwner(final Collection<SimpleTableSegment> simpleTableSegments, final String owner) {
+        for (SimpleTableSegment each : simpleTableSegments) {
+            String tableNameOrAlias = each.getAliasName().orElseGet(() -> each.getTableName().getIdentifier().getValue());
+            if (tableNameOrAlias.equalsIgnoreCase(owner)) {
+                return Optional.of(each.getTableName().getIdentifier().getValue());
+            }
+        }
+        return Optional.empty();
+    }
 }
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
index 87631094ca7..aba9c48fa77 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
@@ -97,11 +97,11 @@ public final class TablesContext {
     
     private Map<String, Collection<SubqueryTableContext>> createSubqueryTables(final Map<Integer, SelectStatementContext> subqueryContexts, final SubqueryTableSegment subqueryTable) {
         SelectStatementContext subqueryContext = subqueryContexts.get(subqueryTable.getSubquery().getStartIndex());
-        Collection<SubqueryTableContext> subqueryTableContexts = new SubqueryTableContextEngine().createSubqueryTableContexts(subqueryContext, subqueryTable.getAliasName().orElse(null));
+        Map<String, SubqueryTableContext> subqueryTableContexts = new SubqueryTableContextEngine().createSubqueryTableContexts(subqueryContext, subqueryTable.getAliasName().orElse(null));
         Map<String, Collection<SubqueryTableContext>> result = new HashMap<>();
-        for (SubqueryTableContext subQuery : subqueryTableContexts) {
-            if (null != subQuery.getAlias()) {
-                result.computeIfAbsent(subQuery.getAlias(), unused -> new LinkedList<>()).add(subQuery);
+        for (SubqueryTableContext each : subqueryTableContexts.values()) {
+            if (null != each.getAliasName()) {
+                result.computeIfAbsent(each.getAliasName(), unused -> new LinkedList<>()).add(each);
             }
         }
         return result;
diff --git a/test/e2e/sql/src/test/resources/cases/dql/dql-integration-select-sub-query.xml b/test/e2e/sql/src/test/resources/cases/dql/dql-integration-select-sub-query.xml
index f9213ddb6cd..928c0eabcfd 100644
--- a/test/e2e/sql/src/test/resources/cases/dql/dql-integration-select-sub-query.xml
+++ b/test/e2e/sql/src/test/resources/cases/dql/dql-integration-select-sub-query.xml
@@ -89,8 +89,18 @@
         <assertion expected-data-source-name="read_dataset" />
     </test-case>
     
-    <test-case sql="SELECT * FROM (SELECT business_code, telephone, (SELECT password FROM t_user LIMIT 1) AS password FROM t_merchant) AS temp;" db-types="MySQL,PostgreSQL,openGauss" scenario-types="encrypt"
+    <test-case sql="SELECT * FROM (SELECT business_code, telephone, (SELECT password FROM t_user LIMIT 1) AS password FROM t_merchant) AS temp" db-types="MySQL,PostgreSQL,openGauss" scenario-types="encrypt"
                scenario-comments="Test shorthand expansion contains subquery projection and subquery projection contains encrypt column and config alias when use encrypt feature.">
         <assertion expected-data-source-name="read_dataset" />
     </test-case>
+
+    <test-case sql="SELECT m.business_code, m.telephone, u.user_id FROM t_merchant AS m INNER JOIN t_user AS u ON m.merchant_id = u.user_id" db-types="MySQL,PostgreSQL,openGauss" scenario-types="encrypt"
+               scenario-comments="Test join contains some encrypt columns in multi tables when use encrypt feature.">
+        <assertion expected-data-source-name="read_dataset" />
+    </test-case>
+
+    <test-case sql="SELECT * FROM (SELECT m.business_code, m.telephone, u.user_id FROM t_merchant AS m INNER JOIN t_user AS u ON m.merchant_id = u.user_id) AS temp" db-types="MySQL,PostgreSQL,openGauss" scenario-types="encrypt"
+               scenario-comments="Test shorthand expansion contains subquery join and join contains some encrypt columns in multi tables when use encrypt feature.">
+        <assertion expected-data-source-name="read_dataset" />
+    </test-case>
 </integration-test-cases>