You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2022/04/04 16:22:38 UTC

[GitHub] [drill] cgivre opened a new pull request, #2516: [WIP] DRILL-8155: Add Impersonation Support for Non-Hadoop Based Storage Plugins

cgivre opened a new pull request, #2516:
URL: https://github.com/apache/drill/pull/2516

   # [DRILL-8155](https://issues.apache.org/jira/browse/DRILL-8155): Add Impersonation Support for Non-Hadoop Based Storage Plugins
   
   ## Description
   
   (Please describe the change. If more than one ticket is fixed, include a reference to those tickets.)
   
   ## Documentation
   (Please describe user-visible changes similar to what should appear in the Drill documentation.)
   
   ## Testing
   Added unit tests.


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] jnturton commented on a diff in pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
jnturton commented on code in PR #2516:
URL: https://github.com/apache/drill/pull/2516#discussion_r857621992


##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcConventionFactory.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.calcite.sql.SqlDialect;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class JdbcConventionFactory {
+
+  private final Map<SqlDialect, DrillJdbcConvention> CACHE = new ConcurrentHashMap<>();

Review Comment:
   @vvysotskyi Thanks, I've replaced it with a Guava cache that has a maximum size of 100 before LRU evictions begin. I wonder if I should set an expiry time on it too?



-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1105565122

   This pull request **introduces 2 alerts** when merging c72ba9af4004ddc0d00a758b923b435b59d3f6c8 into 66bc83233df2aafa6b95d83c650d54df7d91defe - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-eff3f82771070c554fc32081fdf50adaa477acdd)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string
   * 1 for Dereferenced variable may be null


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1103457982

   This pull request **introduces 2 alerts** when merging 861b1585216a384f2a6f232944fa4c9814e70691 into fd836a2a6e933c294a53271f7e9b6ba3fea7cc8d - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-655f1b5b213873cb902377487e1e2240447b8b7d)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string
   * 1 for Dereferenced variable may be null


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1104731624

   This pull request **introduces 2 alerts** when merging c615bac0096d2dd596cec96e5ab8a0ff8972c5a4 into 66bc83233df2aafa6b95d83c650d54df7d91defe - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-7000c38ebc32818d9b81229c0125b13d52dbea99)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string
   * 1 for Dereferenced variable may be null


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1095521738

   This pull request **introduces 1 alert** when merging 82c9418b57e3e2c5bc38d0004149e0a53ae53d47 into 634ffa26ae21d138488946930570c95d858406cc - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-1e35a06b8a0f56c138deab384032f8f6d3425afa)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1102894124

   This pull request **introduces 2 alerts** when merging 70c50dcc56f99d8816c756beb9b6190b6b6cf6be into fd836a2a6e933c294a53271f7e9b6ba3fea7cc8d - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-ae5cb0f8eedcaa9fa183314f4b09173e35aa7f9f)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string
   * 1 for Dereferenced variable may be null


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1110248260

   This pull request **introduces 2 alerts** when merging a99c43497ec138497d273ecc2c27c8b8aa903fe3 into 5080424abbd7cee0f603ede10ab8dbf29ec10c85 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-18b20330426c3833f80dd34cc942f0118fbdab05)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string
   * 1 for Dereferenced variable may be null


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1092761361

   This pull request **introduces 1 alert** when merging 2ab0498d88b96c2aab08f5bbf50d63af7a9c9818 into 355e074858373ab7b42dd303bee9feebe4a671c0 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-ba22572c5fe296c9af29e77be6ec49488f2a465f)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1094345634

   This pull request **introduces 1 alert** when merging 904d00171ebaf874bac51515379d035f0afc23f6 into 355e074858373ab7b42dd303bee9feebe4a671c0 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-5e452825cc18e7946b26add3dd6a6367334f440c)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] jnturton commented on a diff in pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
jnturton commented on code in PR #2516:
URL: https://github.com/apache/drill/pull/2516#discussion_r857621992


