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/11/23 06:22:11 UTC

[GitHub] [drill] vvysotskyi opened a new pull request, #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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

   # [DRILL-8358](https://issues.apache.org/jira/browse/DRILL-8358): Storage plugin for querying other Apache Drill clusters
   
   ## Description
   Using native client to query other drill clusters. Added logic to do various pushdowns when possible.
   Fixed adding extra project for the case of star columns.
   Fixed ignoring column with empty name column for excel format.
   
   ## Documentation
   See README.md
   
   ## Testing
   Tested manually, added UT.
   


-- 
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 #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


##########
contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillSubScan.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.drill.exec.store.drill.plugin;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+@JsonTypeName("drill-read")
+public class DrillSubScan extends AbstractBase implements SubScan {
+
+  public static final String OPERATOR_TYPE = "DRILL_SUB_SCAN";
+
+  private final String query;
+
+  @JsonProperty
+  private final DrillStoragePluginConfig pluginConfig;
+
+  @JsonCreator
+  public DrillSubScan(
+      @JsonProperty("userName") String userName,
+      @JsonProperty("mongoPluginConfig") StoragePluginConfig pluginConfig,

Review Comment:
   Is this supposed to be `mongoPluginConfig`?



-- 
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 #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


##########
contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillStoragePluginConfig.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.drill.plugin;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.calcite.avatica.ConnectStringParser;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.logical.security.CredentialsProvider;
+import org.apache.drill.common.logical.security.PlainCredentialsProvider;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.RpcException;
+
+import java.sql.SQLException;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+@JsonTypeName(DrillStoragePluginConfig.NAME)
+public class DrillStoragePluginConfig extends StoragePluginConfig {
+  public static final String NAME = "drill";
+  public static final String CONNECTION_STRING_PREFIX = "jdbc:drill:";
+
+  private static final String DEFAULT_QUOTING_IDENTIFIER = "`";
+
+  private final String connection;
+  private final Properties properties;
+
+  @JsonCreator
+  public DrillStoragePluginConfig(
+      @JsonProperty("connection") String connection,
+      @JsonProperty("properties") Properties properties,
+      @JsonProperty("credentialsProvider") CredentialsProvider credentialsProvider) {
+    super(getCredentialsProvider(credentialsProvider), credentialsProvider == null);
+    this.connection = connection;
+    this.properties = Optional.ofNullable(properties).orElse(new Properties());
+  }
+
+  private DrillStoragePluginConfig(DrillStoragePluginConfig that,
+    CredentialsProvider credentialsProvider) {
+    super(getCredentialsProvider(credentialsProvider),
+      credentialsProvider == null, that.authMode);
+    this.connection = that.connection;
+    this.properties = that.properties;
+  }
+
+  @JsonProperty("connection")
+  public String getConnection() {
+    return connection;
+  }
+
+  @JsonProperty("properties")
+  public Properties getProperties() {
+    return properties;
+  }
+
+  private static CredentialsProvider getCredentialsProvider(CredentialsProvider credentialsProvider) {
+    return credentialsProvider != null ? credentialsProvider : PlainCredentialsProvider.EMPTY_CREDENTIALS_PROVIDER;
+  }
+
+  @JsonIgnore
+  public String getIdentifierQuoteString() {
+    return properties.getProperty(DrillProperties.QUOTING_IDENTIFIERS, DEFAULT_QUOTING_IDENTIFIER);
+  }
+
+  @Override
+  public DrillStoragePluginConfig updateCredentialProvider(CredentialsProvider credentialsProvider) {
+    return new DrillStoragePluginConfig(this, credentialsProvider);
+  }
+
+  @JsonIgnore
+  public DrillClient getDrillClient(String userName, BufferAllocator allocator) {
+    try {
+      String urlSuffix = connection.substring(CONNECTION_STRING_PREFIX.length());
+      Properties props = ConnectStringParser.parse(urlSuffix, properties);
+      props.putAll(credentialsProvider.getUserCredentials(userName));

Review Comment:
   Thanks, 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] vvysotskyi commented on a diff in pull request #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


##########
contrib/storage-drill/src/test/java/org/apache/drill/exec/store/drill/plugin/DrillPluginQueriesTest.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.drill.plugin;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+
+public class DrillPluginQueriesTest extends ClusterTest {
+
+  private static final String TABLE_NAME = "dfs.tmp.test_table";
+
+  private static ClusterFixture drill;
+  private static ClientFixture drillClient;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    initPlugin();
+  }
+
+  @AfterClass
+  public static void shutdown() throws Exception {
+    AutoCloseables.close(drill, drillClient);
+  }
+
+  private static void initPlugin() throws Exception {
+    startCluster(ClusterFixture.builder(dirTestWatcher));
+    drill = ClusterFixture.builder(dirTestWatcher).build();
+
+    DrillStoragePluginConfig config = new DrillStoragePluginConfig(
+      "jdbc:drill:drillbit=localhost:" + drill.drillbit().getUserPort(),
+      new Properties(), null);
+    config.setEnabled(true);
+    cluster.defineStoragePlugin("drill", config);
+    cluster.defineStoragePlugin("drill2", config);
+    drillClient = drill.clientFixture();
+
+    drillClient.queryBuilder()
+      .sql("create table %s as select * from cp.`tpch/nation.parquet`", TABLE_NAME)
+      .run();
+  }
+
+  @Test
+  public void testSerDe() throws Exception {
+    String plan = queryBuilder().sql("select * from drill.%s", TABLE_NAME).explainJson();
+    long count = queryBuilder().physical(plan).run().recordCount();
+    assertEquals(25, count);
+  }
+
+  @Test
+  public void testShowDatabases() throws Exception {
+    testBuilder()
+      .sqlQuery("show databases where SCHEMA_NAME='drill.dfs.tmp'")
+      .unOrdered()
+      .baselineColumns("SCHEMA_NAME")
+      .baselineValues("drill.dfs.tmp")
+      .go();
+  }
+
+  @Test
+  public void testShowTables() throws Exception {
+    testBuilder()
+      .sqlQuery("show tables IN drill.INFORMATION_SCHEMA")
+      .unOrdered()
+      .baselineColumns("TABLE_SCHEMA", "TABLE_NAME")
+      .baselineValues("drill.information_schema", "VIEWS")
+      .baselineValues("drill.information_schema", "CATALOGS")
+      .baselineValues("drill.information_schema", "COLUMNS")
+      .baselineValues("drill.information_schema", "PARTITIONS")
+      .baselineValues("drill.information_schema", "FILES")
+      .baselineValues("drill.information_schema", "SCHEMATA")
+      .baselineValues("drill.information_schema", "TABLES")
+      .go();
+  }
+
+  @Test
+  public void testProjectPushDown() throws Exception {
+    String query = "select n_nationkey, n_regionkey, n_name from drill.%s";
+
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("query=\"SELECT `n_nationkey`, `n_regionkey`, `n_name`")
+        .exclude("\\*")
+        .match();
+
+    RowSet sets = queryBuilder()
+      .sql(query, TABLE_NAME)
+      .rowSet();
+
+    TupleMetadata schema = new SchemaBuilder()
+      .add("n_nationkey", TypeProtos.MinorType.INT)
+      .add("n_regionkey", TypeProtos.MinorType.INT)
+      .add("n_name", TypeProtos.MinorType.VARCHAR)
+      .build();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), schema)
+      .addRow(0, 0, "ALGERIA")
+      .addRow(1, 1, "ARGENTINA")
+      .addRow(2, 1, "BRAZIL")
+      .addRow(3, 1, "CANADA")
+      .addRow(4, 4, "EGYPT")
+      .addRow(5, 0, "ETHIOPIA")
+      .addRow(6, 3, "FRANCE")
+      .addRow(7, 3, "GERMANY")
+      .addRow(8, 2, "INDIA")
+      .addRow(9, 2, "INDONESIA")
+      .addRow(10, 4, "IRAN")
+      .addRow(11, 4, "IRAQ")
+      .addRow(12, 2, "JAPAN")
+      .addRow(13, 4, "JORDAN")
+      .addRow(14, 0, "KENYA")
+      .addRow(15, 0, "MOROCCO")
+      .addRow(16, 0, "MOZAMBIQUE")
+      .addRow(17, 1, "PERU")
+      .addRow(18, 2, "CHINA")
+      .addRow(19, 3, "ROMANIA")
+      .addRow(20, 4, "SAUDI ARABIA")
+      .addRow(21, 2, "VIETNAM")
+      .addRow(22, 3, "RUSSIA")
+      .addRow(23, 3, "UNITED KINGDOM")
+      .addRow(24, 1, "UNITED STATES")
+      .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(sets);
+  }
+
+  @Test
+  public void testFilterPushDown() throws Exception {
+    String query = "select n_name, n_nationkey from drill.%s where n_nationkey = 0";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("WHERE")
+        .exclude("Filter")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("n_name", "n_nationkey")
+      .baselineValues("ALGERIA", 0)
+      .go();
+  }
+
+  @Test
+  public void testFilterPushDownWithJoin() throws Exception {
+    String query = "select * from drill.%s e\n" +
+        "join drill.%s s on e.n_nationkey = s.n_nationkey where e.n_name = 'BRAZIL'";
+
+    queryBuilder()
+        .sql(query, TABLE_NAME, TABLE_NAME)
+        .planMatcher()
+        .include("INNER JOIN")
+        .match();
+
+    testBuilder()
+      .ordered()
+      .sqlQuery(query, TABLE_NAME, TABLE_NAME)
+      .baselineColumns("n_nationkey", "n_name", "n_regionkey", "n_comment", "n_nationkey0",
+        "n_name0", "n_regionkey0", "n_comment0")
+      .baselineValues(2, "BRAZIL", 1, "y alongside of the pending deposits. carefully special " +
+        "packages are about the ironic forges. slyly special ", 2, "BRAZIL", 1, "y alongside of " +
+        "the pending deposits. carefully special packages are about the ironic forges. slyly special ")
+      .go();
+  }
+
+  @Test
+  public void testJoinDifferentDrillPlugins() throws Exception {
+    String query = "select * from drill.%s e\n" +
+      "join drill2.cp.`tpch/nation.parquet` s on e.n_nationkey = s.n_nationkey where e.n_name = 'BRAZIL'";
+
+    queryBuilder()
+      .sql(query, TABLE_NAME, TABLE_NAME)
+      .planMatcher()
+      .include("HashJoin")
+      .exclude("INNER JOIN")
+      .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME, TABLE_NAME)
+      .baselineColumns("n_nationkey", "n_name", "n_regionkey", "n_comment", "n_nationkey0",
+        "n_name0", "n_regionkey0", "n_comment0")
+      .baselineValues(2, "BRAZIL", 1, "y alongside of the pending deposits. carefully special " +
+        "packages are about the ironic forges. slyly special ", 2, "BRAZIL", 1, "y alongside of " +
+        "the pending deposits. carefully special packages are about the ironic forges. slyly special ")
+      .go();
+  }
+
+  @Test
+  public void testAggregationPushDown() throws Exception {
+    String query = "select count(*) c from drill.%s";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("query=\"SELECT COUNT\\(\\*\\)")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("c")
+      .baselineValues(25L)
+      .go();
+  }
+
+  @Test
+  public void testLimitPushDown() throws Exception {
+    String query = "select n_name from drill.%s FETCH NEXT 1 ROWS ONLY";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("FETCH NEXT 1 ROWS ONLY")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("n_name")
+      .baselineValues("ALGERIA")
+      .go();
+  }
+
+  @Test
+  public void testLimitWithSortPushDown() throws Exception {
+    String query = "select n_nationkey from drill.%s order by n_name limit 3";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("ORDER BY `n_name`", "FETCH NEXT 3 ROWS ONLY")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("n_nationkey")
+      .baselineValues(0)
+      .baselineValues(1)
+      .baselineValues(2)
+      .go();
+  }
+
+  @Test
+  public void testAggregationWithGroupByPushDown() throws Exception {
+    String query = "select sum(n_nationkey) s from drill.%s group by n_regionkey";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("query=\"SELECT SUM\\(`n_nationkey`\\)", "GROUP BY `n_regionkey`")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("s")
+      .baselineValues(47L)
+      .baselineValues(50L)
+      .baselineValues(58L)
+      .baselineValues(68L)
+      .baselineValues(77L)
+      .go();
+  }
+
+  @Test
+  public void testUnionAllPushDown() throws Exception {
+    String query = "select col1, col2 from drill.%s " +
+      "union all " +
+      "select col1, col2 from drill.%s";
+    queryBuilder()
+      .sql(query, TABLE_NAME, TABLE_NAME)
+      .planMatcher()
+      .include("UNION ALL")
+      .match();
+
+    long recordCount = queryBuilder()
+      .sql(query, TABLE_NAME, TABLE_NAME)
+      .run()
+      .recordCount();
+
+    assertEquals(50L, recordCount);
+  }

Review Comment:
   Yes, I'll add a unit test for it in one of the future pull requests.



-- 
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 pull request #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on PR #2709:
URL: https://github.com/apache/drill/pull/2709#issuecomment-1344035229

   @cgivre, yes, you can create a plugin in drill1 with the name drill2, and query all plugins that drill2 has configured from drill1, so if drill2 has file system plugin called dfs2, query for drill1 will be the following:
   ```sql
   SELECT *
   FROM drill2.dfs2.ws.`file`
   ```


-- 
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 #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


##########
contrib/storage-drill/src/test/java/org/apache/drill/exec/store/drill/plugin/DrillPluginQueriesTest.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.drill.plugin;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+
+public class DrillPluginQueriesTest extends ClusterTest {
+
+  private static final String TABLE_NAME = "dfs.tmp.test_table";
+
+  private static ClusterFixture drill;
+  private static ClientFixture drillClient;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    initPlugin();
+  }
+
+  @AfterClass
+  public static void shutdown() throws Exception {
+    AutoCloseables.close(drill, drillClient);
+  }
+
+  private static void initPlugin() throws Exception {
+    startCluster(ClusterFixture.builder(dirTestWatcher));
+    drill = ClusterFixture.builder(dirTestWatcher).build();
+
+    DrillStoragePluginConfig config = new DrillStoragePluginConfig(
+      "jdbc:drill:drillbit=localhost:" + drill.drillbit().getUserPort(),
+      new Properties(), null);
+    config.setEnabled(true);
+    cluster.defineStoragePlugin("drill", config);
+    cluster.defineStoragePlugin("drill2", config);
+    drillClient = drill.clientFixture();
+
+    drillClient.queryBuilder()
+      .sql("create table %s as select * from cp.`tpch/nation.parquet`", TABLE_NAME)
+      .run();
+  }
+
+  @Test
+  public void testSerDe() throws Exception {
+    String plan = queryBuilder().sql("select * from drill.%s", TABLE_NAME).explainJson();
+    long count = queryBuilder().physical(plan).run().recordCount();
+    assertEquals(25, count);
+  }
+
+  @Test
+  public void testShowDatabases() throws Exception {
+    testBuilder()
+      .sqlQuery("show databases where SCHEMA_NAME='drill.dfs.tmp'")
+      .unOrdered()
+      .baselineColumns("SCHEMA_NAME")
+      .baselineValues("drill.dfs.tmp")
+      .go();
+  }
+
+  @Test
+  public void testShowTables() throws Exception {
+    testBuilder()
+      .sqlQuery("show tables IN drill.INFORMATION_SCHEMA")
+      .unOrdered()
+      .baselineColumns("TABLE_SCHEMA", "TABLE_NAME")
+      .baselineValues("drill.information_schema", "VIEWS")
+      .baselineValues("drill.information_schema", "CATALOGS")
+      .baselineValues("drill.information_schema", "COLUMNS")
+      .baselineValues("drill.information_schema", "PARTITIONS")
+      .baselineValues("drill.information_schema", "FILES")
+      .baselineValues("drill.information_schema", "SCHEMATA")
+      .baselineValues("drill.information_schema", "TABLES")
+      .go();
+  }
+
+  @Test
+  public void testProjectPushDown() throws Exception {
+    String query = "select n_nationkey, n_regionkey, n_name from drill.%s";
+
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("query=\"SELECT `n_nationkey`, `n_regionkey`, `n_name`")
+        .exclude("\\*")
+        .match();
+
+    RowSet sets = queryBuilder()
+      .sql(query, TABLE_NAME)
+      .rowSet();
+
+    TupleMetadata schema = new SchemaBuilder()
+      .add("n_nationkey", TypeProtos.MinorType.INT)
+      .add("n_regionkey", TypeProtos.MinorType.INT)
+      .add("n_name", TypeProtos.MinorType.VARCHAR)
+      .build();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), schema)
+      .addRow(0, 0, "ALGERIA")
+      .addRow(1, 1, "ARGENTINA")
+      .addRow(2, 1, "BRAZIL")
+      .addRow(3, 1, "CANADA")
+      .addRow(4, 4, "EGYPT")
+      .addRow(5, 0, "ETHIOPIA")
+      .addRow(6, 3, "FRANCE")
+      .addRow(7, 3, "GERMANY")
+      .addRow(8, 2, "INDIA")
+      .addRow(9, 2, "INDONESIA")
+      .addRow(10, 4, "IRAN")
+      .addRow(11, 4, "IRAQ")
+      .addRow(12, 2, "JAPAN")
+      .addRow(13, 4, "JORDAN")
+      .addRow(14, 0, "KENYA")
+      .addRow(15, 0, "MOROCCO")
+      .addRow(16, 0, "MOZAMBIQUE")
+      .addRow(17, 1, "PERU")
+      .addRow(18, 2, "CHINA")
+      .addRow(19, 3, "ROMANIA")
+      .addRow(20, 4, "SAUDI ARABIA")
+      .addRow(21, 2, "VIETNAM")
+      .addRow(22, 3, "RUSSIA")
+      .addRow(23, 3, "UNITED KINGDOM")
+      .addRow(24, 1, "UNITED STATES")
+      .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(sets);
+  }
+
+  @Test
+  public void testFilterPushDown() throws Exception {
+    String query = "select n_name, n_nationkey from drill.%s where n_nationkey = 0";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("WHERE")
+        .exclude("Filter")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("n_name", "n_nationkey")
+      .baselineValues("ALGERIA", 0)
+      .go();
+  }
+
+  @Test
+  public void testFilterPushDownWithJoin() throws Exception {
+    String query = "select * from drill.%s e\n" +
+        "join drill.%s s on e.n_nationkey = s.n_nationkey where e.n_name = 'BRAZIL'";
+
+    queryBuilder()
+        .sql(query, TABLE_NAME, TABLE_NAME)
+        .planMatcher()
+        .include("INNER JOIN")
+        .match();
+
+    testBuilder()
+      .ordered()
+      .sqlQuery(query, TABLE_NAME, TABLE_NAME)
+      .baselineColumns("n_nationkey", "n_name", "n_regionkey", "n_comment", "n_nationkey0",
+        "n_name0", "n_regionkey0", "n_comment0")
+      .baselineValues(2, "BRAZIL", 1, "y alongside of the pending deposits. carefully special " +
+        "packages are about the ironic forges. slyly special ", 2, "BRAZIL", 1, "y alongside of " +
+        "the pending deposits. carefully special packages are about the ironic forges. slyly special ")
+      .go();
+  }
+
+  @Test
+  public void testJoinDifferentDrillPlugins() throws Exception {
+    String query = "select * from drill.%s e\n" +
+      "join drill2.cp.`tpch/nation.parquet` s on e.n_nationkey = s.n_nationkey where e.n_name = 'BRAZIL'";
+
+    queryBuilder()
+      .sql(query, TABLE_NAME, TABLE_NAME)
+      .planMatcher()
+      .include("HashJoin")
+      .exclude("INNER JOIN")
+      .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME, TABLE_NAME)
+      .baselineColumns("n_nationkey", "n_name", "n_regionkey", "n_comment", "n_nationkey0",
+        "n_name0", "n_regionkey0", "n_comment0")
+      .baselineValues(2, "BRAZIL", 1, "y alongside of the pending deposits. carefully special " +
+        "packages are about the ironic forges. slyly special ", 2, "BRAZIL", 1, "y alongside of " +
+        "the pending deposits. carefully special packages are about the ironic forges. slyly special ")
+      .go();
+  }
+
+  @Test
+  public void testAggregationPushDown() throws Exception {
+    String query = "select count(*) c from drill.%s";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("query=\"SELECT COUNT\\(\\*\\)")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("c")
+      .baselineValues(25L)
+      .go();
+  }
+
+  @Test
+  public void testLimitPushDown() throws Exception {
+    String query = "select n_name from drill.%s FETCH NEXT 1 ROWS ONLY";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("FETCH NEXT 1 ROWS ONLY")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("n_name")
+      .baselineValues("ALGERIA")
+      .go();
+  }
+
+  @Test
+  public void testLimitWithSortPushDown() throws Exception {
+    String query = "select n_nationkey from drill.%s order by n_name limit 3";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("ORDER BY `n_name`", "FETCH NEXT 3 ROWS ONLY")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("n_nationkey")
+      .baselineValues(0)
+      .baselineValues(1)
+      .baselineValues(2)
+      .go();
+  }
+
+  @Test
+  public void testAggregationWithGroupByPushDown() throws Exception {
+    String query = "select sum(n_nationkey) s from drill.%s group by n_regionkey";
+    queryBuilder()
+        .sql(query, TABLE_NAME)
+        .planMatcher()
+        .include("query=\"SELECT SUM\\(`n_nationkey`\\)", "GROUP BY `n_regionkey`")
+        .match();
+
+    testBuilder()
+      .unOrdered()
+      .sqlQuery(query, TABLE_NAME)
+      .baselineColumns("s")
+      .baselineValues(47L)
+      .baselineValues(50L)
+      .baselineValues(58L)
+      .baselineValues(68L)
+      .baselineValues(77L)
+      .go();
+  }
+
+  @Test
+  public void testUnionAllPushDown() throws Exception {
+    String query = "select col1, col2 from drill.%s " +
+      "union all " +
+      "select col1, col2 from drill.%s";
+    queryBuilder()
+      .sql(query, TABLE_NAME, TABLE_NAME)
+      .planMatcher()
+      .include("UNION ALL")
+      .match();
+
+    long recordCount = queryBuilder()
+      .sql(query, TABLE_NAME, TABLE_NAME)
+      .run()
+      .recordCount();
+
+    assertEquals(50L, recordCount);
+  }

