You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by ge...@apache.org on 2017/09/04 08:27:31 UTC

oozie git commit: OOZIE-2858 HiveMain, ShellMain and SparkMain should not overwrite properties and config files locally (gezapeti)

Repository: oozie
Updated Branches:
  refs/heads/master 12ea195dd -> 71d3ddc31


OOZIE-2858 HiveMain, ShellMain and SparkMain should not overwrite properties and config files locally (gezapeti)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/71d3ddc3
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/71d3ddc3
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/71d3ddc3

Branch: refs/heads/master
Commit: 71d3ddc31cfb983dc83c51882f99c4429253230a
Parents: 12ea195
Author: Gezapeti Cseh <ge...@apache.org>
Authored: Mon Sep 4 10:27:27 2017 +0200
Committer: Gezapeti Cseh <ge...@apache.org>
Committed: Mon Sep 4 10:27:27 2017 +0200

----------------------------------------------------------------------
 release-log.txt                                 |  1 +
 .../apache/oozie/action/hadoop/DistcpMain.java  |  7 +--
 .../apache/oozie/action/hadoop/HiveMain.java    | 33 +++---------
 .../oozie/action/hadoop/LauncherMain.java       | 48 ++++++++++++-----
 .../oozie/action/hadoop/MapReduceMain.java      |  3 +-
 .../oozie/action/hadoop/PasswordMasker.java     | 49 ++++++++++++++----
 .../oozie/action/hadoop/TestLauncherMain.java   | 54 ++++++++++++++++++++
 .../org/apache/oozie/action/hadoop/PigMain.java | 22 +++-----
 .../apache/oozie/action/hadoop/SparkMain.java   | 20 ++------
 .../apache/oozie/action/hadoop/SqoopMain.java   | 24 ++-------
 10 files changed, 154 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index e2311ea..82a10aa 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.0.0 release (trunk - unreleased)
 
+OOZIE-2858 HiveMain, ShellMain and SparkMain should not overwrite properties and config files locally (gezapeti)
 OOZIE-3035 HDFS HA and log aggregation: getting HDFS delegation token from YARN renewer within JavaActionExecutor (andras.piros via pbacsko)
 OOZIE-3026 fix openjpa enhancer stage during build for logging (dbdist13, andras.piros via pbacsko)
 OOZIE-2746 Several tests failure in TestV2ValidateServlet.java (Dongying Jiao via asasvari)

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/distcp/src/main/java/org/apache/oozie/action/hadoop/DistcpMain.java
----------------------------------------------------------------------
diff --git a/sharelib/distcp/src/main/java/org/apache/oozie/action/hadoop/DistcpMain.java b/sharelib/distcp/src/main/java/org/apache/oozie/action/hadoop/DistcpMain.java
index d17d5be..052940d 100644
--- a/sharelib/distcp/src/main/java/org/apache/oozie/action/hadoop/DistcpMain.java
+++ b/sharelib/distcp/src/main/java/org/apache/oozie/action/hadoop/DistcpMain.java
@@ -19,9 +19,7 @@
 package org.apache.oozie.action.hadoop;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.regex.Pattern;