##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcConventionFactory.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.calcite.sql.SqlDialect;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class JdbcConventionFactory {
+
+  private final Map<SqlDialect, DrillJdbcConvention> CACHE = new ConcurrentHashMap<>();

Review Comment:
   @vvysotskyi Thanks, I've replaced it with a Guava cache that has a maximum size of 100 before evictions LRU begin. I wonder if I should set an expiry time on it too?



-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] jnturton commented on a diff in pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
jnturton commented on code in PR #2516:
URL: https://github.com/apache/drill/pull/2516#discussion_r857621992


##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcConventionFactory.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.calcite.sql.SqlDialect;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class JdbcConventionFactory {
+
+  private final Map<SqlDialect, DrillJdbcConvention> CACHE = new ConcurrentHashMap<>();

Review Comment:
   Thanks, I've replaced it with a Guava cache that has a maximum size of 100 before evictions LRU begin. I wonder if I should set an expiry time on it too?



-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] cgivre merged pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
cgivre merged PR #2516:
URL: https://github.com/apache/drill/pull/2516


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: [WIP] DRILL-8155: Add Impersonation Support for Non-Hadoop Based Storage Plugins

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1087848858

   This pull request **introduces 2 alerts** when merging c0214cd64562039510fd8b5b1098b7fb9ee585fd into 8e1ab3ad4a9e26de86f029a16f56608dab5f1876 - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-0a5c79c38e6ec9e3a756e1c4a538d8ddf93b4375)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string
   * 1 for Spurious Javadoc @param tags


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] vvysotskyi commented on a diff in pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on code in PR #2516:
URL: https://github.com/apache/drill/pull/2516#discussion_r857767899


##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcConventionFactory.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.calcite.sql.SqlDialect;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class JdbcConventionFactory {
+
+  private final Map<SqlDialect, DrillJdbcConvention> CACHE = new ConcurrentHashMap<>();

Review Comment:
   Yes, it would be also nice to have an expiry time on 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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] vvysotskyi commented on a diff in pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on code in PR #2516:
URL: https://github.com/apache/drill/pull/2516#discussion_r854816913


##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcConventionFactory.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.calcite.sql.SqlDialect;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class JdbcConventionFactory {
+
+  private final Map<SqlDialect, DrillJdbcConvention> CACHE = new ConcurrentHashMap<>();

Review Comment:
   Shouldn't this cache invalidate its entries after some period of time to avoid memory leaks?



##########
contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcWriterWithH2.java:
##########
@@ -437,6 +445,7 @@ public void testWithReallyLongFile() throws Exception {
   }
 
   @Test
+  @Ignore

Review Comment:
   Is there any reason why it is ignored?



##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java:
##########
@@ -31,51 +34,89 @@
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.security.UsernamePasswordCredentials;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.sql.DataSource;
 
 public class JdbcStoragePlugin extends AbstractStoragePlugin {
 
-  private static final Logger logger = LoggerFactory.getLogger(JdbcStoragePlugin.class);
+  static final Logger logger = LoggerFactory.getLogger(JdbcStoragePlugin.class);
 
-  private final JdbcStorageConfig config;
-  private final HikariDataSource dataSource;
-  private final SqlDialect dialect;
-  private final DrillJdbcConvention convention;
-  private final JdbcDialect jdbcDialect;
+  private final JdbcStorageConfig jdbcStorageConfig;
+  private final JdbcDialectFactory dialectFactory;
+  private final JdbcConventionFactory conventionFactory;
+  // DataSources for this storage config keyed on JDBC username
+  private final Map<String, HikariDataSource> dataSources = new ConcurrentHashMap<>();

Review Comment:
   The same regarding caching here.



##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcScanBatchCreator.java:
##########
@@ -55,14 +57,22 @@ public CloseableRecordBatch getBatch(ExecutorFragmentContext context,
     }
   }
 
