From 89d019f02025e31100680bd79fa7602778fa43c9 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 7 Mar 2024 18:47:08 +0530 Subject: [PATCH 01/26] [multistage] support database --- .../MultiStageBrokerRequestHandler.java | 12 +-- .../MultiStageEngineIntegrationTest.java | 87 ++++++++++++++++++- .../calcite/jdbc/CalciteSchemaBuilder.java | 11 +++ .../apache/pinot/query/QueryEnvironment.java | 79 +++++++++++------ .../pinot/query/catalog/PinotCatalog.java | 67 +++++++++++++- .../logical/RelToPlanNodeConverter.java | 16 +++- 6 files changed, 230 insertions(+), 42 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 35aff7efd245..32e1492e73de 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -81,7 +81,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler { private static final Logger LOGGER = LoggerFactory.getLogger(MultiStageBrokerRequestHandler.class); - private final QueryEnvironment _queryEnvironment; + private final WorkerManager _workerManager; private final MailboxService _mailboxService; private final QueryDispatcher _queryDispatcher; @@ -93,9 +93,7 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId LOGGER.info("Using Multi-stage BrokerRequestHandler."); String hostname = config.getProperty(CommonConstants.MultiStageQueryRunner.KEY_OF_QUERY_RUNNER_HOSTNAME); int port = Integer.parseInt(config.getProperty(CommonConstants.MultiStageQueryRunner.KEY_OF_QUERY_RUNNER_PORT)); - _queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()), - CalciteSchemaBuilder.asRootSchema(new PinotCatalog(tableCache)), - new WorkerManager(hostname, port, routingManager), _tableCache); + _workerManager = new WorkerManager(hostname, port, _routingManager); _mailboxService = new MailboxService(hostname, port, config); _queryDispatcher = new QueryDispatcher(_mailboxService); @@ -128,9 +126,11 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S queryTimeoutMs = timeoutMsFromQueryOption == null ? _brokerTimeoutMs : timeoutMsFromQueryOption; // Compile the request compilationStartTimeNs = System.nanoTime(); + QueryEnvironment queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()), + CalciteSchemaBuilder.asRootSchema(new PinotCatalog(_tableCache)), _workerManager, _tableCache); switch (sqlNodeAndOptions.getSqlNode().getKind()) { case EXPLAIN: - queryPlanResult = _queryEnvironment.explainQuery(query, sqlNodeAndOptions, requestId); + queryPlanResult = queryEnvironment.explainQuery(query, sqlNodeAndOptions, requestId); String plan = queryPlanResult.getExplainPlan(); Set tableNames = queryPlanResult.getTableNames(); if (!hasTableAccess(requesterIdentity, tableNames, requestContext, httpHeaders)) { @@ -140,7 +140,7 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S return constructMultistageExplainPlan(query, plan); case SELECT: default: - queryPlanResult = _queryEnvironment.planQuery(query, sqlNodeAndOptions, requestId); + queryPlanResult = queryEnvironment.planQuery(query, sqlNodeAndOptions, requestId); break; } } catch (WebApplicationException e) { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index c18dd026ffac..90052f720f3c 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -25,12 +25,19 @@ import java.time.Instant; import java.time.ZoneId; import java.time.format.DateTimeFormatter; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import javax.annotation.Nullable; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.TransformConfig; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.MetricFieldSpec; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.util.TestUtils; import org.testng.Assert; @@ -47,6 +54,8 @@ public class MultiStageEngineIntegrationTest extends BaseClusterIntegrationTestSet { private static final String SCHEMA_FILE_NAME = "On_Time_On_Time_Performance_2014_100k_subset_nonulls.schema"; + private String _tableName = DEFAULT_TABLE_NAME; + private List _avroFiles = new ArrayList<>(); @Override protected String getSchemaFileName() { @@ -72,17 +81,17 @@ public void setUp() addTableConfig(tableConfig); // Unpack the Avro files - List avroFiles = unpackAvroData(_tempDir); + _avroFiles = unpackAvroData(_tempDir); // Create and upload segments - ClusterIntegrationTestUtils.buildSegmentsFromAvro(avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); uploadSegments(getTableName(), _tarDir); // Set up the H2 connection - setUpH2Connection(avroFiles); + setUpH2Connection(_avroFiles); // Initialize the query generator - setUpQueryGenerator(avroFiles); + setUpQueryGenerator(_avroFiles); // Wait for all documents loaded waitForAllDocsLoaded(600_000L); @@ -749,6 +758,76 @@ public void testSearch() assertNoError(jsonNode); } + @Override + protected String getTableName() { + return _tableName; + } + + @Test + public void testWithDatabaseContext() + throws Exception { + try { + _tableName = "db1." + DEFAULT_TABLE_NAME; + String defaultCol = "ActualElapsedTime"; + String customCol = "ActualElapsedTime_2"; + Schema schema = createSchema(); + schema.addField(new MetricFieldSpec(customCol, FieldSpec.DataType.INT)); + addSchema(schema); + TableConfig tableConfig = createOfflineTableConfig(); + assert tableConfig.getIndexingConfig().getNoDictionaryColumns() != null; + List noDicCols = new ArrayList<>(DEFAULT_NO_DICTIONARY_COLUMNS); + noDicCols.add(customCol); + tableConfig.getIndexingConfig().setNoDictionaryColumns(noDicCols); + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setTransformConfigs(List.of(new TransformConfig(customCol, defaultCol))); + tableConfig.setIngestionConfig(ingestionConfig); + addTableConfig(tableConfig); + + // Create and upload segments to 'db1.mytable' + TestUtils.ensureDirectoriesExistAndEmpty(_segmentDir, _tarDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + uploadSegments(getTableName(), _tarDir); + + // Wait for all documents loaded + waitForAllDocsLoaded(600_000L); + + // default database check. No database context passed + checkQueryResultForDBTest(defaultCol, DEFAULT_TABLE_NAME); + + // default database check. Default database context passed as table prefix + checkQueryResultForDBTest(defaultCol, "default." + DEFAULT_TABLE_NAME); + + // default database check. Default database context passed as SET database='dbName' + checkQueryResultForDBTest(defaultCol, DEFAULT_TABLE_NAME, "default"); + + // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. + // custom database check. Database context passed as table prefix + checkQueryResultForDBTest(customCol, _tableName); + + // custom database check. Database context passed as SET database='dbName' + checkQueryResultForDBTest(customCol, DEFAULT_TABLE_NAME, "db1"); + } finally { + dropOfflineTable(_tableName); + _tableName = DEFAULT_TABLE_NAME; + } + } + + private void checkQueryResultForDBTest(String column, String tableName) + throws Exception { + checkQueryResultForDBTest(column, tableName, null); + } + + private void checkQueryResultForDBTest(String column, String tableName, @Nullable String database) + throws Exception { + String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") + + "select max(" + column + ") from " + tableName + ";"; + // max value of 'ActualElapsedTime' + long expectedValue = 678; + JsonNode jsonNode = postQuery(query); + long result = jsonNode.get("resultTable").get("rows").get(0).get(0).asLong(); + assertEquals(result, expectedValue); + } + @AfterClass public void tearDown() throws Exception { diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java index edb2d74bf07c..b7f64945f9ca 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java +++ b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java @@ -61,4 +61,15 @@ public static CalciteSchema asRootSchema(Schema root) { } return rootSchema; } + + public static CalciteSchema asSubSchema(Schema root, String name) { + CalciteSchema subSchema = CalciteSchema.createRootSchema(false, false, name, root); + SchemaPlus schemaPlus = subSchema.plus(); + for (Map.Entry> e : FunctionRegistry.getRegisteredCalciteFunctionMap().entrySet()) { + for (Function f : e.getValue()) { + schemaPlus.add(e.getKey(), f); + } + } + return subSchema; + } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 32a75c4a3d4c..ac41cbac0afa 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.Set; import javax.annotation.Nullable; @@ -84,6 +85,7 @@ *

