You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2019/08/14 18:19:29 UTC

[hive] branch branch-2 updated: HIVE-22096 Backport HIVE-21584 to branch-2 (Yuming Wang via Alan Gates).

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

gates pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 7d5a3d6  HIVE-22096 Backport HIVE-21584 to branch-2 (Yuming Wang via Alan Gates).
7d5a3d6 is described below

commit 7d5a3d6e58b6c10fe4657eb9861f74d02d45b270
Author: Alan Gates <ga...@hortonworks.com>
AuthorDate: Tue Aug 13 15:17:22 2019 -0700

    HIVE-22096 Backport HIVE-21584 to branch-2 (Yuming Wang via Alan Gates).
---
 .../src/java/org/apache/hive/beeline/Commands.java |   2 +-
 .../hive/beeline/TestClassNameCompleter.java       |   4 +-
 .../org/apache/hadoop/hive/common/JavaUtils.java   |  57 ++------
 .../hive/llap/daemon/impl/FunctionLocalizer.java   |  18 ++-
 .../hadoop/hive/metastore/MetaStoreUtils.java      |  21 ++-
 .../hadoop/hive/ql/exec/AddToClassPathAction.java  |  92 +++++++++++++
 .../org/apache/hadoop/hive/ql/exec/Utilities.java  | 100 +++++++-------
 .../apache/hadoop/hive/ql/exec/mr/ExecDriver.java  |   7 +-
 .../apache/hadoop/hive/ql/exec/mr/ExecMapper.java  |  14 +-
 .../apache/hadoop/hive/ql/exec/mr/ExecReducer.java |  16 +--
 .../hive/ql/exec/spark/SparkRecordHandler.java     |  14 +-
 .../hadoop/hive/ql/exec/tez/RecordProcessor.java   |  17 +--
 .../hadoop/hive/ql/session/SessionState.java       |  32 +++--
 .../hive/ql/exec/TestAddToClassPathAction.java     | 145 +++++++++++++++++++++
 .../hive/spark/client/SparkClientUtilities.java    |  23 +++-
 15 files changed, 371 insertions(+), 191 deletions(-)

diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java
index 35b3b3b..fb5d8f5 100644
--- a/beeline/src/java/org/apache/hive/beeline/Commands.java
+++ b/beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -169,7 +169,7 @@ public class Commands {
       return false;
     }
 
