ayushtkn commented on code in PR #4037:
URL: https://github.com/apache/hive/pull/4037#discussion_r1104501992


##########
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java:
##########
@@ -1146,7 +1146,8 @@ public boolean runDistCp(List<Path> srcPaths, Path dst, 
Configuration conf) thro
 
       // HIVE-13704 states that we should use run() instead of execute() due 
to a hadoop known issue
       // added by HADOOP-10459
-      if (distcp.run(params.toArray(new String[0])) == 0) {
+      int rc = runDistCpInternal(distcp, params);
+      if (rc == 0) {
         return true;
       } else {
         return false;

Review Comment:
   I don't understand why are you introducing a variable or so,
   why not just change:
   ```
   if (distcp.run(params.toArray(new String[0])) == 0) { 
   ```
   to
   ```
         if (runDistCpInternal(distcp, params) == 0) {
   ```
   or if you want to refactor then may be:
   ```
         return runDistCpInternal(distcp, params) == 0;
   ```



##########
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java:
##########
@@ -1211,6 +1212,35 @@ public boolean runDistCpWithSnapshots(String 
oldSnapshot, String newSnapshot, Li
     return false;
   }
 
+  protected int runDistCpInternal(DistCp distcp, List<String> params) {
+    ensureMapReduceQueue(distcp.getConf());
+    return distcp.run(params.toArray(new String[0]));
+  }
+
+  /**
+   * This method ensures if there is an explicit tez.queue.name set, the 
hadoop shim will submit jobs
+   * to the same yarn queue. This solves a security issue where e.g settings 
have the following values:
+   * tez.queue.name=sample
+   * hive.server2.tez.queue.access.check=true
+   * In this case, when a query submits Tez DAGs, the tez client layer checks 
whether the end user has access to
+   * the yarn queue 'sample' via YarnQueueHelper, but this is not respected in 
case of MR jobs that run
+   * even if the query execution engine is Tez. E.g. an EXPORT TABLE can 
submit DistCp MR jobs at some stages when
+   * certain criteria are met. We tend to restrict the setting of 
mapreduce.job.queuename in order to bypass this
+   * security flaw, and even the default queue is unexpected if we explicitly 
set tez.queue.name.
+   * Under the hood the desired behavior is to have DistCp jobs in the same 
yarn queue as other parts
+   * of the query. Most of the time, the user isn't aware that a query 
involves DistCp jobs, hence isn't aware
+   * of these details.
+   */
+  protected void ensureMapReduceQueue(Configuration conf) {
+    String queueName = conf.get(TezConfiguration.TEZ_QUEUE_NAME);
+    LOG.debug("Checking tez.queue.name {}", queueName);
+    if (queueName != null && queueName.length() > 0) {
+      LOG.info("Setting mapreduce.job.queuename (current: '{}') to become 
tez.queue.name: '{}'",
+          conf.get(MRJobConfig.QUEUE_NAME), queueName);
+      conf.set(MRJobConfig.QUEUE_NAME, queueName);
+    }

Review Comment:
   Say during replication, External Table are copied they are simple DistCp 
jobs, they should use that only?
   Second, is this an incompatible change as well?



##########
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java:
##########
@@ -1211,6 +1212,35 @@ public boolean runDistCpWithSnapshots(String 
oldSnapshot, String newSnapshot, Li
     return false;
   }
 
+  protected int runDistCpInternal(DistCp distcp, List<String> params) {
+    ensureMapReduceQueue(distcp.getConf());
+    return distcp.run(params.toArray(new String[0]));
+  }
+
+  /**
+   * This method ensures if there is an explicit tez.queue.name set, the 
hadoop shim will submit jobs
+   * to the same yarn queue. This solves a security issue where e.g settings 
have the following values:
+   * tez.queue.name=sample
+   * hive.server2.tez.queue.access.check=true
+   * In this case, when a query submits Tez DAGs, the tez client layer checks 
whether the end user has access to
+   * the yarn queue 'sample' via YarnQueueHelper, but this is not respected in 
case of MR jobs that run
+   * even if the query execution engine is Tez. E.g. an EXPORT TABLE can 
submit DistCp MR jobs at some stages when
+   * certain criteria are met. We tend to restrict the setting of 
mapreduce.job.queuename in order to bypass this
+   * security flaw, and even the default queue is unexpected if we explicitly 
set tez.queue.name.
+   * Under the hood the desired behavior is to have DistCp jobs in the same 
yarn queue as other parts
+   * of the query. Most of the time, the user isn't aware that a query 
involves DistCp jobs, hence isn't aware
+   * of these details.
+   */
+  protected void ensureMapReduceQueue(Configuration conf) {
+    String queueName = conf.get(TezConfiguration.TEZ_QUEUE_NAME);
+    LOG.debug("Checking tez.queue.name {}", queueName);
+    if (queueName != null && queueName.length() > 0) {
+      LOG.info("Setting mapreduce.job.queuename (current: '{}') to become 
tez.queue.name: '{}'",
+          conf.get(MRJobConfig.QUEUE_NAME), queueName);
+      conf.set(MRJobConfig.QUEUE_NAME, queueName);
+    }

Review Comment:
   I have doubts here, Distcp itself is a MapReduce Job, it will launch a MR 
job only, so why it should not use MR queue?
   Second, Hive on MR might not be widely used but still exists and we haven't 
mentioned anywhere it won't work upstream.
   So, if someone isn't using Tez, will this create problems?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to