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/03/16 18:47:37 UTC

[GitHub] [drill] cgivre opened a new pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

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


   # [DRILL-8169](https://issues.apache.org/jira/browse/DRILL-8169): Add UDFs to HTTP Plugin to Facilitate Joins
   
   ## Description
   
   (Please describe the change. If more than one ticket is fixed, include a reference to those tickets.)
   
   ## Documentation
   There are some situations where a user might want to join data with an API result and the pushdowns prevent that from happening.  The main situation where this happens is when 
   an API has parameters which are part of the URL AND these parameters are dynamically populated via a join. 
   
   In this case, there are two functions `http_get_url` and `http_get` which you can use to faciliate these joins. 
   
   * `http_get('<storage_plugin_name>', <params>)`:  This function accepts a storage plugin as input and an optional list of parameters to include in a URL.
   * `http_get_url(<url>, <params>)`:  This function works in the same way except that it does not pull any configuration information from existing storage plugins.
   
   ### Example Queries
   Let's say that you have a storage plugin called `github` with an endpoint called `repos` which points to the url: https://github.com/orgs/{org}/repos.  It is easy enough to 
   write a query like this:
   
   ```sql
   SELECT * 
   FROM github.repos
   WHERE org='apache'
   ```
   However, if you had a file with organizations and wanted to join this with the API, the query would fail.  Using the functions listed above you could get this data as follows:
   
   ```sql
   SELECT http_get('github.repos', `org`)
   FROM dfs.`some_data.csvh`
   ```
   or
   ```sql
   SELECT http_get('https://github.com/orgs/{org}/repos', `org`)
   FROM dfs.`some_data.csvh`
   ```
   
   **WARNING:  This functionality will execute an HTTP Request FOR EVERY ROW IN YOUR DATA.  Use with caution.**
   
   ## 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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
jnturton commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829080866



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       Thank you, I think it's better to move or absorb it into something in storage-http.  To preserve layering and to eventually decouple our plugins successfully, we will want them not to have introduced plugin-specific code into exec/.  These utils are specific to storage-http in places e.g. `getDefaultParameterValue(...)`

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       Yes I think the other one can become `http_get`.  Maybe later it will be joined by an `http_post` which has a parameter called `content_type` that allows values of 'application/x-www-form-urlencoded' and 'application/json' or something.




-- 
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 pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
jnturton commented on pull request #2496:
URL: https://github.com/apache/drill/pull/2496#issuecomment-1072373589


   Thanks for making changes. +1, I'm fine here if @vvysotskyi's fine.


-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829656249



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];

Review comment:
       Done!




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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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



[GitHub] [drill] cgivre commented on pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2496:
URL: https://github.com/apache/drill/pull/2496#issuecomment-1072362265


   @jnturton @vvysotskyi Thanks for your review.  I believe I addressed all your comments.  Could you please take another look?   Thanks!


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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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



[GitHub] [drill] jnturton commented on pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
jnturton commented on pull request #2496:
URL: https://github.com/apache/drill/pull/2496#issuecomment-1072373589


   Thanks for making changes. +1, I'm fine here if @vvysotskyi's fine.


-- 
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 pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2496:
URL: https://github.com/apache/drill/pull/2496#issuecomment-1072362265


   @jnturton @vvysotskyi Thanks for your review.  I believe I addressed all your comments.  Could you please take another look?   Thanks!


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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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



[GitHub] [drill] cgivre commented on a change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r830049045



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    FieldReader[] inputReaders;

Review comment:
       @vvysotskyi One thing to think about is that the input args may not be strings.  If I make this a `VarCharHolder` would that break things if the user tries to use an int field?




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829066804



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       So... originally, I started work with this in the `udfs` package.  I thought it would be a good idea to make a Drill utils class that can be accessed anywhere that has all the associated methods for making http requests.   Then, I ended up moving the UDF to `storage-http` because it needed the http storage config and never relocated it.  
   
   If you feel strongly about it, I can move all those methods to `SimpleHttp` and eliminate the class.
   




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829065654



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       sure!  Do you think I should rename the other one 'http_get' because that is only a simple http get?




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829656793



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();

Review comment:
       I did this for the UDF that accepts a storage plugin as the first argument, but not the first one.  The reason being that the there are some APIs which return URLs and I could imagine a use case where a user might want to generate a list of URLs then call them. 




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829656195



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       Done!




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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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



[GitHub] [drill] vvysotskyi commented on a change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829053206



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];

