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 2021/05/18 04:11:55 UTC

[shardingsphere] branch master updated: fix projection owner check when exist subquery temporary table (#10363)

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 be09827  fix projection owner check when exist subquery temporary table (#10363)
be09827 is described below

commit be09827d98457fc7454ccd16e1ce509f6ac3dfeb
Author: Zhengqiang Duan <st...@gmail.com>
AuthorDate: Tue May 18 12:11:13 2021 +0800

    fix projection owner check when exist subquery temporary table (#10363)
    
    * fix projection owner check when exist subquery temporary table
    
    * change null return to optional
---
 .../impl/EncryptProjectionTokenGenerator.java      |  2 +-
 .../infra/binder/segment/table/TablesContext.java  | 12 +++---
 .../statement/dml/SelectStatementContext.java      | 18 +++++++-
 .../engine/ProjectionsContextEngineTest.java       | 48 ++++++++++++++++++++++
 4 files changed, 71 insertions(+), 9 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java
index 0eee6de..054c6ff 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java
@@ -90,7 +90,7 @@ public final class EncryptProjectionTokenGenerator extends BaseEncryptSQLTokenGe
             return false;
         }
         Optional<OwnerSegment> ownerSegment = ((ShorthandProjectionSegment) projectionSegment).getOwner();
-        return ownerSegment.map(segment -> selectStatementContext.getTablesContext().findTableNameFromSQL(segment.getIdentifier().getValue()).equalsIgnoreCase(tableName)).orElse(true);
+        return ownerSegment.map(segment -> selectStatementContext.getTablesContext().findTableNameFromSQL(segment.getIdentifier().getValue()).orElse("").equalsIgnoreCase(tableName)).orElse(true);
     }
     
     private SubstitutableColumnNameToken generateSQLToken(final ColumnProjectionSegment segment, final String tableName) {
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
index 0fb75c2..5de6208 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
@@ -19,8 +19,8 @@ package org.apache.shardingsphere.infra.binder.segment.table;
 
 import lombok.Getter;
 import lombok.ToString;
-import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ColumnProjection;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 
@@ -74,7 +74,7 @@ public final class TablesContext {
             return Optional.of(tables.iterator().next().getTableName().getIdentifier().getValue());
         }
         if (column.getOwner().isPresent()) {
-            return Optional.of(findTableNameFromSQL(column.getOwner().get().getIdentifier().getValue()));
+            return findTableNameFromSQL(column.getOwner().get().getIdentifier().getValue());
         }
         return findTableNameFromMetaData(column.getIdentifier().getValue(), schema);
     }
@@ -91,7 +91,7 @@ public final class TablesContext {
             return Optional.of(tables.iterator().next().getTableName().getIdentifier().getValue());
         }
         if (null != column.getOwner()) {
-            return Optional.of(findTableNameFromSQL(column.getOwner()));
+            return findTableNameFromSQL(column.getOwner());
         }
         return findTableNameFromMetaData(column.getName(), schema);
     }
