diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java index 358cbc7..0ab1847 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java @@ -165,14 +165,14 @@ private void refreshLocalResources(SparkWork sparkWork, HiveConf conf) { // add aux jars String auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS); if (StringUtils.isNotEmpty(auxJars) && StringUtils.isNotBlank(auxJars)) { - addResources(auxJars, localJars); + addJars(auxJars); } // add added jars String addedJars = Utilities.getResourceFiles(conf, SessionState.ResourceType.JAR); if (StringUtils.isNotEmpty(addedJars) && StringUtils.isNotBlank(addedJars)) { HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDJARS, addedJars); - addResources(addedJars, localJars); + addJars(addedJars); } // add plugin module jars on demand @@ -200,24 +200,34 @@ private void refreshLocalResources(SparkWork sparkWork, HiveConf conf) { String addedFiles = Utilities.getResourceFiles(conf, SessionState.ResourceType.FILE); if (StringUtils.isNotEmpty(addedFiles) && StringUtils.isNotBlank(addedFiles)) { HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDFILES, addedFiles); - addResources(addedFiles, localFiles); + addResources(addedFiles); } // add added archives String addedArchives = Utilities.getResourceFiles(conf, SessionState.ResourceType.ARCHIVE); if (StringUtils.isNotEmpty(addedArchives) && StringUtils.isNotBlank(addedArchives)) { HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDARCHIVES, addedArchives); - addResources(addedArchives, localFiles); + addResources(addedArchives); } } - private void addResources(String addedFiles, List localCache) { + private void addResources(String addedFiles) { for (String addedFile : addedFiles.split(",")) { if (StringUtils.isNotEmpty(addedFile) && StringUtils.isNotBlank(addedFile) - && !localCache.contains(addedFile)) { - localCache.add(addedFile); + && !localFiles.contains(addedFile)) { + localFiles.add(addedFile); sc.addFile(addedFile); } } } + + private void addJars(String addedJars) { + for (String addedJar : addedJars.split(",")) { + if (StringUtils.isNotEmpty(addedJar) && StringUtils.isNotBlank(addedJar) + && !localJars.contains(addedJar)) { + localJars.add(addedJar); + sc.addJar(addedJar); + } + } + } }