This is an automated email from the ASF dual-hosted git repository. ron pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-1.17 by this push: new d2f93a5527b [FLINK-34379][table] Fix OutOfMemoryError with large queries d2f93a5527b is described below commit d2f93a5527b05583fc97bbae511ca0ac95325c02 Author: Jeyhun Karimov <je.kari...@gmail.com> AuthorDate: Tue Apr 2 00:24:02 2024 +0200 [FLINK-34379][table] Fix OutOfMemoryError with large queries --- .../utils/DynamicPartitionPruningUtils.java | 9 +- .../DynamicPartitionPruningProgramTest.java | 85 +++ .../program/DynamicPartitionPruningProgramTest.xml | 618 +++++++++++++++++++++ 3 files changed, 711 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java index 90f7b40bc0b..089e13333fd 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java @@ -61,8 +61,10 @@ import org.apache.calcite.util.ImmutableIntList; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; /** Planner utils for Dynamic partition Pruning. */ @@ -115,7 +117,7 @@ public class DynamicPartitionPruningUtils { private final RelNode relNode; private boolean hasFilter; private boolean hasPartitionedScan; - private final List<ContextResolvedTable> tables = new ArrayList<>(); + private final Set<ContextResolvedTable> tables = new HashSet<>(); public DppDimSideChecker(RelNode relNode) { this.relNode = relNode; @@ -235,9 +237,14 @@ public class DynamicPartitionPruningUtils { if (tables.size() == 0) { tables.add(catalogTable); } else { + boolean hasAdded = false; for (ContextResolvedTable thisTable : new ArrayList<>(tables)) { + if (hasAdded) { + break; + } if (!thisTable.getIdentifier().equals(catalogTable.getIdentifier())) { tables.add(catalogTable); + hasAdded = true; } } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java index 8e957e2958a..c7ab3e40ef8 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.optimize.program; +import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.flink.table.catalog.ObjectPath; @@ -30,6 +31,11 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.table.api.Expressions.col; + /** * Tests for rules that extend {@link FlinkDynamicPartitionPruningProgram} to create {@link * org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalDynamicFilteringTableSourceScan}. @@ -80,6 +86,85 @@ public class DynamicPartitionPruningProgramTest extends TableTestBase { + ")"); } + @Test + public void testLargeQueryPlanShouldNotOutOfMemoryWithTableApi() { + // TABLE_OPTIMIZER_DYNAMIC_FILTERING_ENABLED is already enabled + List<String> selectStmts = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + util.tableEnv() + .executeSql( + "CREATE TABLE IF NOT EXISTS table" + + i + + "(att STRING,filename STRING) " + + "with(" + + " 'connector' = 'values', " + + " 'runtime-source' = 'NewSource', " + + " 'bounded' = 'true'" + + ")"); + selectStmts.add("select att,filename from table" + i); + } + + final String countName = "CNM"; + Table allUnionTable = util.tableEnv().sqlQuery(String.join(" UNION ALL ", selectStmts)); + Table res = + allUnionTable.join( + allUnionTable + .groupBy(col("att")) + .select(col("att"), col("att").count().as(countName)) + .filter(col(countName).isGreater(1)) + .select(col("att").as("l_key")), + col("att").isEqual(col("l_key"))); + util.verifyExecPlan(res); + + // clear resources + for (int i = 0; i < 100; i++) { + util.tableEnv().executeSql("DROP TABLE IF EXISTS table" + i); + } + } + + @Test + public void testLargeQueryPlanShouldNotOutOfMemoryWithSqlApi() { + // TABLE_OPTIMIZER_DYNAMIC_FILTERING_ENABLED is already enabled + List<String> selectStmts = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + util.tableEnv() + .executeSql( + "CREATE TABLE IF NOT EXISTS table" + + i + + "(att STRING,filename STRING) " + + "with(" + + " 'connector' = 'values', " + + " 'runtime-source' = 'NewSource', " + + " 'bounded' = 'true'" + + ")"); + selectStmts.add("select att,filename from table" + i); + } + + final String countName = "CNM"; + final String unionSelectStmts = String.join(" UNION ALL ", selectStmts); + + final String groupedUnionStmt = + String.format( + "SELECT att as l_key, COUNT(att) AS %s " + + "FROM (%s) " + + "GROUP BY att " + + "HAVING COUNT(att) > 1 ", + countName, unionSelectStmts); + + final String joinedUnionStmt = + String.format( + "SELECT * FROM (%s) as t1 INNER JOIN (%s) as t2 ON t1.att = t2.l_key", + unionSelectStmts, groupedUnionStmt); + Table resultTable = util.tableEnv().sqlQuery(joinedUnionStmt); + + util.verifyExecPlan(resultTable); + + // clear resources + for (int i = 0; i < 100; i++) { + util.tableEnv().executeSql("DROP TABLE IF EXISTS table" + i); + } + } + @Test public void testDimTableFilteringFieldsNotInJoinKey() { // fact_part.id not in dynamic-filtering-fields, so dynamic partition pruning will not diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.xml index 6a86ad4fb0b..191084f4ec8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/program/DynamicPartitionPruningProgramTest.xml @@ -16,6 +16,624 @@ See the License for the specific language governing permissions and limitations under the License. --> <Root> + <TestCase name="testLargeQueryPlanShouldNotOutOfMemoryWithTableApi"> + <Resource name="optimized exec plan"> + <![CDATA[ +MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(att = l_key)], select=[att, filename, l_key], build=[right])\n:- [#1] Exchange(distribution=[hash[att]])\n+- Calc(select=[att AS l_key], where=[(EXPR$0 > 1)])\n +- HashAggregate(isMerge=[true], groupBy=[att], select=[att, Final_COUNT(count$0) AS EXPR$0])\n +- [#2] Exchange(distribution=[hash[att]])\n]) +:- Exchange(distribution=[hash[att]]) +: +- Union(all=[true], union=[att, filename])(reuse_id=[1]) +: :- Union(all=[true], union=[att, filename]) +: : :- Union(all=[true], union=[att, filename]) +: : : :- Union(all=[true], union=[att, filename]) +: : : : :- Union(all=[true], union=[att, filename]) +: : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- TableSourceScan(table=[[testCatalog, test_database, table0]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table1]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table2]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table3]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table4]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table5]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table6]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table7]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table8]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table9]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table10]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table11]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table12]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table13]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table14]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table15]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table16]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table17]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table18]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table19]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table20]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table21]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table22]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table23]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table24]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table25]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table26]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table27]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table28]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table29]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table30]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table31]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table32]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table33]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table34]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table35]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table36]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table37]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table38]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table39]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table40]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table41]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table42]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table43]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table44]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table45]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table46]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table47]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table48]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table49]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table50]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table51]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table52]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table53]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table54]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table55]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table56]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table57]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table58]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table59]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table60]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table61]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table62]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table63]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table64]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table65]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table66]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table67]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table68]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table69]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table70]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table71]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table72]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table73]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table74]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table75]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table76]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table77]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table78]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table79]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table80]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table81]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table82]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table83]], fields=[att, filename]) +: : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table84]], fields=[att, filename]) +: : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table85]], fields=[att, filename]) +: : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table86]], fields=[att, filename]) +: : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table87]], fields=[att, filename]) +: : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table88]], fields=[att, filename]) +: : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table89]], fields=[att, filename]) +: : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table90]], fields=[att, filename]) +: : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table91]], fields=[att, filename]) +: : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table92]], fields=[att, filename]) +: : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table93]], fields=[att, filename]) +: : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table94]], fields=[att, filename]) +: : : : : +- TableSourceScan(table=[[testCatalog, test_database, table95]], fields=[att, filename]) +: : : : +- TableSourceScan(table=[[testCatalog, test_database, table96]], fields=[att, filename]) +: : : +- TableSourceScan(table=[[testCatalog, test_database, table97]], fields=[att, filename]) +: : +- TableSourceScan(table=[[testCatalog, test_database, table98]], fields=[att, filename]) +: +- TableSourceScan(table=[[testCatalog, test_database, table99]], fields=[att, filename]) ++- Exchange(distribution=[hash[att]]) + +- LocalHashAggregate(groupBy=[att], select=[att, Partial_COUNT(att) AS count$0]) + +- Reused(reference_id=[1]) +]]> + </Resource> + </TestCase> + <TestCase name="testLargeQueryPlanShouldNotOutOfMemoryWithSqlApi"> + <Resource name="optimized exec plan"> + <![CDATA[ +MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(att = l_key)], select=[att, filename, l_key, CNM], build=[right])\n:- [#1] Exchange(distribution=[hash[att]])\n+- Calc(select=[att AS l_key, CNM], where=[(CNM > 1)])\n +- HashAggregate(isMerge=[true], groupBy=[att], select=[att, Final_COUNT(count$0) AS CNM])\n +- [#2] Exchange(distribution=[hash[att]])\n]) +:- Exchange(distribution=[hash[att]]) +: +- Union(all=[true], union=[att, filename]) +: :- Union(all=[true], union=[att, filename]) +: : :- Union(all=[true], union=[att, filename]) +: : : :- Union(all=[true], union=[att, filename]) +: : : : :- Union(all=[true], union=[att, filename]) +: : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- TableSourceScan(table=[[testCatalog, test_database, table0]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table1]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table2]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table3]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table4]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table5]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table6]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table7]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table8]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table9]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table10]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table11]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table12]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table13]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table14]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table15]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table16]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table17]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table18]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table19]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table20]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table21]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table22]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table23]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table24]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table25]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table26]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table27]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table28]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table29]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table30]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table31]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table32]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table33]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table34]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table35]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table36]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table37]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table38]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table39]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table40]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table41]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table42]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table43]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table44]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table45]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table46]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table47]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table48]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table49]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table50]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table51]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table52]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table53]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table54]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table55]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table56]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table57]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table58]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table59]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table60]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table61]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table62]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table63]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table64]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table65]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table66]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table67]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table68]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table69]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table70]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table71]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table72]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table73]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table74]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table75]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table76]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table77]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table78]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table79]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table80]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table81]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table82]], fields=[att, filename]) +: : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table83]], fields=[att, filename]) +: : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table84]], fields=[att, filename]) +: : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table85]], fields=[att, filename]) +: : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table86]], fields=[att, filename]) +: : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table87]], fields=[att, filename]) +: : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table88]], fields=[att, filename]) +: : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table89]], fields=[att, filename]) +: : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table90]], fields=[att, filename]) +: : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table91]], fields=[att, filename]) +: : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table92]], fields=[att, filename]) +: : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table93]], fields=[att, filename]) +: : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table94]], fields=[att, filename]) +: : : : : +- TableSourceScan(table=[[testCatalog, test_database, table95]], fields=[att, filename]) +: : : : +- TableSourceScan(table=[[testCatalog, test_database, table96]], fields=[att, filename]) +: : : +- TableSourceScan(table=[[testCatalog, test_database, table97]], fields=[att, filename]) +: : +- TableSourceScan(table=[[testCatalog, test_database, table98]], fields=[att, filename]) +: +- TableSourceScan(table=[[testCatalog, test_database, table99]], fields=[att, filename]) ++- Exchange(distribution=[hash[att]]) + +- LocalHashAggregate(groupBy=[att], select=[att, Partial_COUNT(att) AS count$0]) + +- Union(all=[true], union=[att]) + :- Union(all=[true], union=[att]) + : :- Union(all=[true], union=[att]) + : : :- Union(all=[true], union=[att]) + : : : :- Union(all=[true], union=[att]) + : : : : :- Union(all=[true], union=[att]) + : : : : : :- Union(all=[true], union=[att]) + : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- Union(all=[true], union=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : :- TableSourceScan(table=[[testCatalog, test_database, table0, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table1, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table2, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table3, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table4, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table5, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table6, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table7, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table8, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table9, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table10, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table11, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table12, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table13, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table14, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table15, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table16, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table17, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table18, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table19, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table20, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table21, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table22, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table23, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table24, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table25, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table26, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table27, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table28, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table29, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table30, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table31, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table32, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table33, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table34, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table35, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table36, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table37, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table38, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table39, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table40, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table41, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table42, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table43, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table44, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table45, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table46, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table47, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table48, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table49, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table50, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table51, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table52, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table53, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table54, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table55, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table56, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table57, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table58, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table59, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table60, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table61, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table62, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table63, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table64, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table65, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table66, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table67, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table68, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table69, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table70, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table71, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table72, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table73, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table74, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table75, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table76, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table77, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table78, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table79, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table80, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table81, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table82, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table83, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table84, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table85, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table86, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table87, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table88, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table89, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table90, project=[att], metadata=[]]], fields=[att]) + : : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table91, project=[att], metadata=[]]], fields=[att]) + : : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table92, project=[att], metadata=[]]], fields=[att]) + : : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table93, project=[att], metadata=[]]], fields=[att]) + : : : : : +- TableSourceScan(table=[[testCatalog, test_database, table94, project=[att], metadata=[]]], fields=[att]) + : : : : +- TableSourceScan(table=[[testCatalog, test_database, table95, project=[att], metadata=[]]], fields=[att]) + : : : +- TableSourceScan(table=[[testCatalog, test_database, table96, project=[att], metadata=[]]], fields=[att]) + : : +- TableSourceScan(table=[[testCatalog, test_database, table97, project=[att], metadata=[]]], fields=[att]) + : +- TableSourceScan(table=[[testCatalog, test_database, table98, project=[att], metadata=[]]], fields=[att]) + +- TableSourceScan(table=[[testCatalog, test_database, table99, project=[att], metadata=[]]], fields=[att]) +]]> + </Resource> + </TestCase> <TestCase name="testAntiJoin"> <Resource name="sql"> <![CDATA[Select * from fact_part where not exists (select dim_date_sk from dim where dim.price < 500)]]>