-  private ScanFrameworkBuilder createBuilder(OptionManager options, JdbcSubScan subScan) {
-    JdbcStorageConfig config = subScan.getConfig();
+  private ScanFrameworkBuilder createBuilder(ExecutorFragmentContext context, JdbcSubScan subScan) {
     ScanFrameworkBuilder builder = new ScanFrameworkBuilder();
     builder.projection(subScan.getColumns());
     builder.setUserName(subScan.getUserName());
     JdbcStoragePlugin plugin = subScan.getPlugin();
-    List<ManagedReader<SchemaNegotiator>> readers =
-      Collections.singletonList(new JdbcBatchReader(plugin.getDataSource(), subScan.getSql(), subScan.getColumns()));
+    UserCredentials userCreds = context.getContextInformation().getQueryUserCredentials();
+    DataSource ds = plugin.getDataSource(userCreds)
+      .orElseThrow(() -> UserException.permissionError().message(
+        "Query user %s could obtain a connection to %s, missing credentials?",

Review Comment:
   could -> could not



-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1094493794

   This pull request **introduces 1 alert** when merging ed92204450c5ace029fded2309a8b250f3059247 into 634ffa26ae21d138488946930570c95d858406cc - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-dda44f48a37dd04de6f8df2ca78e0df0a42ce07a)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1095345571

   This pull request **introduces 1 alert** when merging 7277123bf1cd27e1a4ba45396725ba1ee65afc76 into 634ffa26ae21d138488946930570c95d858406cc - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-e14a3917a7eba7ef2af53e1fceda0583ec684197)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] lgtm-com[bot] commented on pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on PR #2516:
URL: https://github.com/apache/drill/pull/2516#issuecomment-1102685834

   This pull request **introduces 2 alerts** when merging ff14630c4045b1f8ad8980f73cd340d3fb317efb into fd836a2a6e933c294a53271f7e9b6ba3fea7cc8d - [view on LGTM.com](https://lgtm.com/projects/g/apache/drill/rev/pr-e5916ece0f12985634861b644275940387d8f679)
   
   **new alerts:**
   
   * 1 for Use of externally\-controlled format string
   * 1 for Dereferenced variable may be null


-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] cgivre commented on a diff in pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
cgivre commented on code in PR #2516:
URL: https://github.com/apache/drill/pull/2516#discussion_r855380007


##########
contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcScanBatchCreator.java:
##########
@@ -55,14 +57,22 @@ public CloseableRecordBatch getBatch(ExecutorFragmentContext context,
     }
   }
 
-  private ScanFrameworkBuilder createBuilder(OptionManager options, JdbcSubScan subScan) {
-    JdbcStorageConfig config = subScan.getConfig();
+  private ScanFrameworkBuilder createBuilder(ExecutorFragmentContext context, JdbcSubScan subScan) {
     ScanFrameworkBuilder builder = new ScanFrameworkBuilder();
     builder.projection(subScan.getColumns());
     builder.setUserName(subScan.getUserName());
     JdbcStoragePlugin plugin = subScan.getPlugin();
-    List<ManagedReader<SchemaNegotiator>> readers =
-      Collections.singletonList(new JdbcBatchReader(plugin.getDataSource(), subScan.getSql(), subScan.getColumns()));
+    UserCredentials userCreds = context.getContextInformation().getQueryUserCredentials();
+    DataSource ds = plugin.getDataSource(userCreds)
+      .orElseThrow(() -> UserException.permissionError().message(
+        "Query user %s could obtain a connection to %s, missing credentials?",

Review Comment:
   Fixed.



-- 
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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] cgivre commented on a diff in pull request #2516: DRILL-8155: Introduce New Plugin Authentication Modes

Posted by GitBox <gi...@apache.org>.
cgivre commented on code in PR #2516:
URL: https://github.com/apache/drill/pull/2516#discussion_r855379275


##########
contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcWriterWithH2.java:
##########
@@ -437,6 +445,7 @@ public void testWithReallyLongFile() throws Exception {
   }
 
   @Test
+  @Ignore

Review Comment:
   Fixed (and un-ignored)



-- 
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: dev-unsubscribe@drill.apache.org

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