Review Comment:
   Can we have a test of a schema path that descends through directories in a filesystem plugin on the remote Drill cluster? E.g.
   ```
   select * from drill.`dfs.tmp`.`/path/to/foo.parquet`
   ```



-- 
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 #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


##########
contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillSubScan.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.drill.exec.store.drill.plugin;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+@JsonTypeName("drill-read")
+public class DrillSubScan extends AbstractBase implements SubScan {
+
+  public static final String OPERATOR_TYPE = "DRILL_SUB_SCAN";
+
+  private final String query;
+
+  @JsonProperty
+  private final DrillStoragePluginConfig pluginConfig;
+
+  @JsonCreator
+  public DrillSubScan(
+      @JsonProperty("userName") String userName,
+      @JsonProperty("mongoPluginConfig") StoragePluginConfig pluginConfig,

Review Comment:
   No, it isn't, thanks, 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: dev-unsubscribe@drill.apache.org

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


[GitHub] [drill] cgivre merged pull request #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


-- 
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 #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


##########
contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillStoragePluginConfig.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.drill.plugin;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.calcite.avatica.ConnectStringParser;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.logical.security.CredentialsProvider;
+import org.apache.drill.common.logical.security.PlainCredentialsProvider;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.RpcException;
+
+import java.sql.SQLException;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+@JsonTypeName(DrillStoragePluginConfig.NAME)
+public class DrillStoragePluginConfig extends StoragePluginConfig {
+  public static final String NAME = "drill";
+  public static final String CONNECTION_STRING_PREFIX = "jdbc:drill:";
+
+  private static final String DEFAULT_QUOTING_IDENTIFIER = "`";
+
+  private final String connection;
+  private final Properties properties;
+
+  @JsonCreator
+  public DrillStoragePluginConfig(
+      @JsonProperty("connection") String connection,
+      @JsonProperty("properties") Properties properties,
+      @JsonProperty("credentialsProvider") CredentialsProvider credentialsProvider) {
+    super(getCredentialsProvider(credentialsProvider), credentialsProvider == null);
+    this.connection = connection;
+    this.properties = Optional.ofNullable(properties).orElse(new Properties());
+  }
+
+  private DrillStoragePluginConfig(DrillStoragePluginConfig that,
+    CredentialsProvider credentialsProvider) {
+    super(getCredentialsProvider(credentialsProvider),
+      credentialsProvider == null, that.authMode);
+    this.connection = that.connection;
+    this.properties = that.properties;
+  }
+
+  @JsonProperty("connection")
+  public String getConnection() {
+    return connection;
+  }
+
+  @JsonProperty("properties")
+  public Properties getProperties() {
+    return properties;
+  }
+
+  private static CredentialsProvider getCredentialsProvider(CredentialsProvider credentialsProvider) {
+    return credentialsProvider != null ? credentialsProvider : PlainCredentialsProvider.EMPTY_CREDENTIALS_PROVIDER;
+  }
+
+  @JsonIgnore
+  public String getIdentifierQuoteString() {
+    return properties.getProperty(DrillProperties.QUOTING_IDENTIFIERS, DEFAULT_QUOTING_IDENTIFIER);
+  }
+
+  @Override
+  public DrillStoragePluginConfig updateCredentialProvider(CredentialsProvider credentialsProvider) {
+    return new DrillStoragePluginConfig(this, credentialsProvider);
+  }
+
+  @JsonIgnore
+  public DrillClient getDrillClient(String userName, BufferAllocator allocator) {
+    try {
+      String urlSuffix = connection.substring(CONNECTION_STRING_PREFIX.length());
+      Properties props = ConnectStringParser.parse(urlSuffix, properties);
+      props.putAll(credentialsProvider.getUserCredentials(userName));

Review Comment:
   This getUserCredentials(String username) method is meant to fetch per-query-user credentials for plugins that are in user translation auth mode while the nullary method getUserCredentials() is meant for shared credentials. Only the plan and Vault providers currently support per-user credentials. You can see some logic for deciding which to call  (via UsernamePasswordCredentials objects) in JdbcStorageConfig on line 142.
   
   Those APIs wound up being a little ugly :/



##########
contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/package-info.java:
##########
@@ -15,24 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.jdbc;
-
-import java.sql.SQLTimeoutException;
-
 /**
- * Indicates that an operation timed out. This is not an error; you can
- * retry the operation.
+ * Drill storage plugin.
+ * <p>
+ * Enables querying Drill as a data store.
  */
-public class SqlTimeoutException extends SQLTimeoutException {
-  private static final long serialVersionUID = 2017_04_03L;
-
-  SqlTimeoutException() {
-    // SQLException(reason, SQLState, vendorCode)
-    // REVIEW mb 19-Jul-05 Is there a standard SQLState?
-    super("timeout", null, 0);
-  }
-
-  public SqlTimeoutException(long timeoutValueInSeconds) {
-    super("Query timed out in "+ timeoutValueInSeconds + " seconds");
-  }
-}
+package org.apache.drill.exec.store.drill.plugin;

Review Comment:
   So Git decided that this was the renaming of a file 😏



-- 
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 #2709: DRILL-8358: Storage plugin for querying other Apache Drill clusters

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


##########
contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillStoragePluginConfig.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.drill.plugin;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.calcite.avatica.ConnectStringParser;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.logical.security.CredentialsProvider;
+import org.apache.drill.common.logical.security.PlainCredentialsProvider;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.RpcException;
+
+import java.sql.SQLException;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+@JsonTypeName(DrillStoragePluginConfig.NAME)
+public class DrillStoragePluginConfig extends StoragePluginConfig {
+  public static final String NAME = "drill";
+  public static final String CONNECTION_STRING_PREFIX = "jdbc:drill:";
+
+  private static final String DEFAULT_QUOTING_IDENTIFIER = "`";
+
+  private final String connection;
+  private final Properties properties;
+
+  @JsonCreator
+  public DrillStoragePluginConfig(
+      @JsonProperty("connection") String connection,
+      @JsonProperty("properties") Properties properties,
+      @JsonProperty("credentialsProvider") CredentialsProvider credentialsProvider) {
+    super(getCredentialsProvider(credentialsProvider), credentialsProvider == null);
+    this.connection = connection;
+    this.properties = Optional.ofNullable(properties).orElse(new Properties());
+  }
+
+  private DrillStoragePluginConfig(DrillStoragePluginConfig that,
+    CredentialsProvider credentialsProvider) {
+    super(getCredentialsProvider(credentialsProvider),
+      credentialsProvider == null, that.authMode);
+    this.connection = that.connection;
+    this.properties = that.properties;
+  }
+
+  @JsonProperty("connection")
+  public String getConnection() {
+    return connection;
+  }
+
+  @JsonProperty("properties")
+  public Properties getProperties() {
+    return properties;
+  }
+
+  private static CredentialsProvider getCredentialsProvider(CredentialsProvider credentialsProvider) {
+    return credentialsProvider != null ? credentialsProvider : PlainCredentialsProvider.EMPTY_CREDENTIALS_PROVIDER;
+  }
+
+  @JsonIgnore
+  public String getIdentifierQuoteString() {
+    return properties.getProperty(DrillProperties.QUOTING_IDENTIFIERS, DEFAULT_QUOTING_IDENTIFIER);
+  }
+
+  @Override
+  public DrillStoragePluginConfig updateCredentialProvider(CredentialsProvider credentialsProvider) {
+    return new DrillStoragePluginConfig(this, credentialsProvider);
+  }
+
+  @JsonIgnore
+  public DrillClient getDrillClient(String userName, BufferAllocator allocator) {
+    try {
+      String urlSuffix = connection.substring(CONNECTION_STRING_PREFIX.length());
+      Properties props = ConnectStringParser.parse(urlSuffix, properties);
+      props.putAll(credentialsProvider.getUserCredentials(userName));

Review Comment:
   This getUserCredentials(String username) method is meant to fetch per-query-user credentials for plugins that are in user translation auth mode while the nullary method getUserCredentials() is meant for shared credentials. Only the plain and Vault providers currently support per-user credentials. You can see some logic for deciding which to call (via UsernamePasswordCredentials objects) in JdbcStorageConfig on line 142.
   
   Those APIs wound up being a little ugly :/



-- 
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