You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "shounakmk219 (via GitHub)" <gi...@apache.org> on 2024/02/14 07:19:07 UTC

[PR] Allow passing database context through `database` http header [pinot]

shounakmk219 opened a new pull request, #12417:
URL: https://github.com/apache/pinot/pull/12417

   # Description
   This PR aims to allow a way of passing the database context along with the API requests without disrupting the existing APIs.
   The database context will be passed as a `database` http header along with the API requests. It is part of the effort to support database concept in Pinot #12333 
   
   To avoid writing the logic for the translation `databaseName` + `tableName` -> `databaseName.tableName` for each API this PR also introduces a request filter for controller and broker endpoints which pre computes the fully qualified table name for the APIs, only condition being that the tableName must be a query param.
   APIs with tableName input as path param and request body will still need to do the translation on their own.
   
   In order to reduce the explicit handling of table translation on each API, this PR also introduces a set of V2 endpoints for APIs which take tableName input as path param and request body. More details [here](https://docs.google.com/document/d/1l1dco4lRhB28y-yKMVKlOMZ0sdFKObjJfhAjG6cmktU/edit#heading=h.plsbi2ybsb2b)
   Any new API that get introduced and needs table name as input should follow this convention for table name input.
   
   # Release Notes
   
   - Allow passing database name context through `database` http header
   - V2 endpoints introduced which take table name input as query param


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "klsince (via GitHub)" <gi...@apache.org>.
klsince commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1503447394


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));

Review Comment:
   call this setRequestUri after finishing the for-loop?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -1660,10 +1640,25 @@ static String getActualColumnName(String rawTableName, String columnName, @Nulla
       return columnName;
     }
     String columnNameToCheck;
-    if (columnName.regionMatches(ignoreCase, 0, rawTableName, 0, rawTableName.length())
-        && columnName.length() > rawTableName.length() && columnName.charAt(rawTableName.length()) == '.') {
-      columnNameToCheck = ignoreCase ? columnName.substring(rawTableName.length() + 1).toLowerCase()
-          : columnName.substring(rawTableName.length() + 1);
+    String resolvedColumnName = columnName;
+    if (rawTableName.contains(".")) { // table name has database prefix
+      String databaseName = rawTableName.split("\\.")[0];

Review Comment:
   What if database, table or column name has '.' in its name? or is it allowed to have '.'. I think currently there is no sanity check to block this. The previous logic above used the name.length() to split.



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {
+        tableName = String.format("%s.%s", databaseName, tableName);
+      }
+    }
+    String actualTableName = null;
+    if (tableCache != null) {
+      actualTableName = tableCache.getActualTableName(tableName);
+    }
+    return actualTableName != null ? actualTableName : tableName;
+  }
+
+  public static boolean isTableNameEquivalent(String name1, String name2) {
+    return Objects.equals(name1, name2) || name1.endsWith("." + name2) || name2.endsWith("." + name1);
+  }

Review Comment:
   looks like the logic heavily assumes the names don't contain `.`, but would this assumption be held true?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1521136795


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableConfigsRestletResource.java:
##########
@@ -322,26 +322,25 @@ public ConfigSuccessResponse updateConfig(
       @ApiParam(value = "Force update the table schema") @DefaultValue("false")
       @QueryParam("forceTableSchemaUpdate") boolean forceTableSchemaUpdate,
       String tableConfigsStr, @Context HttpHeaders headers) throws Exception {
+    tableName = DatabaseUtils.translateTableName(tableName, headers);
     Pair<TableConfigs, Map<String, Object>> tableConfigsAndUnrecognizedProps;
     TableConfigs tableConfigs;
     String translatedTableName;
     try {
-      translatedTableName = DatabaseUtils.translateTableName(tableName, headers);
       tableConfigsAndUnrecognizedProps =
           JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigsStr, TableConfigs.class);
       tableConfigs = tableConfigsAndUnrecognizedProps.getLeft();
-      Preconditions.checkState(
-          DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers).equals(translatedTableName),
-          "Table name mismatch: %s is not equivalent to %s", tableConfigs.getTableName(), tableName);
+      tableConfigs.setTableName(DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers));

Review Comment:
   setting the table name here will make all the validations on table/schema names redundant which are part of `validateConfig(tableConfigs, typesToSkip);` called below. This in a way allows user to pass invalid schema or realtime/offline table names as ultimately it gets overridden by the raw table name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1503667801


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -1660,10 +1640,25 @@ static String getActualColumnName(String rawTableName, String columnName, @Nulla
       return columnName;
     }
     String columnNameToCheck;
