You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "zachjsh (via GitHub)" <gi...@apache.org> on 2023/03/28 05:56:09 UTC

[GitHub] [druid] zachjsh opened a new pull request, #13989: Input source security

zachjsh opened a new pull request, #13989:
URL: https://github.com/apache/druid/pull/13989

   <!-- Thanks for trying to help us make Apache Druid be the best it can be! Please fill out as much of the following information as is possible (where relevant, and remove it when irrelevant) to help make the intention and scope of this PR clear in order to ease review. -->
   
   <!-- Please read the doc for contribution (https://github.com/apache/druid/blob/master/CONTRIBUTING.md) before making this PR. Also, once you open a PR, please _avoid using force pushes and rebasing_ since these make it difficult for reviewers to see what you've changed in response to their reviews. See [the 'If your pull request shows conflicts with master' section](https://github.com/apache/druid/blob/master/CONTRIBUTING.md#if-your-pull-request-shows-conflicts-with-master) for more details. -->
   
   Fixes #XXXX.
   
   <!-- Replace XXXX with the id of the issue fixed in this PR. Remove this section if there is no corresponding issue. Don't reference the issue in the title of this pull-request. -->
   
   <!-- If you are a committer, follow the PR action item checklist for committers:
   https://github.com/apache/druid/blob/master/dev/committer-instructions.md#pr-and-issue-action-item-checklist-for-committers. -->
   
   ### Description
   
   <!-- Describe the goal of this PR, what problem are you fixing. If there is a corresponding issue (referenced above), it's not necessary to repeat the description here, however, you may choose to keep one summary sentence. -->
   
   <!-- Describe your patch: what did you change in code? How did you fix the problem? -->
   
   <!-- If there are several relatively logically separate changes in this PR, create a mini-section for each of them. For example: -->
   
   #### Fixed the bug ...
   #### Renamed the class ...
   #### Added a forbidden-apis entry ...
   
   <!--
   In each section, please describe design decisions made, including:
    - Choice of algorithms
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error conditions handled, such as when there are insufficient resources?
    - Class organization and design (how the logic is split between classes, inheritance, composition, design patterns)
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
   -->
   
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design (or naming) decision point and compare the alternatives with the designs that you've implemented (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), link to that discussion from this PR description and explain what have changed in your final design compared to your original proposal or the consensus version in the end of the discussion. If something hasn't changed since the original discussion, you can omit a detailed discussion of those aspects of the design here, perhaps apart from brief mentioning for the sake of readability of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. -->
   
   #### Release note
   <!-- Give your best effort to summarize your changes in a couple of sentences aimed toward Druid users. 
   
   If your change doesn't have end user impact, you can skip this section.
   
   For tips about how to write a good release note, see [Release notes](https://github.com/apache/druid/blob/master/CONTRIBUTING.md#release-notes).
   
   -->
   
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `MyFoo`
    * `OurBar`
    * `TheirBaz`
   
   <hr>
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] a release note entry in the PR description.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


-- 
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@druid.apache.org

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


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


[GitHub] [druid] abhishekrb19 commented on a diff in pull request #13989: Allow for Input source security in SQL layer

Posted by "abhishekrb19 (via GitHub)" <gi...@apache.org>.
abhishekrb19 commented on code in PR #13989:
URL: https://github.com/apache/druid/pull/13989#discussion_r1152418051


##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java:
##########
@@ -307,6 +308,39 @@ public void testInsertFromExternal()
         .verify();
   }
 