It provide the higher level entry interface to convert a SQL string into a {@link DispatchableSubPlan}. */ public class QueryEnvironment { + private static final String DATABASE_KEY = "database"; // Calcite configurations private final FrameworkConfig _config; @@ -106,26 +108,10 @@ public QueryEnvironment(TypeFactory typeFactory, CalciteSchema rootSchema, Worke _workerManager = workerManager; _tableCache = tableCache; - // catalog - Properties catalogReaderConfigProperties = new Properties(); - catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); - _catalogReader = new PinotCalciteCatalogReader(_rootSchema, _rootSchema.path(null), _typeFactory, - new CalciteConnectionConfigImpl(catalogReaderConfigProperties)); - - _config = Frameworks.newConfigBuilder().traitDefs() - .operatorTable(new PinotChainedSqlOperatorTable(Arrays.asList( - PinotOperatorTable.instance(), - _catalogReader))) - .defaultSchema(_rootSchema.plus()) - .sqlToRelConverterConfig(SqlToRelConverter.config() - .withHintStrategyTable(getHintStrategyTable()) - .withTrimUnusedFields(true) - // SUB-QUERY Threshold is useless as we are encoding all IN clause in-line anyway - .withInSubQueryThreshold(Integer.MAX_VALUE) - .addRelBuilderConfigTransform(c -> c.withPushJoinCondition(true)) - .addRelBuilderConfigTransform(c -> c.withAggregateUnique(true)) - .addRelBuilderConfigTransform(c -> c.withPruneInputOfAggregate(false))) - .build(); + // catalog & config + _catalogReader = getCatalog(null); + _config = getConfig(_catalogReader); + // opt programs _optProgram = getOptProgram(); _traitProgram = getTraitProgram(); } @@ -143,8 +129,7 @@ public QueryEnvironment(TypeFactory typeFactory, CalciteSchema rootSchema, Worke * @return QueryPlannerResult containing the dispatchable query plan and the relRoot. */ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) { - try (PlannerContext plannerContext = new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, - _traitProgram)) { + try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions.getOptions())) { plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(sqlNodeAndOptions.getSqlNode(), plannerContext); // TODO: current code only assume one SubPlan per query, but we should support multiple SubPlans per query. @@ -172,8 +157,7 @@ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAn * @return QueryPlannerResult containing the explained query plan and the relRoot. */ public QueryPlannerResult explainQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) { - try (PlannerContext plannerContext = new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, - _traitProgram)) { + try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions.getOptions())) { SqlExplain explain = (SqlExplain) sqlNodeAndOptions.getSqlNode(); plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(explain.getExplicandum(), plannerContext); @@ -206,8 +190,11 @@ public String explainQuery(String sqlQuery, long requestId) { } public List getTableNamesForQuery(String sqlQuery) { - try (PlannerContext plannerContext = new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, - _traitProgram)) { + return getTableNamesForQuery(sqlQuery, CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery).getOptions()); + } + + public List getTableNamesForQuery(String sqlQuery, Map options) { + try (PlannerContext plannerContext = getPlannerContext(options)) { SqlNode sqlNode = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery).getSqlNode(); if (sqlNode.getKind().equals(SqlKind.EXPLAIN)) { sqlNode = ((SqlExplain) sqlNode).getExplicandum(); @@ -344,6 +331,46 @@ private DispatchableSubPlan toDispatchableSubPlan(RelRoot relRoot, PlannerContex // utils // -------------------------------------------------------------------------- + private Prepare.CatalogReader getCatalog(@Nullable String schemaPath) { + Properties catalogReaderConfigProperties = new Properties(); + catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); + CalciteSchema subSchema = schemaPath == null ? _rootSchema : _rootSchema.getSubSchema(schemaPath, false); + if (subSchema != null) { + return new PinotCalciteCatalogReader(subSchema, subSchema.path(null), _typeFactory, + new CalciteConnectionConfigImpl(catalogReaderConfigProperties)); + } else { + throw new IllegalArgumentException("Cannot find schema with path: " + schemaPath); + } + } + + private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { + return Frameworks.newConfigBuilder().traitDefs() + .operatorTable(new PinotChainedSqlOperatorTable(Arrays.asList( + PinotOperatorTable.instance(), + catalogReader))) + .defaultSchema(catalogReader.getRootSchema().plus()) + .sqlToRelConverterConfig(SqlToRelConverter.config() + .withHintStrategyTable(getHintStrategyTable()) + .withTrimUnusedFields(true) + // SUB-QUERY Threshold is useless as we are encoding all IN clause in-line anyway + .withInSubQueryThreshold(Integer.MAX_VALUE) + .addRelBuilderConfigTransform(c -> c.withPushJoinCondition(true)) + .addRelBuilderConfigTransform(c -> c.withAggregateUnique(true)) + .addRelBuilderConfigTransform(c -> c.withPruneInputOfAggregate(false))) + .build(); + } + + private PlannerContext getPlannerContext(Map options) { + String database = options.getOrDefault(DATABASE_KEY, "default"); + if (database.equalsIgnoreCase("default")) { + return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); + } else { + Prepare.CatalogReader catalogReader = getCatalog(database); + FrameworkConfig config = getConfig(catalogReader); + return new PlannerContext(config, catalogReader, _typeFactory, _optProgram, _traitProgram); + } + } + private HintStrategyTable getHintStrategyTable() { return PinotHintStrategyTable.PINOT_HINT_STRATEGY_TABLE; } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index b777c9bc0350..64296fc35dca 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -20,8 +20,13 @@ import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import javax.annotation.Nullable; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.schema.Function; @@ -30,6 +35,7 @@ import org.apache.calcite.schema.SchemaVersion; import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -44,14 +50,46 @@ */ public class PinotCatalog implements Schema { + public static final String DEFAULT_DB_NAME = "default"; + private final TableCache _tableCache; + private final Map _subCatalog; + + private final String _databaseName; + /** * PinotCatalog needs have access to the actual {@link TableCache} object because TableCache hosts the actual * table available for query and processes table/segment metadata updates when cluster status changes. */ public PinotCatalog(TableCache tableCache) { _tableCache = tableCache; + _databaseName = null; + // create all databases + // TODO: we need to monitor table cache changes to register newly created databases + // TODO: we also need to monitor table that needs to be put into the right places + _subCatalog = constructSubCatalogs(_tableCache); + } + + private PinotCatalog(String databaseName, TableCache tableCache) { + _tableCache = tableCache; + _databaseName = databaseName; + _subCatalog = null; + } + + private Map constructSubCatalogs(TableCache tableCache) { + Map subCatalog = new HashMap<>(); + for (String physicalTableName : tableCache.getTableNameMap().keySet()) { + String[] nameSplit = StringUtils.split(physicalTableName, '.'); + if (nameSplit.length > 1) { + String databaseName = nameSplit[0]; + subCatalog.putIfAbsent(databaseName, + CalciteSchemaBuilder.asSubSchema(new PinotCatalog(databaseName, tableCache), databaseName)); + } + } + subCatalog.put(DEFAULT_DB_NAME, + CalciteSchemaBuilder.asSubSchema(new PinotCatalog(DEFAULT_DB_NAME, tableCache), DEFAULT_DB_NAME)); + return subCatalog; } /** @@ -61,7 +99,13 @@ public PinotCatalog(TableCache tableCache) { */ @Override public Table getTable(String name) { - String tableName = TableNameBuilder.extractRawTableName(name); + String rawTableName = TableNameBuilder.extractRawTableName(name); + String tableName; + if (_databaseName != null) { + tableName = constructPhysicalTableName(_databaseName, rawTableName); + } else { + tableName = rawTableName; + } org.apache.pinot.spi.data.Schema schema = _tableCache.getSchema(tableName); if (schema == null) { throw new IllegalArgumentException(String.format("Could not find schema for table: '%s'", tableName)); @@ -69,13 +113,24 @@ public Table getTable(String name) { return new PinotTable(schema); } + public static String constructPhysicalTableName(String databaseName, String tableName) { + return databaseName.equals(DEFAULT_DB_NAME) ? tableName : databaseName + "." + tableName; + } + /** * acquire a set of available table names. * @return the set of table names at the time of query planning. */ @Override public Set getTableNames() { - return _tableCache.getTableNameMap().keySet(); + if (_databaseName != null) { + return _databaseName.equals(DEFAULT_DB_NAME) ? _tableCache.getTableNameMap().keySet().stream() + .filter(n -> StringUtils.split(n, '.').length == 1).collect(Collectors.toSet()) + : _tableCache.getTableNameMap().keySet().stream().filter(n -> n.startsWith(_databaseName)) + .collect(Collectors.toSet()); + } else { + return Collections.emptySet(); + } } @Override @@ -108,12 +163,16 @@ public Set getFunctionNames() { @Override public Schema getSubSchema(String name) { - return null; + if (_subCatalog != null && _subCatalog.containsKey(name)) { + return _subCatalog.get(name).schema; + } else { + return null; + } } @Override public Set getSubSchemaNames() { - return Collections.emptySet(); + return _subCatalog.keySet(); } @Override diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 54839e52ffd2..6384e806d0e3 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -49,6 +49,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.query.catalog.PinotCatalog; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; import org.apache.pinot.query.planner.plannode.FilterNode; @@ -173,7 +174,13 @@ private static PlanNode convertLogicalFilter(LogicalFilter node, int currentStag } private static PlanNode convertLogicalTableScan(LogicalTableScan node, int currentStageId) { - String tableName = node.getTable().getQualifiedName().get(0); + String tableName; + if (node.getTable().getQualifiedName().size() == 1) { + tableName = node.getTable().getQualifiedName().get(0); + } else { + tableName = PinotCatalog.constructPhysicalTableName( + node.getTable().getQualifiedName().get(0), node.getTable().getQualifiedName().get(1)); + } List columnNames = node.getRowType().getFieldList().stream().map(RelDataTypeField::getName).collect(Collectors.toList()); return new TableScanNode(currentStageId, toDataSchema(node.getRowType()), node.getHints(), tableName, columnNames); @@ -287,7 +294,12 @@ public static Set getTableNamesFromRelRoot(RelNode relRoot) { // Calcite encloses table and schema names in square brackets to properly quote and delimit them in SQL // statements, particularly to handle cases when they contain special characters or reserved keywords. String tableName = qualifiedTableName.replaceAll("^\\[(.*)\\]$", "$1"); - tableNames.add(tableName); + String[] split = tableName.split(", "); + if (split.length == 1) { + tableNames.add(tableName); + } else { + tableNames.add(PinotCatalog.constructPhysicalTableName(split[0], split[1])); + } } return tableNames; } From 9d91b3266535eec9b7bafb1fc6ad802b3c5e42f9 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 7 Mar 2024 18:56:14 +0530 Subject: [PATCH 02/26] lint fix --- .../integration/tests/MultiStageEngineIntegrationTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index 90052f720f3c..335caa27de51 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -819,8 +819,8 @@ private void checkQueryResultForDBTest(String column, String tableName) private void checkQueryResultForDBTest(String column, String tableName, @Nullable String database) throws Exception { - String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") + - "select max(" + column + ") from " + tableName + ";"; + String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") + + "select max(" + column + ") from " + tableName + ";"; // max value of 'ActualElapsedTime' long expectedValue = 678; JsonNode jsonNode = postQuery(query); From 1c0b2408710b89c503351c20cf90aba94ed2f274 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 12:40:50 +0530 Subject: [PATCH 03/26] Move isPartOfDatabase utility to DatabaseUtils --- .../pinot/common/utils/DatabaseUtils.java | 9 +++++++ .../helix/core/PinotHelixResourceManager.java | 27 ++++++++----------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java index f6f9a0ef6807..4fba0209ec7e 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java @@ -84,4 +84,13 @@ public static String translateTableName(String tableName, HttpHeaders headers, b public static String translateTableName(String tableName, HttpHeaders headers) { return translateTableName(tableName, headers, false); } + + public static boolean isPartOfDatabase(String tableName, @Nullable String databaseName) { + // assumes tableName will not have default database prefix ('default.') + if (StringUtils.isEmpty(databaseName) || databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { + return StringUtils.split(tableName, '.').length == 1; + } else { + return tableName.startsWith(databaseName + "."); + } + } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index a54648676caf..5a9155c8f4dd 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -732,22 +732,14 @@ public List getAllTables() { public List getAllTables(@Nullable String databaseName) { List tableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) && + DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { tableNames.add(resourceName); } } return tableNames; } - private boolean isPartOfDatabase(String tableName, @Nullable String databaseName) { - // assumes tableName will not have default database prefix ('default.') - if (StringUtils.isEmpty(databaseName) || databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { - return StringUtils.split(tableName, '.').length == 1; - } else { - return tableName.startsWith(databaseName + "."); - } - } - /** * Get all offline table names from default database. * @@ -766,7 +758,8 @@ public List getAllOfflineTables() { public List getAllOfflineTables(@Nullable String databaseName) { List offlineTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (isPartOfDatabase(resourceName, databaseName) && TableNameBuilder.isOfflineTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && + TableNameBuilder.isOfflineTableResource(resourceName)) { offlineTableNames.add(resourceName); } } @@ -790,7 +783,7 @@ public List getAllDimensionTables() { */ public List getAllDimensionTables(@Nullable String databaseName) { return _tableCache.getAllDimensionTables().stream() - .filter(table -> isPartOfDatabase(table, databaseName)) + .filter(table -> DatabaseUtils.isPartOfDatabase(table, databaseName)) .collect(Collectors.toList()); } @@ -812,7 +805,8 @@ public List getAllRealtimeTables() { public List getAllRealtimeTables(@Nullable String databaseName) { List realtimeTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (isPartOfDatabase(resourceName, databaseName) && TableNameBuilder.isRealtimeTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && + TableNameBuilder.isRealtimeTableResource(resourceName)) { realtimeTableNames.add(resourceName); } } @@ -837,7 +831,8 @@ public List getAllRawTables() { public List getAllRawTables(@Nullable String databaseName) { Set rawTableNames = new HashSet<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) && + DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { rawTableNames.add(TableNameBuilder.extractRawTableName(resourceName)); } } @@ -1616,7 +1611,7 @@ public List getSchemaNames(@Nullable String databaseName) { List schemas = _propertyStore.getChildNames( PinotHelixPropertyStoreZnRecordProvider.forSchema(_propertyStore).getRelativePath(), AccessOption.PERSISTENT); if (schemas != null) { - return schemas.stream().filter(schemaName -> isPartOfDatabase(schemaName, databaseName)) + return schemas.stream().filter(schemaName -> DatabaseUtils.isPartOfDatabase(schemaName, databaseName)) .collect(Collectors.toList()); } return Collections.emptyList(); @@ -4011,7 +4006,7 @@ public Map> getTableToLiveBrokersMapping(@Nullable St ZNRecord znRecord = ev.getRecord(); for (Map.Entry> tableToBrokersEntry : znRecord.getMapFields().entrySet()) { String tableName = tableToBrokersEntry.getKey(); - if (!isPartOfDatabase(tableName, databaseName)) { + if (!DatabaseUtils.isPartOfDatabase(tableName, databaseName)) { continue; } Map brokersToState = tableToBrokersEntry.getValue(); From 98195f4649730b2e4a2a2bb105017d98c7e494aa Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 12:40:59 +0530 Subject: [PATCH 04/26] Reuse utils and constants --- .../apache/pinot/query/QueryEnvironment.java | 4 +-- .../pinot/query/catalog/PinotCatalog.java | 26 ++++++------------- .../logical/RelToPlanNodeConverter.java | 7 ++--- 3 files changed, 14 insertions(+), 23 deletions(-) diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index ac41cbac0afa..9da6b3294702 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -74,6 +74,7 @@ import org.apache.pinot.query.routing.WorkerManager; import org.apache.pinot.query.type.TypeFactory; import org.apache.pinot.query.validate.BytesCastVisitor; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.sql.parsers.CalciteSqlParser; import org.apache.pinot.sql.parsers.SqlNodeAndOptions; import org.apache.pinot.sql.parsers.parser.SqlPhysicalExplain; @@ -85,7 +86,6 @@ *

It provide the higher level entry interface to convert a SQL string into a {@link DispatchableSubPlan}. */ public class QueryEnvironment { - private static final String DATABASE_KEY = "database"; // Calcite configurations private final FrameworkConfig _config; @@ -361,7 +361,7 @@ private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { } private PlannerContext getPlannerContext(Map options) { - String database = options.getOrDefault(DATABASE_KEY, "default"); + String database = options.getOrDefault(CommonConstants.DATABASE, CommonConstants.DEFAULT_DATABASE); if (database.equalsIgnoreCase("default")) { return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); } else { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index 64296fc35dca..87bdc65f84e8 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -37,6 +37,8 @@ import org.apache.calcite.schema.Table; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.config.provider.TableCache; +import org.apache.pinot.common.utils.DatabaseUtils; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import static java.util.Objects.requireNonNull; @@ -50,8 +52,6 @@ */ public class PinotCatalog implements Schema { - public static final String DEFAULT_DB_NAME = "default"; - private final TableCache _tableCache; private final Map _subCatalog; @@ -87,8 +87,8 @@ private Map constructSubCatalogs(TableCache tableCache) { CalciteSchemaBuilder.asSubSchema(new PinotCatalog(databaseName, tableCache), databaseName)); } } - subCatalog.put(DEFAULT_DB_NAME, - CalciteSchemaBuilder.asSubSchema(new PinotCatalog(DEFAULT_DB_NAME, tableCache), DEFAULT_DB_NAME)); + subCatalog.put(CommonConstants.DEFAULT_DATABASE, CalciteSchemaBuilder.asSubSchema( + new PinotCatalog(CommonConstants.DEFAULT_DATABASE, tableCache), CommonConstants.DEFAULT_DATABASE)); return subCatalog; } @@ -100,12 +100,7 @@ private Map constructSubCatalogs(TableCache tableCache) { @Override public Table getTable(String name) { String rawTableName = TableNameBuilder.extractRawTableName(name); - String tableName; - if (_databaseName != null) { - tableName = constructPhysicalTableName(_databaseName, rawTableName); - } else { - tableName = rawTableName; - } + String tableName = DatabaseUtils.translateTableName(rawTableName, _databaseName); org.apache.pinot.spi.data.Schema schema = _tableCache.getSchema(tableName); if (schema == null) { throw new IllegalArgumentException(String.format("Could not find schema for table: '%s'", tableName)); @@ -113,10 +108,6 @@ public Table getTable(String name) { return new PinotTable(schema); } - public static String constructPhysicalTableName(String databaseName, String tableName) { - return databaseName.equals(DEFAULT_DB_NAME) ? tableName : databaseName + "." + tableName; - } - /** * acquire a set of available table names. * @return the set of table names at the time of query planning. @@ -124,10 +115,9 @@ public static String constructPhysicalTableName(String databaseName, String tabl @Override public Set getTableNames() { if (_databaseName != null) { - return _databaseName.equals(DEFAULT_DB_NAME) ? _tableCache.getTableNameMap().keySet().stream() - .filter(n -> StringUtils.split(n, '.').length == 1).collect(Collectors.toSet()) - : _tableCache.getTableNameMap().keySet().stream().filter(n -> n.startsWith(_databaseName)) - .collect(Collectors.toSet()); + return _tableCache.getTableNameMap().keySet().stream() + .filter(n -> DatabaseUtils.isPartOfDatabase(n, _databaseName)) + .collect(Collectors.toSet()); } else { return Collections.emptySet(); } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 6384e806d0e3..5f7fff08c38c 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -49,6 +49,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.query.catalog.PinotCatalog; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; @@ -178,8 +179,8 @@ private static PlanNode convertLogicalTableScan(LogicalTableScan node, int curre if (node.getTable().getQualifiedName().size() == 1) { tableName = node.getTable().getQualifiedName().get(0); } else { - tableName = PinotCatalog.constructPhysicalTableName( - node.getTable().getQualifiedName().get(0), node.getTable().getQualifiedName().get(1)); + tableName = DatabaseUtils.translateTableName(node.getTable().getQualifiedName().get(1), + node.getTable().getQualifiedName().get(0)); } List columnNames = node.getRowType().getFieldList().stream().map(RelDataTypeField::getName).collect(Collectors.toList()); @@ -298,7 +299,7 @@ public static Set getTableNamesFromRelRoot(RelNode relRoot) { if (split.length == 1) { tableNames.add(tableName); } else { - tableNames.add(PinotCatalog.constructPhysicalTableName(split[0], split[1])); + tableNames.add(DatabaseUtils.translateTableName(split[1], split[0])); } } return tableNames; From 246a9b3c5c2cf27a50b96b153123bb1f7f401ec9 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 13:17:52 +0530 Subject: [PATCH 05/26] lint fix --- .../helix/core/PinotHelixResourceManager.java | 16 ++++++++-------- .../org/apache/pinot/query/QueryEnvironment.java | 2 +- .../planner/logical/RelToPlanNodeConverter.java | 1 - 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index 5a9155c8f4dd..1874ecea9f7a 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -732,8 +732,8 @@ public List getAllTables() { public List getAllTables(@Nullable String databaseName) { List tableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && - DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) + && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { tableNames.add(resourceName); } } @@ -758,8 +758,8 @@ public List getAllOfflineTables() { public List getAllOfflineTables(@Nullable String databaseName) { List offlineTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && - TableNameBuilder.isOfflineTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) + && TableNameBuilder.isOfflineTableResource(resourceName)) { offlineTableNames.add(resourceName); } } @@ -805,8 +805,8 @@ public List getAllRealtimeTables() { public List getAllRealtimeTables(@Nullable String databaseName) { List realtimeTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && - TableNameBuilder.isRealtimeTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) + && TableNameBuilder.isRealtimeTableResource(resourceName)) { realtimeTableNames.add(resourceName); } } @@ -831,8 +831,8 @@ public List getAllRawTables() { public List getAllRawTables(@Nullable String databaseName) { Set rawTableNames = new HashSet<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && - DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) + && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { rawTableNames.add(TableNameBuilder.extractRawTableName(resourceName)); } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 9da6b3294702..2f3873f016b3 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -362,7 +362,7 @@ private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { private PlannerContext getPlannerContext(Map options) { String database = options.getOrDefault(CommonConstants.DATABASE, CommonConstants.DEFAULT_DATABASE); - if (database.equalsIgnoreCase("default")) { + if (database.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); } else { Prepare.CatalogReader catalogReader = getCatalog(database); diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 5f7fff08c38c..450c2ccc4549 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -50,7 +50,6 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.DatabaseUtils; -import org.apache.pinot.query.catalog.PinotCatalog; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; import org.apache.pinot.query.planner.plannode.FilterNode; From 8c4649fd8b058ea7367064b5006d471fe021fd19 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 15:09:31 +0530 Subject: [PATCH 06/26] Fix test case --- .../integration/tests/OfflineClusterIntegrationTest.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java index c60b26bde808..4f55279490d4 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java @@ -2772,11 +2772,7 @@ public void testQuerySourceWithDatabaseNameV2() testQuery(pinotQuery, h2Query); pinotQuery = "SELECT DISTINCT Carrier FROM default.mytable LIMIT 1000000"; - JsonNode response = postQuery(pinotQuery); - JsonNode exceptions = response.get("exceptions"); - assertFalse(exceptions.isEmpty(), "At least one exception was expected"); - JsonNode firstException = exceptions.get(0); - assertEquals(firstException.get("errorCode").asInt(), QueryException.QUERY_PLANNING_ERROR_CODE); + testQuery(pinotQuery, h2Query); } @Test(dataProvider = "useBothQueryEngines") From 84fcbdead055b808bae3a4c067297de917a3a336 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Mon, 18 Mar 2024 15:21:06 +0530 Subject: [PATCH 07/26] review fixes --- .../pinot/common/utils/DatabaseUtils.java | 17 ++++++++++++++++- .../apache/pinot/query/QueryEnvironment.java | 14 +++++++++----- .../pinot/query/catalog/PinotCatalog.java | 1 + 3 files changed, 26 insertions(+), 6 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java index 4fba0209ec7e..d00ed916628d 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java @@ -85,10 +85,25 @@ public static String translateTableName(String tableName, HttpHeaders headers) { return translateTableName(tableName, headers, false); } + /** + * Checks if the fully qualified {@code tableName} belongs to the provided {@code databaseName} + * @param tableName fully qualified table name + * @param databaseName database name + * @return true if + *

    + *
  • + * tableName is prefixed with "databaseName." or + *
  • + *
  • + * databaseName is null or "default" and tableName does not have a '.' + *
  • + *
+ * else false + */ public static boolean isPartOfDatabase(String tableName, @Nullable String databaseName) { // assumes tableName will not have default database prefix ('default.') if (StringUtils.isEmpty(databaseName) || databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { - return StringUtils.split(tableName, '.').length == 1; + return !tableName.contains("."); } else { return tableName.startsWith(databaseName + "."); } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 2f3873f016b3..5ea93819b86d 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -334,13 +334,17 @@ private DispatchableSubPlan toDispatchableSubPlan(RelRoot relRoot, PlannerContex private Prepare.CatalogReader getCatalog(@Nullable String schemaPath) { Properties catalogReaderConfigProperties = new Properties(); catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); - CalciteSchema subSchema = schemaPath == null ? _rootSchema : _rootSchema.getSubSchema(schemaPath, false); - if (subSchema != null) { - return new PinotCalciteCatalogReader(subSchema, subSchema.path(null), _typeFactory, - new CalciteConnectionConfigImpl(catalogReaderConfigProperties)); + CalciteSchema schema; + if (schemaPath == null) { + schema = _rootSchema; } else { - throw new IllegalArgumentException("Cannot find schema with path: " + schemaPath); + schema = _rootSchema.getSubSchema(schemaPath, false); + if (schema == null) { + throw new IllegalArgumentException("Cannot find schema with path: " + schemaPath); + } } + CalciteConnectionConfigImpl connConfig = new CalciteConnectionConfigImpl(catalogReaderConfigProperties); + return new PinotCalciteCatalogReader(schema, schema.path(null), _typeFactory, connConfig); } private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index 87bdc65f84e8..9836705b1a41 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -119,6 +119,7 @@ public Set getTableNames() { .filter(n -> DatabaseUtils.isPartOfDatabase(n, _databaseName)) .collect(Collectors.toSet()); } else { + // root schema will not contain tables return Collections.emptySet(); } } From 3b3b87ecd79c896154ca42508d4fbac68668584e Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 7 Mar 2024 18:47:08 +0530 Subject: [PATCH 08/26] [multistage] support database --- .../MultiStageBrokerRequestHandler.java | 12 +-- .../MultiStageEngineIntegrationTest.java | 87 ++++++++++++++++++- .../calcite/jdbc/CalciteSchemaBuilder.java | 11 +++ .../apache/pinot/query/QueryEnvironment.java | 79 +++++++++++------ .../pinot/query/catalog/PinotCatalog.java | 67 +++++++++++++- .../logical/RelToPlanNodeConverter.java | 16 +++- 6 files changed, 230 insertions(+), 42 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 35aff7efd245..32e1492e73de 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -81,7 +81,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler { private static final Logger LOGGER = LoggerFactory.getLogger(MultiStageBrokerRequestHandler.class); - private final QueryEnvironment _queryEnvironment; + private final WorkerManager _workerManager; private final MailboxService _mailboxService; private final QueryDispatcher _queryDispatcher; @@ -93,9 +93,7 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId LOGGER.info("Using Multi-stage BrokerRequestHandler."); String hostname = config.getProperty(CommonConstants.MultiStageQueryRunner.KEY_OF_QUERY_RUNNER_HOSTNAME); int port = Integer.parseInt(config.getProperty(CommonConstants.MultiStageQueryRunner.KEY_OF_QUERY_RUNNER_PORT)); - _queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()), - CalciteSchemaBuilder.asRootSchema(new PinotCatalog(tableCache)), - new WorkerManager(hostname, port, routingManager), _tableCache); + _workerManager = new WorkerManager(hostname, port, _routingManager); _mailboxService = new MailboxService(hostname, port, config); _queryDispatcher = new QueryDispatcher(_mailboxService); @@ -128,9 +126,11 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S queryTimeoutMs = timeoutMsFromQueryOption == null ? _brokerTimeoutMs : timeoutMsFromQueryOption; // Compile the request compilationStartTimeNs = System.nanoTime(); + QueryEnvironment queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()), + CalciteSchemaBuilder.asRootSchema(new PinotCatalog(_tableCache)), _workerManager, _tableCache); switch (sqlNodeAndOptions.getSqlNode().getKind()) { case EXPLAIN: - queryPlanResult = _queryEnvironment.explainQuery(query, sqlNodeAndOptions, requestId); + queryPlanResult = queryEnvironment.explainQuery(query, sqlNodeAndOptions, requestId); String plan = queryPlanResult.getExplainPlan(); Set tableNames = queryPlanResult.getTableNames(); if (!hasTableAccess(requesterIdentity, tableNames, requestContext, httpHeaders)) { @@ -140,7 +140,7 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S return constructMultistageExplainPlan(query, plan); case SELECT: default: - queryPlanResult = _queryEnvironment.planQuery(query, sqlNodeAndOptions, requestId); + queryPlanResult = queryEnvironment.planQuery(query, sqlNodeAndOptions, requestId); break; } } catch (WebApplicationException e) { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index c18dd026ffac..90052f720f3c 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -25,12 +25,19 @@ import java.time.Instant; import java.time.ZoneId; import java.time.format.DateTimeFormatter; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import javax.annotation.Nullable; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.TransformConfig; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.MetricFieldSpec; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.util.TestUtils; import org.testng.Assert; @@ -47,6 +54,8 @@ public class MultiStageEngineIntegrationTest extends BaseClusterIntegrationTestSet { private static final String SCHEMA_FILE_NAME = "On_Time_On_Time_Performance_2014_100k_subset_nonulls.schema"; + private String _tableName = DEFAULT_TABLE_NAME; + private List _avroFiles = new ArrayList<>(); @Override protected String getSchemaFileName() { @@ -72,17 +81,17 @@ public void setUp() addTableConfig(tableConfig); // Unpack the Avro files - List avroFiles = unpackAvroData(_tempDir); + _avroFiles = unpackAvroData(_tempDir); // Create and upload segments - ClusterIntegrationTestUtils.buildSegmentsFromAvro(avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); uploadSegments(getTableName(), _tarDir); // Set up the H2 connection - setUpH2Connection(avroFiles); + setUpH2Connection(_avroFiles); // Initialize the query generator - setUpQueryGenerator(avroFiles); + setUpQueryGenerator(_avroFiles); // Wait for all documents loaded waitForAllDocsLoaded(600_000L); @@ -749,6 +758,76 @@ public void testSearch() assertNoError(jsonNode); } + @Override + protected String getTableName() { + return _tableName; + } + + @Test + public void testWithDatabaseContext() + throws Exception { + try { + _tableName = "db1." + DEFAULT_TABLE_NAME; + String defaultCol = "ActualElapsedTime"; + String customCol = "ActualElapsedTime_2"; + Schema schema = createSchema(); + schema.addField(new MetricFieldSpec(customCol, FieldSpec.DataType.INT)); + addSchema(schema); + TableConfig tableConfig = createOfflineTableConfig(); + assert tableConfig.getIndexingConfig().getNoDictionaryColumns() != null; + List noDicCols = new ArrayList<>(DEFAULT_NO_DICTIONARY_COLUMNS); + noDicCols.add(customCol); + tableConfig.getIndexingConfig().setNoDictionaryColumns(noDicCols); + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setTransformConfigs(List.of(new TransformConfig(customCol, defaultCol))); + tableConfig.setIngestionConfig(ingestionConfig); + addTableConfig(tableConfig); + + // Create and upload segments to 'db1.mytable' + TestUtils.ensureDirectoriesExistAndEmpty(_segmentDir, _tarDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + uploadSegments(getTableName(), _tarDir); + + // Wait for all documents loaded + waitForAllDocsLoaded(600_000L); + + // default database check. No database context passed + checkQueryResultForDBTest(defaultCol, DEFAULT_TABLE_NAME); + + // default database check. Default database context passed as table prefix + checkQueryResultForDBTest(defaultCol, "default." + DEFAULT_TABLE_NAME); + + // default database check. Default database context passed as SET database='dbName' + checkQueryResultForDBTest(defaultCol, DEFAULT_TABLE_NAME, "default"); + + // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. + // custom database check. Database context passed as table prefix + checkQueryResultForDBTest(customCol, _tableName); + + // custom database check. Database context passed as SET database='dbName' + checkQueryResultForDBTest(customCol, DEFAULT_TABLE_NAME, "db1"); + } finally { + dropOfflineTable(_tableName); + _tableName = DEFAULT_TABLE_NAME; + } + } + + private void checkQueryResultForDBTest(String column, String tableName) + throws Exception { + checkQueryResultForDBTest(column, tableName, null); + } + + private void checkQueryResultForDBTest(String column, String tableName, @Nullable String database) + throws Exception { + String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") + + "select max(" + column + ") from " + tableName + ";"; + // max value of 'ActualElapsedTime' + long expectedValue = 678; + JsonNode jsonNode = postQuery(query); + long result = jsonNode.get("resultTable").get("rows").get(0).get(0).asLong(); + assertEquals(result, expectedValue); + } + @AfterClass public void tearDown() throws Exception { diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java index edb2d74bf07c..b7f64945f9ca 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java +++ b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java @@ -61,4 +61,15 @@ public static CalciteSchema asRootSchema(Schema root) { } return rootSchema; } + + public static CalciteSchema asSubSchema(Schema root, String name) { + CalciteSchema subSchema = CalciteSchema.createRootSchema(false, false, name, root); + SchemaPlus schemaPlus = subSchema.plus(); + for (Map.Entry> e : FunctionRegistry.getRegisteredCalciteFunctionMap().entrySet()) { + for (Function f : e.getValue()) { + schemaPlus.add(e.getKey(), f); + } + } + return subSchema; + } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 32a75c4a3d4c..ac41cbac0afa 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.Set; import javax.annotation.Nullable; @@ -84,6 +85,7 @@ *

It provide the higher level entry interface to convert a SQL string into a {@link DispatchableSubPlan}. */ public class QueryEnvironment { + private static final String DATABASE_KEY = "database"; // Calcite configurations private final FrameworkConfig _config; @@ -106,26 +108,10 @@ public QueryEnvironment(TypeFactory typeFactory, CalciteSchema rootSchema, Worke _workerManager = workerManager; _tableCache = tableCache; - // catalog - Properties catalogReaderConfigProperties = new Properties(); - catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); - _catalogReader = new PinotCalciteCatalogReader(_rootSchema, _rootSchema.path(null), _typeFactory, - new CalciteConnectionConfigImpl(catalogReaderConfigProperties)); - - _config = Frameworks.newConfigBuilder().traitDefs() - .operatorTable(new PinotChainedSqlOperatorTable(Arrays.asList( - PinotOperatorTable.instance(), - _catalogReader))) - .defaultSchema(_rootSchema.plus()) - .sqlToRelConverterConfig(SqlToRelConverter.config() - .withHintStrategyTable(getHintStrategyTable()) - .withTrimUnusedFields(true) - // SUB-QUERY Threshold is useless as we are encoding all IN clause in-line anyway - .withInSubQueryThreshold(Integer.MAX_VALUE) - .addRelBuilderConfigTransform(c -> c.withPushJoinCondition(true)) - .addRelBuilderConfigTransform(c -> c.withAggregateUnique(true)) - .addRelBuilderConfigTransform(c -> c.withPruneInputOfAggregate(false))) - .build(); + // catalog & config + _catalogReader = getCatalog(null); + _config = getConfig(_catalogReader); + // opt programs _optProgram = getOptProgram(); _traitProgram = getTraitProgram(); } @@ -143,8 +129,7 @@ public QueryEnvironment(TypeFactory typeFactory, CalciteSchema rootSchema, Worke * @return QueryPlannerResult containing the dispatchable query plan and the relRoot. */ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) { - try (PlannerContext plannerContext = new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, - _traitProgram)) { + try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions.getOptions())) { plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(sqlNodeAndOptions.getSqlNode(), plannerContext); // TODO: current code only assume one SubPlan per query, but we should support multiple SubPlans per query. @@ -172,8 +157,7 @@ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAn * @return QueryPlannerResult containing the explained query plan and the relRoot. */ public QueryPlannerResult explainQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) { - try (PlannerContext plannerContext = new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, - _traitProgram)) { + try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions.getOptions())) { SqlExplain explain = (SqlExplain) sqlNodeAndOptions.getSqlNode(); plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(explain.getExplicandum(), plannerContext); @@ -206,8 +190,11 @@ public String explainQuery(String sqlQuery, long requestId) { } public List getTableNamesForQuery(String sqlQuery) { - try (PlannerContext plannerContext = new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, - _traitProgram)) { + return getTableNamesForQuery(sqlQuery, CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery).getOptions()); + } + + public List getTableNamesForQuery(String sqlQuery, Map options) { + try (PlannerContext plannerContext = getPlannerContext(options)) { SqlNode sqlNode = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery).getSqlNode(); if (sqlNode.getKind().equals(SqlKind.EXPLAIN)) { sqlNode = ((SqlExplain) sqlNode).getExplicandum(); @@ -344,6 +331,46 @@ private DispatchableSubPlan toDispatchableSubPlan(RelRoot relRoot, PlannerContex // utils // -------------------------------------------------------------------------- + private Prepare.CatalogReader getCatalog(@Nullable String schemaPath) { + Properties catalogReaderConfigProperties = new Properties(); + catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); + CalciteSchema subSchema = schemaPath == null ? _rootSchema : _rootSchema.getSubSchema(schemaPath, false); + if (subSchema != null) { + return new PinotCalciteCatalogReader(subSchema, subSchema.path(null), _typeFactory, + new CalciteConnectionConfigImpl(catalogReaderConfigProperties)); + } else { + throw new IllegalArgumentException("Cannot find schema with path: " + schemaPath); + } + } + + private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { + return Frameworks.newConfigBuilder().traitDefs() + .operatorTable(new PinotChainedSqlOperatorTable(Arrays.asList( + PinotOperatorTable.instance(), + catalogReader))) + .defaultSchema(catalogReader.getRootSchema().plus()) + .sqlToRelConverterConfig(SqlToRelConverter.config() + .withHintStrategyTable(getHintStrategyTable()) + .withTrimUnusedFields(true) + // SUB-QUERY Threshold is useless as we are encoding all IN clause in-line anyway + .withInSubQueryThreshold(Integer.MAX_VALUE) + .addRelBuilderConfigTransform(c -> c.withPushJoinCondition(true)) + .addRelBuilderConfigTransform(c -> c.withAggregateUnique(true)) + .addRelBuilderConfigTransform(c -> c.withPruneInputOfAggregate(false))) + .build(); + } + + private PlannerContext getPlannerContext(Map options) { + String database = options.getOrDefault(DATABASE_KEY, "default"); + if (database.equalsIgnoreCase("default")) { + return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); + } else { + Prepare.CatalogReader catalogReader = getCatalog(database); + FrameworkConfig config = getConfig(catalogReader); + return new PlannerContext(config, catalogReader, _typeFactory, _optProgram, _traitProgram); + } + } + private HintStrategyTable getHintStrategyTable() { return PinotHintStrategyTable.PINOT_HINT_STRATEGY_TABLE; } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index b777c9bc0350..64296fc35dca 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -20,8 +20,13 @@ import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import javax.annotation.Nullable; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.schema.Function; @@ -30,6 +35,7 @@ import org.apache.calcite.schema.SchemaVersion; import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -44,14 +50,46 @@ */ public class PinotCatalog implements Schema { + public static final String DEFAULT_DB_NAME = "default"; + private final TableCache _tableCache; + private final Map _subCatalog; + + private final String _databaseName; + /** * PinotCatalog needs have access to the actual {@link TableCache} object because TableCache hosts the actual * table available for query and processes table/segment metadata updates when cluster status changes. */ public PinotCatalog(TableCache tableCache) { _tableCache = tableCache; + _databaseName = null; + // create all databases + // TODO: we need to monitor table cache changes to register newly created databases + // TODO: we also need to monitor table that needs to be put into the right places + _subCatalog = constructSubCatalogs(_tableCache); + } + + private PinotCatalog(String databaseName, TableCache tableCache) { + _tableCache = tableCache; + _databaseName = databaseName; + _subCatalog = null; + } + + private Map constructSubCatalogs(TableCache tableCache) { + Map subCatalog = new HashMap<>(); + for (String physicalTableName : tableCache.getTableNameMap().keySet()) { + String[] nameSplit = StringUtils.split(physicalTableName, '.'); + if (nameSplit.length > 1) { + String databaseName = nameSplit[0]; + subCatalog.putIfAbsent(databaseName, + CalciteSchemaBuilder.asSubSchema(new PinotCatalog(databaseName, tableCache), databaseName)); + } + } + subCatalog.put(DEFAULT_DB_NAME, + CalciteSchemaBuilder.asSubSchema(new PinotCatalog(DEFAULT_DB_NAME, tableCache), DEFAULT_DB_NAME)); + return subCatalog; } /** @@ -61,7 +99,13 @@ public PinotCatalog(TableCache tableCache) { */ @Override public Table getTable(String name) { - String tableName = TableNameBuilder.extractRawTableName(name); + String rawTableName = TableNameBuilder.extractRawTableName(name); + String tableName; + if (_databaseName != null) { + tableName = constructPhysicalTableName(_databaseName, rawTableName); + } else { + tableName = rawTableName; + } org.apache.pinot.spi.data.Schema schema = _tableCache.getSchema(tableName); if (schema == null) { throw new IllegalArgumentException(String.format("Could not find schema for table: '%s'", tableName)); @@ -69,13 +113,24 @@ public Table getTable(String name) { return new PinotTable(schema); } + public static String constructPhysicalTableName(String databaseName, String tableName) { + return databaseName.equals(DEFAULT_DB_NAME) ? tableName : databaseName + "." + tableName; + } + /** * acquire a set of available table names. * @return the set of table names at the time of query planning. */ @Override public Set getTableNames() { - return _tableCache.getTableNameMap().keySet(); + if (_databaseName != null) { + return _databaseName.equals(DEFAULT_DB_NAME) ? _tableCache.getTableNameMap().keySet().stream() + .filter(n -> StringUtils.split(n, '.').length == 1).collect(Collectors.toSet()) + : _tableCache.getTableNameMap().keySet().stream().filter(n -> n.startsWith(_databaseName)) + .collect(Collectors.toSet()); + } else { + return Collections.emptySet(); + } } @Override @@ -108,12 +163,16 @@ public Set getFunctionNames() { @Override public Schema getSubSchema(String name) { - return null; + if (_subCatalog != null && _subCatalog.containsKey(name)) { + return _subCatalog.get(name).schema; + } else { + return null; + } } @Override public Set getSubSchemaNames() { - return Collections.emptySet(); + return _subCatalog.keySet(); } @Override diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 54839e52ffd2..6384e806d0e3 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -49,6 +49,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.query.catalog.PinotCatalog; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; import org.apache.pinot.query.planner.plannode.FilterNode; @@ -173,7 +174,13 @@ private static PlanNode convertLogicalFilter(LogicalFilter node, int currentStag } private static PlanNode convertLogicalTableScan(LogicalTableScan node, int currentStageId) { - String tableName = node.getTable().getQualifiedName().get(0); + String tableName; + if (node.getTable().getQualifiedName().size() == 1) { + tableName = node.getTable().getQualifiedName().get(0); + } else { + tableName = PinotCatalog.constructPhysicalTableName( + node.getTable().getQualifiedName().get(0), node.getTable().getQualifiedName().get(1)); + } List columnNames = node.getRowType().getFieldList().stream().map(RelDataTypeField::getName).collect(Collectors.toList()); return new TableScanNode(currentStageId, toDataSchema(node.getRowType()), node.getHints(), tableName, columnNames); @@ -287,7 +294,12 @@ public static Set getTableNamesFromRelRoot(RelNode relRoot) { // Calcite encloses table and schema names in square brackets to properly quote and delimit them in SQL // statements, particularly to handle cases when they contain special characters or reserved keywords. String tableName = qualifiedTableName.replaceAll("^\\[(.*)\\]$", "$1"); - tableNames.add(tableName); + String[] split = tableName.split(", "); + if (split.length == 1) { + tableNames.add(tableName); + } else { + tableNames.add(PinotCatalog.constructPhysicalTableName(split[0], split[1])); + } } return tableNames; } From ed25b485929fe93bbd717de50050d71f2f1e4da1 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 7 Mar 2024 18:56:14 +0530 Subject: [PATCH 09/26] lint fix --- .../integration/tests/MultiStageEngineIntegrationTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index 90052f720f3c..335caa27de51 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -819,8 +819,8 @@ private void checkQueryResultForDBTest(String column, String tableName) private void checkQueryResultForDBTest(String column, String tableName, @Nullable String database) throws Exception { - String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") + - "select max(" + column + ") from " + tableName + ";"; + String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") + + "select max(" + column + ") from " + tableName + ";"; // max value of 'ActualElapsedTime' long expectedValue = 678; JsonNode jsonNode = postQuery(query); From 62dd3fdaeb991ad15d96165d24a83ebf7f1407fe Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 12:40:50 +0530 Subject: [PATCH 10/26] Move isPartOfDatabase utility to DatabaseUtils --- .../pinot/common/utils/DatabaseUtils.java | 9 +++++++ .../helix/core/PinotHelixResourceManager.java | 27 ++++++++----------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java index f6f9a0ef6807..4fba0209ec7e 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java @@ -84,4 +84,13 @@ public static String translateTableName(String tableName, HttpHeaders headers, b public static String translateTableName(String tableName, HttpHeaders headers) { return translateTableName(tableName, headers, false); } + + public static boolean isPartOfDatabase(String tableName, @Nullable String databaseName) { + // assumes tableName will not have default database prefix ('default.') + if (StringUtils.isEmpty(databaseName) || databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { + return StringUtils.split(tableName, '.').length == 1; + } else { + return tableName.startsWith(databaseName + "."); + } + } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index a54648676caf..5a9155c8f4dd 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -732,22 +732,14 @@ public List getAllTables() { public List getAllTables(@Nullable String databaseName) { List tableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) && + DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { tableNames.add(resourceName); } } return tableNames; } - private boolean isPartOfDatabase(String tableName, @Nullable String databaseName) { - // assumes tableName will not have default database prefix ('default.') - if (StringUtils.isEmpty(databaseName) || databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { - return StringUtils.split(tableName, '.').length == 1; - } else { - return tableName.startsWith(databaseName + "."); - } - } - /** * Get all offline table names from default database. * @@ -766,7 +758,8 @@ public List getAllOfflineTables() { public List getAllOfflineTables(@Nullable String databaseName) { List offlineTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (isPartOfDatabase(resourceName, databaseName) && TableNameBuilder.isOfflineTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && + TableNameBuilder.isOfflineTableResource(resourceName)) { offlineTableNames.add(resourceName); } } @@ -790,7 +783,7 @@ public List getAllDimensionTables() { */ public List getAllDimensionTables(@Nullable String databaseName) { return _tableCache.getAllDimensionTables().stream() - .filter(table -> isPartOfDatabase(table, databaseName)) + .filter(table -> DatabaseUtils.isPartOfDatabase(table, databaseName)) .collect(Collectors.toList()); } @@ -812,7 +805,8 @@ public List getAllRealtimeTables() { public List getAllRealtimeTables(@Nullable String databaseName) { List realtimeTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (isPartOfDatabase(resourceName, databaseName) && TableNameBuilder.isRealtimeTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && + TableNameBuilder.isRealtimeTableResource(resourceName)) { realtimeTableNames.add(resourceName); } } @@ -837,7 +831,8 @@ public List getAllRawTables() { public List getAllRawTables(@Nullable String databaseName) { Set rawTableNames = new HashSet<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) && + DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { rawTableNames.add(TableNameBuilder.extractRawTableName(resourceName)); } } @@ -1616,7 +1611,7 @@ public List getSchemaNames(@Nullable String databaseName) { List schemas = _propertyStore.getChildNames( PinotHelixPropertyStoreZnRecordProvider.forSchema(_propertyStore).getRelativePath(), AccessOption.PERSISTENT); if (schemas != null) { - return schemas.stream().filter(schemaName -> isPartOfDatabase(schemaName, databaseName)) + return schemas.stream().filter(schemaName -> DatabaseUtils.isPartOfDatabase(schemaName, databaseName)) .collect(Collectors.toList()); } return Collections.emptyList(); @@ -4011,7 +4006,7 @@ public Map> getTableToLiveBrokersMapping(@Nullable St ZNRecord znRecord = ev.getRecord(); for (Map.Entry> tableToBrokersEntry : znRecord.getMapFields().entrySet()) { String tableName = tableToBrokersEntry.getKey(); - if (!isPartOfDatabase(tableName, databaseName)) { + if (!DatabaseUtils.isPartOfDatabase(tableName, databaseName)) { continue; } Map brokersToState = tableToBrokersEntry.getValue(); From 86007f9c3afed3b9f558d23a09b4d911e65031bf Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 12:40:59 +0530 Subject: [PATCH 11/26] Reuse utils and constants --- .../apache/pinot/query/QueryEnvironment.java | 4 +-- .../pinot/query/catalog/PinotCatalog.java | 26 ++++++------------- .../logical/RelToPlanNodeConverter.java | 7 ++--- 3 files changed, 14 insertions(+), 23 deletions(-) diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index ac41cbac0afa..9da6b3294702 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -74,6 +74,7 @@ import org.apache.pinot.query.routing.WorkerManager; import org.apache.pinot.query.type.TypeFactory; import org.apache.pinot.query.validate.BytesCastVisitor; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.sql.parsers.CalciteSqlParser; import org.apache.pinot.sql.parsers.SqlNodeAndOptions; import org.apache.pinot.sql.parsers.parser.SqlPhysicalExplain; @@ -85,7 +86,6 @@ *

It provide the higher level entry interface to convert a SQL string into a {@link DispatchableSubPlan}. */ public class QueryEnvironment { - private static final String DATABASE_KEY = "database"; // Calcite configurations private final FrameworkConfig _config; @@ -361,7 +361,7 @@ private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { } private PlannerContext getPlannerContext(Map options) { - String database = options.getOrDefault(DATABASE_KEY, "default"); + String database = options.getOrDefault(CommonConstants.DATABASE, CommonConstants.DEFAULT_DATABASE); if (database.equalsIgnoreCase("default")) { return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); } else { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index 64296fc35dca..87bdc65f84e8 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -37,6 +37,8 @@ import org.apache.calcite.schema.Table; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.config.provider.TableCache; +import org.apache.pinot.common.utils.DatabaseUtils; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import static java.util.Objects.requireNonNull; @@ -50,8 +52,6 @@ */ public class PinotCatalog implements Schema { - public static final String DEFAULT_DB_NAME = "default"; - private final TableCache _tableCache; private final Map _subCatalog; @@ -87,8 +87,8 @@ private Map constructSubCatalogs(TableCache tableCache) { CalciteSchemaBuilder.asSubSchema(new PinotCatalog(databaseName, tableCache), databaseName)); } } - subCatalog.put(DEFAULT_DB_NAME, - CalciteSchemaBuilder.asSubSchema(new PinotCatalog(DEFAULT_DB_NAME, tableCache), DEFAULT_DB_NAME)); + subCatalog.put(CommonConstants.DEFAULT_DATABASE, CalciteSchemaBuilder.asSubSchema( + new PinotCatalog(CommonConstants.DEFAULT_DATABASE, tableCache), CommonConstants.DEFAULT_DATABASE)); return subCatalog; } @@ -100,12 +100,7 @@ private Map constructSubCatalogs(TableCache tableCache) { @Override public Table getTable(String name) { String rawTableName = TableNameBuilder.extractRawTableName(name); - String tableName; - if (_databaseName != null) { - tableName = constructPhysicalTableName(_databaseName, rawTableName); - } else { - tableName = rawTableName; - } + String tableName = DatabaseUtils.translateTableName(rawTableName, _databaseName); org.apache.pinot.spi.data.Schema schema = _tableCache.getSchema(tableName); if (schema == null) { throw new IllegalArgumentException(String.format("Could not find schema for table: '%s'", tableName)); @@ -113,10 +108,6 @@ public Table getTable(String name) { return new PinotTable(schema); } - public static String constructPhysicalTableName(String databaseName, String tableName) { - return databaseName.equals(DEFAULT_DB_NAME) ? tableName : databaseName + "." + tableName; - } - /** * acquire a set of available table names. * @return the set of table names at the time of query planning. @@ -124,10 +115,9 @@ public static String constructPhysicalTableName(String databaseName, String tabl @Override public Set getTableNames() { if (_databaseName != null) { - return _databaseName.equals(DEFAULT_DB_NAME) ? _tableCache.getTableNameMap().keySet().stream() - .filter(n -> StringUtils.split(n, '.').length == 1).collect(Collectors.toSet()) - : _tableCache.getTableNameMap().keySet().stream().filter(n -> n.startsWith(_databaseName)) - .collect(Collectors.toSet()); + return _tableCache.getTableNameMap().keySet().stream() + .filter(n -> DatabaseUtils.isPartOfDatabase(n, _databaseName)) + .collect(Collectors.toSet()); } else { return Collections.emptySet(); } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 6384e806d0e3..5f7fff08c38c 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -49,6 +49,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.query.catalog.PinotCatalog; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; @@ -178,8 +179,8 @@ private static PlanNode convertLogicalTableScan(LogicalTableScan node, int curre if (node.getTable().getQualifiedName().size() == 1) { tableName = node.getTable().getQualifiedName().get(0); } else { - tableName = PinotCatalog.constructPhysicalTableName( - node.getTable().getQualifiedName().get(0), node.getTable().getQualifiedName().get(1)); + tableName = DatabaseUtils.translateTableName(node.getTable().getQualifiedName().get(1), + node.getTable().getQualifiedName().get(0)); } List columnNames = node.getRowType().getFieldList().stream().map(RelDataTypeField::getName).collect(Collectors.toList()); @@ -298,7 +299,7 @@ public static Set getTableNamesFromRelRoot(RelNode relRoot) { if (split.length == 1) { tableNames.add(tableName); } else { - tableNames.add(PinotCatalog.constructPhysicalTableName(split[0], split[1])); + tableNames.add(DatabaseUtils.translateTableName(split[1], split[0])); } } return tableNames; From fc6bf1d546d7d7d827c1dd07eab13b4682371092 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 13:17:52 +0530 Subject: [PATCH 12/26] lint fix --- .../helix/core/PinotHelixResourceManager.java | 16 ++++++++-------- .../org/apache/pinot/query/QueryEnvironment.java | 2 +- .../planner/logical/RelToPlanNodeConverter.java | 1 - 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index 5a9155c8f4dd..1874ecea9f7a 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -732,8 +732,8 @@ public List getAllTables() { public List getAllTables(@Nullable String databaseName) { List tableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && - DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) + && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { tableNames.add(resourceName); } } @@ -758,8 +758,8 @@ public List getAllOfflineTables() { public List getAllOfflineTables(@Nullable String databaseName) { List offlineTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && - TableNameBuilder.isOfflineTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) + && TableNameBuilder.isOfflineTableResource(resourceName)) { offlineTableNames.add(resourceName); } } @@ -805,8 +805,8 @@ public List getAllRealtimeTables() { public List getAllRealtimeTables(@Nullable String databaseName) { List realtimeTableNames = new ArrayList<>(); for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) && - TableNameBuilder.isRealtimeTableResource(resourceName)) { + if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) + && TableNameBuilder.isRealtimeTableResource(resourceName)) { realtimeTableNames.add(resourceName); } } @@ -831,8 +831,8 @@ public List getAllRawTables() { public List getAllRawTables(@Nullable String databaseName) { Set rawTableNames = new HashSet<>(); for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) && - DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { + if (TableNameBuilder.isTableResource(resourceName) + && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { rawTableNames.add(TableNameBuilder.extractRawTableName(resourceName)); } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 9da6b3294702..2f3873f016b3 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -362,7 +362,7 @@ private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { private PlannerContext getPlannerContext(Map options) { String database = options.getOrDefault(CommonConstants.DATABASE, CommonConstants.DEFAULT_DATABASE); - if (database.equalsIgnoreCase("default")) { + if (database.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); } else { Prepare.CatalogReader catalogReader = getCatalog(database); diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 5f7fff08c38c..450c2ccc4549 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -50,7 +50,6 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.DatabaseUtils; -import org.apache.pinot.query.catalog.PinotCatalog; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; import org.apache.pinot.query.planner.plannode.FilterNode; From 601b896dc983d6fd64d588c29e96feb81f9f0a12 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Fri, 15 Mar 2024 15:09:31 +0530 Subject: [PATCH 13/26] Fix test case --- .../integration/tests/OfflineClusterIntegrationTest.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java index c60b26bde808..4f55279490d4 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java @@ -2772,11 +2772,7 @@ public void testQuerySourceWithDatabaseNameV2() testQuery(pinotQuery, h2Query); pinotQuery = "SELECT DISTINCT Carrier FROM default.mytable LIMIT 1000000"; - JsonNode response = postQuery(pinotQuery); - JsonNode exceptions = response.get("exceptions"); - assertFalse(exceptions.isEmpty(), "At least one exception was expected"); - JsonNode firstException = exceptions.get(0); - assertEquals(firstException.get("errorCode").asInt(), QueryException.QUERY_PLANNING_ERROR_CODE); + testQuery(pinotQuery, h2Query); } @Test(dataProvider = "useBothQueryEngines") From 96cee4843e204efc76fe18d6b4ab3175621d64b0 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Mon, 18 Mar 2024 15:21:06 +0530 Subject: [PATCH 14/26] review fixes --- .../pinot/common/utils/DatabaseUtils.java | 17 ++++++++++++++++- .../apache/pinot/query/QueryEnvironment.java | 14 +++++++++----- .../pinot/query/catalog/PinotCatalog.java | 1 + 3 files changed, 26 insertions(+), 6 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java index 4fba0209ec7e..d00ed916628d 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java @@ -85,10 +85,25 @@ public static String translateTableName(String tableName, HttpHeaders headers) { return translateTableName(tableName, headers, false); } + /** + * Checks if the fully qualified {@code tableName} belongs to the provided {@code databaseName} + * @param tableName fully qualified table name + * @param databaseName database name + * @return true if + *

    + *
  • + * tableName is prefixed with "databaseName." or + *
  • + *
  • + * databaseName is null or "default" and tableName does not have a '.' + *
  • + *
+ * else false + */ public static boolean isPartOfDatabase(String tableName, @Nullable String databaseName) { // assumes tableName will not have default database prefix ('default.') if (StringUtils.isEmpty(databaseName) || databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { - return StringUtils.split(tableName, '.').length == 1; + return !tableName.contains("."); } else { return tableName.startsWith(databaseName + "."); } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 2f3873f016b3..5ea93819b86d 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -334,13 +334,17 @@ private DispatchableSubPlan toDispatchableSubPlan(RelRoot relRoot, PlannerContex private Prepare.CatalogReader getCatalog(@Nullable String schemaPath) { Properties catalogReaderConfigProperties = new Properties(); catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); - CalciteSchema subSchema = schemaPath == null ? _rootSchema : _rootSchema.getSubSchema(schemaPath, false); - if (subSchema != null) { - return new PinotCalciteCatalogReader(subSchema, subSchema.path(null), _typeFactory, - new CalciteConnectionConfigImpl(catalogReaderConfigProperties)); + CalciteSchema schema; + if (schemaPath == null) { + schema = _rootSchema; } else { - throw new IllegalArgumentException("Cannot find schema with path: " + schemaPath); + schema = _rootSchema.getSubSchema(schemaPath, false); + if (schema == null) { + throw new IllegalArgumentException("Cannot find schema with path: " + schemaPath); + } } + CalciteConnectionConfigImpl connConfig = new CalciteConnectionConfigImpl(catalogReaderConfigProperties); + return new PinotCalciteCatalogReader(schema, schema.path(null), _typeFactory, connConfig); } private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index 87bdc65f84e8..9836705b1a41 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -119,6 +119,7 @@ public Set getTableNames() { .filter(n -> DatabaseUtils.isPartOfDatabase(n, _databaseName)) .collect(Collectors.toSet()); } else { + // root schema will not contain tables return Collections.emptySet(); } } From 345cc0e20cfdedc58edae43b8c43c1f99288c7a5 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Tue, 19 Mar 2024 14:43:41 +0530 Subject: [PATCH 15/26] keep separate tests for all database context checks --- .../MultiStageEngineIntegrationTest.java | 109 ++++++++++-------- 1 file changed, 64 insertions(+), 45 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index 335caa27de51..23b2eb643b24 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -54,6 +54,7 @@ public class MultiStageEngineIntegrationTest extends BaseClusterIntegrationTestSet { private static final String SCHEMA_FILE_NAME = "On_Time_On_Time_Performance_2014_100k_subset_nonulls.schema"; + private static final String TABLE_NAME_WITH_DATABASE = "db1." + DEFAULT_TABLE_NAME; private String _tableName = DEFAULT_TABLE_NAME; private List _avroFiles = new ArrayList<>(); @@ -95,6 +96,36 @@ public void setUp() // Wait for all documents loaded waitForAllDocsLoaded(600_000L); + + setupTableWithNonDefaultDatabase(); + } + + private void setupTableWithNonDefaultDatabase() + throws Exception { + _tableName = TABLE_NAME_WITH_DATABASE; + String defaultCol = "ActualElapsedTime"; + String customCol = "ActualElapsedTime_2"; + Schema schema = createSchema(); + schema.addField(new MetricFieldSpec(customCol, FieldSpec.DataType.INT)); + addSchema(schema); + TableConfig tableConfig = createOfflineTableConfig(); + assert tableConfig.getIndexingConfig().getNoDictionaryColumns() != null; + List noDicCols = new ArrayList<>(DEFAULT_NO_DICTIONARY_COLUMNS); + noDicCols.add(customCol); + tableConfig.getIndexingConfig().setNoDictionaryColumns(noDicCols); + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setTransformConfigs(List.of(new TransformConfig(customCol, defaultCol))); + tableConfig.setIngestionConfig(ingestionConfig); + addTableConfig(tableConfig); + + // Create and upload segments to 'db1.mytable' + TestUtils.ensureDirectoriesExistAndEmpty(_segmentDir, _tarDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + uploadSegments(getTableName(), _tarDir); + + // Wait for all documents loaded + waitForAllDocsLoaded(600_000L); + _tableName = DEFAULT_TABLE_NAME; } protected void setupTenants() @@ -764,52 +795,39 @@ protected String getTableName() { } @Test - public void testWithDatabaseContext() + public void testWithoutDatabaseContext() throws Exception { - try { - _tableName = "db1." + DEFAULT_TABLE_NAME; - String defaultCol = "ActualElapsedTime"; - String customCol = "ActualElapsedTime_2"; - Schema schema = createSchema(); - schema.addField(new MetricFieldSpec(customCol, FieldSpec.DataType.INT)); - addSchema(schema); - TableConfig tableConfig = createOfflineTableConfig(); - assert tableConfig.getIndexingConfig().getNoDictionaryColumns() != null; - List noDicCols = new ArrayList<>(DEFAULT_NO_DICTIONARY_COLUMNS); - noDicCols.add(customCol); - tableConfig.getIndexingConfig().setNoDictionaryColumns(noDicCols); - IngestionConfig ingestionConfig = new IngestionConfig(); - ingestionConfig.setTransformConfigs(List.of(new TransformConfig(customCol, defaultCol))); - tableConfig.setIngestionConfig(ingestionConfig); - addTableConfig(tableConfig); - - // Create and upload segments to 'db1.mytable' - TestUtils.ensureDirectoriesExistAndEmpty(_segmentDir, _tarDir); - ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); - uploadSegments(getTableName(), _tarDir); - - // Wait for all documents loaded - waitForAllDocsLoaded(600_000L); - - // default database check. No database context passed - checkQueryResultForDBTest(defaultCol, DEFAULT_TABLE_NAME); - - // default database check. Default database context passed as table prefix - checkQueryResultForDBTest(defaultCol, "default." + DEFAULT_TABLE_NAME); - - // default database check. Default database context passed as SET database='dbName' - checkQueryResultForDBTest(defaultCol, DEFAULT_TABLE_NAME, "default"); - - // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. - // custom database check. Database context passed as table prefix - checkQueryResultForDBTest(customCol, _tableName); - - // custom database check. Database context passed as SET database='dbName' - checkQueryResultForDBTest(customCol, DEFAULT_TABLE_NAME, "db1"); - } finally { - dropOfflineTable(_tableName); - _tableName = DEFAULT_TABLE_NAME; - } + // default database check. No database context passed + checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_TABLE_NAME); + } + + @Test + public void testWithDefaultDatabaseContextAsTableNamePrefix() + throws Exception { + // default database check. Default database context passed as table prefix + checkQueryResultForDBTest("ActualElapsedTime", "default." + DEFAULT_TABLE_NAME); + } + + @Test + public void testWithDefaultDatabaseContextAsQueryOption() + throws Exception { + // default database check. Default database context passed as SET database='dbName' + checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_TABLE_NAME, "default"); + } + + @Test + public void testWithDatabaseContextAsTableNamePrefix() + throws Exception { + // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. + // custom database check. Database context passed as table prefix + checkQueryResultForDBTest("ActualElapsedTime_2", TABLE_NAME_WITH_DATABASE); + } + + @Test + public void testWithDatabaseContextAsQueryOption() + throws Exception { + // custom database check. Database context passed as SET database='dbName' + checkQueryResultForDBTest("ActualElapsedTime_2", DEFAULT_TABLE_NAME, "db1"); } private void checkQueryResultForDBTest(String column, String tableName) @@ -832,6 +850,7 @@ private void checkQueryResultForDBTest(String column, String tableName, @Nullabl public void tearDown() throws Exception { dropOfflineTable(DEFAULT_TABLE_NAME); + dropOfflineTable(TABLE_NAME_WITH_DATABASE); stopServer(); stopBroker(); From 49d6c7fdf4ca4eb97c1a14cb5fdcf0313fc0e2be Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Wed, 20 Mar 2024 17:35:50 +0530 Subject: [PATCH 16/26] remove subCatalog and use root catalog itself for query specific database context --- .../MultiStageBrokerRequestHandler.java | 5 +- .../pinot/common/utils/DatabaseUtils.java | 22 +++++++ .../api/resources/PinotQueryResource.java | 18 +++++- .../calcite/jdbc/CalciteSchemaBuilder.java | 19 ++----- .../apache/pinot/query/QueryEnvironment.java | 36 +++++------- .../pinot/query/catalog/PinotCatalog.java | 57 ++++--------------- .../pinot/query/QueryEnvironmentTestBase.java | 2 +- 7 files changed, 72 insertions(+), 87 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 32e1492e73de..7f856b6a54b8 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -52,6 +52,7 @@ import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.common.response.broker.ResultTable; import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.common.utils.ExceptionUtils; import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.common.utils.request.RequestUtils; @@ -124,10 +125,12 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S try { Long timeoutMsFromQueryOption = QueryOptionsUtils.getTimeoutMs(sqlNodeAndOptions.getOptions()); queryTimeoutMs = timeoutMsFromQueryOption == null ? _brokerTimeoutMs : timeoutMsFromQueryOption; + String database = DatabaseUtils.extractDatabaseFromQueryRequest(sqlNodeAndOptions.getOptions(), httpHeaders); // Compile the request compilationStartTimeNs = System.nanoTime(); QueryEnvironment queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()), - CalciteSchemaBuilder.asRootSchema(new PinotCatalog(_tableCache)), _workerManager, _tableCache); + CalciteSchemaBuilder.asRootSchema(new PinotCatalog(database, _tableCache), database), _workerManager, + _tableCache); switch (sqlNodeAndOptions.getSqlNode().getKind()) { case EXPLAIN: queryPlanResult = queryEnvironment.explainQuery(query, sqlNodeAndOptions, requestId); diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java index d00ed916628d..cbb2779675e5 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java @@ -19,6 +19,7 @@ package org.apache.pinot.common.utils; import com.google.common.base.Preconditions; +import java.util.Map; import javax.annotation.Nullable; import javax.ws.rs.core.HttpHeaders; import org.apache.commons.lang3.StringUtils; @@ -108,4 +109,25 @@ public static boolean isPartOfDatabase(String tableName, @Nullable String databa return tableName.startsWith(databaseName + "."); } } + + /** + * Extract database context from headers and query options + * @param queryOptions Query option from request + * @param headers http headers from request + * @return extracted database name. + * If queryOptions and headers have conflicting database context an {@link IllegalArgumentException} is thrown + */ + public static @Nullable String extractDatabaseFromQueryRequest( + Map queryOptions, HttpHeaders headers) { + String databaseFromOptions = queryOptions.get(CommonConstants.DATABASE); + String databaseFromHeaders = headers.getHeaderString(CommonConstants.DATABASE); + if (databaseFromHeaders != null && databaseFromOptions != null) { + Preconditions.checkArgument(databaseFromOptions.equals(databaseFromHeaders), "Database context mismatch : " + + "from headers %s, from query options %s", databaseFromHeaders, databaseFromOptions); + } + if (databaseFromHeaders != null) { + return databaseFromHeaders; + } + return databaseFromOptions; + } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java index f3249abff54f..b9f13a53f4cf 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java @@ -58,6 +58,7 @@ import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.common.utils.config.TagNameUtils; import org.apache.pinot.common.utils.request.RequestUtils; import org.apache.pinot.controller.ControllerConf; @@ -205,8 +206,14 @@ private String getMultiStageQueryResponse(String query, String queryOptions, Htt throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN); } + Map queryOptionsMap = RequestUtils.parseQuery(query).getOptions(); + if (queryOptions != null) { + queryOptionsMap.putAll(RequestUtils.getOptionsFromString(queryOptions)); + } + String database = DatabaseUtils.extractDatabaseFromQueryRequest(queryOptionsMap, httpHeaders); QueryEnvironment queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()), - CalciteSchemaBuilder.asRootSchema(new PinotCatalog(_pinotHelixResourceManager.getTableCache())), null, null); + CalciteSchemaBuilder.asRootSchema(new PinotCatalog(database, _pinotHelixResourceManager.getTableCache()), + database), null, null); List tableNames; try { tableNames = queryEnvironment.getTableNamesForQuery(query); @@ -242,6 +249,11 @@ private String getQueryResponse(String query, @Nullable SqlNode sqlNode, String HttpHeaders httpHeaders) { // Get resource table name. String tableName; + Map queryOptionsMap = RequestUtils.parseQuery(query).getOptions(); + if (queryOptions != null) { + queryOptionsMap.putAll(RequestUtils.getOptionsFromString(queryOptions)); + } + String database = DatabaseUtils.extractDatabaseFromQueryRequest(queryOptionsMap, httpHeaders); try { String inputTableName = sqlNode != null ? RequestUtils.getTableNames(CalciteSqlParser.compileSqlNodeToPinotQuery(sqlNode)).iterator() @@ -254,8 +266,8 @@ private String getQueryResponse(String query, @Nullable SqlNode sqlNode, String // try to compile the query using multi-stage engine and suggest using it if it succeeds. LOGGER.info("Trying to compile query {} using multi-stage engine", query); QueryEnvironment queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()), - CalciteSchemaBuilder.asRootSchema(new PinotCatalog(_pinotHelixResourceManager.getTableCache())), null, - null); + CalciteSchemaBuilder.asRootSchema(new PinotCatalog(database, _pinotHelixResourceManager.getTableCache()), + database), null, null); queryEnvironment.getTableNamesForQuery(query); LOGGER.info("Successfully compiled query using multi-stage engine: {}", query); return QueryException.getException(QueryException.SQL_PARSING_ERROR, new Exception( diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java index b7f64945f9ca..0d01228c7f42 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java +++ b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import org.apache.calcite.schema.Function; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; @@ -51,8 +52,11 @@ private CalciteSchemaBuilder() { * @param root schema to use as a root schema * @return calcite schema with given schema as the root */ - public static CalciteSchema asRootSchema(Schema root) { - CalciteSchema rootSchema = CalciteSchema.createRootSchema(false, false, "", root); + public static CalciteSchema asRootSchema(Schema root, @Nullable String name) { + if (name == null) { + name = ""; + } + CalciteSchema rootSchema = CalciteSchema.createRootSchema(false, false, name, root); SchemaPlus schemaPlus = rootSchema.plus(); for (Map.Entry> e : FunctionRegistry.getRegisteredCalciteFunctionMap().entrySet()) { for (Function f : e.getValue()) { @@ -61,15 +65,4 @@ public static CalciteSchema asRootSchema(Schema root) { } return rootSchema; } - - public static CalciteSchema asSubSchema(Schema root, String name) { - CalciteSchema subSchema = CalciteSchema.createRootSchema(false, false, name, root); - SchemaPlus schemaPlus = subSchema.plus(); - for (Map.Entry> e : FunctionRegistry.getRegisteredCalciteFunctionMap().entrySet()) { - for (Function f : e.getValue()) { - schemaPlus.add(e.getKey(), f); - } - } - return subSchema; - } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 5ea93819b86d..3f85173fe8cf 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Properties; @@ -61,6 +62,7 @@ import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.RelBuilder; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.query.context.PlannerContext; import org.apache.pinot.query.planner.PlannerUtils; @@ -109,7 +111,7 @@ public QueryEnvironment(TypeFactory typeFactory, CalciteSchema rootSchema, Worke _tableCache = tableCache; // catalog & config - _catalogReader = getCatalog(null); + _catalogReader = getCatalog(_rootSchema.getName()); _config = getConfig(_catalogReader); // opt programs _optProgram = getOptProgram(); @@ -129,7 +131,7 @@ public QueryEnvironment(TypeFactory typeFactory, CalciteSchema rootSchema, Worke * @return QueryPlannerResult containing the dispatchable query plan and the relRoot. */ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) { - try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions.getOptions())) { + try (PlannerContext plannerContext = getPlannerContext()) { plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(sqlNodeAndOptions.getSqlNode(), plannerContext); // TODO: current code only assume one SubPlan per query, but we should support multiple SubPlans per query. @@ -157,7 +159,7 @@ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAn * @return QueryPlannerResult containing the explained query plan and the relRoot. */ public QueryPlannerResult explainQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) { - try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions.getOptions())) { + try (PlannerContext plannerContext = getPlannerContext()) { SqlExplain explain = (SqlExplain) sqlNodeAndOptions.getSqlNode(); plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(explain.getExplicandum(), plannerContext); @@ -194,7 +196,7 @@ public List getTableNamesForQuery(String sqlQuery) { } public List getTableNamesForQuery(String sqlQuery, Map options) { - try (PlannerContext plannerContext = getPlannerContext(options)) { + try (PlannerContext plannerContext = getPlannerContext()) { SqlNode sqlNode = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery).getSqlNode(); if (sqlNode.getKind().equals(SqlKind.EXPLAIN)) { sqlNode = ((SqlExplain) sqlNode).getExplicandum(); @@ -334,17 +336,12 @@ private DispatchableSubPlan toDispatchableSubPlan(RelRoot relRoot, PlannerContex private Prepare.CatalogReader getCatalog(@Nullable String schemaPath) { Properties catalogReaderConfigProperties = new Properties(); catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); - CalciteSchema schema; - if (schemaPath == null) { - schema = _rootSchema; - } else { - schema = _rootSchema.getSubSchema(schemaPath, false); - if (schema == null) { - throw new IllegalArgumentException("Cannot find schema with path: " + schemaPath); - } - } CalciteConnectionConfigImpl connConfig = new CalciteConnectionConfigImpl(catalogReaderConfigProperties); - return new PinotCalciteCatalogReader(schema, schema.path(null), _typeFactory, connConfig); + if (StringUtils.isEmpty(schemaPath) || schemaPath.equals(CommonConstants.DEFAULT_DATABASE)) { + return new PinotCalciteCatalogReader(_rootSchema, Collections.emptyList(), _typeFactory, + connConfig); + } + return new PinotCalciteCatalogReader(_rootSchema, Collections.singletonList(schemaPath), _typeFactory, connConfig); } private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { @@ -364,15 +361,8 @@ private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { .build(); } - private PlannerContext getPlannerContext(Map options) { - String database = options.getOrDefault(CommonConstants.DATABASE, CommonConstants.DEFAULT_DATABASE); - if (database.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) { - return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); - } else { - Prepare.CatalogReader catalogReader = getCatalog(database); - FrameworkConfig config = getConfig(catalogReader); - return new PlannerContext(config, catalogReader, _typeFactory, _optProgram, _traitProgram); - } + private PlannerContext getPlannerContext() { + return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); } private HintStrategyTable getHintStrategyTable() { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index 9836705b1a41..792e8d19006e 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -20,13 +20,9 @@ import java.util.Collection; import java.util.Collections; -import java.util.HashMap; -import java.util.Map; import java.util.Set; import java.util.stream.Collectors; import javax.annotation.Nullable; -import org.apache.calcite.jdbc.CalciteSchema; -import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.schema.Function; @@ -35,10 +31,8 @@ import org.apache.calcite.schema.SchemaVersion; import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.common.utils.DatabaseUtils; -import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import static java.util.Objects.requireNonNull; @@ -53,9 +47,6 @@ public class PinotCatalog implements Schema { private final TableCache _tableCache; - - private final Map _subCatalog; - private final String _databaseName; /** @@ -65,31 +56,11 @@ public class PinotCatalog implements Schema { public PinotCatalog(TableCache tableCache) { _tableCache = tableCache; _databaseName = null; - // create all databases - // TODO: we need to monitor table cache changes to register newly created databases - // TODO: we also need to monitor table that needs to be put into the right places - _subCatalog = constructSubCatalogs(_tableCache); } - private PinotCatalog(String databaseName, TableCache tableCache) { + public PinotCatalog(String databaseName, TableCache tableCache) { _tableCache = tableCache; _databaseName = databaseName; - _subCatalog = null; - } - - private Map constructSubCatalogs(TableCache tableCache) { - Map subCatalog = new HashMap<>(); - for (String physicalTableName : tableCache.getTableNameMap().keySet()) { - String[] nameSplit = StringUtils.split(physicalTableName, '.'); - if (nameSplit.length > 1) { - String databaseName = nameSplit[0]; - subCatalog.putIfAbsent(databaseName, - CalciteSchemaBuilder.asSubSchema(new PinotCatalog(databaseName, tableCache), databaseName)); - } - } - subCatalog.put(CommonConstants.DEFAULT_DATABASE, CalciteSchemaBuilder.asSubSchema( - new PinotCatalog(CommonConstants.DEFAULT_DATABASE, tableCache), CommonConstants.DEFAULT_DATABASE)); - return subCatalog; } /** @@ -101,6 +72,9 @@ private Map constructSubCatalogs(TableCache tableCache) { public Table getTable(String name) { String rawTableName = TableNameBuilder.extractRawTableName(name); String tableName = DatabaseUtils.translateTableName(rawTableName, _databaseName); + if (!_tableCache.getTableNameMap().containsKey(tableName.toLowerCase())) { + return null; + } org.apache.pinot.spi.data.Schema schema = _tableCache.getSchema(tableName); if (schema == null) { throw new IllegalArgumentException(String.format("Could not find schema for table: '%s'", tableName)); @@ -114,14 +88,9 @@ public Table getTable(String name) { */ @Override public Set getTableNames() { - if (_databaseName != null) { - return _tableCache.getTableNameMap().keySet().stream() - .filter(n -> DatabaseUtils.isPartOfDatabase(n, _databaseName)) - .collect(Collectors.toSet()); - } else { - // root schema will not contain tables - return Collections.emptySet(); - } + return _tableCache.getTableNameMap().keySet().stream() + .filter(n -> DatabaseUtils.isPartOfDatabase(n, _databaseName)) + .collect(Collectors.toSet()); } @Override @@ -136,7 +105,7 @@ public Set getTypeNames() { /** * {@code PinotCatalog} doesn't need to return function collections b/c they are already registered. - * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema)} + * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema, String)} */ @Override public Collection getFunctions(String name) { @@ -145,7 +114,7 @@ public Collection getFunctions(String name) { /** * {@code PinotCatalog} doesn't need to return function name set b/c they are already registered. - * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema)} + * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema, String)} */ @Override public Set getFunctionNames() { @@ -154,16 +123,12 @@ public Set getFunctionNames() { @Override public Schema getSubSchema(String name) { - if (_subCatalog != null && _subCatalog.containsKey(name)) { - return _subCatalog.get(name).schema; - } else { - return null; - } + return null; } @Override public Set getSubSchemaNames() { - return _subCatalog.keySet(); + return Collections.emptySet(); } @Override diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java index 6f423e253a6b..8ab00d8e598f 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java @@ -272,7 +272,7 @@ public static QueryEnvironment getQueryEnvironment(int reducerPort, int port1, i RoutingManager routingManager = factory.buildRoutingManager(partitionInfoMap); TableCache tableCache = factory.buildTableCache(); return new QueryEnvironment(new TypeFactory(new TypeSystem()), - CalciteSchemaBuilder.asRootSchema(new PinotCatalog(tableCache)), + CalciteSchemaBuilder.asRootSchema(new PinotCatalog(tableCache), null), new WorkerManager("localhost", reducerPort, routingManager), tableCache); } From 74d9bf65543abdd519297ec956ad5f346b240d1e Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Wed, 20 Mar 2024 23:35:14 +0530 Subject: [PATCH 17/26] table cache mock fix --- .../pinot/query/testutils/MockRoutingManagerFactory.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java index 4539627a2337..37454f93af81 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java @@ -81,8 +81,10 @@ public void registerTable(Schema schema, String tableName) { } private void registerTableNameWithType(Schema schema, String tableNameWithType) { + String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); _tableNameMap.put(tableNameWithType, tableNameWithType); - _schemaMap.put(TableNameBuilder.extractRawTableName(tableNameWithType), schema); + _tableNameMap.put(rawTableName, rawTableName); + _schemaMap.put(rawTableName, schema); } public void registerSegment(int insertToServerPort, String tableNameWithType, String segmentName) { From c1f67c015ff0b6d4cce2f3e841ae87253c7a58a7 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 11:36:41 +0530 Subject: [PATCH 18/26] Bugfix. Table name extraction from TableCache --- .../org/apache/pinot/query/catalog/PinotCatalog.java | 12 +++++------- .../query/testutils/MockRoutingManagerFactory.java | 4 ++++ 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index 792e8d19006e..7becc2a562d9 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.query.catalog; +import com.google.common.base.Preconditions; import java.util.Collection; import java.util.Collections; import java.util.Set; @@ -71,14 +72,11 @@ public PinotCatalog(String databaseName, TableCache tableCache) { @Override public Table getTable(String name) { String rawTableName = TableNameBuilder.extractRawTableName(name); - String tableName = DatabaseUtils.translateTableName(rawTableName, _databaseName); - if (!_tableCache.getTableNameMap().containsKey(tableName.toLowerCase())) { - return null; - } + String physicalTableName = DatabaseUtils.translateTableName(rawTableName, _databaseName); + String tableName = _tableCache.getActualTableName(physicalTableName); + Preconditions.checkArgument(tableName != null, String.format("Table does not exist: '%s'", physicalTableName)); org.apache.pinot.spi.data.Schema schema = _tableCache.getSchema(tableName); - if (schema == null) { - throw new IllegalArgumentException(String.format("Could not find schema for table: '%s'", tableName)); - } + Preconditions.checkArgument(schema != null, String.format("Could not find schema for table: '%s'", tableName)); return new PinotTable(schema); } diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java index 37454f93af81..d01185214077 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/testutils/MockRoutingManagerFactory.java @@ -105,6 +105,10 @@ public RoutingManager buildRoutingManager(@Nullable Map { + String tableName = invocationOnMock.getArgument(0); + return _tableNameMap.get(tableName); + }); when(mock.getSchema(anyString())).thenAnswer(invocationOnMock -> { String schemaName = invocationOnMock.getArgument(0); return _schemaMap.get(schemaName); From 929bced6b02501c79cac69b3a7bab5ae4076ce4c Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 13:31:26 +0530 Subject: [PATCH 19/26] Use "default" database context when no context is passed --- .../pinot/common/utils/DatabaseUtils.java | 18 +++++++++--------- .../calcite/jdbc/CalciteSchemaBuilder.java | 5 +---- .../apache/pinot/query/QueryEnvironment.java | 9 ++++----- .../pinot/query/QueryEnvironmentTestBase.java | 3 ++- 4 files changed, 16 insertions(+), 19 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java index cbb2779675e5..6913c409a6ab 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import java.util.Map; +import java.util.Objects; import javax.annotation.Nullable; import javax.ws.rs.core.HttpHeaders; import org.apache.commons.lang3.StringUtils; @@ -115,19 +116,18 @@ public static boolean isPartOfDatabase(String tableName, @Nullable String databa * @param queryOptions Query option from request * @param headers http headers from request * @return extracted database name. - * If queryOptions and headers have conflicting database context an {@link IllegalArgumentException} is thrown + *
If database context is not provided at all return {@link CommonConstants#DEFAULT_DATABASE}. + *
If queryOptions and headers have conflicting database context an {@link IllegalArgumentException} is thrown. */ - public static @Nullable String extractDatabaseFromQueryRequest( - Map queryOptions, HttpHeaders headers) { - String databaseFromOptions = queryOptions.get(CommonConstants.DATABASE); - String databaseFromHeaders = headers.getHeaderString(CommonConstants.DATABASE); + public static String extractDatabaseFromQueryRequest( + @Nullable Map queryOptions, @Nullable HttpHeaders headers) { + String databaseFromOptions = queryOptions == null ? null : queryOptions.get(CommonConstants.DATABASE); + String databaseFromHeaders = headers == null ? null : headers.getHeaderString(CommonConstants.DATABASE); if (databaseFromHeaders != null && databaseFromOptions != null) { Preconditions.checkArgument(databaseFromOptions.equals(databaseFromHeaders), "Database context mismatch : " + "from headers %s, from query options %s", databaseFromHeaders, databaseFromOptions); } - if (databaseFromHeaders != null) { - return databaseFromHeaders; - } - return databaseFromOptions; + String database = databaseFromHeaders != null ? databaseFromHeaders : databaseFromOptions; + return Objects.requireNonNullElse(database, CommonConstants.DEFAULT_DATABASE); } } diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java index 0d01228c7f42..b32ee32ca123 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java +++ b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java @@ -52,10 +52,7 @@ private CalciteSchemaBuilder() { * @param root schema to use as a root schema * @return calcite schema with given schema as the root */ - public static CalciteSchema asRootSchema(Schema root, @Nullable String name) { - if (name == null) { - name = ""; - } + public static CalciteSchema asRootSchema(Schema root, String name) { CalciteSchema rootSchema = CalciteSchema.createRootSchema(false, false, name, root); SchemaPlus schemaPlus = rootSchema.plus(); for (Map.Entry> e : FunctionRegistry.getRegisteredCalciteFunctionMap().entrySet()) { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 3f85173fe8cf..c586cfb89af5 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Set; import javax.annotation.Nullable; @@ -337,11 +338,9 @@ private Prepare.CatalogReader getCatalog(@Nullable String schemaPath) { Properties catalogReaderConfigProperties = new Properties(); catalogReaderConfigProperties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(), "true"); CalciteConnectionConfigImpl connConfig = new CalciteConnectionConfigImpl(catalogReaderConfigProperties); - if (StringUtils.isEmpty(schemaPath) || schemaPath.equals(CommonConstants.DEFAULT_DATABASE)) { - return new PinotCalciteCatalogReader(_rootSchema, Collections.emptyList(), _typeFactory, - connConfig); - } - return new PinotCalciteCatalogReader(_rootSchema, Collections.singletonList(schemaPath), _typeFactory, connConfig); + return new PinotCalciteCatalogReader(_rootSchema, + Collections.singletonList(Objects.requireNonNullElse(schemaPath, CommonConstants.DEFAULT_DATABASE)), + _typeFactory, connConfig); } private FrameworkConfig getConfig(Prepare.CatalogReader catalogReader) { diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java index 8ab00d8e598f..615f9bace302 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java @@ -43,6 +43,7 @@ import org.apache.pinot.query.type.TypeSystem; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.CommonConstants; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; @@ -272,7 +273,7 @@ public static QueryEnvironment getQueryEnvironment(int reducerPort, int port1, i RoutingManager routingManager = factory.buildRoutingManager(partitionInfoMap); TableCache tableCache = factory.buildTableCache(); return new QueryEnvironment(new TypeFactory(new TypeSystem()), - CalciteSchemaBuilder.asRootSchema(new PinotCatalog(tableCache), null), + CalciteSchemaBuilder.asRootSchema(new PinotCatalog(tableCache), CommonConstants.DEFAULT_DATABASE), new WorkerManager("localhost", reducerPort, routingManager), tableCache); } From 724f392f8f62b9b9851afbd136e1055ba3f46f22 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 13:32:33 +0530 Subject: [PATCH 20/26] cosmetic fixes --- .../tests/MultiStageEngineIntegrationTest.java | 15 +++++++-------- .../apache/calcite/jdbc/CalciteSchemaBuilder.java | 1 - .../org/apache/pinot/query/QueryEnvironment.java | 1 - 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index 23b2eb643b24..5f2c53184282 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -56,7 +56,6 @@ public class MultiStageEngineIntegrationTest extends BaseClusterIntegrationTestS private static final String SCHEMA_FILE_NAME = "On_Time_On_Time_Performance_2014_100k_subset_nonulls.schema"; private static final String TABLE_NAME_WITH_DATABASE = "db1." + DEFAULT_TABLE_NAME; private String _tableName = DEFAULT_TABLE_NAME; - private List _avroFiles = new ArrayList<>(); @Override protected String getSchemaFileName() { @@ -82,25 +81,25 @@ public void setUp() addTableConfig(tableConfig); // Unpack the Avro files - _avroFiles = unpackAvroData(_tempDir); + List avroFiles = unpackAvroData(_tempDir); // Create and upload segments - ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); uploadSegments(getTableName(), _tarDir); // Set up the H2 connection - setUpH2Connection(_avroFiles); + setUpH2Connection(avroFiles); // Initialize the query generator - setUpQueryGenerator(_avroFiles); + setUpQueryGenerator(avroFiles); // Wait for all documents loaded waitForAllDocsLoaded(600_000L); - setupTableWithNonDefaultDatabase(); + setupTableWithNonDefaultDatabase(avroFiles); } - private void setupTableWithNonDefaultDatabase() + private void setupTableWithNonDefaultDatabase(List avroFiles) throws Exception { _tableName = TABLE_NAME_WITH_DATABASE; String defaultCol = "ActualElapsedTime"; @@ -120,7 +119,7 @@ private void setupTableWithNonDefaultDatabase() // Create and upload segments to 'db1.mytable' TestUtils.ensureDirectoriesExistAndEmpty(_segmentDir, _tarDir); - ClusterIntegrationTestUtils.buildSegmentsFromAvro(_avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); uploadSegments(getTableName(), _tarDir); // Wait for all documents loaded diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java index b32ee32ca123..bab3ac045d91 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java +++ b/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import javax.annotation.Nullable; import org.apache.calcite.schema.Function; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index c586cfb89af5..c42b14dbce0c 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -63,7 +63,6 @@ import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.RelBuilder; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.query.context.PlannerContext; import org.apache.pinot.query.planner.PlannerUtils; From cdae3c03c09c1ff11a0ce1edd335f274dc28957e Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 15:29:48 +0530 Subject: [PATCH 21/26] Fix the expected explain plan with default path --- .../pinot/query/QueryCompilationTest.java | 6 +- .../resources/queries/AggregatePlans.json | 20 +- .../resources/queries/BasicQueryPlans.json | 18 +- .../test/resources/queries/GroupByPlans.json | 34 +- .../src/test/resources/queries/JoinPlans.json | 122 ++--- .../queries/LiteralEvaluationPlans.json | 42 +- .../test/resources/queries/OrderByPlans.json | 20 +- .../resources/queries/PinotHintablePlans.json | 104 ++-- .../test/resources/queries/SetOpPlans.json | 28 +- .../queries/WindowFunctionPlans.json | 498 +++++++++--------- 10 files changed, 446 insertions(+), 446 deletions(-) diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java index b8c6e971783d..82ae97cf7848 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java @@ -88,7 +88,7 @@ public void testAggregateCaseToFilter() { + " PinotLogicalExchange(distribution=[hash])\n" + " LogicalAggregate(group=[{}], agg#0=[COUNT() FILTER $0])\n" + " LogicalProject($f1=[=($0, _UTF-8'a')])\n" - + " LogicalTableScan(table=[[a]])\n"); + + " LogicalTableScan(table=[[default, a]])\n"); } private static void assertGroupBySingletonAfterJoin(DispatchableSubPlan dispatchableSubPlan, boolean shouldRewrite) { @@ -466,10 +466,10 @@ private Object[][] provideQueriesWithExplainedLogicalPlan() { + " LogicalJoin(condition=[=($0, $1)], joinType=[inner])\n" + " PinotLogicalExchange(distribution=[hash[0]])\n" + " LogicalProject(col1=[$0])\n" - + " LogicalTableScan(table=[[a]])\n" + + " LogicalTableScan(table=[[default, a]])\n" + " PinotLogicalExchange(distribution=[hash[0]])\n" + " LogicalProject(col1=[$0], col3=[$2])\n" - + " LogicalTableScan(table=[[b]])\n" + + " LogicalTableScan(table=[[default, b]])\n" }, }; //@formatter:on diff --git a/pinot-query-planner/src/test/resources/queries/AggregatePlans.json b/pinot-query-planner/src/test/resources/queries/AggregatePlans.json index 1072ccd03c84..325357d73559 100644 --- a/pinot-query-planner/src/test/resources/queries/AggregatePlans.json +++ b/pinot-query-planner/src/test/resources/queries/AggregatePlans.json @@ -13,13 +13,13 @@ "\n LogicalJoin(condition=[>=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[random])", "\n LogicalProject(col3=[$2], col4=[$3])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast])", "\n LogicalProject(EXPR$0=[CAST(/(CASE(=($1, 0), null:DECIMAL(1000, 0), $0), $1)):DECIMAL(1000, 0)])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($0)], agg#1=[COUNT($1)])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($3)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -33,7 +33,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($3)], agg#1=[COUNT()])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'pink floyd'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -47,7 +47,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($3)], agg#1=[COUNT()], agg#2=[MAX($3)])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'pink floyd'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -61,7 +61,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'pink floyd'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -74,7 +74,7 @@ "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($0)], agg#1=[COUNT($1)])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -88,7 +88,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -102,7 +102,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'pink floyd'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -116,7 +116,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'pink floyd'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -130,7 +130,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'pink floyd'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] } diff --git a/pinot-query-planner/src/test/resources/queries/BasicQueryPlans.json b/pinot-query-planner/src/test/resources/queries/BasicQueryPlans.json index 59acc30e6611..2d4a52fa822e 100644 --- a/pinot-query-planner/src/test/resources/queries/BasicQueryPlans.json +++ b/pinot-query-planner/src/test/resources/queries/BasicQueryPlans.json @@ -6,7 +6,7 @@ "sql": "EXPLAIN PLAN FOR SELECT * FROM d", "output": [ "Execution Plan", - "\nLogicalTableScan(table=[[d]])", + "\nLogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -18,7 +18,7 @@ "\nLogicalSort(offset=[0], fetch=[0])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[]], isSortOnSender=[false], isSortOnReceiver=[false])", "\n LogicalSort(fetch=[0])", - "\n LogicalTableScan(table=[[d]])", + "\n LogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -30,7 +30,7 @@ "\nLogicalSort(offset=[0], fetch=[0])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[]], isSortOnSender=[false], isSortOnReceiver=[false])", "\n LogicalSort(fetch=[0])", - "\n LogicalTableScan(table=[[d]])", + "\n LogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -41,7 +41,7 @@ "Execution Plan", "\nLogicalProject(col1=[$0], EXPR$1=[+($2, $6)])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -52,7 +52,7 @@ "Execution Plan", "\nLogicalProject(col1=[$0], colsum=[+($2, $6)])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -65,7 +65,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[]], isSortOnSender=[false], isSortOnReceiver=[false])", "\n LogicalSort(fetch=[10])", "\n LogicalProject(EXPR$0=[DATETRUNC(_UTF-8'DAY', $6)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -78,7 +78,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[]], isSortOnSender=[false], isSortOnReceiver=[false])", "\n LogicalSort(fetch=[10])", "\n LogicalProject(day=[DATETRUNC(_UTF-8'DAY', $6)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -92,7 +92,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[$SUM0($0)], agg#1=[COUNT()])", "\n LogicalProject($f0=[CAST(CASE(>($2, 10), _UTF-8'1':VARCHAR CHARACTER SET \"UTF-8\", >($2, 20), _UTF-8'2':VARCHAR CHARACTER SET \"UTF-8\", >($2, 30), _UTF-8'3':VARCHAR CHARACTER SET \"UTF-8\", >($2, 40), _UTF-8'4':VARCHAR CHARACTER SET \"UTF-8\", >($2, 50), _UTF-8'5':VARCHAR CHARACTER SET \"UTF-8\", _UTF-8'0':VARCHAR CHARACTER SET \"UTF-8\")):DECIMAL(1000, 500) NOT NULL])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -106,7 +106,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n LogicalProject(EXPR$1=[ARRAYTOMV($6)], col3=[$2])", - "\n LogicalTableScan(table=[[e]])", + "\n LogicalTableScan(table=[[default, e]])", "\n" ] } diff --git a/pinot-query-planner/src/test/resources/queries/GroupByPlans.json b/pinot-query-planner/src/test/resources/queries/GroupByPlans.json index 2a67b4455ac7..a7a4b1a8be21 100644 --- a/pinot-query-planner/src/test/resources/queries/GroupByPlans.json +++ b/pinot-query-planner/src/test/resources/queries/GroupByPlans.json @@ -9,7 +9,7 @@ "\nLogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -22,7 +22,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)], agg#2=[MAX($3)], agg#3=[MIN($4)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[COUNT()], agg#2=[MAX($2)], agg#3=[MIN($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -35,7 +35,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -49,7 +49,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[COUNT()])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -63,7 +63,7 @@ "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($0, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -78,7 +78,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[COUNT()], agg#1=[$SUM0($2)], agg#2=[MAX($2)], agg#3=[MIN($2)])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -93,7 +93,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[COUNT()], agg#1=[$SUM0($2)], agg#2=[MAX($2)], agg#3=[MIN($2)])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -105,7 +105,7 @@ "\nLogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -118,7 +118,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT()])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -131,7 +131,7 @@ "\n LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)], agg#1=[COUNT()], EXPR$3=[MAX($1)], EXPR$4=[MIN($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -144,7 +144,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -157,7 +157,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -171,7 +171,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -185,7 +185,7 @@ "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($0, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -200,7 +200,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -215,7 +215,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -230,7 +230,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] } diff --git a/pinot-query-planner/src/test/resources/queries/JoinPlans.json b/pinot-query-planner/src/test/resources/queries/JoinPlans.json index 26cfc6bea3b9..e75e69cc579f 100644 --- a/pinot-query-planner/src/test/resources/queries/JoinPlans.json +++ b/pinot-query-planner/src/test/resources/queries/JoinPlans.json @@ -12,10 +12,10 @@ "\n LogicalJoin(condition=[=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], ts=[$6])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -30,10 +30,10 @@ "\n LogicalJoin(condition=[=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], ts=[$6])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -44,9 +44,9 @@ "Execution Plan", "\nLogicalJoin(condition=[=($0, $8)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[1]])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -58,9 +58,9 @@ "\nLogicalJoin(condition=[=($0, $8)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalFilter(condition=[>=($2, 0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[1]])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -72,9 +72,9 @@ "\nLogicalJoin(condition=[AND(=($0, $8), >($2, $9))], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalFilter(condition=[>=($2, 0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[1]])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -85,9 +85,9 @@ "Execution Plan", "\nLogicalJoin(condition=[AND(=($0, $7), =($1, $8))], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -101,11 +101,11 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], ts=[$6])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[<($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -122,11 +122,11 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[<($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -139,10 +139,10 @@ "\n LogicalJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], ts=[$6])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], ts=[$6])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -158,10 +158,10 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[>=($2, 0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -177,10 +177,10 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[>=($2, 0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -194,11 +194,11 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[OR(=($1, _UTF-8'bar'), =($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'alice':VARCHAR(7) CHARACTER SET \"UTF-8\"), <>($1, _UTF-8'charlie':VARCHAR(7) CHARACTER SET \"UTF-8\"))])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -212,11 +212,11 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[OR(=($1, _UTF-8'bar'), =($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'alice':VARCHAR(7) CHARACTER SET \"UTF-8\"), <>($1, _UTF-8'charlie':VARCHAR(7) CHARACTER SET \"UTF-8\"))])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -229,10 +229,10 @@ "\n LogicalJoin(condition=[=($2, $3)], joinType=[semi])", "\n PinotLogicalExchange(distribution=[hash[2]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col3=[$2])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -250,19 +250,19 @@ "\n PinotLogicalExchange(distribution=[hash[2]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[=($1, _UTF-8'test')])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col3=[$2])", "\n LogicalFilter(condition=[=($0, _UTF-8'foo')])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col3=[$2])", "\n LogicalFilter(condition=[=($0, _UTF-8'bar')])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col3=[$2])", "\n LogicalFilter(condition=[=($0, _UTF-8'foobar')])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -285,28 +285,28 @@ "\n PinotLogicalExchange(distribution=[hash[3]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2], col30=[$2])", "\n LogicalFilter(condition=[=($1, _UTF-8'test')])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[MIN($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[MIN($1)])", "\n LogicalProject(col3=[$2], $f1=[true])", "\n LogicalFilter(condition=[=($0, _UTF-8'foo')])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[MIN($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[MIN($1)])", "\n LogicalProject(col3=[$2], $f1=[true])", "\n LogicalFilter(condition=[=($0, _UTF-8'bar')])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[MIN($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[MIN($1)])", "\n LogicalProject(col3=[$2], $f1=[true])", "\n LogicalFilter(condition=[=($0, _UTF-8'foobar')])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -318,13 +318,13 @@ "\nLogicalProject(col1=[$0])", "\n LogicalJoin(condition=[AND(=($0, $7), =($1, $8), >($3, $12))], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col10=[CAST($0):VARCHAR CHARACTER SET \"UTF-8\"], col20=[CAST($1):VARCHAR CHARACTER SET \"UTF-8\"], $f2=[CAST($2):BIGINT], EXPR$3=[*(0.5:DECIMAL(2, 1), $2)])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -336,14 +336,14 @@ "\nLogicalProject(col1=[$0])", "\n LogicalJoin(condition=[AND(=($0, $7), =($1, $8), >($3, $9))], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], EXPR$0=[$2], col10=[CAST($0):VARCHAR CHARACTER SET \"UTF-8\"], col20=[CAST($1):VARCHAR CHARACTER SET \"UTF-8\"], EXPR$05=[CAST($2):DECIMAL(12, 1)])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)])", "\n LogicalProject(col1=[$0], col2=[$1], $f0=[*(0.5:DECIMAL(2, 1), $2)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -355,14 +355,14 @@ "\nLogicalProject(col1=[$0])", "\n LogicalJoin(condition=[AND(=($0, $7), =($1, $8), >($3, $9))], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], EXPR$0=[CAST(/($2, $3)):DECIMAL(12, 1) NOT NULL], col10=[CAST($0):VARCHAR CHARACTER SET \"UTF-8\"], col20=[CAST($1):VARCHAR CHARACTER SET \"UTF-8\"], EXPR$05=[CAST(CAST(/($2, $3)):DECIMAL(12, 1) NOT NULL):DECIMAL(12, 1)])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($3)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", "\n LogicalProject(col1=[$0], col2=[$1], $f0=[*(0.5:DECIMAL(2, 1), $2)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -374,13 +374,13 @@ "\nLogicalProject(col1=[$0])", "\n LogicalJoin(condition=[AND(=($0, $7), =($1, $8), >($3, $12))], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col10=[CAST($0):VARCHAR CHARACTER SET \"UTF-8\"], col20=[CAST($1):VARCHAR CHARACTER SET \"UTF-8\"], $f2=[CAST(/(CAST($2):DOUBLE NOT NULL, $3)):DOUBLE], EXPR$3=[*(0.5:DECIMAL(2, 1), /(CAST($2):DOUBLE NOT NULL, $3))])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($3)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -393,12 +393,12 @@ "\n LogicalJoin(condition=[=($1, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{1}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -414,10 +414,10 @@ "\n LogicalJoin(condition=[=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], ts=[$6])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -429,19 +429,19 @@ "\nLogicalJoin(condition=[=($2, $7)], joinType=[semi])", "\n LogicalJoin(condition=[=($1, $7)], joinType=[semi])", "\n LogicalJoin(condition=[=($1, $7)], joinType=[semi])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 100)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col3=[$2])", "\n LogicalFilter(condition=[<($2, 100)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -457,15 +457,15 @@ "\n LogicalProject(col3=[$1])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col3=[$2])", "\n LogicalFilter(condition=[<($2, 100)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -483,15 +483,15 @@ "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalJoin(condition=[=($1, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col3=[$2])", "\n LogicalFilter(condition=[<($2, 100)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -503,15 +503,15 @@ "\nLogicalJoin(condition=[=($2, $7)], joinType=[semi])", "\n LogicalJoin(condition=[=($0, $7)], joinType=[semi])", "\n LogicalFilter(condition=[<($2, 100)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col3=[$2])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] } diff --git a/pinot-query-planner/src/test/resources/queries/LiteralEvaluationPlans.json b/pinot-query-planner/src/test/resources/queries/LiteralEvaluationPlans.json index cbbfa67d9e14..950b504548ea 100644 --- a/pinot-query-planner/src/test/resources/queries/LiteralEvaluationPlans.json +++ b/pinot-query-planner/src/test/resources/queries/LiteralEvaluationPlans.json @@ -16,7 +16,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[30.0], EXPR$1=[11.0])", - "\n LogicalTableScan(table=[[d]])", + "\n LogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -26,7 +26,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[1997-02-01 00:00:00])", - "\n LogicalTableScan(table=[[d]])", + "\n LogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -36,7 +36,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[CAST(854755200000:BIGINT):BIGINT])", - "\n LogicalTableScan(table=[[d]])", + "\n LogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -46,7 +46,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[TIMESTAMPDIFF(FLAG(DAY), CAST($6):TIMESTAMP(0) NOT NULL, CAST(854755200000:BIGINT):TIMESTAMP(0))])", - "\n LogicalTableScan(table=[[d]])", + "\n LogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -56,7 +56,7 @@ "output": [ "Execution Plan", "\nLogicalFilter(condition=[=(CAST($6):TIMESTAMP(0) NOT NULL, 2019-01-01 00:00:00)])", - "\n LogicalTableScan(table=[[d]])", + "\n LogicalTableScan(table=[[default, d]])", "\n" ] }, @@ -66,7 +66,7 @@ "output": [ "Execution Plan", "\nLogicalProject(day=[CAST(854755200000:BIGINT):BIGINT])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -76,7 +76,7 @@ "output": [ "Execution Plan", "\nLogicalFilter(condition=[=(DATETRUNC(_UTF-8'MONTH', $6), 1546300800000)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -86,7 +86,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[_UTF-8'MONTH'])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -96,7 +96,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[CAST(_UTF-8'month 1':VARCHAR CHARACTER SET \"UTF-8\"):VARCHAR CHARACTER SET \"UTF-8\"])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -106,7 +106,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[CAST(_UTF-8'nth':VARCHAR CHARACTER SET \"UTF-8\"):VARCHAR CHARACTER SET \"UTF-8\"])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -116,7 +116,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[CAST(_UTF-8'NTH':VARCHAR CHARACTER SET \"UTF-8\"):VARCHAR CHARACTER SET \"UTF-8\"])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -126,7 +126,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[599041711439609855:BIGINT])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -136,7 +136,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[X'8040340000000000004024000000000000':VARBINARY])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -146,7 +146,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[1.3416951966757335E7:DOUBLE])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -156,7 +156,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[12345678901234567890123456789.1234567890123456789:DECIMAL(1000, 0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -166,7 +166,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[14:BIGINT])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -176,7 +176,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[15.0:BIGINT])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -186,7 +186,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[ARRAY(1, 2, 3)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -206,7 +206,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[ARRAY(0.1:DECIMAL(2, 1), 0.2:DECIMAL(2, 1), 0.3:DECIMAL(2, 1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -226,7 +226,7 @@ "output": [ "Execution Plan", "\nLogicalProject(EXPR$0=[ARRAY(_UTF-8'a', _UTF-8'b', _UTF-8'c')])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -249,7 +249,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalAggregate(group=[{}], agg#0=[COUNT()])", "\n LogicalFilter(condition=[>(CAST($0):BIGINT NOT NULL, 14)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, diff --git a/pinot-query-planner/src/test/resources/queries/OrderByPlans.json b/pinot-query-planner/src/test/resources/queries/OrderByPlans.json index dbe27dcff116..7b97f583ea03 100644 --- a/pinot-query-planner/src/test/resources/queries/OrderByPlans.json +++ b/pinot-query-planner/src/test/resources/queries/OrderByPlans.json @@ -9,7 +9,7 @@ "\nLogicalSort(sort0=[$0], dir0=[ASC])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -21,7 +21,7 @@ "\nLogicalSort(sort0=[$0], dir0=[ASC])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(value1=[$0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -33,7 +33,7 @@ "\nLogicalSort(sort0=[$0], dir0=[ASC], offset=[0], fetch=[10])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalSort(sort0=[$0], dir0=[ASC], fetch=[10])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -45,7 +45,7 @@ "\nLogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], offset=[0], fetch=[10])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], fetch=[10])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -57,7 +57,7 @@ "\nLogicalSort(offset=[0], fetch=[10000000])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[]], isSortOnSender=[false], isSortOnReceiver=[false])", "\n LogicalSort(fetch=[10000000])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -68,7 +68,7 @@ "Execution Plan", "\nLogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], fetch=[10000000])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -82,7 +82,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -96,7 +96,7 @@ "\n LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -110,7 +110,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -124,7 +124,7 @@ "\n LogicalAggregate(group=[{0}], sum=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] } diff --git a/pinot-query-planner/src/test/resources/queries/PinotHintablePlans.json b/pinot-query-planner/src/test/resources/queries/PinotHintablePlans.json index a16765e558bc..351438740b55 100644 --- a/pinot-query-planner/src/test/resources/queries/PinotHintablePlans.json +++ b/pinot-query-planner/src/test/resources/queries/PinotHintablePlans.json @@ -32,11 +32,11 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[<($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -47,11 +47,11 @@ "Execution Plan", "\nLogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -63,15 +63,15 @@ "\nLogicalJoin(condition=[=($1, $2)], joinType=[semi])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[c]])", + "\n LogicalTableScan(table=[[default, c]])", "\n" ] }, @@ -86,15 +86,15 @@ "\n LogicalJoin(condition=[=($1, $3)], joinType=[semi])", "\n LogicalJoin(condition=[=($0, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[c]])", + "\n LogicalTableScan(table=[[default, c]])", "\n" ] }, @@ -106,11 +106,11 @@ "\nLogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -124,11 +124,11 @@ "\n LogicalAggregate(group=[{1}], agg#0=[$SUM0($2)])", "\n LogicalJoin(condition=[=($0, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -142,7 +142,7 @@ "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($0, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -157,7 +157,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2], $f2=[CAST($0):DECIMAL(1000, 500) NOT NULL])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -170,7 +170,7 @@ "\n LogicalAggregate(group=[{0}], EXPR$1=[COUNT()], EXPR$2=[$SUM0($1)], EXPR$3=[$SUM0($2)])", "\n LogicalProject(col2=[$1], col3=[$2], $f2=[CAST($0):DECIMAL(1000, 500) NOT NULL])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -182,7 +182,7 @@ "\nLogicalAggregate(group=[{0}], EXPR$1=[COUNT()], EXPR$2=[$SUM0($1)], EXPR$3=[$SUM0($2)])", "\n LogicalProject(col2=[$1], col3=[$2], $f2=[CAST($0):DECIMAL(1000, 500) NOT NULL])", "\n LogicalFilter(condition=[AND(>=($2, 0), =($1, _UTF-8'a'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] } @@ -199,11 +199,11 @@ "\n LogicalJoin(condition=[=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -215,7 +215,7 @@ "\nLogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{1}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -227,7 +227,7 @@ "\nLogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -242,11 +242,11 @@ "\n LogicalJoin(condition=[=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -261,11 +261,11 @@ "\n LogicalJoin(condition=[=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -280,11 +280,11 @@ "\n LogicalJoin(condition=[=($0, $2)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -298,11 +298,11 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -316,11 +316,11 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -334,11 +334,11 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", "\n LogicalJoin(condition=[=($1, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -352,7 +352,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", "\n LogicalJoin(condition=[=($1, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($1, 1)])", @@ -360,7 +360,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[COUNT()])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -376,11 +376,11 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", "\n LogicalJoin(condition=[=($1, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -396,11 +396,11 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($2)])", "\n LogicalJoin(condition=[=($1, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -412,11 +412,11 @@ "\nLogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -428,11 +428,11 @@ "\nLogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -446,11 +446,11 @@ "\n LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)], agg#1=[COUNT()])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -464,11 +464,11 @@ "\n LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($1)])", "\n LogicalJoin(condition=[=($0, $2)], joinType=[semi])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -481,11 +481,11 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalJoin(condition=[=($0, $1)], joinType=[semi])", "\n LogicalProject(col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -499,11 +499,11 @@ "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalJoin(condition=[=($1, $3)], joinType=[semi])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[broadcast], relExchangeType=[PIPELINE_BREAKER])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 0)])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] } diff --git a/pinot-query-planner/src/test/resources/queries/SetOpPlans.json b/pinot-query-planner/src/test/resources/queries/SetOpPlans.json index 0cc8111ec3dc..a43214d75579 100644 --- a/pinot-query-planner/src/test/resources/queries/SetOpPlans.json +++ b/pinot-query-planner/src/test/resources/queries/SetOpPlans.json @@ -9,10 +9,10 @@ "\nLogicalUnion(all=[true])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -26,13 +26,13 @@ "\n LogicalUnion(all=[true])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[c]])", + "\n LogicalTableScan(table=[[default, c]])", "\n" ] }, @@ -49,13 +49,13 @@ "\n LogicalUnion(all=[true])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[c]])", + "\n LogicalTableScan(table=[[default, c]])", "\n" ] }, @@ -69,13 +69,13 @@ "\n LogicalIntersect(all=[false])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[c]])", + "\n LogicalTableScan(table=[[default, c]])", "\n" ] }, @@ -89,13 +89,13 @@ "\n LogicalMinus(all=[false])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[c]])", + "\n LogicalTableScan(table=[[default, c]])", "\n" ] } diff --git a/pinot-query-planner/src/test/resources/queries/WindowFunctionPlans.json b/pinot-query-planner/src/test/resources/queries/WindowFunctionPlans.json index 3b621c386b0b..9b1032d1a3af 100644 --- a/pinot-query-planner/src/test/resources/queries/WindowFunctionPlans.json +++ b/pinot-query-planner/src/test/resources/queries/WindowFunctionPlans.json @@ -10,7 +10,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($0)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -23,7 +23,7 @@ "\n LogicalWindow(window#0=[window(aggs [COUNT()])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(winLiteral=[0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -37,7 +37,7 @@ "\n LogicalWindow(window#0=[window(aggs [COUNT()])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(winLiteral=[0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -50,7 +50,7 @@ "\n LogicalWindow(window#0=[window( rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(winLiteral=[0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -63,7 +63,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -75,7 +75,7 @@ "\nLogicalWindow(window#0=[window( rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -89,7 +89,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -103,7 +103,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($0)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -118,7 +118,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($2)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -133,7 +133,7 @@ "\n LogicalWindow(window#0=[window( rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -149,7 +149,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -165,7 +165,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($2)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -178,7 +178,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($0)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col3=[$2], $1=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -192,7 +192,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[>($2, 10)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -205,7 +205,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[>($2, 10)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -219,7 +219,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col3=[$2], $1=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(=($0, _UTF-8'bar'), =($0, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -234,7 +234,7 @@ "\n LogicalAggregate(group=[{0}])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -249,7 +249,7 @@ "\n LogicalAggregate(group=[{0, 1}])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -264,7 +264,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -280,7 +280,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -298,7 +298,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -316,7 +316,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -329,7 +329,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($1), COUNT($0)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -342,7 +342,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -356,7 +356,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -370,7 +370,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($0), MIN($0)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -385,7 +385,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -401,7 +401,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -417,7 +417,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -430,7 +430,7 @@ "\n LogicalWindow(window#0=[window(aggs [SUM($0), MAX($0)])])", "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col3=[$2], $1=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -444,7 +444,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[>($2, 100)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -458,7 +458,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject(col3=[$2], $1=[LENGTH(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'baz'), <>($0, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -473,7 +473,7 @@ "\n PinotLogicalExchange(distribution=[hash])", "\n LogicalProject($0=[LENGTH(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'baz'), <>($0, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -488,7 +488,7 @@ "\n LogicalAggregate(group=[{0}])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -503,7 +503,7 @@ "\n LogicalAggregate(group=[{0, 1}])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -518,7 +518,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -534,7 +534,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -552,7 +552,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -565,7 +565,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [SUM($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -578,7 +578,7 @@ "\n LogicalWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -592,7 +592,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [SUM($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -605,7 +605,7 @@ "\n LogicalWindow(window#0=[window(partition {1} aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -618,7 +618,7 @@ "\n LogicalWindow(window#0=[window(partition {1} aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -632,7 +632,7 @@ "\n LogicalWindow(window#0=[window(partition {1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -646,7 +646,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [MAX($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -661,7 +661,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [MIN($2)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -677,7 +677,7 @@ "\n LogicalWindow(window#0=[window(partition {1} aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -693,7 +693,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [MIN($2)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -709,7 +709,7 @@ "\n LogicalWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -722,7 +722,7 @@ "\n LogicalWindow(window#0=[window(partition {1} aggs [COUNT($0)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[SUBSTR($0, 0, 2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -735,7 +735,7 @@ "\n LogicalWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col3=[$2], $1=[SUBSTR($0, 0, 2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -749,7 +749,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -763,7 +763,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -776,7 +776,7 @@ "\n LogicalWindow(window#0=[window(partition {1} aggs [SUM($0), COUNT($0)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col3=[$2], $1=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -789,7 +789,7 @@ "\n LogicalWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject($0=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -804,7 +804,7 @@ "\n LogicalAggregate(group=[{0}])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -819,7 +819,7 @@ "\n LogicalAggregate(group=[{0, 1}])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -834,7 +834,7 @@ "\n LogicalAggregate(group=[{0, 1}])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -849,7 +849,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -865,7 +865,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -883,7 +883,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -896,7 +896,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -910,7 +910,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -923,7 +923,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -936,7 +936,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -950,7 +950,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -965,7 +965,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} aggs [SUM($2), MAX($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -981,7 +981,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -997,7 +997,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} aggs [SUM($2), MAX($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1012,7 +1012,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} aggs [SUM($2), MAX($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1027,7 +1027,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1043,7 +1043,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER(), ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1058,7 +1058,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} aggs [SUM($2), SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1073,7 +1073,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1086,7 +1086,7 @@ "\n LogicalWindow(window#0=[window(partition {0} aggs [SUM($1), MAX($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[REVERSE($0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1100,7 +1100,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 42), OR(=($0, _UTF-8'chewbacca':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'vader':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'yoda':VARCHAR(9) CHARACTER SET \"UTF-8\")))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1114,7 +1114,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[REVERSE(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'baz'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1127,7 +1127,7 @@ "\n LogicalWindow(window#0=[window(partition {2} aggs [SUM($1), COUNT($1), COUNT($0)])])", "\n PinotLogicalExchange(distribution=[hash[2]])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1142,7 +1142,7 @@ "\n LogicalAggregate(group=[{0}])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1157,7 +1157,7 @@ "\n LogicalAggregate(group=[{0, 1}])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1172,7 +1172,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1188,7 +1188,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1206,7 +1206,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{2}], agg#0=[$SUM0($2)], agg#1=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1219,7 +1219,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [SUM($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1232,7 +1232,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1246,7 +1246,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [SUM($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1259,7 +1259,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1272,7 +1272,7 @@ "\n LogicalWindow(window#0=[window(order by [1 DESC] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1285,7 +1285,7 @@ "\n LogicalWindow(window#0=[window(order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1300,7 +1300,7 @@ "\n LogicalWindow(window#0=[window(order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1315,7 +1315,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1330,7 +1330,7 @@ "\n LogicalWindow(window#0=[window(order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1345,7 +1345,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1360,7 +1360,7 @@ "\n LogicalWindow(window#0=[window(order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1375,7 +1375,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1390,7 +1390,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1403,7 +1403,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1417,7 +1417,7 @@ "\n LogicalWindow(window#0=[window(order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1431,7 +1431,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1444,7 +1444,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [MAX($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1459,7 +1459,7 @@ "\n LogicalWindow(window#0=[window(order by [0 DESC] aggs [MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1475,7 +1475,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1491,7 +1491,7 @@ "\n LogicalWindow(window#0=[window(order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1507,7 +1507,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1523,7 +1523,7 @@ "\n LogicalWindow(window#0=[window(order by [0 DESC] aggs [MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1539,7 +1539,7 @@ "\n LogicalWindow(window#0=[window(order by [0 DESC] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1552,7 +1552,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [COUNT($0)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[SUBSTR($0, 0, 2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1565,7 +1565,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col3=[$2], $1=[SUBSTR($0, 0, 2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1579,7 +1579,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1592,7 +1592,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1606,7 +1606,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1620,7 +1620,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], $1=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1634,7 +1634,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], $1=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1647,7 +1647,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [SUM($0), COUNT($0)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col3=[$2], $1=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1660,7 +1660,7 @@ "\n LogicalWindow(window#0=[window(order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject($0=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1673,7 +1673,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject($0=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1686,7 +1686,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1699,7 +1699,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [RANK(), DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1713,7 +1713,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1726,7 +1726,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1739,7 +1739,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [DENSE_RANK(), RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1752,7 +1752,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1765,7 +1765,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [DENSE_RANK(), MIN($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1778,7 +1778,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [COUNT($1), MIN($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1791,7 +1791,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [COUNT($1), RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1806,7 +1806,7 @@ "\n LogicalWindow(window#0=[window(order by [1, 0 DESC] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1, 0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1821,7 +1821,7 @@ "\n LogicalWindow(window#0=[window(order by [1, 0 DESC] aggs [RANK(), SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1, 0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1837,7 +1837,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1853,7 +1853,7 @@ "\n LogicalWindow(window#0=[window(order by [1, 0 DESC] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1, 0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1869,7 +1869,7 @@ "\n LogicalWindow(window#0=[window(order by [1, 0 DESC] aggs [DENSE_RANK(), RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1, 0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1882,7 +1882,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [SUM($1), MAX($1)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[REVERSE($0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1896,7 +1896,7 @@ "\n LogicalWindow(window#0=[window(order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], $1=[REVERSE($0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1909,7 +1909,7 @@ "\n LogicalWindow(window#0=[window(order by [0] aggs [DENSE_RANK(), RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], $1=[REVERSE($0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1923,7 +1923,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 42), OR(=($0, _UTF-8'chewbacca':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'vader':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'yoda':VARCHAR(9) CHARACTER SET \"UTF-8\")))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1937,7 +1937,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[REVERSE(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'baz'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1951,7 +1951,7 @@ "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], $1=[REVERSE(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'baz'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1964,7 +1964,7 @@ "\n LogicalWindow(window#0=[window(order by [2] aggs [SUM($1), COUNT($1), COUNT($0)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[2]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1977,7 +1977,7 @@ "\n LogicalWindow(window#0=[window(order by [1] aggs [DENSE_RANK(), COUNT($0)])])", "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], $1=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -1990,7 +1990,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [SUM($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2003,7 +2003,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2017,7 +2017,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [SUM($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2031,7 +2031,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2044,7 +2044,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2057,7 +2057,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2070,7 +2070,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] aggs [DENSE_RANK()])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2083,7 +2083,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2097,7 +2097,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] aggs [RANK()])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2110,7 +2110,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [MAX($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2125,7 +2125,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [MIN($2)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2140,7 +2140,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [DENSE_RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2156,7 +2156,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2172,7 +2172,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [MIN($2)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2188,7 +2188,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2204,7 +2204,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2217,7 +2217,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] aggs [COUNT($0)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[SUBSTR($0, 0, 2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2231,7 +2231,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2244,7 +2244,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2257,7 +2257,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2271,7 +2271,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2285,7 +2285,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], $1=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2298,7 +2298,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1] aggs [SUM($0), COUNT($0)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col3=[$2], $1=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2311,7 +2311,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [DENSE_RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject($0=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2324,7 +2324,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1 DESC] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2337,7 +2337,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1 DESC] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2350,7 +2350,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1 ASC-nulls-first] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2363,7 +2363,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [1 DESC-nulls-last] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2376,7 +2376,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2389,7 +2389,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [DENSE_RANK(), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2403,7 +2403,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2416,7 +2416,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2428,7 +2428,7 @@ "\nLogicalWindow(window#0=[window(partition {0} order by [0] aggs [RANK(), DENSE_RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2441,7 +2441,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2454,7 +2454,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2467,7 +2467,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [COUNT($1), DENSE_RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2482,7 +2482,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [1, 0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2497,7 +2497,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [1, 0] aggs [DENSE_RANK(), RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2513,7 +2513,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2529,7 +2529,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [1, 0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2545,7 +2545,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [1, 0] aggs [RANK(), DENSE_RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2558,7 +2558,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [SUM($1), MAX($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1], col3=[$2], $2=[REVERSE($0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2572,7 +2572,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 42), OR(=($0, _UTF-8'chewbacca':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'vader':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'yoda':VARCHAR(9) CHARACTER SET \"UTF-8\")))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2585,7 +2585,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0])", "\n LogicalFilter(condition=[AND(>($2, 42), OR(=($0, _UTF-8'chewbacca':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'vader':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'yoda':VARCHAR(9) CHARACTER SET \"UTF-8\")))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2599,7 +2599,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[REVERSE(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'baz'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2613,7 +2613,7 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[REVERSE(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'baz'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2626,7 +2626,7 @@ "\n LogicalWindow(window#0=[window(partition {2} order by [2] aggs [SUM($1), COUNT($1), COUNT($0)])])", "\n PinotLogicalExchange(distribution=[hash[2]])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2640,7 +2640,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject($0=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2653,7 +2653,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0] aggs [RANK(), DENSE_RANK()])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject($0=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2666,7 +2666,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0 DESC] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2679,7 +2679,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0 ASC-nulls-first] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2692,7 +2692,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [0 DESC-nulls-last] aggs [SUM($1), COUNT($1), MIN($1)])])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2705,7 +2705,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2718,7 +2718,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2731,7 +2731,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2744,7 +2744,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2757,7 +2757,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2770,7 +2770,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2783,7 +2783,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2796,7 +2796,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2809,7 +2809,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [MAX($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2824,7 +2824,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2839,7 +2839,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2854,7 +2854,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2870,7 +2870,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2886,7 +2886,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2902,7 +2902,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2915,7 +2915,7 @@ "\n LogicalWindow(window#0=[window(partition {2} order by [0] aggs [COUNT($1)])])", "\n PinotLogicalSortExchange(distribution=[hash[2]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2], $3=[SUBSTR($0, 0, 2)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2929,7 +2929,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2943,7 +2943,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", "\n LogicalFilter(condition=[AND(>($2, 10), <=($2, 500))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2957,7 +2957,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2], $3=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2971,7 +2971,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], $2=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2985,7 +2985,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], $2=[CONCAT($0, _UTF-8'-', $1)])", "\n LogicalFilter(condition=[OR(AND(<>($0, _UTF-8'bar'), <>($0, _UTF-8'foo')), >=($2, 42))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -2998,7 +2998,7 @@ "\n LogicalWindow(window#0=[window(partition {2} order by [1] aggs [SUM($0), COUNT($0)])])", "\n PinotLogicalSortExchange(distribution=[hash[2]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col3=[$2], $1=[REVERSE($1)], $2=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3011,7 +3011,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject($0=[REVERSE($1)], $1=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3024,7 +3024,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject($0=[REVERSE($1)], $1=[CONCAT($0, _UTF-8'-', $1)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3037,7 +3037,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [2] aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[2]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3050,7 +3050,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [MAX($1), DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3064,7 +3064,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [2] aggs [MAX($2), COUNT($1)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[2]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3077,7 +3077,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [SUM($2), COUNT($2), MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3090,7 +3090,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [DENSE_RANK(), RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3103,7 +3103,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [SUM($2), COUNT($2), MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3116,7 +3116,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [COUNT($2), MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3129,7 +3129,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [COUNT($2), DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3144,7 +3144,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [2, 0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0, 1]], collation=[[2, 0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3159,7 +3159,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [2, 0] aggs [RANK(), DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[0, 1]], collation=[[2, 0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3175,7 +3175,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1] aggs [SUM($2), COUNT($2), MIN($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3191,7 +3191,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [2, 0] aggs [SUM($2), COUNT($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[0, 1]], collation=[[2, 0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3207,7 +3207,7 @@ "\n LogicalWindow(window#0=[window(partition {0, 1} order by [2, 0] aggs [DENSE_RANK(), RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[0, 1]], collation=[[2, 0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3220,7 +3220,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [0] aggs [SUM($2), MAX($2)])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[0]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2], $3=[REVERSE($0)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3234,7 +3234,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[AND(>($2, 42), OR(=($0, _UTF-8'chewbacca':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'vader':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'yoda':VARCHAR(9) CHARACTER SET \"UTF-8\")))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3248,7 +3248,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", "\n LogicalFilter(condition=[AND(>($2, 42), OR(=($0, _UTF-8'chewbacca':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'vader':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'yoda':VARCHAR(9) CHARACTER SET \"UTF-8\")))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3262,7 +3262,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1])", "\n LogicalFilter(condition=[AND(>($2, 42), OR(=($0, _UTF-8'chewbacca':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'vader':VARCHAR(9) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'yoda':VARCHAR(9) CHARACTER SET \"UTF-8\")))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3276,7 +3276,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2], $3=[REVERSE(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'baz'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3290,7 +3290,7 @@ "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], $2=[REVERSE(CONCAT($0, _UTF-8' ', $1))])", "\n LogicalFilter(condition=[AND(<>($1, _UTF-8'bar'), <>($1, _UTF-8'baz'), <>($1, _UTF-8'foo'))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3303,7 +3303,7 @@ "\n LogicalWindow(window#0=[window(partition {3} order by [2] aggs [SUM($1), COUNT($1), COUNT($0)])])", "\n PinotLogicalSortExchange(distribution=[hash[3]], collation=[[2]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col3=[$2], $2=[CONCAT($0, _UTF-8'-', $1)], $3=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3316,7 +3316,7 @@ "\n LogicalWindow(window#0=[window(partition {2} order by [1] aggs [SUM($0), COUNT($0), RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[2]], collation=[[1]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col3=[$2], $1=[CONCAT($0, _UTF-8'-', $1)], $2=[REVERSE(CONCAT($0, _UTF-8'-', $1))])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3332,10 +3332,10 @@ "\n LogicalJoin(condition=[=($0, $3)], joinType=[inner])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[1]])", "\n LogicalProject(col1=[$0], col2=[$1])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -3350,7 +3350,7 @@ "\n LogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}], agg#0=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3365,7 +3365,7 @@ "\n LogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}], agg#0=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3380,7 +3380,7 @@ "\n LogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}], agg#0=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3395,7 +3395,7 @@ "\n LogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)])", "\n PinotLogicalExchange(distribution=[hash[0, 1]])", "\n LogicalAggregate(group=[{0, 2}], agg#0=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3409,7 +3409,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [2] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[2]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3423,7 +3423,7 @@ "\n LogicalWindow(window#0=[window(partition {1} order by [2] aggs [RANK(), DENSE_RANK()])])", "\n PinotLogicalSortExchange(distribution=[hash[1]], collation=[[2]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3437,7 +3437,7 @@ "\n LogicalWindow(window#0=[window(partition {0} order by [1 DESC] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])])", "\n PinotLogicalSortExchange(distribution=[hash[0]], collation=[[1 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", "\n LogicalProject(col2=[$1], col3=[$2])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3452,7 +3452,7 @@ "\n LogicalAggregate(group=[{0}], agg#0=[COUNT($1)])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalAggregate(group=[{0}], agg#0=[COUNT()])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n" ] }, @@ -3470,11 +3470,11 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[>($2, 100)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[OR(=($0, _UTF-8'brandon sanderson':VARCHAR(17) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'douglas adams':VARCHAR(17) CHARACTER SET \"UTF-8\"))])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, @@ -3492,11 +3492,11 @@ "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col1=[$0], col2=[$1], col3=[$2])", "\n LogicalFilter(condition=[>($2, 100)])", - "\n LogicalTableScan(table=[[a]])", + "\n LogicalTableScan(table=[[default, a]])", "\n PinotLogicalExchange(distribution=[hash[0]])", "\n LogicalProject(col2=[$1])", "\n LogicalFilter(condition=[OR(=($0, _UTF-8'brandon sanderson':VARCHAR(17) CHARACTER SET \"UTF-8\"), =($0, _UTF-8'douglas adams':VARCHAR(17) CHARACTER SET \"UTF-8\"))])", - "\n LogicalTableScan(table=[[b]])", + "\n LogicalTableScan(table=[[default, b]])", "\n" ] }, From 7bd212cad5585a57d06d21afdbe7b77034fb20ab Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 15:30:46 +0530 Subject: [PATCH 22/26] Add cross database query test --- .../MultiStageEngineIntegrationTest.java | 40 +++++++++++++++++-- 1 file changed, 36 insertions(+), 4 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index 5f2c53184282..3cd1ef789819 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -39,6 +39,7 @@ import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.MetricFieldSpec; import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.util.TestUtils; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -50,11 +51,14 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; public class MultiStageEngineIntegrationTest extends BaseClusterIntegrationTestSet { private static final String SCHEMA_FILE_NAME = "On_Time_On_Time_Performance_2014_100k_subset_nonulls.schema"; - private static final String TABLE_NAME_WITH_DATABASE = "db1." + DEFAULT_TABLE_NAME; + private static final String DEFAULT_DATABASE_NAME = CommonConstants.DEFAULT_DATABASE; + private static final String DATABASE_NAME = "db1"; + private static final String TABLE_NAME_WITH_DATABASE = DATABASE_NAME + "." + DEFAULT_TABLE_NAME; private String _tableName = DEFAULT_TABLE_NAME; @Override @@ -804,14 +808,22 @@ public void testWithoutDatabaseContext() public void testWithDefaultDatabaseContextAsTableNamePrefix() throws Exception { // default database check. Default database context passed as table prefix - checkQueryResultForDBTest("ActualElapsedTime", "default." + DEFAULT_TABLE_NAME); + checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_DATABASE_NAME + "." + DEFAULT_TABLE_NAME); } @Test public void testWithDefaultDatabaseContextAsQueryOption() throws Exception { // default database check. Default database context passed as SET database='dbName' - checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_TABLE_NAME, "default"); + checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_TABLE_NAME, DEFAULT_DATABASE_NAME); + } + + @Test + public void testWithDefaultDatabaseContextAsTableNamePrefixAndQueryOption() + throws Exception { + // default database check. Default database context passed as table prefix as well as query option + checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_DATABASE_NAME + "." + DEFAULT_TABLE_NAME, + DEFAULT_DATABASE_NAME); } @Test @@ -825,8 +837,28 @@ public void testWithDatabaseContextAsTableNamePrefix() @Test public void testWithDatabaseContextAsQueryOption() throws Exception { + // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. // custom database check. Database context passed as SET database='dbName' - checkQueryResultForDBTest("ActualElapsedTime_2", DEFAULT_TABLE_NAME, "db1"); + checkQueryResultForDBTest("ActualElapsedTime_2", DEFAULT_TABLE_NAME, DATABASE_NAME); + } + + @Test + public void testWithDatabaseContextAsTableNamePrefixAndQueryOption() + throws Exception { + // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. + // custom database check. Database context passed as table prefix as well as query option + checkQueryResultForDBTest("ActualElapsedTime_2", TABLE_NAME_WITH_DATABASE, DATABASE_NAME); + } + + @Test + public void testCrossDatabaseQuery() + throws Exception { + String query = "SELECT tb1.Carrier, maxTime, distance FROM (SELECT max(AirTime) AS maxTime, Carrier FROM " + + DEFAULT_TABLE_NAME + " GROUP BY Carrier ORDER BY maxTime DESC) AS tb1 JOIN (SELECT sum(Distance) AS distance," + + " Carrier FROM " + TABLE_NAME_WITH_DATABASE + " GROUP BY Carrier) AS tb2 " + + "ON tb1.Carrier = tb2.Carrier; "; + JsonNode result = postQuery(query); + assertEquals(result.get("exceptions").get(0).get("errorCode").asInt(), QueryException.QUERY_PLANNING_ERROR_CODE); } private void checkQueryResultForDBTest(String column, String tableName) From ae665478ad6c7d7bf8ef5b0fe2c28693519d1886 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 16:21:27 +0530 Subject: [PATCH 23/26] Add more tests for database context from header --- .../pinot/integration/tests/ClusterTest.java | 9 ++ .../MultiStageEngineIntegrationTest.java | 100 ++++++++++++++++-- 2 files changed, 100 insertions(+), 9 deletions(-) diff --git a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java index 77e84e1e9129..278f378b1ab7 100644 --- a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java +++ b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java @@ -526,6 +526,15 @@ protected JsonNode postQuery(String query) getExtraQueryProperties()); } + /** + * Queries the broker's sql query endpoint (/query/sql) + */ + protected JsonNode postQuery(String query, Map headers) + throws Exception { + return postQuery(query, getBrokerQueryApiUrl(getBrokerBaseApiUrl(), useMultiStageQueryEngine()), headers, + getExtraQueryProperties()); + } + protected Map getExtraQueryProperties() { return Collections.emptyMap(); } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index 3cd1ef789819..0f5c5f611ee1 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -26,7 +26,9 @@ import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import javax.annotation.Nullable; @@ -51,7 +53,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; public class MultiStageEngineIntegrationTest extends BaseClusterIntegrationTestSet { @@ -818,6 +819,14 @@ public void testWithDefaultDatabaseContextAsQueryOption() checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_TABLE_NAME, DEFAULT_DATABASE_NAME); } + @Test + public void testWithDefaultDatabaseContextAsHttpHeader() + throws Exception { + // default database check. Default database context passed as "database" http header + checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_TABLE_NAME, + Collections.singletonMap(CommonConstants.DATABASE, DEFAULT_DATABASE_NAME)); + } + @Test public void testWithDefaultDatabaseContextAsTableNamePrefixAndQueryOption() throws Exception { @@ -826,12 +835,21 @@ public void testWithDefaultDatabaseContextAsTableNamePrefixAndQueryOption() DEFAULT_DATABASE_NAME); } + @Test + public void testWithDefaultDatabaseContextAsTableNamePrefixAndHttpHeader() + throws Exception { + // default database check. Default database context passed as table prefix as well as http header + checkQueryResultForDBTest("ActualElapsedTime", DEFAULT_DATABASE_NAME + "." + DEFAULT_TABLE_NAME, + Collections.singletonMap(CommonConstants.DATABASE, DEFAULT_DATABASE_NAME)); + } + @Test public void testWithDatabaseContextAsTableNamePrefix() throws Exception { // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. - // custom database check. Database context passed as table prefix - checkQueryResultForDBTest("ActualElapsedTime_2", TABLE_NAME_WITH_DATABASE); + // custom database check. Database context passed only as table prefix. Will + JsonNode result = getQueryResultForDBTest("ActualElapsedTime_2", TABLE_NAME_WITH_DATABASE, null, null); + checkQueryPlanningErrorForDBTest(result); } @Test @@ -842,6 +860,15 @@ public void testWithDatabaseContextAsQueryOption() checkQueryResultForDBTest("ActualElapsedTime_2", DEFAULT_TABLE_NAME, DATABASE_NAME); } + @Test + public void testWithDatabaseContextAsHttpHeader() + throws Exception { + // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. + // custom database check. Database context passed as "database" http header + checkQueryResultForDBTest("ActualElapsedTime_2", DEFAULT_TABLE_NAME, + Collections.singletonMap(CommonConstants.DATABASE, DATABASE_NAME)); + } + @Test public void testWithDatabaseContextAsTableNamePrefixAndQueryOption() throws Exception { @@ -850,6 +877,39 @@ public void testWithDatabaseContextAsTableNamePrefixAndQueryOption() checkQueryResultForDBTest("ActualElapsedTime_2", TABLE_NAME_WITH_DATABASE, DATABASE_NAME); } + @Test + public void testWithDatabaseContextAsTableNamePrefixAndHttpHeader() + throws Exception { + // Using renamed column "ActualElapsedTime_2" to ensure that the same table is not being queried. + // custom database check. Database context passed as table prefix as well as http header + checkQueryResultForDBTest("ActualElapsedTime_2", TABLE_NAME_WITH_DATABASE, + Collections.singletonMap(CommonConstants.DATABASE, DATABASE_NAME)); + } + + @Test + public void testWithConflictingDatabaseContextFromTableNamePrefixAndQueryOption() + throws Exception { + JsonNode result = getQueryResultForDBTest("ActualElapsedTime", TABLE_NAME_WITH_DATABASE, DEFAULT_DATABASE_NAME, + null); + checkQueryPlanningErrorForDBTest(result); + } + + @Test + public void testWithConflictingDatabaseContextFromTableNamePrefixAndHttpHeader() + throws Exception { + JsonNode result = getQueryResultForDBTest("ActualElapsedTime", TABLE_NAME_WITH_DATABASE, null, + Collections.singletonMap(CommonConstants.DATABASE, DEFAULT_DATABASE_NAME)); + checkQueryPlanningErrorForDBTest(result); + } + + @Test + public void testWithConflictingDatabaseContextFromHttpHeaderAndQueryOption() + throws Exception { + JsonNode result = getQueryResultForDBTest("ActualElapsedTime", TABLE_NAME_WITH_DATABASE, DATABASE_NAME, + Collections.singletonMap(CommonConstants.DATABASE, DEFAULT_DATABASE_NAME)); + checkQueryPlanningErrorForDBTest(result); + } + @Test public void testCrossDatabaseQuery() throws Exception { @@ -858,25 +918,47 @@ public void testCrossDatabaseQuery() + " Carrier FROM " + TABLE_NAME_WITH_DATABASE + " GROUP BY Carrier) AS tb2 " + "ON tb1.Carrier = tb2.Carrier; "; JsonNode result = postQuery(query); - assertEquals(result.get("exceptions").get(0).get("errorCode").asInt(), QueryException.QUERY_PLANNING_ERROR_CODE); + checkQueryPlanningErrorForDBTest(result); } private void checkQueryResultForDBTest(String column, String tableName) throws Exception { - checkQueryResultForDBTest(column, tableName, null); + checkQueryResultForDBTest(column, tableName, null, null); } - private void checkQueryResultForDBTest(String column, String tableName, @Nullable String database) + private void checkQueryResultForDBTest(String column, String tableName, Map headers) + throws Exception { + checkQueryResultForDBTest(column, tableName, null, headers); + } + + private void checkQueryResultForDBTest(String column, String tableName, String database) + throws Exception { + checkQueryResultForDBTest(column, tableName, database, null); + } + + private void checkQueryResultForDBTest(String column, String tableName, @Nullable String database, + Map headers) throws Exception { - String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") - + "select max(" + column + ") from " + tableName + ";"; // max value of 'ActualElapsedTime' long expectedValue = 678; - JsonNode jsonNode = postQuery(query); + JsonNode jsonNode = getQueryResultForDBTest(column, tableName, database, headers); long result = jsonNode.get("resultTable").get("rows").get(0).get(0).asLong(); assertEquals(result, expectedValue); } + private void checkQueryPlanningErrorForDBTest(JsonNode queryResult) { + long result = queryResult.get("exceptions").get(0).get("errorCode").asInt(); + assertEquals(result, QueryException.QUERY_PLANNING_ERROR_CODE); + } + + private JsonNode getQueryResultForDBTest(String column, String tableName, @Nullable String database, + Map headers) + throws Exception { + String query = (StringUtils.isNotBlank(database) ? "SET database='" + database + "'; " : "") + + "select max(" + column + ") from " + tableName + ";"; + return postQuery(query, headers); + } + @AfterClass public void tearDown() throws Exception { From 071757d7f11f799c35862c26a5a6e0065916c2b4 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 17:07:37 +0530 Subject: [PATCH 24/26] fix missed LogicalTableScan content updates --- .../integration/tests/OfflineClusterIntegrationTest.java | 4 ++-- .../java/org/apache/pinot/query/QueryCompilationTest.java | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java index 4f55279490d4..8de4713aed41 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java @@ -2910,7 +2910,7 @@ public void testExplainPlanQueryV2() + " LogicalAggregate(group=[{0}], agg#0=[COUNT($1)])\\n" + " PinotLogicalExchange(distribution=[hash[0]])\\n" + " LogicalAggregate(group=[{17}], agg#0=[COUNT()])\\n" - + " LogicalTableScan(table=[[mytable]])\\n" + + " LogicalTableScan(table=[[default, mytable]])\\n" + "\"]]}"); // In the query below, FlightNum column has an inverted index and there is no data satisfying the predicate @@ -2924,7 +2924,7 @@ public void testExplainPlanQueryV2() + "\"Execution Plan.." + "LogicalProject\\(.*\\).." + " LogicalFilter\\(condition=\\[<\\(.*, 0\\)]\\).." - + " LogicalTableScan\\(table=\\[\\[mytable]]\\)..\"" + + " LogicalTableScan\\(table=\\[\\[default, mytable]]\\)..\"" + "]]}"); boolean found = pattern.matcher(response2).find(); assertTrue(found, "Pattern " + pattern + " not found in " + response2); diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java index 82ae97cf7848..dddc422bcdef 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java @@ -429,6 +429,7 @@ private Object[][] provideQueriesWithExplainedLogicalPlan() { + " \"id\": \"0\",\n" + " \"relOp\": \"LogicalTableScan\",\n" + " \"table\": [\n" + + " \"default\"\n" + " \"a\"\n" + " ],\n" + " \"inputs\": []\n" From 2c61d5b2631942d6e4d0163cf5fe859182ce5698 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 17:48:07 +0530 Subject: [PATCH 25/26] typo --- .../test/java/org/apache/pinot/query/QueryCompilationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java index dddc422bcdef..523cd1a7bb59 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java @@ -429,7 +429,7 @@ private Object[][] provideQueriesWithExplainedLogicalPlan() { + " \"id\": \"0\",\n" + " \"relOp\": \"LogicalTableScan\",\n" + " \"table\": [\n" - + " \"default\"\n" + + " \"default\",\n" + " \"a\"\n" + " ],\n" + " \"inputs\": []\n" From 3294fcfd7cdd2194fa049d9a3b6233d9332197e0 Mon Sep 17 00:00:00 2001 From: Shounak kulkarni Date: Thu, 21 Mar 2024 20:16:41 +0530 Subject: [PATCH 26/26] test fix --- .../apache/calcite/prepare/PinotCalciteCatalogReader.java | 3 +-- .../apache/pinot/query/runtime/queries/QueryRunnerTest.java | 6 +++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java b/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java index 84c71be601f0..64d1e54e4db0 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java +++ b/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java @@ -98,8 +98,7 @@ public class PinotCalciteCatalogReader implements Prepare.CatalogReader { public PinotCalciteCatalogReader(CalciteSchema rootSchema, List defaultSchema, RelDataTypeFactory typeFactory, CalciteConnectionConfig config) { this(rootSchema, SqlNameMatchers.withCaseSensitive(config != null && config.caseSensitive()), - ImmutableList.of(Objects.requireNonNull(defaultSchema, "defaultSchema"), - ImmutableList.of()), + ImmutableList.of(Objects.requireNonNull(defaultSchema, "defaultSchema")), typeFactory, config); } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java index b4f174eb7f86..0aea318c12a4 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java @@ -89,13 +89,13 @@ public void setUp() factory1.registerTable(SCHEMA_BUILDER.setSchemaName("b").build(), "b_REALTIME"); factory1.registerTable(SCHEMA_BUILDER.setSchemaName("c").build(), "c_OFFLINE"); factory1.registerTable(SCHEMA_BUILDER.setSchemaName("d").build(), "d"); - factory1.registerTable(SCHEMA_BUILDER.setSchemaName("tbl-escape.naming").build(), "tbl-escape.naming_OFFLINE"); + factory1.registerTable(SCHEMA_BUILDER.setSchemaName("tbl-escape-naming").build(), "tbl-escape-naming_OFFLINE"); factory1.addSegment("a_REALTIME", buildRows("a_REALTIME")); factory1.addSegment("a_REALTIME", buildRows("a_REALTIME")); factory1.addSegment("b_REALTIME", buildRows("b_REALTIME")); factory1.addSegment("c_OFFLINE", buildRows("c_OFFLINE")); factory1.addSegment("d_OFFLINE", buildRows("d_OFFLINE")); - factory1.addSegment("tbl-escape.naming_OFFLINE", buildRows("tbl-escape.naming_OFFLINE")); + factory1.addSegment("tbl-escape-naming_OFFLINE", buildRows("tbl-escape-naming_OFFLINE")); MockInstanceDataManagerFactory factory2 = new MockInstanceDataManagerFactory("server2"); factory2.registerTable(SCHEMA_BUILDER.setSchemaName("a").build(), "a_REALTIME"); @@ -258,7 +258,7 @@ private Object[][] provideTestSqlAndRowCount() { + "col1, COUNT(*) FROM a GROUP BY col1", 2}, new Object[]{"SET numGroupsLimit = 2; SELECT /*+ aggOptions(num_groups_limit='1') */ " + "a.col2, b.col2, COUNT(*) FROM a JOIN b USING (col1) GROUP BY a.col2, b.col2", 1}, - new Object[]{"SELECT * FROM \"tbl-escape.naming\"", 5} + new Object[]{"SELECT * FROM \"default.tbl-escape-naming\"", 5} }; }