@@ -143,10 +141,7 @@ public class DistcpMain extends JavaMain {
         log4jProperties.setProperty("log4j.logger.org.apache.hadoop.yarn.client.api.impl.YarnClientImpl", "INFO, jobid");
 
         String localProps = new File(DISTCP_LOG4J_PROPS).getAbsolutePath();
-        try (OutputStream os1 = new FileOutputStream(localProps)) {
-            log4jProperties.store(os1, "");
-        }
-
+        createFileWithContentIfNotExists(localProps, log4jProperties);
         PropertyConfigurator.configure(DISTCP_LOG4J_PROPS);
 
         return logFile;

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
----------------------------------------------------------------------
diff --git a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
index fb3fabf..657fcfb 100644
--- a/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
+++ b/sharelib/hive/src/main/java/org/apache/oozie/action/hadoop/HiveMain.java
@@ -21,10 +21,9 @@ package org.apache.oozie.action.hadoop;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.io.OutputStream;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -164,15 +163,8 @@ public class HiveMain extends LauncherMain {
         log4jProperties.setProperty("log4j.logger.org.apache.hadoop.yarn.client.api.impl.YarnClientImpl", "INFO, jobid");
         log4jProperties.setProperty("log4j.additivity.org.apache.hadoop.yarn.client.api.impl.YarnClientImpl", "false");
 
-        String localProps = new File(HIVE_L4J_PROPS).getAbsolutePath();
-        try (OutputStream os1 = new FileOutputStream(localProps)) {
-            log4jProperties.store(os1, "");
-        }
-
-        localProps = new File(HIVE_EXEC_L4J_PROPS).getAbsolutePath();
-        try (OutputStream os2 = new FileOutputStream(localProps)) {
-            log4jProperties.store(os2, "");
-        }
+        createFileWithContentIfNotExists(new File(HIVE_L4J_PROPS).getAbsolutePath(), log4jProperties);
+        createFileWithContentIfNotExists(new File(HIVE_EXEC_L4J_PROPS).getAbsolutePath(), log4jProperties);
 
         return logFile;
     }
@@ -181,23 +173,12 @@ public class HiveMain extends LauncherMain {
         Configuration hiveConf = initActionConf();
 
         // Write the action configuration out to hive-site.xml
-         try (OutputStream os = new FileOutputStream(HIVE_SITE_CONF)) {
-             hiveConf.writeXml(os);
-         }
-
-        System.out.println();
-        System.out.println("Hive Configuration Properties:");
-        System.out.println("------------------------");
-        for (Entry<String, String> entry : hiveConf) {
-            System.out.println(entry.getKey() + "=" + entry.getValue());
-        }
-        System.out.flush();
-        System.out.println("------------------------");
-        System.out.println();
+        URL hiveSiteURL = createFileWithContentIfNotExists(HIVE_SITE_CONF, hiveConf);
+        logMasking("Hive Configuration Properties:", hiveConf);
 
         // Reset the hiveSiteURL static variable as we just created hive-site.xml.
         // If prepare block had a drop partition it would have been initialized to null.
-        HiveConf.setHiveSiteLocation(HiveConf.class.getClassLoader().getResource("hive-site.xml"));
+        HiveConf.setHiveSiteLocation(hiveSiteURL);
         return hiveConf;
     }
 
@@ -219,7 +200,7 @@ public class HiveMain extends LauncherMain {
         arguments.add("hive.exec.log4j.file=" + new File(HIVE_EXEC_L4J_PROPS).getAbsolutePath());
 
         //setting oozie workflow id as caller context id for hive
-        String callerId = "oozie:" + System.getProperty("oozie.job.id");
+        String callerId = "oozie:" + System.getProperty(LauncherAM.OOZIE_JOB_ID);
         arguments.add("--hiveconf");
         arguments.add("hive.log.trace.id=" + callerId);
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
index f1f52c6..3317764 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherMain.java
@@ -32,7 +32,6 @@ import java.io.StringWriter;
 import java.io.Writer;
 import java.net.URL;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -280,25 +279,18 @@ public abstract class LauncherMain {
      * any of the strings in the maskSet will be masked when writting it to STDOUT.
      *
      * @param header message for the beginning of the Configuration/Properties dump.
-     * @param maskSet set with substrings of property names to mask.
      * @param conf Configuration/Properties object to dump to STDOUT
      * @throws IOException thrown if an IO error ocurred.
      */
 
-    protected static void logMasking(String header, Collection<String> maskSet, Iterable<Map.Entry<String,String>> conf)
+    protected static void logMasking(String header, Iterable<Map.Entry<String,String>> conf)
             throws IOException {
         StringWriter writer = new StringWriter();
         writer.write(header + "\n");
         writer.write("--------------------\n");
+        PasswordMasker masker = new PasswordMasker();
         for (Map.Entry<String, String> entry : conf) {
-            String name = (String) entry.getKey();
-            String value = (String) entry.getValue();
-            for (String mask : maskSet) {
-                if (name.contains(mask)) {
-                    value = "*MASKED*";
-                }
-            }
-            writer.write(" " + name + " : " + value + "\n");
+            writer.write(" " + entry.getKey() + " : " + masker.mask(entry) + "\n");
         }
         writer.write("--------------------\n");
         writer.close();
@@ -463,7 +455,39 @@ public abstract class LauncherMain {
 
       Configuration.dumpConfiguration(propagationConf, new OutputStreamWriter(System.out));
       Configuration.addDefaultResource(PROPAGATION_CONF_XML);
-  }
+    }
+
+    protected static URL createFileWithContentIfNotExists(String filename, Configuration content) throws IOException {
+        File output = new File(filename);
+        try (OutputStream os = createStreamIfFileNotExists(output)) {
+            if(os != null) {
+                content.writeXml(os);
+            }
+        }
+        return output.toURI().toURL();
+    }
+
+    protected static URL createFileWithContentIfNotExists(String filename, Properties content) throws IOException {
+        File output = new File(filename);
+        try (OutputStream os = createStreamIfFileNotExists(output)) {
+            if(os != null) {
+                content.store(os, "");
+            }
+        }
+        return output.toURI().toURL();
+    }
+
+    static FileOutputStream createStreamIfFileNotExists(File output) throws IOException {
+        if (output.exists()) {
+            System.out.println(output + " exists, skipping. The action will use the "
+                    + output.getName() + " defined in the workflow.");
+            return null;
+        } else {
+            System.out.println("Creating " + output.getAbsolutePath());
+            return new FileOutputStream(output);
+        }
+    }
+
 }
 
 class LauncherMainException extends Exception {

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/MapReduceMain.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/MapReduceMain.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/MapReduceMain.java
index 77aaa62..a22e792 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/MapReduceMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/MapReduceMain.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RunningJob;
 
-import java.util.ArrayList;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.io.FileOutputStream;
@@ -64,7 +63,7 @@ public class MapReduceMain extends LauncherMain {
             maskedJobConf.set(entry.getKey(), passwordMasker.maskPasswordsIfNecessary(entry.getValue()));
         }
 
-        logMasking("Map-Reduce job configuration:", new ArrayList<String>(), maskedJobConf);
+        logMasking("Map-Reduce job configuration:", maskedJobConf);
 
         File idFile = new File(System.getProperty(LauncherAMUtils.ACTION_PREFIX + LauncherAMUtils.ACTION_DATA_NEW_ID));
         System.out.println("Submitting Oozie action Map-Reduce job");

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PasswordMasker.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PasswordMasker.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PasswordMasker.java
index 6a73bf9..fd265e8 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PasswordMasker.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PasswordMasker.java
@@ -67,7 +67,7 @@ public class PasswordMasker {
             .compile(PASSWORD_EXTRACTING_REGEX);
 
     /**
-     * Returns a map where keys are masked if they are considered a password.
+     * Returns a map where values are masked if they are considered a password.
      * There are two cases when passwords are masked:
      * 1. The key contains the string "pass". In this case, the entire value is considered a password and replaced completely with
      * a masking string.
@@ -82,19 +82,50 @@ public class PasswordMasker {
         return Maps.transformEntries(unmasked, new Maps.EntryTransformer<String, String, String>() {
             @Override
             public String transformEntry(@Nonnull String key, @Nonnull String value) {
-                checkNotNull(key, "key has to be set");
-                checkNotNull(value, "value has to be set");
-
-                if (isPasswordKey(key)) {
-                    return PASSWORD_MASK;
-                }
-
-                return maskPasswordsIfNecessary(value);
+                return mask(key, value);
             }
         });
     }
 
     /**
+     * Returns a the value of the entry masked if its considered as a password
+     * There are two cases when passwords are masked:
+     * 1. The key contains the string "pass". In this case, the entire value is considered a password and replaced completely with
+     * a masking string.
+     * 2. The value matches a regular expression. Strings like "HADOOP_CREDSTORE_PASSWORD=pwd123" or
+     * "-Djavax.net.ssl.trustStorePassword=password" are considered password definition strings and the text after the equal sign
+     * is replaced with a masking string.
+     *
+     * @param unmasked key-value entry
+     * @return The value of the entry changed based on the replace algorithm described above
+     */
+    public String mask(Map.Entry<String, String> unmasked) {
+        return mask(unmasked.getKey(), unmasked.getValue());
+    }
+
+    /**
+     * Returns a the value of the entry masked if its considered as a password
+     * There are two cases when passwords are masked:
+     * 1. The key contains the string "pass". In this case, the entire value is considered a password and replaced completely with
+     * a masking string.
+     * 2. The value matches a regular expression. Strings like "HADOOP_CREDSTORE_PASSWORD=pwd123" or
+     * "-Djavax.net.ssl.trustStorePassword=password" are considered password definition strings and the text after the equal sign
+     * is replaced with a masking string.
+     * @param key key of entry
+     * @param value value of entry
+     * @return The value of the entry changed based on the replace algorithm described above
+     */
+    private String mask(String key, String value) {
+        checkNotNull(key, "key has to be set");
+        checkNotNull(value, "value has to be set");
+
+        if (isPasswordKey(key)) {
+            return PASSWORD_MASK;
+        }
+        return maskPasswordsIfNecessary(value);
+    }
+
+    /**
      * Masks passwords inside a string. A substring is subject to password masking if it looks like
      * "HADOOP_CREDSTORE_PASSWORD=pwd123" or "-Djavax.net.ssl.trustStorePassword=password". The text after the equal sign is
      * replaced with a masking string.

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherMain.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherMain.java b/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherMain.java
index a793df1..5fb7cf5 100644
--- a/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherMain.java
+++ b/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherMain.java
@@ -20,18 +20,31 @@ package org.apache.oozie.action.hadoop;
 
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
 import java.io.PrintStream;
+import java.nio.file.Files;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.rules.TemporaryFolder;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 public class TestLauncherMain {
     private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
     private PrintStream originalStream;
 
+    @Rule
+    public TemporaryFolder tmp = new TemporaryFolder();
     @Before
     public void setUpStreams() {
         originalStream = System.out;
@@ -56,4 +69,45 @@ public class TestLauncherMain {
         @Override
         protected void run(String[] args) throws Exception {}
     }
+
+    @Test
+    public void testDontCreateStreamIfFileExists() throws IOException {
+        File f = tmp.newFile();
+        try (FileOutputStream fos = new FileOutputStream(f)) {
+            fos.write("foo".getBytes());
+        }
+
+        try (FileOutputStream fos = LauncherMain.createStreamIfFileNotExists(f)) {
+            assertNull(fos);
+        }
+    }
+
+    @Test
+    public void testConfigWrite() throws IOException {
+        File f = new File(tmp.newFolder(), "nonExistentFile");
+        assertFalse(f.exists());
+        try (FileOutputStream fos = LauncherMain.createStreamIfFileNotExists(f)) {
+            Configuration c = new Configuration(false);
+            c.set("foo", "bar");
+            c.writeXml(fos);
+        }
+        String contents = new String(Files.readAllBytes(f.toPath()));
+        assertTrue(contents.contains("foo"));
+        assertTrue(contents.contains("bar"));
+        assertTrue(contents.contains("<configuration>"));
+        assertTrue(contents.contains("<property"));
+    }
+
+    @Test
+    public void testPropertiesWrite() throws IOException {
+        File f = new File(tmp.newFolder(), "nonExistentFile");
+        assertFalse(f.exists());
+        try (FileOutputStream fos = LauncherMain.createStreamIfFileNotExists(f)) {
+            Properties p = new Properties();
+            p.setProperty("foo", "bar");
+            p.store(fos, "");
+        }
+        String contents = new String(Files.readAllBytes(f.toPath()));
+        assertTrue(contents.contains("foo=bar"));
+    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
----------------------------------------------------------------------
diff --git a/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java b/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
index 13ee04f..a0b7165 100644
--- a/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
+++ b/sharelib/pig/src/main/java/org/apache/oozie/action/hadoop/PigMain.java
@@ -18,7 +18,6 @@
 
 package org.apache.oozie.action.hadoop;
 
-import org.apache.pig.Main;
 import org.apache.pig.PigRunner;
 import org.apache.pig.scripting.ScriptEngine;
 import org.apache.pig.tools.pigstats.JobStats;
@@ -31,17 +30,15 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
-import java.io.BufferedWriter;
-import java.io.FileNotFoundException;
-import java.io.OutputStream;
-import java.io.FileOutputStream;
 import java.io.BufferedReader;
+import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
-import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.List;
@@ -123,14 +120,11 @@ public class PigMain extends LauncherMain {
         }
 
         //setting oozie workflow id as caller context id for pig
-        String callerId = "oozie:" + System.getProperty("oozie.job.id");
+        String callerId = "oozie:" + System.getProperty(LauncherAM.OOZIE_JOB_ID);
         pigProperties.setProperty("pig.log.trace.id", callerId);
 
-        try (OutputStream os = new FileOutputStream("pig.properties")) {
-            pigProperties.store(os, "");
-        }
-
-        logMasking("pig.properties:", Arrays.asList("password"),
+        createFileWithContentIfNotExists("pig.properties", pigProperties);
+        logMasking("pig.properties:",
                 (Iterable<Map.Entry<String, String>>)(Iterable<?>) pigProperties.entrySet());
 
         List<String> arguments = new ArrayList<>();
@@ -178,9 +172,7 @@ public class PigMain extends LauncherMain {
         log4jProperties.setProperty("log4j.logger.org.apache.hadoop.yarn.client.api.impl.YarnClientImpl", "INFO, B");
 
         String localProps = new File("piglog4j.properties").getAbsolutePath();
-        try (OutputStream os1 = new FileOutputStream(localProps)) {
-            log4jProperties.store(os1, "");
-        }
+        createFileWithContentIfNotExists(localProps, log4jProperties);
 
         arguments.add("-log4jconf");
         arguments.add(localProps);

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
----------------------------------------------------------------------
diff --git a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
index 8aa8343..47172be 100644
--- a/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
+++ b/sharelib/spark/src/main/java/org/apache/oozie/action/hadoop/SparkMain.java
@@ -19,11 +19,10 @@
 package org.apache.oozie.action.hadoop;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.URL;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -208,9 +207,7 @@ public class SparkMain extends LauncherMain {
         log4jProperties.setProperty("log4j.logger.org.apache.hadoop.yarn.client.api.impl.YarnClientImpl", "INFO, jobid");
 
         final String localProps = new File(SPARK_LOG4J_PROPS).getAbsolutePath();
-        try (OutputStream os1 = new FileOutputStream(localProps)) {
-            log4jProperties.store(os1, "");
-        }
+        createFileWithContentIfNotExists(localProps, log4jProperties);
 
         PropertyConfigurator.configure(SPARK_LOG4J_PROPS);
         return logFile;
@@ -256,20 +253,11 @@ public class SparkMain extends LauncherMain {
         hiveConf.unset("hive.exec.local.scratchdir");
 
         // Write the action configuration out to hive-site.xml
-        OutputStream os = null;
-        try {
-            os = new FileOutputStream(HIVE_SITE_CONF);
-            hiveConf.writeXml(os);
-        }
-        finally {
-            if (os != null) {
-                os.close();
-            }
-        }
+        URL hiveSiteURL = createFileWithContentIfNotExists(HIVE_SITE_CONF, hiveConf);
         // Reset the hiveSiteURL static variable as we just created
         // hive-site.xml.
         // If prepare block had a drop partition it would have been initialized
         // to null.
-        HiveConf.setHiveSiteLocation(HiveConf.class.getClassLoader().getResource("hive-site.xml"));
+        HiveConf.setHiveSiteLocation(hiveSiteURL);
     }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/71d3ddc3/sharelib/sqoop/src/main/java/org/apache/oozie/action/hadoop/SqoopMain.java
----------------------------------------------------------------------
diff --git a/sharelib/sqoop/src/main/java/org/apache/oozie/action/hadoop/SqoopMain.java b/sharelib/sqoop/src/main/java/org/apache/oozie/action/hadoop/SqoopMain.java
index e889ef9..27f9306 100644
--- a/sharelib/sqoop/src/main/java/org/apache/oozie/action/hadoop/SqoopMain.java
+++ b/sharelib/sqoop/src/main/java/org/apache/oozie/action/hadoop/SqoopMain.java
@@ -19,7 +19,6 @@
 package org.apache.oozie.action.hadoop;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Map;
@@ -84,23 +83,8 @@ public class SqoopMain extends LauncherMain {
         Configuration sqoopConf = initActionConf();
 
         // Write the action configuration out to sqoop-site.xml
-        OutputStream os = new FileOutputStream(SQOOP_SITE_CONF);
-        try {
-            sqoopConf.writeXml(os);
-        }
-        finally {
-            os.close();
-        }
-
-        System.out.println();
-        System.out.println("Sqoop Configuration Properties:");
-        System.out.println("------------------------");
-        for (Map.Entry<String, String> entry : sqoopConf) {
-            System.out.println(entry.getKey() + "=" + entry.getValue());
-        }
-        System.out.flush();
-        System.out.println("------------------------");
-        System.out.println();
+        createFileWithContentIfNotExists(SQOOP_SITE_CONF, sqoopConf);
+        logMasking("Sqoop Configuration Properties:", sqoopConf);
         return sqoopConf;
     }
 
@@ -132,9 +116,7 @@ public class SqoopMain extends LauncherMain {
         log4jProperties.setProperty("log4j.logger.org.apache.hadoop.yarn.client.api.impl.YarnClientImpl", "INFO, jobid");
 
         String localProps = new File(SQOOP_LOG4J_PROPS).getAbsolutePath();
-        try (OutputStream os1 = new FileOutputStream(localProps)) {
-            log4jProperties.store(os1, "");
-        }
+        createFileWithContentIfNotExists(localProps, log4jProperties);
 
         PropertyConfigurator.configure(SQOOP_LOG4J_PROPS);