+  @Test
+  public void testInsertFromExternalWithInputSourceSecurityEnabled()
+  {
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
+        .authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setEnableInputSourceSecurity(true).build())
+        .expectTarget("dst", externalDataSource.getSignature())
+        .expectResources(dataSourceWrite("dst"), externalRead("inline"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource(externalDataSource)
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("x", "y", "z")
+                .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
+                .build()
+        )
+        .expectLogicalPlanFrom("insertFromExternal")
+        .verify();
+  }
+
+  @Test
+  public void testUnauthorizedInsertFromExternalWithInputSourceSecurityEnabled()
+  {
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
+        .authentication(CalciteTests.REGULAR_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setEnableInputSourceSecurity(true).build())

Review Comment:
   All the tests either implicitly disable the input source security (via a default) or explicitly enable it. It'll be nice also to have a test where we explicitly disable it.
   
   Perhaps a unit test similar to this, where we `setEnableInputSourceSecurity` set to `false`, and we shouldn't get a `ForbiddenException`. 



##########
sql/src/main/java/org/apache/druid/sql/calcite/external/DruidExternTableMacro.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.druid.sql.calcite.external;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.NlsString;
+import org.apache.druid.server.security.Action;
+import org.apache.druid.server.security.Resource;
+import org.apache.druid.server.security.ResourceAction;
+import org.apache.druid.server.security.ResourceType;
+import org.apache.druid.sql.calcite.table.DruidTable;
+
+import javax.validation.constraints.NotNull;
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * Used by {@link ExternalOperatorConversion} to generate a {@link DruidTable}
+ * that references an {@link ExternalDataSource}.
+ */
+public class DruidExternTableMacro extends DruidUserDefinedTableMacro
+{
+  public DruidExternTableMacro(DruidTableMacro macro)
+  {
+    super(macro);
+  }
+
+  @Override
+  public Set<ResourceAction> computeResources(final SqlCall call, boolean inputSourceTypeSecurityEnabled)
+  {
+    if (!inputSourceTypeSecurityEnabled) {
+      return Collections.singleton(Externals.EXTERNAL_RESOURCE_ACTION);
+    }
+    String inputSourceStr = getInputSourceArgument(call);
+
+    try {
+      JsonNode jsonNode = ((DruidTableMacro) macro).getJsonMapper().readTree(inputSourceStr);
+      return Collections.singleton(new ResourceAction(new Resource(
+          ResourceType.EXTERNAL,
+          jsonNode.get("type").asText()
+      ), Action.READ));
+    }
+    catch (JsonProcessingException e) {
+      // this shouldn't happen, the input source paraemeter should have been validated before this
+      throw new RuntimeException(e);
+    }
+  }
+
+  @NotNull
+  private String getInputSourceArgument(final SqlCall call)
+  {
+    // this covers case where parameters are used positionally
+    if (call.getOperandList().size() > 0) {
+      if (call.getOperandList().get(0) instanceof SqlCharStringLiteral) {
+        return ((SqlCharStringLiteral) call.getOperandList().get(0)).toValue();
+      }
+    }
+
+    // this covers case where named parameters are used.
+    for (SqlNode sqlNode : call.getOperandList()) {
+      if (sqlNode instanceof SqlCall) {
+        String argumentName = ((SqlCall) sqlNode).getOperandList().size() > 1 ?
+                             ((SqlCall) sqlNode).getOperandList().get(1).toString()
+                             : null;
+        if (ExternalOperatorConversion.INPUT_SOURCE_PARAM.equals(argumentName)) {
+          return ((NlsString) ((SqlCharStringLiteral) ((SqlCall) call.getOperandList().get(0))
+              .getOperandList()
+              .get(0))
+              .getValue())
+              .getValue();
+        }
+      }
+    }
+    // this shouldn't happen, as the sqlCall should have been validated by this point,

Review Comment:
   nit: `sqlCall` -> `call` or if you're referring to the type `SqlCall`



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlResourceCollectorShuttle.java:
##########
@@ -51,19 +51,24 @@ public class SqlResourceCollectorShuttle extends SqlShuttle
   private final Set<ResourceAction> resourceActions;
   private final PlannerContext plannerContext;
   private final SqlValidator validator;
+  private final boolean inputSourceTypeSecurityEnabled;
 
   public SqlResourceCollectorShuttle(SqlValidator validator, PlannerContext plannerContext)
   {
     this.validator = validator;
     this.resourceActions = new HashSet<>();
     this.plannerContext = plannerContext;
+    inputSourceTypeSecurityEnabled = plannerContext.getPlannerToolbox().getAuthConfig().isEnableInputSourceSecurity();

Review Comment:
   This class already holds `plannerContext` state, so we could just reference this line inline directly in the `visit()` call?



-- 
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@druid.apache.org

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


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


[GitHub] [druid] zachjsh commented on pull request #13989: Allow for Input source security in SQL layer

Posted by "zachjsh (via GitHub)" <gi...@apache.org>.
zachjsh commented on PR #13989:
URL: https://github.com/apache/druid/pull/13989#issuecomment-1489586709

   thanks @abhishekrb19 ! I will take care of the minor comments in follow up pr.


-- 
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@druid.apache.org

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


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


[GitHub] [druid] abhishekrb19 commented on a diff in pull request #13989: Allow for Input source security in SQL layer

Posted by "abhishekrb19 (via GitHub)" <gi...@apache.org>.
abhishekrb19 commented on code in PR #13989:
URL: https://github.com/apache/druid/pull/13989#discussion_r1152418051


##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java:
##########
@@ -307,6 +308,39 @@ public void testInsertFromExternal()
         .verify();
   }
 
+  @Test
+  public void testInsertFromExternalWithInputSourceSecurityEnabled()
+  {
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
+        .authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setEnableInputSourceSecurity(true).build())
+        .expectTarget("dst", externalDataSource.getSignature())
+        .expectResources(dataSourceWrite("dst"), externalRead("inline"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource(externalDataSource)
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("x", "y", "z")
+                .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
+                .build()
+        )
+        .expectLogicalPlanFrom("insertFromExternal")
+        .verify();
+  }
+
+  @Test
+  public void testUnauthorizedInsertFromExternalWithInputSourceSecurityEnabled()
+  {
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
+        .authentication(CalciteTests.REGULAR_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setEnableInputSourceSecurity(true).build())

Review Comment:
   All the tests either implicitly disable the input source security (via a default) or explicitly enable it. It'll be nice also to have a test where we explicitly disable it.
   
   Perhaps a unit test similar to this, where `setEnableInputSourceSecurity` us set to `false`, and we shouldn't get a `ForbiddenException`. 



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java:
##########
@@ -307,6 +308,39 @@ public void testInsertFromExternal()
         .verify();
   }
 
+  @Test
+  public void testInsertFromExternalWithInputSourceSecurityEnabled()
+  {
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
+        .authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setEnableInputSourceSecurity(true).build())
+        .expectTarget("dst", externalDataSource.getSignature())
+        .expectResources(dataSourceWrite("dst"), externalRead("inline"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource(externalDataSource)
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("x", "y", "z")
+                .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
+                .build()
+        )
+        .expectLogicalPlanFrom("insertFromExternal")
+        .verify();
+  }
+
+  @Test
+  public void testUnauthorizedInsertFromExternalWithInputSourceSecurityEnabled()
+  {
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
+        .authentication(CalciteTests.REGULAR_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setEnableInputSourceSecurity(true).build())

Review Comment:
   All the tests either implicitly disable the input source security (via a default) or explicitly enable it. It'll be nice also to have a test where we explicitly disable it.
   
   Perhaps a unit test similar to this, where `setEnableInputSourceSecurity` is set to `false`, and we shouldn't get a `ForbiddenException`. 



-- 
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@druid.apache.org

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


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


[GitHub] [druid] paul-rogers commented on a diff in pull request #13989: Allow for Input source security in SQL layer

Posted by "paul-rogers (via GitHub)" <gi...@apache.org>.
paul-rogers commented on code in PR #13989:
URL: https://github.com/apache/druid/pull/13989#discussion_r1151107789


##########
extensions-core/s3-extensions/src/test/java/org/apache/druid/catalog/model/table/S3InputSourceDefnTest.java:
##########
@@ -345,6 +345,7 @@ public void testAdHocUri()
         CatalogUtils.stringListToUriList(uris),
         s3InputSource.getUris()
     );
+    assertEquals(Collections.singleton(S3StorageDruidModule.SCHEME), externSpec.inputSourceTypes);

Review Comment:
   While this is OK, it muddies the semantics. There should be only one input source type, not a list. Yes, a list with one item is the same as a single value, but the list suggests that there could be more values, which there cannot be.
   
   Second, the input source type should be the value returned from `typeValue()` that happens to be the same as the `SCHEME` constant, but that is an implementation detail.
   
   Actually, the core problem here may be the lack of a `TYPE_KEY` constant in `S3InputSourceDefn`. Go ahead and add one, then use it here:
   
   ```java
   public class S3InputSourceDefn extends FormattedInputSourceDefn
   {
     public static final String TYPE_KEY = S3StorageDruidModule.SCHEME;
   ...
   
     @Override
     public String typeValue()
     {
       return TYPE_KEY;
     }
   `
   



##########
server/src/main/java/org/apache/druid/server/security/AuthConfig.java:
##########
@@ -97,6 +97,9 @@ public AuthConfig()
   @JsonProperty
   private final Set<String> securedContextKeys;
 
+  @JsonProperty
+  private final boolean inputSourceTypeSecurityEnabled;

Review Comment:
   Maybe `enableInputSourceSecurity` shorter. Starts with a verb.



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java:
##########
@@ -351,6 +373,94 @@ public void testInsertFromExternalWithSchema()
         .verify();
   }
 
+  @Test
+  public void testInsertFromExternalWithSchemaWithInputsourceSecurity()
+  {
+    String extern;
+    ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper();
+    try {
+      extern = StringUtils.format(
+          "TABLE(extern(%s, %s))",
+          Calcites.escapeStringLiteral(
+              queryJsonMapper.writeValueAsString(
+                  new InlineInputSource("a,b,1\nc,d,2\n")
+              )
+          ),
+          Calcites.escapeStringLiteral(
+              queryJsonMapper.writeValueAsString(
+                  new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0)
+              )
+          )
+      );
+    }
+    catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s\n" +
+             "  (x VARCHAR, y VARCHAR, z BIGINT)\n" +
+             "PARTITIONED BY ALL TIME",
+             extern
+        )
+        .authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setInputSourceTypeSecurityEnabled(true).build())
+        .expectTarget("dst", externalDataSource.getSignature())
+        .expectResources(dataSourceWrite("dst"), externalRead("inline"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource(externalDataSource)
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("x", "y", "z")
+                .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
+                .build()
+        )
+        .expectLogicalPlanFrom("insertFromExternal")
+        .verify();
+  }
+
+  @Test
+  public void testInsertFromExternalFunctionalStyleWithSchemaWithInputsourceSecurity()
+  {
+    String extern;
+    ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper();
+    try {
+      extern = StringUtils.format(
+          "TABLE(extern("
+          + "inputSource => '%s',"
+          + "inputFormat => '%s'))",
+          queryJsonMapper.writeValueAsString(
+              new InlineInputSource("a,b,1\nc,d,2\n")
+          ),
+          queryJsonMapper.writeValueAsString(
+              new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0)
+          )
+      );
+    }
+    catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+    testIngestionQuery()
+        .sql("INSERT INTO dst SELECT * FROM %s\n" +
+             "  (x VARCHAR, y VARCHAR, z BIGINT)\n" +
+             "PARTITIONED BY ALL TIME",
+             extern
+        )
+        .authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
+        .authConfig(AuthConfig.newBuilder().setInputSourceTypeSecurityEnabled(true).build())
+        .expectTarget("dst", externalDataSource.getSignature())
+        .expectResources(dataSourceWrite("dst"), externalRead("inline"))
+        .expectQuery(
+            newScanQueryBuilder()
+                .dataSource(externalDataSource)
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("x", "y", "z")
+                .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
+                .build()
+        )
+        .expectLogicalPlanFrom("insertFromExternal")

