You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2021/06/01 13:46:28 UTC

[shardingsphere] branch master updated: Improve the handling of FederateRowExecutor (#10611)

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

duanzhengqiang 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 bf74c30  Improve the handling of FederateRowExecutor (#10611)
bf74c30 is described below

commit bf74c308de705de108bbafb7c329de3e38e38f16
Author: Juan Pan(Trista) <pa...@apache.org>
AuthorDate: Tue Jun 1 21:45:54 2021 +0800

    Improve the handling of FederateRowExecutor (#10611)
    
    * Improve the handling of FederateRowExecutor
    
    * java doc
---
 .../schema/FederateLogicSchemaFactory.java         | 51 ----------------------
 .../federate/schema/row/FederateRowExecutor.java   | 21 ++++++---
 .../schema/table/FederateFilterableTable.java      |  6 +--
 ...java => FederateExecutionContextGenerator.java} | 22 +++++-----
 ...tor.java => FederateExecutionSQLGenerator.java} | 16 ++++---
 5 files changed, 38 insertions(+), 78 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/FederateLogicSchemaFactory.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/FederateLogicSchemaFactory.java
deleted file mode 100644
index 240c174..0000000
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/FederateLogicSchemaFactory.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.executor.sql.federate.schema;
-
-import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.infra.executor.sql.federate.schema.row.FederateRowExecutor;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.optimize.core.schema.LogicSchemaMetadatas;
-
-import java.util.Map;
-
-/**
- * Federate logic schema factory.
- */
-public final class FederateLogicSchemaFactory {
-    
-    private final LogicSchemaMetadatas metadatas;
-    
-    public FederateLogicSchemaFactory(final Map<String, ShardingSphereMetaData> metaDataMap) {
-        metadatas = new LogicSchemaMetadatas(metaDataMap);
-    }
-    
-    /**
-     * Create schema.
-     *
-     * @param name name
-     * @param executor executor
-     * @return schema
-     */
-    public FederateLogicSchema create(final String name, final FederateRowExecutor executor) {
-        if (!metadatas.getSchemas().containsKey(name)) {
-            throw new ShardingSphereException("No `%s` schema.", name);
-        }
-        return new FederateLogicSchema(metadatas.getSchemas().get(name), executor);
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java
index 207706c..30e1dd9 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java
@@ -17,8 +17,9 @@
 
 package org.apache.shardingsphere.infra.executor.sql.federate.schema.row;
 
-import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.rex.RexNode;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
@@ -29,6 +30,8 @@ import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.J
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.ExecuteResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
+import org.apache.shardingsphere.infra.executor.sql.federate.schema.table.generator.FederateExecutionContextGenerator;
+import org.apache.shardingsphere.infra.executor.sql.federate.schema.table.generator.FederateExecutionSQLGenerator;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.ExecutorJDBCManager;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.JDBCDriverType;
@@ -39,6 +42,7 @@ import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.Collection;
+import java.util.List;
 import java.util.stream.Collectors;
 
 /**
@@ -56,18 +60,25 @@ public final class FederateRowExecutor {
     
     private final JDBCExecutor jdbcExecutor;
     
-    @Getter
-    private final ExecutionContext initialExecutionContext;
+    private final ExecutionContext routeExecutionContext;
     
     private final JDBCExecutorCallback<? extends ExecuteResult> callback;
     
     /**
      * Execute.
      *
-     * @param context context
+     * @param logicTable logic table
+     * @param root root
+     * @param filters filter
+     * @param projects projects
      * @return a query result list
      */
-    public Collection<QueryResult> execute(final ExecutionContext context) {
+    public Collection<QueryResult> execute(final String logicTable, final DataContext root, final List<RexNode> filters, final int[] projects) {
+        FederateExecutionContextGenerator generator = new FederateExecutionContextGenerator(logicTable, routeExecutionContext, new FederateExecutionSQLGenerator(root, filters, projects));
+        return execute(generator.generate());
+    }
+    
+    private Collection<QueryResult> execute(final ExecutionContext context) {
         try {
             ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = createExecutionGroupContext(context);
             ExecuteProcessEngine.initialize(context.getSqlStatementContext(), executionGroupContext, props);
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/FederateFilterableTable.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/FederateFilterableTable.java
index 22b3a169..9f7dca1 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/FederateFilterableTable.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/FederateFilterableTable.java
@@ -25,8 +25,6 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.ProjectableFilterableTable;
 import org.apache.shardingsphere.infra.executor.sql.federate.schema.row.FederateRowEnumerator;
 import org.apache.shardingsphere.infra.executor.sql.federate.schema.row.FederateRowExecutor;
-import org.apache.shardingsphere.infra.executor.sql.federate.schema.table.generator.FederateExecuteContextGenerator;
-import org.apache.shardingsphere.infra.executor.sql.federate.schema.table.generator.FederateExecuteSQLGenerator;
 import org.apache.shardingsphere.infra.optimize.core.schema.LogicTableMetadata;
 
 import java.util.List;
@@ -47,9 +45,7 @@ public final class FederateFilterableTable extends AbstractFederateTable impleme
 
             @Override
             public Enumerator<Object[]> enumerator() {
-                FederateExecuteContextGenerator generator = new FederateExecuteContextGenerator(getMetadata().getName(), 
-                        getExecutor().getInitialExecutionContext(), new FederateExecuteSQLGenerator(root, filters, projects));
-                return new FederateRowEnumerator(getExecutor().execute(generator.generate()));
+                return new FederateRowEnumerator(getExecutor().execute(getMetadata().getName(), root, filters, projects));
             }
         };
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecuteContextGenerator.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionContextGenerator.java
similarity index 80%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecuteContextGenerator.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionContextGenerator.java
index 45f429b..df3d0e9 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecuteContextGenerator.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionContextGenerator.java
@@ -30,16 +30,16 @@ import java.util.Collections;
 import java.util.LinkedHashSet;
 
 /**
- * Federate table execute context generator.
+ * Federate table execution context generator.
  */
 @RequiredArgsConstructor
-public final class FederateExecuteContextGenerator {
+public final class FederateExecutionContextGenerator {
     
     private final String table;
     
-    private final ExecutionContext initialExecutionContext;
+    private final ExecutionContext routeExecutionContext;
     
-    private final FederateExecuteSQLGenerator filter;
+    private final FederateExecutionSQLGenerator generator;
     
     /**
      * Create execution context.
@@ -47,24 +47,24 @@ public final class FederateExecuteContextGenerator {
      * @return execution context
      */
     public ExecutionContext generate() {
-        RouteContext routeContext = getRouteContext(initialExecutionContext.getRouteContext());
-        return new ExecutionContext(initialExecutionContext.getSqlStatementContext(),
-                getExecutionUnits(routeContext.getRouteUnits(), filter), routeContext);
+        RouteContext routeContext = getRouteContext(routeExecutionContext.getRouteContext());
+        return new ExecutionContext(routeExecutionContext.getSqlStatementContext(),
+                getExecutionUnits(routeContext.getRouteUnits(), generator), routeContext);
     }
     
-    private Collection<ExecutionUnit> getExecutionUnits(final Collection<RouteUnit> routeUnits, final FederateExecuteSQLGenerator filter) {
+    private Collection<ExecutionUnit> getExecutionUnits(final Collection<RouteUnit> routeUnits, final FederateExecutionSQLGenerator generator) {
         Collection<ExecutionUnit> result = new LinkedHashSet<>();
         for (RouteUnit each: routeUnits) {
-            fillExecutionUnits(result, filter, each);
+            fillExecutionUnits(result, generator, each);
         }
         return result;
     }
     
-    private void fillExecutionUnits(final Collection<ExecutionUnit> executionUnits, final FederateExecuteSQLGenerator filter, final RouteUnit routeUnit) {
+    private void fillExecutionUnits(final Collection<ExecutionUnit> executionUnits, final FederateExecutionSQLGenerator generator, final RouteUnit routeUnit) {
         for (RouteMapper mapper : routeUnit.getTableMappers()) {
             if (mapper.getLogicName().equals(table)) {
                 executionUnits.add(new ExecutionUnit(routeUnit.getDataSourceMapper().getActualName(),
-                        new SQLUnit(filter.generate(mapper.getActualName()), Collections.emptyList(), Collections.singletonList(mapper))));
+                        new SQLUnit(generator.generate(mapper.getActualName()), Collections.emptyList(), Collections.singletonList(mapper))));
             }
         }
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecuteSQLGenerator.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionSQLGenerator.java
similarity index 80%
rename from shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecuteSQLGenerator.java
rename to shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionSQLGenerator.java
index 890487b..2df025b 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecuteSQLGenerator.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionSQLGenerator.java
@@ -17,22 +17,26 @@
 
 package org.apache.shardingsphere.infra.executor.sql.federate.schema.table.generator;
 
+import lombok.RequiredArgsConstructor;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.rex.RexNode;
 
 import java.util.List;
 
 /**
- * Federate execute sql generator.
+ * Federate execution sql generator.
  */
-public final class FederateExecuteSQLGenerator {
+@RequiredArgsConstructor
+public final class FederateExecutionSQLGenerator {
     
-    public FederateExecuteSQLGenerator(final DataContext root, final List<RexNode> filters, final int[] projects) {
-        // TODO
-    }
+    private final DataContext root;
+    
+    private final List<RexNode> filters;
+    
+    private final int[] projects;
     
     /**
-     * Create sql.
+     * Generate sql.
      *
      * @param table table
      * @return sql