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/10/06 19:30:30 UTC

[drill] branch master updated: DRILL-8328: HTTP UDF Not Resolving Storage Aliases (#2668)

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 2174f8ac05 DRILL-8328: HTTP UDF Not Resolving Storage Aliases (#2668)
2174f8ac05 is described below

commit 2174f8ac05bf1a59ebf97ed7da1fe14ceb5a438e
Author: Charles S. Givre <cg...@apache.org>
AuthorDate: Thu Oct 6 15:30:23 2022 -0400

    DRILL-8328: HTTP UDF Not Resolving Storage Aliases (#2668)
---
 .../exec/store/http/udfs/HttpHelperFunctions.java  |   5 +-
 .../drill/exec/store/http/util/SimpleHttp.java     |  20 +-
 .../exec/store/http/TestHttpUDFWithAliases.java    | 276 +++++++++++++++++++++
 .../org/apache/drill/exec/ops/QueryContext.java    |   2 +-
 .../drill/exec/planner/logical/DrillOptiq.java     |  50 ++++
 .../exec/planner/physical/PlannerSettings.java     |  18 +-
 .../exec/work/metadata/ServerMetaProvider.java     |   2 +-
 .../test/java/org/apache/drill/PlanningBase.java   |   2 +-
 8 files changed, 364 insertions(+), 11 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 84a9262d3a..b21e04caa2 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
@@ -152,10 +152,7 @@ public class HttpHelperFunctions {
       }
       String pluginName = parts[0], endpointName = parts[1];
 
-      plugin = org.apache.drill.exec.store.http.util.SimpleHttp.getStoragePlugin(
-        drillbitContext,
-        pluginName
-      );
+      plugin = org.apache.drill.exec.store.http.util.SimpleHttp.getStoragePlugin(pluginName, drillbitContext);
 
       endpointConfig = org.apache.drill.exec.store.http.util.SimpleHttp.getEndpointConfig(
         endpointName,
diff --git a/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/util/SimpleHttp.java b/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/util/SimpleHttp.java
index 2c8589649e..41c03995c6 100644
--- a/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/util/SimpleHttp.java
+++ b/contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/util/SimpleHttp.java
@@ -832,7 +832,15 @@ public class SimpleHttp implements AutoCloseable {
     return inputArguments;
   }
 
-  public static HttpApiConfig getEndpointConfig(String endpoint, HttpStoragePluginConfig pluginConfig) {
+  /**
+   * This function is used to obtain the configuration information for a given API in the HTTP UDF.
+   * If aliasing is enabled, this function will resolve aliases for connections.
+   * @param endpoint The name of the endpoint.  Should be a {@link String}
+   * @param pluginConfig The {@link HttpStoragePluginConfig} the configuration from the plugin
+   * @return The {@link HttpApiConfig} corresponding with the endpoint.
+   */
+  public static HttpApiConfig getEndpointConfig(String endpoint,
+                                                HttpStoragePluginConfig pluginConfig) {
     HttpApiConfig endpointConfig = pluginConfig.getConnection(endpoint);
     if (endpointConfig == null) {
       throw UserException.functionError()
@@ -847,7 +855,14 @@ public class SimpleHttp implements AutoCloseable {
     return endpointConfig;
   }
 
-  public static HttpStoragePlugin getStoragePlugin(DrillbitContext context, String pluginName) {
+  /**
+   * This function will return a {@link HttpStoragePlugin} for use in the HTTP UDFs.  If user or public aliases
+   * are used, the function will resolve those aliases.
+   * @param context A {@link DrillbitContext} from the current query
+   * @param pluginName A {@link String} of the plugin name.  Note that the function will resolve aliases.
+   * @return A {@link HttpStoragePlugin} of the plugin.
+   */
+  public static HttpStoragePlugin getStoragePlugin(String pluginName, DrillbitContext context) {
     StoragePluginRegistry storage = context.getStorage();
     try {
       StoragePlugin pluginInstance = storage.getPlugin(pluginName);
@@ -870,7 +885,6 @@ public class SimpleHttp implements AutoCloseable {
     }
   }
 
-
   /**
    * 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
diff --git a/contrib/storage-http/src/test/java/org/apache/drill/exec/store/http/TestHttpUDFWithAliases.java b/contrib/storage-http/src/test/java/org/apache/drill/exec/store/http/TestHttpUDFWithAliases.java
new file mode 100644
index 0000000000..15152de5cd
--- /dev/null
+++ b/contrib/storage-http/src/test/java/org/apache/drill/exec/store/http/TestHttpUDFWithAliases.java
@@ -0,0 +1,276 @@
+/*
+ * 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;
+
+import okhttp3.mockwebserver.MockResponse;
+import okhttp3.mockwebserver.MockWebServer;
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.common.logical.StoragePluginConfig.AuthMode;
+import org.apache.drill.common.logical.security.PlainCredentialsProvider;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.alias.AliasRegistry;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl;
+import org.apache.drill.exec.store.security.UsernamePasswordCredentials;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
+import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl.ADMIN_GROUP;
+import static org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl.ADMIN_USER;
+import static org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl.ADMIN_USER_PASSWORD;
+import static org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl.PROCESS_USER;
+import static org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl.PROCESS_USER_PASSWORD;
+import static org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl.TEST_USER_2;
+import static org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl.TEST_USER_2_PASSWORD;
+import static org.apache.drill.test.rowSet.RowSetUtilities.mapValue;
+import static org.apache.drill.test.rowSet.RowSetUtilities.singleMap;
+import static org.junit.Assert.assertEquals;
+
+public class TestHttpUDFWithAliases extends ClusterTest {
+
+  private static AliasRegistry storageAliasesRegistry;
+  private static AliasRegistry tableAliasesRegistry;
+  private static final int MOCK_SERVER_PORT = 47778;
+  private static String TEST_JSON_PAGE1;
+  private static final String DUMMY_URL = "http://localhost:" + MOCK_SERVER_PORT;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+
+    TEST_JSON_PAGE1 = Files.asCharSource(DrillFileUtils.getResourceAsFile("/data/p1.json"), Charsets.UTF_8).read();
+
+    cluster = ClusterFixture.bareBuilder(dirTestWatcher)
+      .configProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, true)
+      .configProperty(ExecConstants.IMPERSONATION_ENABLED, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, UserAuthenticatorTestImpl.TYPE)
+      .configProperty(DrillProperties.USER, PROCESS_USER)
+      .configProperty(DrillProperties.PASSWORD, PROCESS_USER_PASSWORD)
+      .build();
+
+    ClientFixture admin = cluster.clientBuilder()
+      .property(DrillProperties.USER, PROCESS_USER)
+      .property(DrillProperties.PASSWORD, PROCESS_USER_PASSWORD)
+      .build();
+
+    admin.alterSystem(ExecConstants.ADMIN_USERS_KEY, ADMIN_USER + "," + PROCESS_USER);
+    admin.alterSystem(ExecConstants.ADMIN_USER_GROUPS_KEY, ADMIN_GROUP);
+
+    client = cluster.clientBuilder()
+      .property(DrillProperties.USER, ADMIN_USER)
+      .property(DrillProperties.PASSWORD, ADMIN_USER_PASSWORD)
+      .build();
+    storageAliasesRegistry = cluster.drillbit().getContext().getAliasRegistryProvider().getStorageAliasesRegistry();
+    tableAliasesRegistry = cluster.drillbit().getContext().getAliasRegistryProvider().getTableAliasesRegistry();
+
+    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("basicJson", basicJson);
+
+    HttpStoragePluginConfig mockStorageConfigWithWorkspace =
+      new HttpStoragePluginConfig(false, configs, 200, "globaluser", "globalpass", "",
+        80, "", "", "", null, new PlainCredentialsProvider(ImmutableMap.of(
+        UsernamePasswordCredentials.USERNAME, "globaluser",
+        UsernamePasswordCredentials.PASSWORD, "globalpass")), AuthMode.SHARED_USER.name());
+    mockStorageConfigWithWorkspace.setEnabled(true);
+    cluster.defineStoragePlugin("local", mockStorageConfigWithWorkspace);
+  }
+
+  @Test
+  public void testSeveralRowsAndRequestsAndPublicStorageAlias() throws Exception {
+    storageAliasesRegistry.createPublicAliases();
+    storageAliasesRegistry.getPublicAliases().put("`foobar`", "`local`", false);
+
+    String sql = "SELECT http_request('foobar.basicJson', `col1`) as data FROM cp.`/data/p4.json`";
+    try (MockWebServer server = startServer()) {
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+
+      RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+      assertEquals(2, results.rowCount());
+
+      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)))
+        .addRow(singleMap(mapValue(4.0, 5.0, 6.0)))
+        .build();
+
+      RowSetUtilities.verify(expected, results);
+    } finally {
+      storageAliasesRegistry.deletePublicAliases();
+    }
+  }
+
+  @Test
+  public void testSeveralRowsAndRequestsAndUserStorageAlias() throws Exception {
+    String sql = "SELECT http_request('foobar.basicJson', `col1`) as data FROM cp.`/data/p4.json`";
+    try (MockWebServer server = startServer()) {
+
+      ClientFixture client = cluster.clientBuilder()
+        .property(DrillProperties.USER, TEST_USER_2)
+        .property(DrillProperties.PASSWORD, TEST_USER_2_PASSWORD)
+        .build();
+
+      storageAliasesRegistry.createUserAliases(TEST_USER_2);
+      storageAliasesRegistry.getUserAliases(TEST_USER_2).put("`foobar`", "`local`", false);
+
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+
+      RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+      assertEquals(2, results.rowCount());
+
+      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)))
+        .addRow(singleMap(mapValue(4.0, 5.0, 6.0)))
+        .build();
+
+      RowSetUtilities.verify(expected, results);
+    } finally {
+      storageAliasesRegistry.deleteUserAliases(TEST_USER_2);
+    }
+  }
+
+  @Test
+  public void testSeveralRowsAndRequestsAndPublicTableAlias() throws Exception {
+    tableAliasesRegistry.createPublicAliases();
+    tableAliasesRegistry.getPublicAliases().put("`foobar`", "`basicJson`", false);
+
+    String sql = "SELECT http_request('local.foobar', `col1`) as data FROM cp.`/data/p4.json`";
+    try (MockWebServer server = startServer()) {
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+
+      RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+      assertEquals(2, results.rowCount());
+
+      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)))
+        .addRow(singleMap(mapValue(4.0, 5.0, 6.0)))
+        .build();
+
+      RowSetUtilities.verify(expected, results);
+    } finally {
+      tableAliasesRegistry.deletePublicAliases();
+    }
+  }
+
+  @Test
+  public void testSeveralRowsAndRequestsAndUserTableAlias() throws Exception {
+    String sql = "SELECT http_request('local.foobar', `col1`) as data FROM cp.`/data/p4.json`";
+    try (MockWebServer server = startServer()) {
+
+      ClientFixture client = cluster.clientBuilder()
+        .property(DrillProperties.USER, TEST_USER_2)
+        .property(DrillProperties.PASSWORD, TEST_USER_2_PASSWORD)
+        .build();
+
+      tableAliasesRegistry.createUserAliases(TEST_USER_2);
+      tableAliasesRegistry.getUserAliases(TEST_USER_2).put("`foobar`", "`basicJson`", false);
+
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+      server.enqueue(new MockResponse().setResponseCode(200).setBody(TEST_JSON_PAGE1));
+
+      RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+      assertEquals(2, results.rowCount());
+
+      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)))
+        .addRow(singleMap(mapValue(4.0, 5.0, 6.0)))
+        .build();
+
+      RowSetUtilities.verify(expected, results);
+    } finally {
+      tableAliasesRegistry.deleteUserAliases(TEST_USER_2);
+    }
+  }
+
+  public static MockWebServer startServer() throws IOException {
+    MockWebServer server = new MockWebServer();
+    server.start(MOCK_SERVER_PORT);
+    return server;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index fadc8a348d..3aef43c01b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -102,7 +102,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     this.skipProfileWrite = false;
     queryOptions = new QueryOptionManager(session.getOptions());
     executionControls = new ExecutionControls(queryOptions, drillbitContext.getEndpoint());
-    plannerSettings = new PlannerSettings(queryOptions, getFunctionRegistry());
+    plannerSettings = new PlannerSettings(queryOptions, getFunctionRegistry(), session.getCredentials().getUserName(), drillbitContext.getAliasRegistryProvider());
     plannerSettings.setNumEndPoints(drillbitContext.getBits().size());
 
     // If we do not need to support dynamic UDFs for this query, just use static operator table
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index e6e7a785fd..022a9b8c2c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -50,6 +50,8 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.alias.AliasRegistry;
+import org.apache.drill.exec.alias.AliasRegistryProvider;
 import org.apache.drill.exec.planner.StarColumnHelper;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -713,6 +715,54 @@ public class DrillOptiq {
         case "date_trunc": {
           return handleDateTruncFunction(args);
         }
+        case "httprequest":
+        case "http_request": {
+          // This code resolves aliases in the http_request function.
+          String completeRawPluginName = ((QuotedString) args.get(0)).value;
+          String username = context.getPlannerSettings().getQueryUser();
+
+          AliasRegistryProvider aliasRegistryProvider = context.getPlannerSettings().getAliasRegistryProvider();
+          AliasRegistry storageAliasRegistry = aliasRegistryProvider.getStorageAliasesRegistry();
+          AliasRegistry tableAliasRegistry = aliasRegistryProvider.getTableAliasesRegistry();
+
+          // Split into plugin and endpoint
+          SchemaPath schemaPath = SchemaPath.parseFromString(completeRawPluginName);
+          String rawPluginName = SchemaPath.getSimplePath(schemaPath.rootName()).toExpr();
+          String rawEndpoint = SchemaPath.getSimplePath(schemaPath.getLastSegment().getNameSegment().getPath()).toExpr();
+
+          // Now resolve plugin name
+          String actualPluginName = storageAliasRegistry.getUserAliases(username).get(rawPluginName);
+          if (StringUtils.isEmpty(actualPluginName)) {
+            // If it is empty, assign it the original name,
+            actualPluginName = rawPluginName;
+          }
+
+          // Finally remove backticks
+          actualPluginName = SchemaPath.parseFromString(actualPluginName).getRootSegmentPath();
+
+          // Now do the same for the endpoint name
+          String actualEndpointName = tableAliasRegistry.getUserAliases(username).get(rawEndpoint);
+          if (StringUtils.isEmpty(actualEndpointName)) {
+            // If it is empty, assign it the original name,
+            actualEndpointName = rawEndpoint;
+          }
+
+          // Now remove backticks
+          actualEndpointName = SchemaPath.parseFromString(actualEndpointName).getRootSegmentPath();
+
+          String finalPluginName = SchemaPath
+            .getCompoundPath(actualPluginName, actualEndpointName)
+            .getAsUnescapedPath();
+
+          QuotedString q = new QuotedString(finalPluginName, finalPluginName.length(), ExpressionPosition.UNKNOWN);
+
+          // Add args to new arg lists
+          List<LogicalExpression> requestArgs = new ArrayList<>();
+          requestArgs.add(q);
+          requestArgs.addAll(args.subList(1, args.size()));
+
+          return FunctionCallFactory.createExpression(functionName, requestArgs);
+        }
       }
 
       return FunctionCallFactory.createExpression(functionName, args);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index 4e079aaf63..6fa145a1b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.planner.physical;
 import org.apache.calcite.avatica.util.Quoting;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.alias.AliasRegistryProvider;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValidator;
@@ -242,10 +243,18 @@ public class PlannerSettings implements Context{
 
   public OptionManager options = null;
   public FunctionImplementationRegistry functionImplementationRegistry = null;
+  private final String queryUser;
+  private final AliasRegistryProvider aliasRegistryProvider;
 
-  public PlannerSettings(OptionManager options, FunctionImplementationRegistry functionImplementationRegistry){
+
+  public PlannerSettings(OptionManager options,
+                         FunctionImplementationRegistry functionImplementationRegistry,
+                         String queryUser,
+                         AliasRegistryProvider aliasRegistryProvider){
     this.options = options;
     this.functionImplementationRegistry = functionImplementationRegistry;
+    this.queryUser = queryUser;
+    this.aliasRegistryProvider = aliasRegistryProvider;
   }
 
   public OptionManager getOptions() {
@@ -373,6 +382,13 @@ public class PlannerSettings implements Context{
   public long getPlanningMemoryLimit() {
     return options.getOption(PLANNER_MEMORY_LIMIT.getOptionName()).num_val;
   }
+  public String getQueryUser() {
+    return this.queryUser;
+  }
+
+  public AliasRegistryProvider getAliasRegistryProvider() {
+    return this.aliasRegistryProvider;
+  }
 
   public static long getInitialPlanningMemorySize() {
     return INITIAL_OFF_HEAP_ALLOCATION_IN_BYTES;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/ServerMetaProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/ServerMetaProvider.java
index f18bf0e0ef..71a7f665ce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/ServerMetaProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/ServerMetaProvider.java
@@ -121,7 +121,7 @@ public class ServerMetaProvider {
       final GetServerMetaResp.Builder respBuilder = GetServerMetaResp.newBuilder();
       try {
         final ServerMeta.Builder metaBuilder = ServerMeta.newBuilder(DEFAULT);
-        PlannerSettings plannerSettings = new PlannerSettings(session.getOptions(), context.getFunctionImplementationRegistry());
+        PlannerSettings plannerSettings = new PlannerSettings(session.getOptions(), context.getFunctionImplementationRegistry(), session.getCredentials().getUserName(), context.getAliasRegistryProvider());
 
         SqlParser.Config config = SqlParser.Config.DEFAULT
           .withIdentifierMaxLength((int) plannerSettings.getIdentifierMaxLength())
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
index fd5046e899..e5f9badc39 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
@@ -119,7 +119,7 @@ public class PlanningBase extends ExecTest {
         UserSession.Builder.newBuilder().withOptionManager(sessionOptions).setSupportComplexTypes(true).build());
     when(context.getCurrentEndpoint()).thenReturn(DrillbitEndpoint.getDefaultInstance());
     when(context.getActiveEndpoints()).thenReturn(ImmutableList.of(DrillbitEndpoint.getDefaultInstance()));
-    when(context.getPlannerSettings()).thenReturn(new PlannerSettings(queryOptions, functionRegistry));
+    when(context.getPlannerSettings()).thenReturn(new PlannerSettings(queryOptions, functionRegistry, null, null));
     when(context.getOptions()).thenReturn(queryOptions);
     when(context.getConfig()).thenReturn(config);
     when(context.getDrillOperatorTable()).thenReturn(table);