Review Comment:
   Should we include a test in which authorization fails?



##########
server/src/main/java/org/apache/druid/catalog/model/table/BaseInputSourceDefn.java:
##########
@@ -151,7 +152,8 @@ protected ExternalTableSpec convertArgsToTable(
     return new ExternalTableSpec(
         convertArgsToSource(args, jsonMapper),
         convertArgsToFormat(args, columns, jsonMapper),
-        Columns.convertSignature(columns)
+        Columns.convertSignature(columns),
+        Collections.singleton(typeValue())

Review Comment:
   Again, there can only ever be a single value.



##########
sql/src/main/java/org/apache/druid/sql/calcite/external/DruidExternTableMacro.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.druid.sql.calcite.external;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.NlsString;
+import org.apache.druid.server.security.Action;
+import org.apache.druid.server.security.Resource;
+import org.apache.druid.server.security.ResourceAction;
+import org.apache.druid.server.security.ResourceType;
+import org.apache.druid.sql.calcite.table.DruidTable;
+
+import javax.validation.constraints.NotNull;
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * Used by {@link ExternalOperatorConversion} to generate a {@link DruidTable}
+ * that references an {@link ExternalDataSource}.
+ */
+public class DruidExternTableMacro extends DruidUserDefinedTableMacro
+{
+  public DruidExternTableMacro(DruidTableMacro macro)
+  {
+    super(macro);
+  }
+
+  @Override
+  public Set<ResourceAction> computeResources(final SqlCall call, boolean inputSourceTypeSecurityEnabled)
+  {
+    if (!inputSourceTypeSecurityEnabled) {
+      return Collections.singleton(Externals.EXTERNAL_RESOURCE_ACTION);
+    }
+    String inputSourceStr = getInputSourceArgument(call);
+
+    try {
+      JsonNode jsonNode = ((DruidTableMacro) macro).getJsonMapper().readTree(inputSourceStr);
+      return Collections.singleton(new ResourceAction(new Resource(
+          ResourceType.EXTERNAL,
+          jsonNode.get("type").asText()
+      ), Action.READ));
+    }
+    catch (JsonProcessingException e) {
+      // this shouldn't happen, the input source paraemeter should have been validated before this
+      throw new RuntimeException(e);
+    }
+  }
+
+  @NotNull
+  private String getInputSourceArgument(final SqlCall call)
+  {
+    // this covers case where parameters are used positionally
+    if (call.getOperandList().size() > 0) {
+      if (call.getOperandList().get(0) instanceof SqlCharStringLiteral) {
+        return ((SqlCharStringLiteral) call.getOperandList().get(0)).toValue();
+      }
+    }
+
+    // this covers case where named parameters are used.
+    for (SqlNode sqlNode : call.getOperandList()) {
+      if (sqlNode instanceof SqlCall) {
+        String argumentName = ((SqlCall) sqlNode).getOperandList().size() > 1 ?
+                             ((SqlCall) sqlNode).getOperandList().get(1).toString()
+                             : null;
+        if (ExternalOperatorConversion.INPUT_SOURCE_PARAM.equals(argumentName)) {
+          return ((NlsString) ((SqlCharStringLiteral) ((SqlCall) call.getOperandList().get(0))
+              .getOperandList()
+              .get(0))
+              .getValue())
+              .getValue();
+        }
+      }
+    }
+    // this shouldn't happen, as the sqlCall should have been validated by this point,
+    // and be guarenteed to have this parameter.
+    throw new RuntimeException("inputSource paramter not found in extern function");

Review Comment:
   paramter -> parameter



##########
server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableSpec.java:
##########
@@ -36,14 +37,17 @@
   public final InputSource inputSource;
   public final InputFormat inputFormat;
   @Nullable public final RowSignature signature;
+  public final Set<String> inputSourceTypes;

Review Comment:
   Again, there can only ever be a single value.



##########
sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalTableMacro.java:
##########


Review Comment:
   Please remove it if it is not used.



##########
sql/src/main/java/org/apache/druid/sql/calcite/external/Externals.java:
##########
@@ -290,15 +294,19 @@ public static ExternalTable buildExternalTable(ExternalTableSpec spec, ObjectMap
                     + "Please change the column name to something other than __time");
     }
 
-    return toExternalTable(spec, jsonMapper);
+    return toExternalTable(spec, jsonMapper, spec.inputSourceTypes);
   }
 
   public static ResourceAction externalRead(String name)
   {
     return new ResourceAction(new Resource(name, ResourceType.EXTERNAL), Action.READ);
   }
 
-  public static ExternalTable toExternalTable(ExternalTableSpec spec, ObjectMapper jsonMapper)
+  public static ExternalTable toExternalTable(
+      ExternalTableSpec spec,
+      ObjectMapper jsonMapper,
+      Set<String> inputSourceTypes

Review Comment:
   There will only ever be one input source type.



-- 
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@druid.apache.org

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


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


[GitHub] [druid] zachjsh commented on a diff in pull request #13989: Input source security

Posted by "zachjsh (via GitHub)" <gi...@apache.org>.
zachjsh commented on code in PR #13989:
URL: https://github.com/apache/druid/pull/13989#discussion_r1150981382


##########
sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalTableMacro.java:
##########


Review Comment:
   Can this class be removed? Doesn't seem to be used anywhere.



-- 
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@druid.apache.org

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


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


[GitHub] [druid] paul-rogers commented on a diff in pull request #13989: Allow for Input source security in SQL layer

Posted by "paul-rogers (via GitHub)" <gi...@apache.org>.
paul-rogers commented on code in PR #13989:
URL: https://github.com/apache/druid/pull/13989#discussion_r1151261659


##########
extensions-core/s3-extensions/src/test/java/org/apache/druid/catalog/model/table/S3InputSourceDefnTest.java:
##########
@@ -695,14 +706,15 @@ public void testTableSpecAsConnection()
     Map<String, Object> args = new HashMap<>();
     args.put(S3InputSourceDefn.PATHS_PARAMETER, Collections.singletonList("bar/file.csv"));
     args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
-    ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
+    ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);;

Review Comment:
   Nit: extra semicolon. Here and below. A static check probably complained.



-- 
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@druid.apache.org

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


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


[GitHub] [druid] zachjsh merged pull request #13989: Allow for Input source security in SQL layer

Posted by "zachjsh (via GitHub)" <gi...@apache.org>.
zachjsh merged PR #13989:
URL: https://github.com/apache/druid/pull/13989


-- 
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@druid.apache.org

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


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