Review comment:
       If the first argument is the URL string, is it makes sense to handle it separately instead of the list? As a bonus, this UDF wouldn't match if the URL wasn't specified.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();

Review comment:
       If it is a String and we assume that it should be constant, why not declare the input with `@Param(constant = true)` and use `VarCharHolder`?

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];

Review comment:
       In this case, we should document that a fully qualified schema should be used, i.e. `use http;` before running the query wouldn't affect anything.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);

Review comment:
       Instead of obtaining `pluginConfig` here and in `getEndpointConfig`, you could change `getEndpointConfig` to accept obtained `pluginConfig` earlier.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;
+
+import okhttp3.HttpUrl;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class HttpUtils {

Review comment:
       This class could be moved to http plugin instead of exec.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+
+    // Get proxy settings
+    HttpProxyConfig proxyConfig = SimpleHttp.getProxySettings(pluginConfig, context.getConfig(), endpointConfig.getHttpUrl());
+
+    // For this use case, we will replace the URL parameters here, rather than doing it in the SimpleHttp client
+    // because we are using positional mapping rather than k/v pairs for this.
+    String finalUrl;

Review comment:
       could some of this logic be combined with the code from `HttpBatchReader`, so we will have a single place with the logic related to handling it?

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    FieldReader[] inputReaders;

Review comment:
       This one could also be `VarCharHolder[]`.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    FieldReader[] inputReaders;

Review comment:
       I'm not sure regarding current implementation, whether it would work with arguments of different types.




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r830124255



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    FieldReader[] inputReaders;

Review comment:
       Thanks @vvysotskyi.  I changed to `VarCharHolder[]` for both functions.




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r830022508



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    FieldReader[] inputReaders;

Review comment:
       This one could also be `VarCharHolder[]`.




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829656132



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       Done!  I called them `http_get` for the simple get request, and `http_request` for the request from storage.




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
jnturton commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r828820399



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       If this UDF is not only for for GET requests, but for whatever request type is configured in the named storage config then can we name it differently?  `http_req_storage` or something?

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       This class looks like it belongs in contrib/storage-http.  Is there a reason why it's here?




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

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

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



[GitHub] [drill] cgivre commented on a change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829637402



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;
+
+import okhttp3.HttpUrl;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class HttpUtils {

Review comment:
       Done!  Removed this class entirely. 




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
jnturton commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r828820399



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       If this UDF is not only for for GET requests, but for whatever request type is configured in the named storage config then can we name it differently?  `http_req_storage` or something?

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       This class looks like it belongs in contrib/storage-http.  Is there a reason why it's here?

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       Thank you, I think it's better to move or absorb it into something in storage-http.  To preserve layering and to eventually decouple our plugins successfully, we will want them not to have introduced plugin-specific code into exec/.  These utils are specific to storage-http in places e.g. `getDefaultParameterValue(...)`

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       Yes I think the other one can become `http_get`.  Maybe later it will be joined by an `http_post` which has a parameter called `content_type` that allows values of 'application/x-www-form-urlencoded' and 'application/json' or something.




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829065654



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       sure!  Do you think I should rename the other one 'http_get' because that is only a simple http get?

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       So... originally, I started work with this in the `udfs` package.  I thought it would be a good idea to make a Drill utils class that can be accessed anywhere that has all the associated methods for making http requests.   Then, I ended up moving the UDF to `storage-http` because it needed the http storage config and never relocated it.  
   
   If you feel strongly about it, I can move all those methods to `SimpleHttp` and eliminate the class.
   

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;
+
+import okhttp3.HttpUrl;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class HttpUtils {

Review comment:
       Done!  Removed this class entirely. 

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();
+
+        // Process Positional Arguments
+        java.util.List args = org.apache.drill.exec.store.http.udfs.HttpHelperUtils.buildParameterList(inputReaders);
+        String finalUrl = org.apache.drill.exec.util.HttpUtils.mapPositionalParameters(url, args);
+
+        // Make the API call
+        String results = org.apache.drill.exec.util.HttpUtils.makeSimpleGetRequest(finalUrl);
+
+        // If the result string is null or empty, return an empty map
+        if (results == null || results.length() == 0) {
+          // Return empty map
+          org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = writer.rootAsMap();
+          mapWriter.start();
+          mapWriter.end();
+          return;
+        }
+
+        try {
+          jsonReader.setSource(results);
+          jsonReader.setIgnoreJSONParseErrors(true);  // Reduce number of errors
+          jsonReader.write(writer);
+          buffer = jsonReader.getWorkBuf();
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Error while converting from JSON. ", e);
+        }
+      }
+    }
+  }
+
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},

