You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by cg...@apache.org on 2022/06/09 01:17:45 UTC

[drill] branch master updated: DRILL-8244: HTTP_Request Not Passing Down Config Variables (#2571)

This is an automated email from the ASF dual-hosted git repository.

cgivre pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new 20f65465cf DRILL-8244: HTTP_Request Not Passing Down Config Variables (#2571)
20f65465cf is described below

commit 20f65465cfb20a37c693dd3bc6a1befd594c8dad
Author: Charles S. Givre <cg...@apache.org>
AuthorDate: Wed Jun 8 21:17:37 2022 -0400

    DRILL-8244: HTTP_Request Not Passing Down Config Variables (#2571)
    
    * Initial Commit
    
    * Addressed review comments, removed logging statement
---
 .../exec/store/http/udfs/HttpHelperFunctions.java  |  9 ++--
 .../drill/exec/store/http/udfs/HttpUdfUtils.java   | 62 ++++++++++++++++++++++
 .../exec/store/http/TestHttpUDFFunctions.java      | 46 ++++++++++++++++
 3 files changed, 113 insertions(+), 4 deletions(-)

diff --git a/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java b/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
index 1ae05a501f..5e73b716ca 100644
--- a/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
+++ b/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpHelperFunctions.java
@@ -135,10 +135,6 @@ public class HttpHelperFunctions {
 
     @Override
     public void setup() {
-      jsonLoaderBuilder = new org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl.JsonLoaderBuilder()
-        .resultSetLoader(loader)
-        .standardOptions(options);
-
       String schemaPath = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(rawInput.start, rawInput.end, rawInput.buffer);
       // Get the plugin name and endpoint name
       String[] parts = schemaPath.split("\\.");
@@ -153,10 +149,15 @@ public class HttpHelperFunctions {
         drillbitContext,
         pluginName
       );
+
       endpointConfig = org.apache.drill.exec.store.http.util.SimpleHttp.getEndpointConfig(
         endpointName,
         plugin.getConfig()
       );
+
+      // Add JSON configuration from Storage plugin, if present.
+      jsonLoaderBuilder = org.apache.drill.exec.store.http.udfs.HttpUdfUtils.setupJsonBuilder(endpointConfig, loader, options);
+
     }
 
     @Override
diff --git a/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpUdfUtils.java b/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpUdfUtils.java
new file mode 100644
index 0000000000..93f078f7e2
--- /dev/null
+++ b/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/udfs/HttpUdfUtils.java
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.commons.lang3.StringUtils;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl.JsonLoaderBuilder;
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderOptions;
+import org.apache.drill.exec.store.http.HttpApiConfig;
+import org.apache.drill.exec.store.http.HttpJsonOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpUdfUtils {
+
+  private static final Logger logger = LoggerFactory.getLogger(HttpUdfUtils.class);
+
+  public static JsonLoaderBuilder setupJsonBuilder(HttpApiConfig endpointConfig, ResultSetLoader loader, OptionManager options) {
+    loader.setTargetRowCount(1);
+    // Add JSON configuration from Storage plugin, if present.
+    HttpJsonOptions jsonOptions = endpointConfig.jsonOptions();
+    JsonLoaderBuilder jsonLoaderBuilder = new JsonLoaderBuilder()
+      .resultSetLoader(loader)
+      .maxRows(1)
+      .standardOptions(options);
+
+    // Add data path if present
+    if (StringUtils.isNotEmpty(endpointConfig.dataPath())) {
+      jsonLoaderBuilder.dataPath(endpointConfig.dataPath());
+    }
+
+    if (jsonOptions != null) {
+      // Add options from endpoint configuration to jsonLoader
+      JsonLoaderOptions jsonLoaderOptions = jsonOptions.getJsonOptions(options);
+      jsonLoaderBuilder.options(jsonLoaderOptions);
+
+      // Add provided schema if present
+      if (jsonOptions.schema() != null) {
+        logger.debug("Found schema: {}", jsonOptions.schema());
+        jsonLoaderBuilder.providedSchema(jsonOptions.schema());
+      }
+    }
+    return jsonLoaderBuilder;
+  }
+}
diff --git a/contrib/storage-http/src/test/java/org/apache/drill/exec/store/http/TestHttpUDFFunctions.java b/contrib/storage-http/src/test/java/org/apache/drill/exec/store/http/TestHttpUDFFunctions.java
index fb30bef5aa..2cbe8d4b85 100644
--- a/contrib/storage-http/src/test/java/org/apache/drill/exec/store/http/TestHttpUDFFunctions.java
+++ b/contrib/storage-http/src/test/java/org/apache/drill/exec/store/http/TestHttpUDFFunctions.java
@@ -25,6 +25,7 @@ import okhttp3.mockwebserver.RecordedRequest;
 import org.apache.drill.common.logical.StoragePluginConfig.AuthMode;
 import org.apache.drill.common.logical.security.PlainCredentialsProvider;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.physical.impl.project.ProjectMemoryManager;
 import org.apache.drill.exec.physical.impl.project.ProjectRecordBatch;
@@ -64,6 +65,7 @@ public class TestHttpUDFFunctions extends ClusterTest {
 
   private static final int MOCK_SERVER_PORT = 47771;
   private static String TEST_JSON_RESPONSE;
+  private static String TEST_JSON_PAGE1;
   private static String DUMMY_URL = "http://localhost:" + MOCK_SERVER_PORT;
   protected static LogFixture logFixture;
   private final static Level CURRENT_LOG_LEVEL = Level.INFO;
@@ -80,6 +82,7 @@ public class TestHttpUDFFunctions extends ClusterTest {
       .build();
     startCluster(ClusterFixture.builder(dirTestWatcher));
     TEST_JSON_RESPONSE = Files.asCharSource(DrillFileUtils.getResourceAsFile("/data/simple.json"), Charsets.UTF_8).read();
+    TEST_JSON_PAGE1 = Files.asCharSource(DrillFileUtils.getResourceAsFile("/data/p1.json"), Charsets.UTF_8).read();
 
     HttpApiConfig mockGithubWithDuplicateParam = HttpApiConfig.builder()
       .url(String.format("%s/orgs/{org}/repos", DUMMY_URL))
@@ -89,8 +92,27 @@ public class TestHttpUDFFunctions extends ClusterTest {
       .requireTail(false)
       .build();
 
+    TupleMetadata simpleSchema = new SchemaBuilder()
+      .addNullable("col_1", MinorType.FLOAT8)
+      .addNullable("col_2", MinorType.FLOAT8)
+      .addNullable("col_3", MinorType.FLOAT8)
+      .build();
+
+    HttpJsonOptions jsonOptions = new HttpJsonOptions.HttpJsonOptionsBuilder()
+      .schema(simpleSchema)
+      .build();
+
+    HttpApiConfig basicJson = HttpApiConfig.builder()
+      .url(String.format("%s/json", DUMMY_URL))
+      .method("get")
+      .jsonOptions(jsonOptions)
+      .requireTail(false)
+      .inputType("json")
+      .build();
+
     Map<String, HttpApiConfig> configs = new HashMap<>();
     configs.put("github", mockGithubWithDuplicateParam);
+    configs.put("basicJson", basicJson);
 
     HttpStoragePluginConfig mockStorageConfigWithWorkspace =
       new HttpStoragePluginConfig(false, configs, 2, "globaluser", "globalpass", "",
@@ -101,6 +123,30 @@ public class TestHttpUDFFunctions extends ClusterTest {
     cluster.defineStoragePlugin("local", mockStorageConfigWithWorkspace);
   }
 
+  @Test
+  public void testProvidedSchema() throws Exception {
+    String sql = "SELECT http_request('local.basicJson') as data FROM (values(1))";
+    try (MockWebServer server = startServer()) {
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+      RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+      TupleMetadata expectedSchema = new SchemaBuilder()
+        .addMap("data")
+          .addNullable("col_1", MinorType.FLOAT8)
+          .addNullable("col_2", MinorType.FLOAT8)
+          .addNullable("col_3", MinorType.FLOAT8)
+        .resumeSchema()
+        .build();
+
+      RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+        .addRow(singleMap(
+          mapValue(1.0, 2.0, 3.0)))
+        .build();
+
+      RowSetUtilities.verify(expected, results);
+    }
+  }
+
   @Test
   public void testHttpGetWithNoParams() throws Exception {
     try (MockWebServer server = startServer()) {