[ 
https://issues.apache.org/jira/browse/HIVE-23519?focusedWorklogId=436425&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-436425
 ]

ASF GitHub Bot logged work on HIVE-23519:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 22/May/20 06:56
            Start Date: 22/May/20 06:56
    Worklog Time Spent: 10m 
      Work Description: pkumarsinha commented on a change in pull request #1028:
URL: https://github.com/apache/hive/pull/1028#discussion_r428938361



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/RangerDumpTask.java
##########
@@ -81,12 +79,16 @@ public int execute() {
       if (rangerRestClient == null) {
         rangerRestClient = getRangerRestClient();
       }
-      String rangerEndpoint = 
conf.getVar(REPL_AUTHORIZATION_PROVIDER_SERVICE_ENDPOINT);
+      InputStream inputStream = RangerDumpTask.class.getClassLoader()

Review comment:
       Move the conf reading to another method. Does conf.addResource also 
closes the stream? If not close the input stream.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/RangerDumpTask.java
##########
@@ -81,12 +79,16 @@ public int execute() {
       if (rangerRestClient == null) {
         rangerRestClient = getRangerRestClient();
       }
-      String rangerEndpoint = 
conf.getVar(REPL_AUTHORIZATION_PROVIDER_SERVICE_ENDPOINT);
+      InputStream inputStream = RangerDumpTask.class.getClassLoader()
+          .getResourceAsStream(ReplUtils.RANGER_CONFIGURATION_RESOURCE_NAME);
+      if (inputStream != null) {
+        conf.addResource(inputStream);
+      }
+      String rangerHiveServiceName = 
conf.get(ReplUtils.RANGER_HIVE_SERVICE_NAME);
+      String rangerEndpoint = conf.get(ReplUtils.RANGER_REST_URL);
       if (StringUtils.isEmpty(rangerEndpoint) || 
!rangerRestClient.checkConnection(rangerEndpoint)) {
-        throw new Exception("Ranger endpoint is not valid. "
-                + "Please pass a valid config 
hive.repl.authorization.provider.service.endpoint");
+        throw new Exception("Ranger endpoint is not valid.");

Review comment:
       One another thing missing in log is, say if rangerEndpoint is null, we 
will not know whether the the conf for it is not present or the inputStream 
itself is null.
   May be, you may want to switch to another version of getResource(), (URL 
based on).
   In that case we will have URL of conf file  to log and we need not bother to 
close inputStream. 

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/RangerDumpTask.java
##########
@@ -81,12 +79,16 @@ public int execute() {
       if (rangerRestClient == null) {
         rangerRestClient = getRangerRestClient();
       }
-      String rangerEndpoint = 
conf.getVar(REPL_AUTHORIZATION_PROVIDER_SERVICE_ENDPOINT);
+      InputStream inputStream = RangerDumpTask.class.getClassLoader()
+          .getResourceAsStream(ReplUtils.RANGER_CONFIGURATION_RESOURCE_NAME);
+      if (inputStream != null) {
+        conf.addResource(inputStream);
+      }
+      String rangerHiveServiceName = 
conf.get(ReplUtils.RANGER_HIVE_SERVICE_NAME);
+      String rangerEndpoint = conf.get(ReplUtils.RANGER_REST_URL);
       if (StringUtils.isEmpty(rangerEndpoint) || 
!rangerRestClient.checkConnection(rangerEndpoint)) {
-        throw new Exception("Ranger endpoint is not valid. "
-                + "Please pass a valid config 
hive.repl.authorization.provider.service.endpoint");
+        throw new Exception("Ranger endpoint is not valid.");

Review comment:
       Adding the endpoint in log message will help debug when required.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/RangerLoadTask.java
##########
@@ -83,10 +81,15 @@ public int execute() {
       if (rangerRestClient == null) {
         rangerRestClient = getRangerRestClient();
       }
-      String rangerEndpoint = 
conf.getVar(REPL_AUTHORIZATION_PROVIDER_SERVICE_ENDPOINT);
+      InputStream inputStream = RangerDumpTask.class.getClassLoader()

Review comment:
       Even though it works, change it to, 
RangerLoadTask.class.getClassLoader() or even just getClass().getClassLoader() 
is fine.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/RangerLoadTask.java
##########
@@ -83,10 +81,15 @@ public int execute() {
       if (rangerRestClient == null) {
         rangerRestClient = getRangerRestClient();
       }
-      String rangerEndpoint = 
conf.getVar(REPL_AUTHORIZATION_PROVIDER_SERVICE_ENDPOINT);
+      InputStream inputStream = RangerDumpTask.class.getClassLoader()
+          .getResourceAsStream(ReplUtils.RANGER_CONFIGURATION_RESOURCE_NAME);
+      if (inputStream != null) {
+        conf.addResource(inputStream);
+      }
+      String rangerHiveServiceName = 
conf.get(ReplUtils.RANGER_HIVE_SERVICE_NAME);
+      String rangerEndpoint = conf.get(ReplUtils.RANGER_REST_URL);
       if (StringUtils.isEmpty(rangerEndpoint) || 
!rangerRestClient.checkConnection(rangerEndpoint)) {
-        throw new Exception("Ranger endpoint is not valid. "
-                + "Please pass a valid config 
hive.repl.authorization.provider.service.endpoint");
+        throw new Exception("Ranger endpoint is not valid.");

Review comment:
       Same comments 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.

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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 436425)
    Time Spent: 20m  (was: 10m)

> Read Ranger Configs from Classpath
> ----------------------------------
>
>                 Key: HIVE-23519
>                 URL: https://issues.apache.org/jira/browse/HIVE-23519
>             Project: Hive
>          Issue Type: Task
>            Reporter: Aasha Medhi
>            Assignee: Aasha Medhi
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: HIVE-23519.01.patch, HIVE-23519.02.patch, 
> HIVE-23519.03.patch, HIVE-23519.04.patch
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to