Review comment:
       Done!  I called them `http_get` for the simple get request, and `http_request` for the request from storage.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;

Review comment:
       Done!

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];

Review comment:
       Done!

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();

Review comment:
       I did this for the UDF that accepts a storage plugin as the first argument, but not the first one.  The reason being that the there are some APIs which return URLs and I could imagine a use case where a user might want to generate a list of URLs then call them. 

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);

Review comment:
       Fixed!

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+
+    // Get proxy settings
+    HttpProxyConfig proxyConfig = SimpleHttp.getProxySettings(pluginConfig, context.getConfig(), endpointConfig.getHttpUrl());
+
+    // For this use case, we will replace the URL parameters here, rather than doing it in the SimpleHttp client
+    // because we are using positional mapping rather than k/v pairs for this.
+    String finalUrl;

Review comment:
       Fixed.  Moved everything to `SimpleHttp`.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];

Review comment:
       Thanks @vvysotskyi.  In this case, the URL is first validated in the `mapPositionalArguments` function.   I added this to the documentation as well. 

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get", "httpGet"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    VarCharHolder rawInput;
+
+    @Param
+    FieldReader[] inputReaders;

Review comment:
       @vvysotskyi One thing to think about is that the input args may not be strings.  If I make this a `VarCharHolder` would that break things if the user tries to use an int field?




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829659506



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+
+    // Get proxy settings
+    HttpProxyConfig proxyConfig = SimpleHttp.getProxySettings(pluginConfig, context.getConfig(), endpointConfig.getHttpUrl());
+
+    // For this use case, we will replace the URL parameters here, rather than doing it in the SimpleHttp client
+    // because we are using positional mapping rather than k/v pairs for this.
+    String finalUrl;

Review comment:
       Fixed.  Moved everything to `SimpleHttp`.




-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829658108



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);

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] vvysotskyi commented on a change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
vvysotskyi commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829053206



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];

Review comment:
       If the first argument is the URL string, is it makes sense to handle it separately instead of the list? As a bonus, this UDF wouldn't match if the URL wasn't specified.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.http.udfs;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+