-    if (columnName.regionMatches(ignoreCase, 0, rawTableName, 0, rawTableName.length())
-        && columnName.length() > rawTableName.length() && columnName.charAt(rawTableName.length()) == '.') {
-      columnNameToCheck = ignoreCase ? columnName.substring(rawTableName.length() + 1).toLowerCase()
-          : columnName.substring(rawTableName.length() + 1);
+    String resolvedColumnName = columnName;
+    if (rawTableName.contains(".")) { // table name has database prefix
+      String databaseName = rawTableName.split("\\.")[0];

Review Comment:
   As for the sanity check, the actual database name or logical table name will not have '.' as that's ensured during table creation by `TableConfigUtils.validateTableName`.
   But I get your point for the column name with '.' case. Will update the condition logic for `if column name only has table prefix` case.
   Nice catch!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #12417:
URL: https://github.com/apache/pinot/pull/12417#issuecomment-1943241974

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   Attention: `54 lines` in your changes are missing coverage. Please review.
   > Comparison is base [(`04b279e`)](https://app.codecov.io/gh/apache/pinot/commit/04b279e8dae03d46a01ed1587ae335813bcc176f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 61.70% compared to head [(`a29f5f5`)](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 46.68%.
   > Report is 6 commits behind head on master.
   
   | [Files](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Patch % | Lines |
   |---|---|---|
   | [...a/org/apache/pinot/common/utils/DatabaseUtils.java](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvRGF0YWJhc2VVdGlscy5qYXZh) | 0.00% | [35 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [...pache/pinot/common/config/provider/TableCache.java](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL3Byb3ZpZGVyL1RhYmxlQ2FjaGUuamF2YQ==) | 0.00% | [10 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [...java/org/apache/pinot/spi/config/TableConfigs.java](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL1RhYmxlQ29uZmlncy5qYXZh) | 0.00% | [7 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [...org/apache/pinot/spi/config/table/TableConfig.java](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1RhYmxlQ29uZmlnLmphdmE=) | 33.33% | [2 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #12417       +/-   ##
   =============================================
   - Coverage     61.70%   46.68%   -15.03%     
   - Complexity      207      948      +741     
   =============================================
     Files          2428     1830      -598     
     Lines        132823    96596    -36227     
     Branches      20544    15658     -4886     
   =============================================
   - Hits          81958    45092    -36866     
   - Misses        44846    48268     +3422     
   + Partials       6019     3236     -2783     
   ```
   
   | [Flag](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flags&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [custom-integration1](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration1](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration2](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [java-11](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [java-21](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.68% <1.81%> (-14.91%)` | :arrow_down: |
   | [skip-bytebuffers-false](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.68% <1.81%> (-15.01%)` | :arrow_down: |
   | [skip-bytebuffers-true](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [temurin](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.68% <1.81%> (-15.03%)` | :arrow_down: |
   | [unittests](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.68% <1.81%> (-15.02%)` | :arrow_down: |
   | [unittests1](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.68% <1.81%> (-0.21%)` | :arrow_down: |
   | [unittests2](https://app.codecov.io/gh/apache/pinot/pull/12417/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/pinot/pull/12417?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1503669959


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));

Review Comment:
   Makes sense, will fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1506102610


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1508053974


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerRouting.java:
##########
@@ -61,8 +69,23 @@ public class PinotBrokerRouting {
       @ApiResponse(code = 200, message = "Success"),
       @ApiResponse(code = 500, message = "Internal server error")
   })
-  public String buildRouting(
+  public String buildRouting(@Context HttpHeaders headers,

Review Comment:
   (nit, convention) We usually put this as the last argument



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1516957220


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -709,68 +709,131 @@ public List<String> getAllResources() {
   }
 
   /**
-   * Get all table names (with type suffix).
+   * Get all table names (with type suffix) in default database.
    *
-   * @return List of table names
+   * @return List of table names in default database
    */
   public List<String> getAllTables() {
+    return getAllTables(null);
+  }
+
+  /**
+   * Get all table names (with type suffix) from provided database.
+   *
+   * @param databaseName database name
+   * @return List of table names in provided database name
+   */
+  public List<String> getAllTables(String databaseName) {
     List<String> tableNames = new ArrayList<>();
     for (String resourceName : getAllResources()) {
-      if (TableNameBuilder.isTableResource(resourceName)) {
+      if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) {
         tableNames.add(resourceName);
       }
     }
     return tableNames;
   }
 
+  private boolean isPartOfDatabase(String tableName, String databaseName) {
+    if (databaseName == null) {

Review Comment:
   Consider allowing empty database name



##########
pinot-common/src/main/java/org/apache/pinot/common/config/provider/TableCache.java:
##########
@@ -71,6 +73,8 @@ public class TableCache implements PinotConfigProvider {
   private static final String LOWER_CASE_OFFLINE_TABLE_SUFFIX = OFFLINE_TABLE_SUFFIX.toLowerCase();
   private static final String LOWER_CASE_REALTIME_TABLE_SUFFIX = REALTIME_TABLE_SUFFIX.toLowerCase();
 
+  private static final String DEFAULT_DATABASE_PREFIX = CommonConstants.DEFAULT_DATABASE + ".";

Review Comment:
   I don't fully follow the logic here. Ideally `DatabaseUtils` should be able to trim off the `default.` prefix, and we should never see it within this class



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -709,68 +709,131 @@ public List<String> getAllResources() {
   }
 
   /**
-   * Get all table names (with type suffix).
+   * Get all table names (with type suffix) in default database.
    *
-   * @return List of table names
+   * @return List of table names in default database
    */
   public List<String> getAllTables() {
+    return getAllTables(null);
+  }
+
+  /**
+   * Get all table names (with type suffix) from provided database.
+   *
+   * @param databaseName database name
+   * @return List of table names in provided database name
+   */
+  public List<String> getAllTables(String databaseName) {

Review Comment:
   Same for other places
   ```suggestion
     public List<String> getAllTables(@Nullable String databaseName) {
   ```



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import com.google.common.base.Preconditions;
+import javax.annotation.Nullable;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  /**
+   * Construct the fully qualified table name i.e. {databaseName}.{tableName} from given table name and database name
+   * @param tableName table/schema name
+   * @param databaseName database name
+   * @return translated table name. Throws {@link IllegalArgumentException} if {@code tableName} contains
+   * more than 1 dot or if {@code tableName} has database prefix, and it does not match with {@code databaseName}
+   */
+  public static String translateTableName(String tableName, @Nullable String databaseName) {
+    Preconditions.checkArgument(tableName != null, "'tableName' cannot be null");
+    String[] tableSplit = StringUtils.split(tableName, '.');
+    switch (tableSplit.length) {
+      case 1:
+        // do not concat the database name prefix if it's a 'default' database
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+          return databaseName + "." + tableName;
+        }
+        return tableName;
+      case 2:
+        String databasePrefix = tableSplit[0];
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equals(databasePrefix)) {
+          throw new IllegalArgumentException("Database name '" + databasePrefix

Review Comment:
   (minor) Use `Preconditions.checkArgument()`



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -709,68 +709,131 @@ public List<String> getAllResources() {
   }
 
   /**
-   * Get all table names (with type suffix).
+   * Get all table names (with type suffix) in default database.
    *
-   * @return List of table names
+   * @return List of table names in default database
    */
   public List<String> getAllTables() {
+    return getAllTables(null);
+  }
+
+  /**
+   * Get all table names (with type suffix) from provided database.
+   *
+   * @param databaseName database name
+   * @return List of table names in provided database name
+   */
+  public List<String> getAllTables(String databaseName) {
     List<String> tableNames = new ArrayList<>();
     for (String resourceName : getAllResources()) {
-      if (TableNameBuilder.isTableResource(resourceName)) {
+      if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) {
         tableNames.add(resourceName);
       }
     }
     return tableNames;
   }
 
+  private boolean isPartOfDatabase(String tableName, String databaseName) {
+    if (databaseName == null) {
+      return StringUtils.split(tableName, '.').length == 1;
+    } else if (databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+      String[] split = StringUtils.split(tableName, '.');
+      return split.length == 1 || split[0].equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE);

Review Comment:
   To simplify the handling, we should prevent table with `default.` prefix. Table should either have non-default database prefix, or no prefix



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  /**
+   * Construct the fully qualified table name i.e. {databaseName}.{tableName} from given table name and database name
+   * @param tableName table/schema name
+   * @param databaseName database name
+   * @return translated table name. Throws {@link IllegalStateException} if {@code tableName} contains more than 1 dot
+   * or if {@code tableName} has database prefix, and it does not match with {@code databaseName}
+   */
+  public static String translateTableName(String tableName, @Nullable String databaseName) {
+    if (tableName == null) {
+      throw new IllegalArgumentException("'tableName' cannot be null");
+    }
+    String[] tableSplit = StringUtils.split(tableName, '.');
+    switch (tableSplit.length) {
+      case 1:
+        // do not concat the database name prefix if it's a 'default' database
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+          return String.format("%s.%s", databaseName, tableName);
+        }
+        return tableName;
+      case 2:
+        String databasePrefix = tableSplit[0];
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equals(databasePrefix)) {
+          throw new IllegalArgumentException("Database name '" + databasePrefix
+              + "' from table prefix does not match database name '" + databaseName + "' from header");
+        }
+        // skip database name prefix if it's a 'default' database
+        return databasePrefix.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE) ? tableSplit[1] : tableName;
+      default:
+      throw new IllegalArgumentException("Table name: '" + tableName + "' containing more than one '.' is not allowed");

Review Comment:
   ^^



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on PR #12417:
URL: https://github.com/apache/pinot/pull/12417#issuecomment-1968346227

   > Can we also process the path parameter during the request filtering? IIRC there is a filter that can be applied after matching the path, and I don't like the idea of adding all these v2 APIs
   
   Hey @Jackie-Jiang we can't update the URI in post match phase and path param context is only available there ([doc ref](https://docs.google.com/document/d/1l1dco4lRhB28y-yKMVKlOMZ0sdFKObjJfhAjG6cmktU/edit#heading=h.o32s8vubl0gc)).
   Due to this limitation we are forced to manually handle the translation for the endpoints having table name as path param. Intent of the v2 APIs is to discourage the table name input as path param and eventually deprecate it so that we have a centralised place where translation is performed and avoid the maintenance overhead around it.
   Let me know your thoughts.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1513822889


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -320,6 +321,11 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S
         // Compile the request into PinotQuery
         compilationStartTimeNs = System.nanoTime();
         pinotQuery = CalciteSqlParser.compileToPinotQuery(sqlNodeAndOptions);
+        if (pinotQuery.getDataSource() != null && pinotQuery.getDataSource().getTableName() != null) {
+          String tableName = getActualTableName(DatabaseUtils.translateTableName(
+              pinotQuery.getDataSource().getTableName(), httpHeaders), _tableCache);
+        pinotQuery.getDataSource().setTableName(tableName);
+        }

Review Comment:
   Revert this? Table name parsing is handled below



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -1652,6 +1648,7 @@ private static void fixColumnName(String rawTableName, Expression expression, Ma
    * Returns the actual column name for the given column name for:
    * - Case-insensitive cluster
    * - Column name in the format of [table_name].[column_name]
+   * - Column name in the format of [database_name].[table_name].[column_name]

Review Comment:
   I don't think we need to support such complicated column name (at least for database isolation purpose)



##########
pinot-common/src/main/java/org/apache/pinot/common/config/provider/TableCache.java:
##########
@@ -71,6 +73,8 @@ public class TableCache implements PinotConfigProvider {
   private static final String LOWER_CASE_OFFLINE_TABLE_SUFFIX = OFFLINE_TABLE_SUFFIX.toLowerCase();
   private static final String LOWER_CASE_REALTIME_TABLE_SUFFIX = REALTIME_TABLE_SUFFIX.toLowerCase();
 
+  private static final String DEFAULT_DATABASE_PREFIX = CommonConstants.DEFAULT_DATABASE + ".";

Review Comment:
   Are the changes in this class required?



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  /**
+   * Construct the fully qualified table name i.e. {databaseName}.{tableName} from given table name and database name
+   * @param tableName table/schema name
+   * @param databaseName database name
+   * @return translated table name. Throws {@link IllegalStateException} if {@code tableName} contains more than 1 dot
+   * or if {@code tableName} has database prefix, and it does not match with {@code databaseName}
+   */
+  public static String translateTableName(String tableName, @Nullable String databaseName) {
+    if (tableName == null) {
+      throw new IllegalArgumentException("'tableName' cannot be null");
+    }
+    String[] tableSplit = StringUtils.split(tableName, '.');
+    switch (tableSplit.length) {
+      case 1:
+        // do not concat the database name prefix if it's a 'default' database
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {

Review Comment:
   Why skipping "default"? There shouldn't be default concept



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  /**
+   * Construct the fully qualified table name i.e. {databaseName}.{tableName} from given table name and database name
+   * @param tableName table/schema name
+   * @param databaseName database name
+   * @return translated table name. Throws {@link IllegalStateException} if {@code tableName} contains more than 1 dot
+   * or if {@code tableName} has database prefix, and it does not match with {@code databaseName}
+   */
+  public static String translateTableName(String tableName, @Nullable String databaseName) {
+    if (tableName == null) {
+      throw new IllegalArgumentException("'tableName' cannot be null");
+    }
+    String[] tableSplit = StringUtils.split(tableName, '.');
+    switch (tableSplit.length) {
+      case 1:
+        // do not concat the database name prefix if it's a 'default' database
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+          return String.format("%s.%s", databaseName, tableName);

Review Comment:
   (minor) Simple concatenation should be faster: `databaseName + "." + tableName`



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  /**
+   * Construct the fully qualified table name i.e. {databaseName}.{tableName} from given table name and database name
+   * @param tableName table/schema name
+   * @param databaseName database name
+   * @return translated table name. Throws {@link IllegalStateException} if {@code tableName} contains more than 1 dot
+   * or if {@code tableName} has database prefix, and it does not match with {@code databaseName}
+   */
+  public static String translateTableName(String tableName, @Nullable String databaseName) {
+    if (tableName == null) {
+      throw new IllegalArgumentException("'tableName' cannot be null");
+    }

Review Comment:
   (minor) 
   ```suggestion
       Preconditions.checkArgument(tableName != null, "'tableName' cannot be null");
   ```



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  /**
+   * Construct the fully qualified table name i.e. {databaseName}.{tableName} from given table name and database name
+   * @param tableName table/schema name
+   * @param databaseName database name
+   * @return translated table name. Throws {@link IllegalStateException} if {@code tableName} contains more than 1 dot
+   * or if {@code tableName} has database prefix, and it does not match with {@code databaseName}
+   */
+  public static String translateTableName(String tableName, @Nullable String databaseName) {
+    if (tableName == null) {
+      throw new IllegalArgumentException("'tableName' cannot be null");
+    }
+    String[] tableSplit = StringUtils.split(tableName, '.');
+    switch (tableSplit.length) {
+      case 1:
+        // do not concat the database name prefix if it's a 'default' database
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+          return String.format("%s.%s", databaseName, tableName);
+        }
+        return tableName;
+      case 2:
+        String databasePrefix = tableSplit[0];
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equals(databasePrefix)) {
+          throw new IllegalArgumentException("Database name '" + databasePrefix
+              + "' from table prefix does not match database name '" + databaseName + "' from header");
+        }
+        // skip database name prefix if it's a 'default' database
+        return databasePrefix.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE) ? tableSplit[1] : tableName;
+      default:
+      throw new IllegalArgumentException("Table name: '" + tableName + "' containing more than one '.' is not allowed");

Review Comment:
   (minor) reformat



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {
+        tableName = String.format("%s.%s", databaseName, tableName);
+      }
+    }
+    String actualTableName = null;
+    if (tableCache != null) {
+      actualTableName = tableCache.getActualTableName(tableName);
+    }
+    return actualTableName != null ? actualTableName : tableName;
+  }
+
+  public static boolean isTableNameEquivalent(String name1, String name2) {

Review Comment:
   Is this still in use?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1505491708


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {
+        tableName = String.format("%s.%s", databaseName, tableName);
+      }
+    }
+    String actualTableName = null;
+    if (tableCache != null) {
+      actualTableName = tableCache.getActualTableName(tableName);

Review Comment:
   Oh this was done to handle the tables under `default` database as those tables don't have any database prefix in their physical table names but we must also allow table name inputs with `default.` prefix.
   I didn't think of the case sensitivity handling as I assumed it will be governed by `enable.case.insensitive`. 
   Will refactor the util to only focus on the default database case.
   Thanks for catching it!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1508834724


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerRouting.java:
##########
@@ -61,8 +69,23 @@ public class PinotBrokerRouting {
       @ApiResponse(code = 200, message = "Success"),
       @ApiResponse(code = 500, message = "Internal server error")
   })
-  public String buildRouting(
+  public String buildRouting(@Context HttpHeaders headers,

Review Comment:
   Noted. Fixed it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1514109569


##########
pinot-common/src/main/java/org/apache/pinot/common/config/provider/TableCache.java:
##########
@@ -71,6 +73,8 @@ public class TableCache implements PinotConfigProvider {
   private static final String LOWER_CASE_OFFLINE_TABLE_SUFFIX = OFFLINE_TABLE_SUFFIX.toLowerCase();
   private static final String LOWER_CASE_REALTIME_TABLE_SUFFIX = REALTIME_TABLE_SUFFIX.toLowerCase();
 
+  private static final String DEFAULT_DATABASE_PREFIX = CommonConstants.DEFAULT_DATABASE + ".";

Review Comment:
   They are required to handle the `actualTableNameWithoutDot` and `default.actualTableNameWithoutDot` table name equivalence.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1517978582


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -709,68 +709,131 @@ public List<String> getAllResources() {
   }
 
   /**
-   * Get all table names (with type suffix).
+   * Get all table names (with type suffix) in default database.
    *
-   * @return List of table names
+   * @return List of table names in default database
    */
   public List<String> getAllTables() {
+    return getAllTables(null);
+  }
+
+  /**
+   * Get all table names (with type suffix) from provided database.
+   *
+   * @param databaseName database name
+   * @return List of table names in provided database name
+   */
+  public List<String> getAllTables(String databaseName) {
     List<String> tableNames = new ArrayList<>();
     for (String resourceName : getAllResources()) {
-      if (TableNameBuilder.isTableResource(resourceName)) {
+      if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) {
         tableNames.add(resourceName);
       }
     }
     return tableNames;
   }
 
+  private boolean isPartOfDatabase(String tableName, String databaseName) {
+    if (databaseName == null) {
+      return StringUtils.split(tableName, '.').length == 1;
+    } else if (databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+      String[] split = StringUtils.split(tableName, '.');
+      return split.length == 1 || split[0].equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE);

Review Comment:
   Agree. The APIs will ensure that constraint going forward. Should we drop the ball on existing tables with `default.` prefix?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "xiangfu0 (via GitHub)" <gi...@apache.org>.
xiangfu0 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1494739384


##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BaseBrokerStarter.java:
##########
@@ -420,6 +422,12 @@ public void start()
    * @param brokerAdminApplication is the application
    */
   protected void registerExtraComponents(BrokerAdminApiApplication brokerAdminApplication) {
+    brokerAdminApplication.register(new AbstractBinder() {

Review Comment:
   I thought this is for override?
   Shall we move this `bind(_tableCache).to(TableCache.class);` to `start()` method.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BaseBrokerStarter.java:
##########
@@ -420,6 +422,12 @@ public void start()
    * @param brokerAdminApplication is the application
    */
   protected void registerExtraComponents(BrokerAdminApiApplication brokerAdminApplication) {
+    brokerAdminApplication.register(new AbstractBinder() {

Review Comment:
   I thought this is for override?
   Shall we move this `bind(_tableCache).to(TableCache.class);` to `start()` method?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1517986003


##########
pinot-common/src/main/java/org/apache/pinot/common/config/provider/TableCache.java:
##########
@@ -71,6 +73,8 @@ public class TableCache implements PinotConfigProvider {
   private static final String LOWER_CASE_OFFLINE_TABLE_SUFFIX = OFFLINE_TABLE_SUFFIX.toLowerCase();
   private static final String LOWER_CASE_REALTIME_TABLE_SUFFIX = REALTIME_TABLE_SUFFIX.toLowerCase();
 
+  private static final String DEFAULT_DATABASE_PREFIX = CommonConstants.DEFAULT_DATABASE + ".";

Review Comment:
   iirc this was preemptively introduced to make life easy for v2 engine. But we anyways need to handle v2 engine part separately so I'll remove this part form here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #12417:
URL: https://github.com/apache/pinot/pull/12417


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1521136795


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableConfigsRestletResource.java:
##########
@@ -322,26 +322,25 @@ public ConfigSuccessResponse updateConfig(
       @ApiParam(value = "Force update the table schema") @DefaultValue("false")
       @QueryParam("forceTableSchemaUpdate") boolean forceTableSchemaUpdate,
       String tableConfigsStr, @Context HttpHeaders headers) throws Exception {
+    tableName = DatabaseUtils.translateTableName(tableName, headers);
     Pair<TableConfigs, Map<String, Object>> tableConfigsAndUnrecognizedProps;
     TableConfigs tableConfigs;
     String translatedTableName;
     try {
-      translatedTableName = DatabaseUtils.translateTableName(tableName, headers);
       tableConfigsAndUnrecognizedProps =
           JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigsStr, TableConfigs.class);
       tableConfigs = tableConfigsAndUnrecognizedProps.getLeft();
-      Preconditions.checkState(
-          DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers).equals(translatedTableName),
-          "Table name mismatch: %s is not equivalent to %s", tableConfigs.getTableName(), tableName);
+      tableConfigs.setTableName(DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers));

Review Comment:
   setting the table name here will make all the validations on table/schema names redundant which are part of `validateConfig(tableConfigs, typesToSkip);` called below. This in a way will allow user to pass invalid schema or realtime/offline table names as ultimately it gets overridden by the translated raw table name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1514487635


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {
+        tableName = String.format("%s.%s", databaseName, tableName);
+      }
+    }
+    String actualTableName = null;
+    if (tableCache != null) {
+      actualTableName = tableCache.getActualTableName(tableName);
+    }
+    return actualTableName != null ? actualTableName : tableName;
+  }
+
+  public static boolean isTableNameEquivalent(String name1, String name2) {

Review Comment:
   Got rid of it. Now the expectation is user should either have translated table/schema names in the payload or logical table/schema names with the proper database header. Partially translated names i.e. table name is translated but schema name is not translated in TableConfigs payload, will throw error.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1514104794


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -1652,6 +1648,7 @@ private static void fixColumnName(String rawTableName, Expression expression, Ma
    * Returns the actual column name for the given column name for:
    * - Case-insensitive cluster
    * - Column name in the format of [table_name].[column_name]
+   * - Column name in the format of [database_name].[table_name].[column_name]

Review Comment:
   Ack. Will skip this handling.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1508835134


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   Agree, made the changes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1503661878


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -1660,10 +1640,25 @@ static String getActualColumnName(String rawTableName, String columnName, @Nulla
       return columnName;
     }
     String columnNameToCheck;
-    if (columnName.regionMatches(ignoreCase, 0, rawTableName, 0, rawTableName.length())
-        && columnName.length() > rawTableName.length() && columnName.charAt(rawTableName.length()) == '.') {
-      columnNameToCheck = ignoreCase ? columnName.substring(rawTableName.length() + 1).toLowerCase()
-          : columnName.substring(rawTableName.length() + 1);
+    String resolvedColumnName = columnName;
+    if (rawTableName.contains(".")) { // table name has database prefix
+      String databaseName = rawTableName.split("\\.")[0];

Review Comment:
   Previous logic is still valid and present at new line # 1657. 
   This code is only trying to handle the case when user queries something like `select tb1.col1 from db1.tb1` or `select tb1.col1 from tb1` with right database header. In such cases we need to prefix the whole column literal with the database name as the following logic is based on rawTableName which is the physical table name and will have database prefix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1521129920


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -1617,9 +1619,9 @@ public List<String> getSchemaNames() {
   public List<String> getSchemaNames(@Nullable String databaseName) {
     List<String> schemas = _propertyStore.getChildNames(
         PinotHelixPropertyStoreZnRecordProvider.forSchema(_propertyStore).getRelativePath(), AccessOption.PERSISTENT);
-    if (schemas != null) {
-        return schemas.stream().filter(schemaName -> isPartOfDatabase(schemaName, databaseName))
-            .collect(Collectors.toList());
+    if (databaseName != null) {

Review Comment:
   This will return all schemas when `databaseName` is `null`, shouldn't the `null` database name be considered as the `default` database and only return schemas under the `default` database?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableConfigsRestletResource.java:
##########
@@ -322,26 +322,25 @@ public ConfigSuccessResponse updateConfig(
       @ApiParam(value = "Force update the table schema") @DefaultValue("false")
       @QueryParam("forceTableSchemaUpdate") boolean forceTableSchemaUpdate,
       String tableConfigsStr, @Context HttpHeaders headers) throws Exception {
+    tableName = DatabaseUtils.translateTableName(tableName, headers);
     Pair<TableConfigs, Map<String, Object>> tableConfigsAndUnrecognizedProps;
     TableConfigs tableConfigs;
     String translatedTableName;
     try {
-      translatedTableName = DatabaseUtils.translateTableName(tableName, headers);
       tableConfigsAndUnrecognizedProps =
           JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigsStr, TableConfigs.class);
       tableConfigs = tableConfigsAndUnrecognizedProps.getLeft();
-      Preconditions.checkState(
-          DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers).equals(translatedTableName),
-          "Table name mismatch: %s is not equivalent to %s", tableConfigs.getTableName(), tableName);
+      tableConfigs.setTableName(DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers));

Review Comment:
   setting the table name here will make all the validations on table/schema names redundant which are part of `validateConfig(tableConfigs, typesToSkip);` called below



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1497355320


##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BaseBrokerStarter.java:
##########
@@ -420,6 +422,12 @@ public void start()
    * @param brokerAdminApplication is the application
    */
   protected void registerExtraComponents(BrokerAdminApiApplication brokerAdminApplication) {
+    brokerAdminApplication.register(new AbstractBinder() {

Review Comment:
   Got it. Moved it to start() above.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1505457039


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   Not in current usage but intent is to return `null` if `null` tableName is passed and avoid throwing any exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1505051848


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   (minor) Can `tableName` ever be null?



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");

Review Comment:
   Use `StringUtils.split(tableName, ',')` instead to avoid regex matching. Same for other places



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   This logic doesn't seem correct. When `database` is provided, why are we still trying to split the table name? What if the provided `database` doesn't match the table prefix?



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {

Review Comment:
   Please add some javadoc on the expected behavior of this method. Same for other public methods



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {
+        tableName = String.format("%s.%s", databaseName, tableName);
+      }
+    }
+    String actualTableName = null;
+    if (tableCache != null) {
+      actualTableName = tableCache.getActualTableName(tableName);

Review Comment:
   Why are we looking up actual name here? Are we planning to support case insensitive table name in rest API?



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {

Review Comment:
   (minor) Use `StringUtils.isNotEmpty(databaseName)`



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {
+        tableName = String.format("%s.%s", databaseName, tableName);
+      }
+    }
+    String actualTableName = null;
+    if (tableCache != null) {
+      actualTableName = tableCache.getActualTableName(tableName);
+    }
+    return actualTableName != null ? actualTableName : tableName;
+  }
+
+  public static boolean isTableNameEquivalent(String name1, String name2) {

Review Comment:
   I don't think this check is good enough. We need to take database into this check, or it might match wrong database



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1514093985


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -320,6 +321,11 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S
         // Compile the request into PinotQuery
         compilationStartTimeNs = System.nanoTime();
         pinotQuery = CalciteSqlParser.compileToPinotQuery(sqlNodeAndOptions);
+        if (pinotQuery.getDataSource() != null && pinotQuery.getDataSource().getTableName() != null) {
+          String tableName = getActualTableName(DatabaseUtils.translateTableName(
+              pinotQuery.getDataSource().getTableName(), httpHeaders), _tableCache);
+        pinotQuery.getDataSource().setTableName(tableName);
+        }

Review Comment:
   Yes, earlier I was not sure how the `GapfillUtils.stripGapfill(pinotQuery)` worked but as it was reassigning the PinotQuery and original PinotQuery was also being used later in the method by `getEmptyBrokerOnlyResponse`. I played safe and added translation for both of them. Now that you pointed it out, it seems that translation on just the reassigned PinotQuery should be sufficient. Will revert the first translation.
   Thanks for catching it!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1514117098


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  /**
+   * Construct the fully qualified table name i.e. {databaseName}.{tableName} from given table name and database name
+   * @param tableName table/schema name
+   * @param databaseName database name
+   * @return translated table name. Throws {@link IllegalStateException} if {@code tableName} contains more than 1 dot
+   * or if {@code tableName} has database prefix, and it does not match with {@code databaseName}
+   */
+  public static String translateTableName(String tableName, @Nullable String databaseName) {
+    if (tableName == null) {
+      throw new IllegalArgumentException("'tableName' cannot be null");
+    }
+    String[] tableSplit = StringUtils.split(tableName, '.');
+    switch (tableSplit.length) {
+      case 1:
+        // do not concat the database name prefix if it's a 'default' database
+        if (StringUtils.isNotEmpty(databaseName) && !databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {

Review Comment:
   We need a database to occupy the existing tables which don't have any database prefix that's why introducing this reserved `default` database for tables that don't have the database prefix. And to maintain consistency within the `default` database we are skipping the `default.` prefix on table names.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on PR #12417:
URL: https://github.com/apache/pinot/pull/12417#issuecomment-1944413043

   cc @zhtaoxiang to also take a look


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on PR #12417:
URL: https://github.com/apache/pinot/pull/12417#issuecomment-1972579675

   @Jackie-Jiang I have reverted the v2 APIs and the request filter approach to achieve table translation. We can talk around what's the best way to handle the translation separately in the design [doc](https://docs.google.com/document/d/1l1dco4lRhB28y-yKMVKlOMZ0sdFKObjJfhAjG6cmktU/edit?usp=sharing) .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1508836250


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   Fixed it to throw `IllegalArgumentException` upon null table name



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1505907187


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   database prefix in table name will take precedence over the database name passed explicitly. Will add java doc



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1506101339


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");

Review Comment:
   done, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1506100806


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1520835443


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -709,68 +709,131 @@ public List<String> getAllResources() {
   }
 
   /**
-   * Get all table names (with type suffix).
+   * Get all table names (with type suffix) in default database.
    *
-   * @return List of table names
+   * @return List of table names in default database
    */
   public List<String> getAllTables() {
+    return getAllTables(null);
+  }
+
+  /**
+   * Get all table names (with type suffix) from provided database.
+   *
+   * @param databaseName database name
+   * @return List of table names in provided database name
+   */
+  public List<String> getAllTables(String databaseName) {
     List<String> tableNames = new ArrayList<>();
     for (String resourceName : getAllResources()) {
-      if (TableNameBuilder.isTableResource(resourceName)) {
+      if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) {
         tableNames.add(resourceName);
       }
     }
     return tableNames;
   }
 
+  private boolean isPartOfDatabase(String tableName, String databaseName) {
+    if (databaseName == null) {
+      return StringUtils.split(tableName, '.').length == 1;
+    } else if (databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+      String[] split = StringUtils.split(tableName, '.');
+      return split.length == 1 || split[0].equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE);

Review Comment:
   Thanks for confirming. Removed the check



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1518048377


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -709,68 +709,131 @@ public List<String> getAllResources() {
   }
 
   /**
-   * Get all table names (with type suffix).
+   * Get all table names (with type suffix) in default database.
    *
-   * @return List of table names
+   * @return List of table names in default database
    */
   public List<String> getAllTables() {
+    return getAllTables(null);
+  }
+
+  /**
+   * Get all table names (with type suffix) from provided database.
+   *
+   * @param databaseName database name
+   * @return List of table names in provided database name
+   */
+  public List<String> getAllTables(String databaseName) {
     List<String> tableNames = new ArrayList<>();
     for (String resourceName : getAllResources()) {
-      if (TableNameBuilder.isTableResource(resourceName)) {
+      if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) {
         tableNames.add(resourceName);
       }
     }
     return tableNames;
   }
 
+  private boolean isPartOfDatabase(String tableName, String databaseName) {
+    if (databaseName == null) {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1518165520


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -709,68 +709,131 @@ public List<String> getAllResources() {
   }
 
   /**
-   * Get all table names (with type suffix).
+   * Get all table names (with type suffix) in default database.
    *
-   * @return List of table names
+   * @return List of table names in default database
    */
   public List<String> getAllTables() {
+    return getAllTables(null);
+  }
+
+  /**
+   * Get all table names (with type suffix) from provided database.
+   *
+   * @param databaseName database name
+   * @return List of table names in provided database name
+   */
+  public List<String> getAllTables(String databaseName) {
     List<String> tableNames = new ArrayList<>();
     for (String resourceName : getAllResources()) {
-      if (TableNameBuilder.isTableResource(resourceName)) {
+      if (TableNameBuilder.isTableResource(resourceName) && isPartOfDatabase(resourceName, databaseName)) {
         tableNames.add(resourceName);
       }
     }
     return tableNames;
   }
 
+  private boolean isPartOfDatabase(String tableName, String databaseName) {
+    if (databaseName == null) {
+      return StringUtils.split(tableName, '.').length == 1;
+    } else if (databaseName.equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE)) {
+      String[] split = StringUtils.split(tableName, '.');
+      return split.length == 1 || split[0].equalsIgnoreCase(CommonConstants.DEFAULT_DATABASE);

Review Comment:
   I think that should be okay. I'm not aware of anyone explicitly create table with `default.` prefix, and we should prevent that going forward



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1522166994


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/TableConfigsRestletResource.java:
##########
@@ -322,26 +322,25 @@ public ConfigSuccessResponse updateConfig(
       @ApiParam(value = "Force update the table schema") @DefaultValue("false")
       @QueryParam("forceTableSchemaUpdate") boolean forceTableSchemaUpdate,
       String tableConfigsStr, @Context HttpHeaders headers) throws Exception {
+    tableName = DatabaseUtils.translateTableName(tableName, headers);
     Pair<TableConfigs, Map<String, Object>> tableConfigsAndUnrecognizedProps;
     TableConfigs tableConfigs;
     String translatedTableName;
     try {
-      translatedTableName = DatabaseUtils.translateTableName(tableName, headers);
       tableConfigsAndUnrecognizedProps =
           JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigsStr, TableConfigs.class);
       tableConfigs = tableConfigsAndUnrecognizedProps.getLeft();
-      Preconditions.checkState(
-          DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers).equals(translatedTableName),
-          "Table name mismatch: %s is not equivalent to %s", tableConfigs.getTableName(), tableName);
+      tableConfigs.setTableName(DatabaseUtils.translateTableName(tableConfigs.getTableName(), headers));

Review Comment:
   Good point. Changed the order



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1522028826


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -1617,9 +1619,9 @@ public List<String> getSchemaNames() {
   public List<String> getSchemaNames(@Nullable String databaseName) {
     List<String> schemas = _propertyStore.getChildNames(
         PinotHelixPropertyStoreZnRecordProvider.forSchema(_propertyStore).getRelativePath(), AccessOption.PERSISTENT);
-    if (schemas != null) {
-        return schemas.stream().filter(schemaName -> isPartOfDatabase(schemaName, databaseName))
-            .collect(Collectors.toList());
+    if (databaseName != null) {

Review Comment:
   Good call. We need to loop over all schemas



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "shounakmk219 (via GitHub)" <gi...@apache.org>.
shounakmk219 commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1503671751


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {
+      String[] tableSplit = tableName.split("\\.");
+      if (tableSplit.length > 2) {
+        throw new IllegalStateException("Table name: '" + tableName + "' containing more than one '.' is not allowed");
+      } else if (tableSplit.length == 2) {
+        databaseName = tableSplit[0];
+        tableName = tableSplit[1];
+      }
+      if (databaseName != null && !databaseName.isBlank()) {
+        tableName = String.format("%s.%s", databaseName, tableName);
+      }
+    }
+    String actualTableName = null;
+    if (tableCache != null) {
+      actualTableName = tableCache.getActualTableName(tableName);
+    }
+    return actualTableName != null ? actualTableName : tableName;
+  }
+
+  public static boolean isTableNameEquivalent(String name1, String name2) {
+    return Objects.equals(name1, name2) || name1.endsWith("." + name2) || name2.endsWith("." + name1);
+  }

Review Comment:
   The actual database name or logical table name will not have '.' as that's ensured during table creation by `TableConfigUtils.validateTableName`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Allow passing database context through `database` http header [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12417:
URL: https://github.com/apache/pinot/pull/12417#discussion_r1506799194


##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   We should never invoke this method when `tableName` is `null`, and let's annotate `databaseName` as nullable if it can be `null`. Currently this method will NPE when `tableName` is `null` but `tableCache` is not `null`.



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/DatabaseUtils.java:
##########
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.common.utils;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.pinot.common.config.provider.TableCache;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DatabaseUtils {
+  private DatabaseUtils() {
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseUtils.class);
+
+  private static final List<String> TABLE_NAME_KEYS = List.of("tableName", "tableNameWithType", "schemaName");
+
+  public static void translateTableNameQueryParam(ContainerRequestContext requestContext, TableCache tableCache) {
+    MultivaluedMap<String, String> queryParams = requestContext.getUriInfo().getQueryParameters();
+    String uri = requestContext.getUriInfo().getRequestUri().toString();
+    String databaseName = null;
+    if (requestContext.getHeaders().containsKey(CommonConstants.DATABASE)) {
+      databaseName = requestContext.getHeaderString(CommonConstants.DATABASE);
+    }
+    for (String key : TABLE_NAME_KEYS) {
+      if (queryParams.containsKey(key)) {
+        String tableName = queryParams.getFirst(key);
+        String actualTableName = translateTableName(tableName, databaseName, tableCache);
+        // table is not part of default database
+        if (!actualTableName.equals(tableName)) {
+          uri = uri.replaceAll(String.format("%s=%s", key, tableName),
+              String.format("%s=%s", key, actualTableName));
+          try {
+            requestContext.setRequestUri(new URI(uri));
+          } catch (URISyntaxException e) {
+            LOGGER.error("Unable to translate the table name from {} to {}", tableName, actualTableName);
+          }
+        }
+      }
+    }
+  }
+
+  public static String translateTableName(String tableName, String databaseName, @Nullable TableCache tableCache) {
+    if (tableName != null && databaseName != null) {

Review Comment:
   Can we throw exception when they mismatch? Silently dropping one is dangerous.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org