@@ -101,13 +101,13 @@ public final class TablesContext {
      * @param tableNameOrAlias table name or alias
      * @return table name
      */
-    public String findTableNameFromSQL(final String tableNameOrAlias) {
+    public Optional<String> findTableNameFromSQL(final String tableNameOrAlias) {
         for (SimpleTableSegment each : tables) {
             if (tableNameOrAlias.equalsIgnoreCase(each.getTableName().getIdentifier().getValue()) || tableNameOrAlias.equals(each.getAlias().orElse(null))) {
-                return each.getTableName().getIdentifier().getValue();
+                return Optional.of(each.getTableName().getIdentifier().getValue());
             }
         }
-        throw new IllegalStateException("Can not find owner from table.");
+        return Optional.empty();
     }
     
     private Optional<String> findTableNameFromMetaData(final String columnName, final ShardingSphereSchema schema) {
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java
index 00aa1dd..3d1cad5 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.infra.binder.statement.dml;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
-import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.binder.segment.select.groupby.GroupByContext;
 import org.apache.shardingsphere.infra.binder.segment.select.groupby.engine.GroupByContextEngine;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByContext;
@@ -35,6 +34,7 @@ import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
@@ -43,14 +43,19 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.Te
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 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.SubqueryTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.util.SQLUtil;
 import org.apache.shardingsphere.sql.parser.sql.common.util.WhereSegmentExtractUtils;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 
 import java.util.Collection;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 /**
  * Select SQL statement context.
@@ -213,6 +218,15 @@ public final class SelectStatementContext extends CommonSQLStatementContext<Sele
      * @return tables with from clause
      */
     public Collection<SimpleTableSegment> getFromSimpleTableSegments() {
-        return new TableExtractor().extractTablesWithFromClause(getSqlStatement());
+        Collection<SimpleTableSegment> result = new LinkedList<>();
+        TableExtractor extractor = new TableExtractor();
+        result.addAll(extractor.extractTablesWithFromClause(getSqlStatement()));
+        result.addAll(getTemporarySimpleTableSegments(extractor.getTableContext()));
+        return result;
+    }
+    
+    private Collection<SimpleTableSegment> getTemporarySimpleTableSegments(final Collection<TableSegment> tableSegments) {
+        return tableSegments.stream().filter(each -> each instanceof SubqueryTableSegment).map(each
+            -> new SimpleTableSegment(each.getStartIndex(), each.getStopIndex(), new IdentifierValue(each.getAlias().orElse("")))).collect(Collectors.toList());
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionsContextEngineTest.java b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionsContextEngineTest.java
index 226a5b6..23e5e48 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionsContextEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/segment/select/projection/engine/ProjectionsContextEngineTest.java
@@ -26,14 +26,17 @@ import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementConte
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.OrderDirection;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.subquery.SubquerySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ColumnProjectionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ShorthandProjectionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.AliasSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.OwnerSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SubqueryTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
@@ -343,4 +346,49 @@ public final class ProjectionsContextEngineTest {
                 .createProjectionsContext(tables, projectionsSegment, new GroupByContext(Collections.emptyList(), 0), orderByContext);
         assertNotNull(actual);
     }
+        
+    @Test
+    public void assertCreateProjectionsContextWithTemporaryTableForMySQL() {
+        assertCreateProjectionsContextWithTemporaryTable(new MySQLSelectStatement(), new MySQLSelectStatement());
+    }
+    
+    @Test
+    public void assertCreateProjectionsContextWithTemporaryTableForOracle() {
+        assertCreateProjectionsContextWithTemporaryTable(new OracleSelectStatement(), new OracleSelectStatement());
+    }
+    
+    @Test
+    public void assertCreateProjectionsContextWithTemporaryTableForPostgreSQL() {
+        assertCreateProjectionsContextWithTemporaryTable(new PostgreSQLSelectStatement(), new PostgreSQLSelectStatement());
+    }
+    
+    @Test
+    public void assertCreateProjectionsContextWithTemporaryTableForSQL92() {
+        assertCreateProjectionsContextWithTemporaryTable(new SQL92SelectStatement(), new SQL92SelectStatement());
+    }
+    
+    @Test
+    public void assertCreateProjectionsContextWithTemporaryTableForSQLServer() {
+        assertCreateProjectionsContextWithTemporaryTable(new SQLServerSelectStatement(), new SQLServerSelectStatement());
+    }
+    
+    private void assertCreateProjectionsContextWithTemporaryTable(final SelectStatement selectStatement, final SelectStatement subquerySelectStatement) {
+        ProjectionsSegment projectionsSegment = new ProjectionsSegment(0, 0);
+        ShorthandProjectionSegment projectionSegment = new ShorthandProjectionSegment(0, 0);
+        projectionSegment.setOwner(new OwnerSegment(0, 0, new IdentifierValue("d")));
+        projectionsSegment.getProjections().addAll(Collections.singletonList(projectionSegment));
+        selectStatement.setProjections(projectionsSegment);
+        subquerySelectStatement.setProjections(new ProjectionsSegment(0, 0));
+        SubqueryTableSegment subqueryTableSegment = new SubqueryTableSegment(new SubquerySegment(0, 0, subquerySelectStatement));
+        subqueryTableSegment.setAlias(new AliasSegment(0, 0, new IdentifierValue("d")));
+        selectStatement.setFrom(subqueryTableSegment);
+        ColumnSegment columnSegment = new ColumnSegment(0, 0, new IdentifierValue("name"));
+        columnSegment.setOwner(new OwnerSegment(0, 0, new IdentifierValue("d")));
+        OrderByItem groupByItem = new OrderByItem(new ColumnOrderByItemSegment(columnSegment, OrderDirection.ASC));
+        GroupByContext groupByContext = new GroupByContext(Collections.singleton(groupByItem), 0);
+        SelectStatementContext selectStatementContext = new SelectStatementContext(schema, new LinkedList<>(), selectStatement);
+        ProjectionsContext actual = new ProjectionsContextEngine(schema)
+                .createProjectionsContext(selectStatementContext.getFromSimpleTableSegments(), projectionsSegment, groupByContext, new OrderByContext(Collections.emptyList(), false));
+        assertNotNull(actual);
+    }
 }