-    URLClassLoader classLoader = (URLClassLoader) Thread.currentThread().getContextClassLoader();
+    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
     try {
       beeLine.debug(jarPath + " is added to the local beeline.");
       URLClassLoader newClassLoader = new URLClassLoader(new URL[]{p.toURL()}, classLoader);
diff --git a/beeline/src/test/org/apache/hive/beeline/TestClassNameCompleter.java b/beeline/src/test/org/apache/hive/beeline/TestClassNameCompleter.java
index 1999937..6c3e57f 100644
--- a/beeline/src/test/org/apache/hive/beeline/TestClassNameCompleter.java
+++ b/beeline/src/test/org/apache/hive/beeline/TestClassNameCompleter.java
@@ -40,7 +40,7 @@ public class TestClassNameCompleter {
     String fileName = "empty.file.jar";
     File p = tmpFolder.newFile(fileName);
 
-    URLClassLoader classLoader = (URLClassLoader) Thread.currentThread().getContextClassLoader();
+    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
     try {
       URLClassLoader newClassLoader = new URLClassLoader(new URL[] { p.toURL() }, classLoader);
 
@@ -62,7 +62,7 @@ public class TestClassNameCompleter {
     String fileName = "empty.file";
     File p = tmpFolder.newFile(fileName);
 
-    URLClassLoader classLoader = (URLClassLoader) Thread.currentThread().getContextClassLoader();
+    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
     try {
       URLClassLoader newClassLoader = new URLClassLoader(new URL[] { p.toURL() }, classLoader);
 
diff --git a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
index 3916fe3..c53d1a2 100644
--- a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
@@ -18,12 +18,8 @@
 
 package org.apache.hadoop.hive.common;
 
-import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
 import java.io.IOException;
-import java.io.PrintStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
 import java.net.URLClassLoader;
 import java.util.Arrays;
 import java.util.List;
@@ -38,22 +34,6 @@ import org.slf4j.LoggerFactory;
 public final class JavaUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(JavaUtils.class);
-  private static final Method SUN_MISC_UTIL_RELEASE;
-
-  static {
-    if (Closeable.class.isAssignableFrom(URLClassLoader.class)) {
-      SUN_MISC_UTIL_RELEASE = null;
-    } else {
-      Method release = null;
-      try {
-        Class<?> clazz = Class.forName("sun.misc.ClassLoaderUtil");
-        release = clazz.getMethod("releaseLoader", URLClassLoader.class);
-      } catch (Exception e) {
-        // ignore
-      }
-      SUN_MISC_UTIL_RELEASE = release;
-    }
-  }
 
   /**
    * Standard way of getting classloader in Hive code (outside of Hadoop).
@@ -87,8 +67,10 @@ public final class JavaUtils {
       try {
         closeClassLoader(current);
       } catch (IOException e) {
-        LOG.info("Failed to close class loader " + current +
-            Arrays.toString(((URLClassLoader) current).getURLs()), e);
+        String detailedMessage = current instanceof URLClassLoader ?
+            Arrays.toString(((URLClassLoader) current).getURLs()) :
+            "";
+        LOG.info("Failed to close class loader " + current + " " + detailedMessage, e);
       }
     }
     return true;
@@ -104,35 +86,12 @@ public final class JavaUtils {
     return current == stop;
   }
 
-  // best effort to close
-  // see https://issues.apache.org/jira/browse/HIVE-3969 for detail
   public static void closeClassLoader(ClassLoader loader) throws IOException {
     if (loader instanceof Closeable) {
-      ((Closeable)loader).close();
-    } else if (SUN_MISC_UTIL_RELEASE != null && loader instanceof URLClassLoader) {
-      PrintStream outputStream = System.out;
-      ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-      PrintStream newOutputStream = new PrintStream(byteArrayOutputStream);
-      try {
-        // SUN_MISC_UTIL_RELEASE.invoke prints to System.out
-        // So we're changing the outputstream for that call,
-        // and setting it back to original System.out when we're done
-        System.setOut(newOutputStream);
-        SUN_MISC_UTIL_RELEASE.invoke(null, loader);
-        String output = byteArrayOutputStream.toString("UTF8");
-        LOG.debug(output);
-      } catch (InvocationTargetException e) {
-        if (e.getTargetException() instanceof IOException) {
-          throw (IOException)e.getTargetException();
-        }
-        throw new IOException(e.getTargetException());
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      finally {
-        System.setOut(outputStream);
-        newOutputStream.close();
-      }
+      ((Closeable) loader).close();
+    } else {
+      LOG.warn("Ignoring attempt to close class loader ({}) -- not instance of UDFClassLoader.",
+        loader == null ? "mull" : loader.getClass().getSimpleName());
     }
   }
 
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/FunctionLocalizer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/FunctionLocalizer.java
index 2a6ef3a..5af0122 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/FunctionLocalizer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/FunctionLocalizer.java
@@ -18,8 +18,10 @@ import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.net.URLClassLoader;
+import java.security.AccessController;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.IdentityHashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -33,10 +35,10 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.ql.exec.AddToClassPathAction;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.FunctionTask;
-import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.UDFClassLoader;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -60,7 +62,7 @@ public class FunctionLocalizer implements GenericUDFBridge.UdfWhitelistChecker {
   private final Thread workThread;
   private final File localDir;
   private final Configuration conf;
-  private final URLClassLoader executorClassloader;
+  private final UDFClassLoader executorClassloader;
 
 
   private final IdentityHashMap<Class<?>, Boolean> allowedUdfClasses = new IdentityHashMap<>();
@@ -70,8 +72,9 @@ public class FunctionLocalizer implements GenericUDFBridge.UdfWhitelistChecker {
   public FunctionLocalizer(Configuration conf, String localDir) {
     this.conf = conf;
     this.localDir = new File(localDir, DIR_NAME);
-    this.executorClassloader = (URLClassLoader)Utilities.createUDFClassLoader(
-        (URLClassLoader)Thread.currentThread().getContextClassLoader(), new String[]{});
+    AddToClassPathAction addAction = new AddToClassPathAction(
+        Thread.currentThread().getContextClassLoader(), Collections.EMPTY_LIST, true);
+    this.executorClassloader = AccessController.doPrivileged(addAction);
     this.workThread = new Thread(new Runnable() {
       @Override
       public void run() {
@@ -223,7 +226,8 @@ public class FunctionLocalizer implements GenericUDFBridge.UdfWhitelistChecker {
     recentlyLocalizedJars.clear();
     ClassLoader updatedCl = null;
     try {
-      updatedCl = Utilities.addToClassPath(executorClassloader, jars);
+      AddToClassPathAction addAction = new AddToClassPathAction(executorClassloader, Arrays.asList(jars));
+      updatedCl = AccessController.doPrivileged(addAction);
       if (LOG.isInfoEnabled()) {
         LOG.info("Added " + jars.length + " jars to classpath");
       }
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
index 1b90170..5904a1b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
@@ -29,7 +29,7 @@ import java.net.Socket;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -42,6 +42,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import com.google.common.base.Predicates;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 import org.apache.commons.lang.StringUtils;
@@ -1882,14 +1883,26 @@ public class MetaStoreUtils {
   }
 
   /**
+   * Returns currently known class paths as best effort. For system class loader, this may return empty.
+   * In such cases we will anyway create new child class loader in {@link #addToClassPath(ClassLoader, String[])},
+   * so all new class paths will be added and next time we will have a URLClassLoader to work with.
+   */
+  private static List<URL> getCurrentClassPaths(ClassLoader parentLoader) {
+    if(parentLoader instanceof URLClassLoader) {
+      return Lists.newArrayList(((URLClassLoader) parentLoader).getURLs());
+    } else {
+      return Collections.emptyList();
+    }
+  }
+
+  /**
    * Add new elements to the classpath.
    *
    * @param newPaths
    *          Array of classpath elements
    */
   public static ClassLoader addToClassPath(ClassLoader cloader, String[] newPaths) throws Exception {
-    URLClassLoader loader = (URLClassLoader) cloader;
-    List<URL> curPath = Arrays.asList(loader.getURLs());
+    List<URL> curPath = getCurrentClassPaths(cloader);
     ArrayList<URL> newPath = new ArrayList<URL>();
 
     // get a list with the current classpath components
@@ -1905,7 +1918,7 @@ public class MetaStoreUtils {
       }
     }
 
-    return new URLClassLoader(curPath.toArray(new URL[0]), loader);
+    return new URLClassLoader(curPath.toArray(new URL[0]), cloader);
   }
 
   public static String encodeTableName(String name) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/AddToClassPathAction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/AddToClassPathAction.java
new file mode 100644
index 0000000..7ee7a3b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/AddToClassPathAction.java
@@ -0,0 +1,92 @@
+/*
+ * 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.hadoop.hive.ql.exec;
+
+import java.net.URL;
+import java.security.PrivilegedAction;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Helper class to create UDFClassLoader when running under a security manager. To create a class loader:
+ * > AddToClassPathAction addAction = new AddToClassPathAction(parentLoader, newPaths, true);
+ * > UDFClassLoader childClassLoader = AccessController.doPrivileged(addAction);
+ * To try to add to the class path of the existing class loader; call the above without forceNewClassLoader=true.
+ * Note that a class loader might be still created as fallback method.
+ * <p>
+ * This is slightly inconvenient, but forces the caller code to make the doPriviliged call, rather than us making the
+ * call on the caller's behalf, in accordance with the security guidelines at:
+ * https://docs.oracle.com/javase/8/docs/technotes/guides/security/doprivileged.html
+ */
+public class AddToClassPathAction implements PrivilegedAction<UDFClassLoader> {
+
+  private final ClassLoader parentLoader;
+  private final Collection<String> newPaths;
+  private final boolean forceNewClassLoader;
+
+  public AddToClassPathAction(ClassLoader parentLoader, Collection<String> newPaths, boolean forceNewClassLoader) {
+    this.parentLoader = parentLoader;
+    this.newPaths = newPaths != null ? newPaths : Collections.EMPTY_LIST;
+    this.forceNewClassLoader = forceNewClassLoader;
+    if (parentLoader == null) {
+      throw new IllegalArgumentException("UDFClassLoader is not designed to be a bootstrap class loader!");
+    }
+  }
+
+  public AddToClassPathAction(ClassLoader parentLoader, Collection<String> newPaths) {
+    this(parentLoader, newPaths, false);
+  }
+
+  @Override
+  public UDFClassLoader run() {
+    if (useExistingClassLoader()) {
+      final UDFClassLoader udfClassLoader = (UDFClassLoader) parentLoader;
+      for (String path : newPaths) {
+        udfClassLoader.addURL(Utilities.urlFromPathString(path));
+      }
+      return udfClassLoader;
+    } else {
+      return createUDFClassLoader();
+    }
+  }
+
+  private boolean useExistingClassLoader() {
+    if (!forceNewClassLoader && parentLoader instanceof UDFClassLoader) {
+      final UDFClassLoader udfClassLoader = (UDFClassLoader) parentLoader;
+      // The classloader may have been closed, Cannot add to the same instance
+      return !udfClassLoader.isClosed();
+    }
+    // Cannot use the same classloader if it is not an instance of {@code UDFClassLoader}, or new loader was explicily
+    // requested
+    return false;
+  }
+
+  private UDFClassLoader createUDFClassLoader() {
+    List<URL> urls = new ArrayList<>();
+    for (String path : newPaths) {
+      URL url = Utilities.urlFromPathString(path);
+      if (url != null) {
+        urls.add(url);
+      }
+    }
+    return new UDFClassLoader(urls.toArray(new URL[urls.size()]), parentLoader);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index ef78b35..f50b82d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -22,7 +22,6 @@ import com.esotericsoftware.kryo.Kryo;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang.StringUtils;
@@ -167,6 +166,7 @@ import java.net.URI;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.net.URLDecoder;
+import java.security.AccessController;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
@@ -397,8 +397,10 @@ public final class Utilities {
         // threads, should be unnecessary while SPARK-5377 is resolved.
         String addedJars = conf.get(HIVE_ADDED_JARS);
         if (addedJars != null && !addedJars.isEmpty()) {
-          ClassLoader loader = Thread.currentThread().getContextClassLoader();
-          ClassLoader newLoader = addToClassPath(loader, addedJars.split(";"));
+          AddToClassPathAction addAction = new AddToClassPathAction(
+              Thread.currentThread().getContextClassLoader(), Arrays.asList(addedJars.split(";"))
+          );
+          ClassLoader newLoader = AccessController.doPrivileged(addAction);
           Thread.currentThread().setContextClassLoader(newLoader);
           kryo.setClassLoader(newLoader);
         }
@@ -1436,14 +1438,13 @@ public final class Utilities {
    * Check the existence of buckets according to bucket specification. Create empty buckets if
    * needed.
    *
-   * @param hconf
+   * @param hconf The definition of the FileSink.
    * @param paths A list of empty buckets to create
-   * @param conf The definition of the FileSink.
    * @param reporter The mapreduce reporter object
    * @throws HiveException
    * @throws IOException
    */
-  private static void createEmptyBuckets(Configuration hconf, List<Path> paths,
+  static void createEmptyBuckets(Configuration hconf, List<Path> paths,
       FileSinkDesc conf, Reporter reporter)
       throws HiveException, IOException {
 
@@ -1793,7 +1794,7 @@ public final class Utilities {
    * @param onestr  path string
    * @return
    */
-  private static URL urlFromPathString(String onestr) {
+  static URL urlFromPathString(String onestr) {
     URL oneurl = null;
     try {
       if (StringUtils.indexOf(onestr, "file:/") == 0) {
@@ -1807,59 +1808,26 @@ public final class Utilities {
     return oneurl;
   }
 
-  private static boolean useExistingClassLoader(ClassLoader cl) {
-    if (!(cl instanceof UDFClassLoader)) {
-      // Cannot use the same classloader if it is not an instance of {@code UDFClassLoader}
-      return false;
-    }
-    final UDFClassLoader udfClassLoader = (UDFClassLoader) cl;
-    if (udfClassLoader.isClosed()) {
-      // The classloader may have been closed, Cannot add to the same instance
-      return false;
-    }
-    return true;
-  }
-
   /**
-   * Add new elements to the classpath.
-   *
-   * @param newPaths
-   *          Array of classpath elements
-   */
-  public static ClassLoader addToClassPath(ClassLoader cloader, String[] newPaths) {
-    final URLClassLoader loader = (URLClassLoader) cloader;
-    if (useExistingClassLoader(cloader)) {
-      final UDFClassLoader udfClassLoader = (UDFClassLoader) loader;
-      for (String path : newPaths) {
-        udfClassLoader.addURL(urlFromPathString(path));
-      }
-      return udfClassLoader;
-    } else {
-      return createUDFClassLoader(loader, newPaths);
-    }
-  }
-
-  public static ClassLoader createUDFClassLoader(URLClassLoader loader, String[] newPaths) {
-    final Set<URL> curPathsSet = Sets.newHashSet(loader.getURLs());
-    final List<URL> curPaths = Lists.newArrayList(curPathsSet);
-    for (String onestr : newPaths) {
-      final URL oneurl = urlFromPathString(onestr);
-      if (oneurl != null && !curPathsSet.contains(oneurl)) {
-        curPaths.add(oneurl);
-      }
-    }
-    return new UDFClassLoader(curPaths.toArray(new URL[0]), loader);
-  }
-
-  /**
-   * remove elements from the classpath.
+   * Remove elements from the classpath, if possible. This will only work if the current thread context class loader is
+   * an UDFClassLoader (i.e. if we have created it).
    *
    * @param pathsToRemove
    *          Array of classpath elements
    */
   public static void removeFromClassPath(String[] pathsToRemove) throws IOException {
     Thread curThread = Thread.currentThread();
-    URLClassLoader loader = (URLClassLoader) curThread.getContextClassLoader();
+    ClassLoader currentLoader = curThread.getContextClassLoader();
+    // If current class loader is NOT UDFClassLoader, then it is a system class loader, we should not mess with it.
+    if (!(currentLoader instanceof UDFClassLoader)) {
+      LOG.warn("Ignoring attempt to manipulate {}; probably means we have closed more UDF loaders than opened.",
+              currentLoader == null ? "null" : currentLoader.getClass().getSimpleName());
+      return;
+    }
+    // Otherwise -- for UDFClassLoaders -- we close the current one and create a new one, with more limited class path.
+
+    UDFClassLoader loader = (UDFClassLoader) currentLoader;
+
     Set<URL> newPath = new HashSet<URL>(Arrays.asList(loader.getURLs()));
 
     for (String onestr : pathsToRemove) {
@@ -1869,9 +1837,9 @@ public final class Utilities {
       }
     }
     JavaUtils.closeClassLoader(loader);
-   // This loader is closed, remove it from cached registry loaders to avoid removing it again.
+    // This loader is closed, remove it from cached registry loaders to avoid removing it again.
     Registry reg = SessionState.getRegistry();
-    if(reg != null) {
+    if (reg != null) {
       reg.removeFromUDFLoaders(loader);
     }
 
@@ -3889,4 +3857,26 @@ public final class Utilities {
     return AcidUtils.ORIGINAL_PATTERN.matcher(path.getName()).matches() ||
       AcidUtils.ORIGINAL_PATTERN_COPY.matcher(path.getName()).matches();
   }
+
+  /**
+   * Logs the class paths of the job class loader and the thread context class loader to the passed logger.
+   * Checks both loaders if getURLs method is available; if not, prints a message about this (instead of the class path)
+   *
+   * Note: all messages will always be logged with INFO log level.
+   */
+  public static void tryLoggingClassPaths(JobConf job, Logger logger) {
+    if (logger != null && logger.isInfoEnabled()) {
+      tryToLogClassPath("conf", job.getClassLoader(), logger);
+      tryToLogClassPath("thread", Thread.currentThread().getContextClassLoader(), logger);
+    }
+  }
+
+  private static void tryToLogClassPath(String prefix, ClassLoader loader, Logger logger) {
+    if(loader instanceof URLClassLoader) {
+      logger.info("{} class path = {}", prefix, Arrays.asList(((URLClassLoader) loader).getURLs()).toString());
+    } else {
+      logger.info("{} class path = unavailable for {}", prefix,
+          loader == null ? "null" : loader.getClass().getSimpleName());
+    }
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
index 1945163..57cc44d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
@@ -24,7 +24,9 @@ import java.io.OutputStream;
 import java.io.Serializable;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
+import java.security.AccessController;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -53,6 +55,7 @@ import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.AddToClassPathAction;
 import org.apache.hadoop.hive.ql.exec.FetchOperator;
 import org.apache.hadoop.hive.ql.exec.HiveTotalOrderPartitioner;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -744,7 +747,9 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
       // see also - code in CliDriver.java
       ClassLoader loader = conf.getClassLoader();
       if (StringUtils.isNotBlank(libjars)) {
-        loader = Utilities.addToClassPath(loader, StringUtils.split(libjars, ","));
+        AddToClassPathAction addAction = new AddToClassPathAction(
+            loader, Arrays.asList(StringUtils.split(libjars, ",")));
+        loader = AccessController.doPrivileged(addAction);
       }
       conf.setClassLoader(loader);
       // Also set this to the Thread ContextClassLoader, so new threads will
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java
index f90a788..267da5b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hive.ql.exec.mr;
 
 import java.io.IOException;
-import java.net.URLClassLoader;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
@@ -74,17 +72,7 @@ public class ExecMapper extends MapReduceBase implements Mapper {
   @Override
   public void configure(JobConf job) {
     execContext = new ExecMapperContext(job);
-    // Allocate the bean at the beginning -
-      try {
-      l4j.info("conf classpath = "
-          + Arrays.asList(((URLClassLoader) job.getClassLoader()).getURLs()));
-      l4j.info("thread classpath = "
-          + Arrays.asList(((URLClassLoader) Thread.currentThread()
-          .getContextClassLoader()).getURLs()));
-    } catch (Exception e) {
-      l4j.info("cannot get classpath: " + e.getMessage());
-    }
-
+    Utilities.tryLoggingClassPaths(job, l4j);
     setDone(false);
 
     try {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java
index 1dffff2..d6df1ca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java
@@ -19,11 +19,7 @@
 package org.apache.hadoop.hive.ql.exec.mr;
 
 import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryMXBean;
-import java.net.URLClassLoader;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 
@@ -96,17 +92,7 @@ public class ExecReducer extends MapReduceBase implements Reducer {
     ObjectInspector[] valueObjectInspector = new ObjectInspector[Byte.MAX_VALUE];
     ObjectInspector keyObjectInspector;
 
-    if (isInfoEnabled) {
-      try {
-        LOG.info("conf classpath = "
-            + Arrays.asList(((URLClassLoader) job.getClassLoader()).getURLs()));
-        LOG.info("thread classpath = "
-            + Arrays.asList(((URLClassLoader) Thread.currentThread()
-            .getContextClassLoader()).getURLs()));
-      } catch (Exception e) {
-        LOG.info("cannot get classpath: " + e.getMessage());
-      }
-    }
+    Utilities.tryLoggingClassPaths(job, LOG);
     jc = job;
 
     ReduceWork gWork = Utilities.getReduceWork(job);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java
index 2421885..433af90 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.spark;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.MapredContext;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.mapred.JobConf;
@@ -30,8 +31,6 @@ import org.apache.hadoop.mapred.Reporter;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
-import java.net.URLClassLoader;
-import java.util.Arrays;
 import java.util.Iterator;
 
 public abstract class SparkRecordHandler {
@@ -58,16 +57,7 @@ public abstract class SparkRecordHandler {
     rp = reporter;
 
     LOG.info("maximum memory = " + memoryMXBean.getHeapMemoryUsage().getMax());
-
-    try {
-      LOG.info("conf classpath = "
-        + Arrays.asList(((URLClassLoader) job.getClassLoader()).getURLs()));
-      LOG.info("thread classpath = "
-        + Arrays.asList(((URLClassLoader) Thread.currentThread()
-        .getContextClassLoader()).getURLs()));
-    } catch (Exception e) {
-      LOG.info("cannot get classpath: " + e.getMessage());
-    }
+    Utilities.tryLoggingClassPaths(job, LOG);
   }
 
   /**
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/RecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/RecordProcessor.java
index 106a534..5e4b4b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/RecordProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/RecordProcessor.java
@@ -16,9 +16,8 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hive.ql.exec.tez;
-import java.net.URLClassLoader;
+
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -86,19 +85,7 @@ public abstract class RecordProcessor extends InterruptibleProcessing {
     isLogTraceEnabled = l4j.isTraceEnabled();
 
     checkAbortCondition();
-
-    //log classpaths
-    try {
-      if (l4j.isDebugEnabled()) {
-        l4j.debug("conf classpath = "
-            + Arrays.asList(((URLClassLoader) jconf.getClassLoader()).getURLs()));
-        l4j.debug("thread classpath = "
-            + Arrays.asList(((URLClassLoader) Thread.currentThread()
-            .getContextClassLoader()).getURLs()));
-      }
-    } catch (Exception e) {
-      l4j.info("cannot get classpath: " + e.getMessage());
-    }
+    Utilities.tryLoggingClassPaths(jconf, l4j);
   }
 
   /**
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 9768793..57090f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -26,11 +26,12 @@ import java.io.PrintStream;
 import java.lang.management.ManagementFactory;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.net.URLClassLoader;
+import java.security.AccessController;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hive.conf.HiveConfUtil;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.ql.MapRedStats;
+import org.apache.hadoop.hive.ql.exec.AddToClassPathAction;
 import org.apache.hadoop.hive.ql.exec.Registry;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession;
@@ -392,7 +394,9 @@ public class SessionState {
     // classloader as parent can pollute the session. See HIVE-11878
     parentLoader = SessionState.class.getClassLoader();
     // Make sure that each session has its own UDFClassloader. For details see {@link UDFClassLoader}
-    final ClassLoader currentLoader = Utilities.createUDFClassLoader((URLClassLoader) parentLoader, new String[]{});
+    AddToClassPathAction addAction = new AddToClassPathAction(
+        parentLoader, Collections.EMPTY_LIST, true);
+    final ClassLoader currentLoader = AccessController.doPrivileged(addAction);
     this.sessionConf.setClassLoader(currentLoader);
     resourceDownloader = new ResourceDownloader(conf,
         HiveConf.getVar(conf, ConfVars.DOWNLOADED_RESOURCES_DIR));
@@ -1180,16 +1184,17 @@ public class SessionState {
     String[] jarPaths = StringUtils.split(sessionConf.getAuxJars(), ',');
     if (ArrayUtils.isEmpty(jarPaths)) return;
 
-    URLClassLoader currentCLoader =
-        (URLClassLoader) SessionState.get().getConf().getClassLoader();
-    currentCLoader =
-        (URLClassLoader) Utilities.addToClassPath(currentCLoader, jarPaths);
+    AddToClassPathAction addAction = new AddToClassPathAction(
+        SessionState.get().getConf().getClassLoader(), Arrays.asList(jarPaths)
+        );
+    final ClassLoader currentCLoader = AccessController.doPrivileged(addAction);
     sessionConf.setClassLoader(currentCLoader);
     Thread.currentThread().setContextClassLoader(currentCLoader);
   }
 
   /**
    * Reload the jars under the path specified in hive.reloadable.aux.jars.path property.
+   *
    * @throws IOException
    */
   public void loadReloadableAuxJars() throws IOException {
@@ -1204,7 +1209,7 @@ public class SessionState {
     Set<String> jarPaths = FileUtils.getJarFilesByPath(renewableJarPath, sessionConf);
 
     // load jars under the hive.reloadable.aux.jars.path
-    if(!jarPaths.isEmpty()){
+    if (!jarPaths.isEmpty()) {
       reloadedAuxJars.addAll(jarPaths);
     }
 
@@ -1214,11 +1219,9 @@ public class SessionState {
     }
 
     if (reloadedAuxJars != null && !reloadedAuxJars.isEmpty()) {
-      URLClassLoader currentCLoader =
-          (URLClassLoader) SessionState.get().getConf().getClassLoader();
-      currentCLoader =
-          (URLClassLoader) Utilities.addToClassPath(currentCLoader,
-              reloadedAuxJars.toArray(new String[0]));
+      AddToClassPathAction addAction = new AddToClassPathAction(
+          SessionState.get().getConf().getClassLoader(), reloadedAuxJars);
+      final ClassLoader currentCLoader = AccessController.doPrivileged(addAction);
       sessionConf.setClassLoader(currentCLoader);
       Thread.currentThread().setContextClassLoader(currentCLoader);
     }
@@ -1229,8 +1232,9 @@ public class SessionState {
   static void registerJars(List<String> newJars) throws IllegalArgumentException {
     LogHelper console = getConsole();
     try {
-      ClassLoader loader = Thread.currentThread().getContextClassLoader();
-      ClassLoader newLoader = Utilities.addToClassPath(loader, newJars.toArray(new String[0]));
+      AddToClassPathAction addAction = new AddToClassPathAction(
+          Thread.currentThread().getContextClassLoader(), newJars);
+      final ClassLoader newLoader = AccessController.doPrivileged(addAction);
       Thread.currentThread().setContextClassLoader(newLoader);
       SessionState.get().getConf().setClassLoader(newLoader);
       console.printInfo("Added " + newJars + " to class path");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestAddToClassPathAction.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestAddToClassPathAction.java
new file mode 100644
index 0000000..7fa24f2
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestAddToClassPathAction.java
@@ -0,0 +1,145 @@
+/*
+ * 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.hadoop.hive.ql.exec;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URL;
+import java.security.AccessController;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.fail;
+
+/**
+ * Minimal tests for AddToClassPathAction class. Most of the tests don't use
+ * {@link java.security.AccessController#doPrivileged(java.security.PrivilegedAction)},
+ * presumably the tests will not be executed under security manager.
+ */
+public class TestAddToClassPathAction {
+
+  private ClassLoader originalClassLoader;
+
+  private static void assertURLsMatch(String message, List<String> expected, URL[] actual) {
+    List<String> actualStrings = new ArrayList<>();
+    for (URL url : actual) {
+      actualStrings.add(url.toExternalForm());
+    }
+    assertEquals(message, expected, actualStrings);
+  }
+
+  private static void assertURLsMatch(List<String> expected, URL[] actual) {
+    assertURLsMatch("", expected, actual);
+  }
+
+  @Before
+  public void saveClassLoader() {
+    originalClassLoader = Thread.currentThread().getContextClassLoader();
+  }
+
+  @After
+  public void restoreClassLoader() {
+    Thread.currentThread().setContextClassLoader(originalClassLoader);
+  }
+
+  @Test
+  public void testNullClassLoader() {
+    try {
+      new AddToClassPathAction(null, Collections.EMPTY_LIST);
+      fail("When pafrent class loader is null, IllegalArgumentException is expected!");
+    } catch (IllegalArgumentException e) {
+      // pass
+    }
+  }
+
+  @Test
+  public void testNullPaths() {
+    ClassLoader rootLoader = Thread.currentThread().getContextClassLoader();
+    AddToClassPathAction action = new AddToClassPathAction(rootLoader, null);
+    UDFClassLoader childLoader = action.run();
+    assertURLsMatch(
+            "When newPaths is null, loader shall be created normally with no extra paths.",
+            Collections.EMPTY_LIST, childLoader.getURLs());
+  }
+
+  @Test
+  public void testUseExisting() {
+    ClassLoader rootLoader = Thread.currentThread().getContextClassLoader();
+    AddToClassPathAction action1 = new AddToClassPathAction(rootLoader, Arrays.asList("/a/1", "/c/3"));
+    UDFClassLoader parentLoader = action1.run();
+    AddToClassPathAction action2 = new AddToClassPathAction(parentLoader, Arrays.asList("/b/2", "/d/4"));
+    UDFClassLoader childLoader = action2.run();
+    assertSame(
+            "Normally, the existing class loader should be reused (not closed, no force new).",
+            parentLoader, childLoader);
+    assertURLsMatch(
+            "The class path of the class loader should be updated.",
+            Arrays.asList("file:/a/1", "file:/c/3", "file:/b/2", "file:/d/4"), childLoader.getURLs());
+  }
+
+  @Test
+  public void testClosed() throws IOException {
+    ClassLoader rootLoader = Thread.currentThread().getContextClassLoader();
+    AddToClassPathAction action1 = new AddToClassPathAction(rootLoader, Arrays.asList("/a/1", "/c/3"));
+    UDFClassLoader parentLoader = action1.run();
+    parentLoader.close();
+    AddToClassPathAction action2 = new AddToClassPathAction(parentLoader, Arrays.asList("/b/2", "/d/4"));
+    UDFClassLoader childLoader = action2.run();
+    assertNotSame(
+            "When the parent class loader is closed, a new instance must be created.",
+            parentLoader, childLoader);
+    assertURLsMatch(Arrays.asList("file:/b/2", "file:/d/4"), childLoader.getURLs());
+  }
+
+  @Test
+  public void testForceNew() {
+    ClassLoader rootLoader = Thread.currentThread().getContextClassLoader();
+    AddToClassPathAction action1 = new AddToClassPathAction(rootLoader, Arrays.asList("/a/1", "/c/3"));
+    UDFClassLoader parentLoader = action1.run();
+    AddToClassPathAction action2 = new AddToClassPathAction(parentLoader, Arrays.asList("/b/2", "/d/4"), true);
+    UDFClassLoader childLoader = action2.run();
+    assertNotSame(
+            "When forceNewClassLoader is set, a new instance must be created.",
+            parentLoader, childLoader);
+    assertURLsMatch(Arrays.asList("file:/b/2", "file:/d/4"), childLoader.getURLs());
+  }
+
+  @Test
+  public void testLegalPaths() {
+    ClassLoader rootLoader = Thread.currentThread().getContextClassLoader();
+    List<String> newPaths = Arrays.asList("file://a/aa", "c/cc", "/bb/b");
+    String userDir = System.getProperty("user.dir");
+    List<String> expectedURLs = Arrays.asList(
+            "file://a/aa",
+            "file:" + userDir + "/c/cc",
+            "file:/bb/b");
+    AddToClassPathAction action = new AddToClassPathAction(rootLoader, newPaths);
+    UDFClassLoader loader = AccessController.doPrivileged(action);
+    assertURLsMatch(expectedURLs, loader.getURLs());
+  }
+
+}
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java
index 9ef3f38..653c65f 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java
@@ -24,6 +24,7 @@ import java.io.File;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -42,14 +43,30 @@ public class SparkClientUtilities {
 
   /**
    * Add new elements to the classpath.
+   * Returns currently known class paths as best effort. For system class loader, this may return empty.
+   * In such cases we will anyway create new child class loader in {@link #addToClassPath(Map, Configuration, File)},
+   * so all new class paths will be added and next time we will have a URLClassLoader to work with.
+   */
+  private static List<URL> getCurrentClassPaths(ClassLoader parentLoader) {
+    if(parentLoader instanceof URLClassLoader) {
+      return Lists.newArrayList(((URLClassLoader) parentLoader).getURLs());
+    } else {
+      return Collections.emptyList();
+    }
+  }
+
+  /**
+   * Add new elements to the classpath by creating a child ClassLoader containing both old and new paths.
+   * This method supports downloading HDFS files to local FS if missing from cache or later timestamp.
+   * However, this method has no tricks working around HIVE-11878, like UDFClassLoader....
    *
    * @param newPaths Map of classpath elements and corresponding timestamp
    * @return locally accessible files corresponding to the newPaths
    */
   public static List<String> addToClassPath(Map<String, Long> newPaths, Configuration conf,
       File localTmpDir) throws Exception {
-    URLClassLoader loader = (URLClassLoader) Thread.currentThread().getContextClassLoader();
-    List<URL> curPath = Lists.newArrayList(loader.getURLs());
+    ClassLoader parentLoader = Thread.currentThread().getContextClassLoader();
+    List<URL> curPath = getCurrentClassPaths(parentLoader);
     List<String> localNewPaths = new ArrayList<>();
 
     boolean newPathAdded = false;
@@ -65,7 +82,7 @@ public class SparkClientUtilities {
 
     if (newPathAdded) {
       URLClassLoader newLoader =
-          new URLClassLoader(curPath.toArray(new URL[curPath.size()]), loader);
+          new URLClassLoader(curPath.toArray(new URL[curPath.size()]), parentLoader);
       Thread.currentThread().setContextClassLoader(newLoader);
     }
     return localNewPaths;