+import javax.inject.Inject;
+
+public class HttpHelperFunctions {
+
+  @FunctionTemplate(names = {"http_get_url", "httpGetUrl"},
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    isVarArg = true)
+  public static class HttpGetFunction implements DrillSimpleFunc {
+
+    @Param
+    FieldReader[] inputReaders;
+
+    @Output
+    ComplexWriter writer;
+
+    @Inject
+    OptionManager options;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    org.apache.drill.exec.vector.complex.fn.JsonReader jsonReader;
+
+    @Override
+    public void setup() {
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader.Builder(buffer)
+        .defaultSchemaPathColumns()
+        .readNumbersAsDouble(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE).bool_val)
+        .allTextMode(options.getOption(org.apache.drill.exec.ExecConstants.JSON_ALL_TEXT_MODE).bool_val)
+        .enableNanInf(options.getOption(org.apache.drill.exec.ExecConstants.JSON_READER_NAN_INF_NUMBERS).bool_val)
+        .build();
+    }
+
+    @Override
+    public void eval() {
+      if (inputReaders.length > 0) {
+        // Get the URL
+        FieldReader urlReader = inputReaders[0];
+        String url = urlReader.readObject().toString();

Review comment:
       If it is a String and we assume that it should be constant, why not declare the input with `@Param(constant = true)` and use `VarCharHolder`?

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];

Review comment:
       In this case, we should document that a fully qualified schema should be used, i.e. `use http;` before running the query wouldn't affect anything.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);

Review comment:
       Instead of obtaining `pluginConfig` here and in `getEndpointConfig`, you could change `getEndpointConfig` to accept obtained `pluginConfig` earlier.

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/util/HttpUtils.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.util;
+
+import okhttp3.HttpUrl;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class HttpUtils {

Review comment:
       This class could be moved to http plugin instead of exec.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];
+
+    HttpStoragePlugin plugin = getStoragePlugin(context, pluginName);
+
+    try {
+      pluginConfig = getPluginConfig(pluginName, context);
+      endpointConfig = getEndpointConfig(schemaPath, context);
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+
+    // Get proxy settings
+    HttpProxyConfig proxyConfig = SimpleHttp.getProxySettings(pluginConfig, context.getConfig(), endpointConfig.getHttpUrl());
+
+    // For this use case, we will replace the URL parameters here, rather than doing it in the SimpleHttp client
+    // because we are using positional mapping rather than k/v pairs for this.
+    String finalUrl;

Review comment:
       could some of this logic be combined with the code from `HttpBatchReader`, so we will have a single place with the logic related to handling 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 #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

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


   


-- 
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 change in pull request #2496: DRILL-8169: Add UDFs to HTTP Plugin to Facilitate Joins

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2496:
URL: https://github.com/apache/drill/pull/2496#discussion_r829950722



##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperUtils.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.http.udfs;
+
+import okhttp3.HttpUrl;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpStoragePlugin;
+import org.apache.drill.exec.store.http.HttpStoragePluginConfig;
+import org.apache.drill.exec.store.http.util.HttpProxyConfig;
+import org.apache.drill.exec.store.http.util.SimpleHttp;
+import org.apache.drill.exec.store.http.util.SimpleHttp.SimpleHttpBuilder;
+import org.apache.drill.exec.util.HttpUtils;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+public class HttpHelperUtils {
+
+  private static Logger logger = LoggerFactory.getLogger(HttpHelperUtils.class);
+  /**
+   * Accepts a list of input readers and converts that into an ArrayList of Strings
+   * @param inputReaders The array of FieldReaders
+   * @return A List of Strings containing the values from the FieldReaders.
+   */
+  public static List<String> buildParameterList(FieldReader[] inputReaders) {
+    List<String> inputArguments = new ArrayList<>();
+
+    // Skip the first argument because that is the input URL
+    for (int i = 1; i < inputReaders.length; i++) {
+      inputArguments.add(inputReaders[i].readObject().toString());
+    }
+
+    return inputArguments;
+  }
+
+  public static HttpStoragePluginConfig getPluginConfig(String name, DrillbitContext context) throws PluginException {
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, name);
+    return httpStoragePlugin.getConfig();
+  }
+
+  public static HttpApiConfig getEndpointConfig(String name, DrillbitContext context) {
+    // Get the plugin name and endpoint name
+    String[] parts = name.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String plugin = parts[0];
+    String endpoint = parts[1];
+
+    HttpStoragePlugin httpStoragePlugin = getStoragePlugin(context, plugin);
+    HttpStoragePluginConfig config = httpStoragePlugin.getConfig();
+
+    HttpApiConfig endpointConfig = config.getConnection(endpoint);
+    if (endpointConfig == null) {
+      throw UserException.functionError()
+        .message("You must call this function with a valid endpoint name.")
+        .build(logger);
+    } else if (endpointConfig.inputType() != "json") {
+      throw UserException.functionError()
+        .message("Http_get only supports API endpoints which return json.")
+        .build(logger);
+    }
+
+    return endpointConfig;
+  }
+
+  private static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+    StoragePluginRegistry storage = context.getStorage();
+    try {
+      StoragePlugin pluginInstance = storage.getPlugin(pluginName);
+      if (pluginInstance == null) {
+        throw UserException.functionError()
+          .message(pluginName + " is not a valid plugin.")
+          .build(logger);
+      }
+
+      if (!(pluginInstance instanceof HttpStoragePlugin)) {
+        throw UserException.functionError()
+          .message("You can only include HTTP plugins in this function.")
+          .build(logger);
+      }
+      return (HttpStoragePlugin) pluginInstance;
+    } catch (PluginException e) {
+      throw UserException.functionError()
+        .message("Could not access plugin " + pluginName)
+        .build(logger);
+    }
+  }
+
+
+  /**
+   * This function makes an API call and returns a string of the parsed results. It is used in the http_get() UDF
+   * and retrieves all the configuration parameters contained in the storage plugin and endpoint configuration. The exception
+   * is pagination.  This does not support pagination.
+   * @param schemaPath The path of storage_plugin.endpoint from which the data will be retrieved
+   * @param context {@link DrillbitContext} The context from the current query
+   * @param args An optional list of parameter arguments which will be included in the URL
+   * @return A String of the results.
+   */
+  public static String makeAPICall(String schemaPath, DrillbitContext context, List<String> args) {
+    HttpStoragePluginConfig pluginConfig;
+    HttpApiConfig endpointConfig;
+
+    // Get the plugin name and endpoint name
+    String[] parts = schemaPath.split("\\.");
+    if (parts.length < 2) {
+      throw UserException.functionError()
+        .message("You must call this function with a connection name and endpoint.")
+        .build(logger);
+    }
+    String pluginName = parts[0];

Review comment:
       Thanks @vvysotskyi.  In this case, the URL is first validated in the `mapPositionalArguments` function.   I added this to the documentation as well. 




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