You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by an...@apache.org on 2018/01/09 10:18:28 UTC

oozie git commit: OOZIE-2150 Shell launcher should print shell script (jtolar via andras.piros)

Repository: oozie
Updated Branches:
  refs/heads/master dde128448 -> c7b2bb74b


OOZIE-2150 Shell launcher should print shell script (jtolar via andras.piros)


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

Branch: refs/heads/master
Commit: c7b2bb74b492c7b727930bc87638788389d18cb8
Parents: dde1284
Author: Andras Piros <an...@cloudera.com>
Authored: Tue Jan 9 11:16:00 2018 +0100
Committer: Andras Piros <an...@cloudera.com>
Committed: Tue Jan 9 11:16:00 2018 +0100

----------------------------------------------------------------------
 core/src/main/resources/oozie-default.xml       |   9 ++
 .../action/hadoop/TestShellContentWriter.java   | 132 +++++++++++++++++++
 release-log.txt                                 |   1 +
 .../oozie/action/hadoop/ShellContentWriter.java | 123 +++++++++++++++++
 .../apache/oozie/action/hadoop/ShellMain.java   |  20 ++-
 5 files changed, 282 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/c7b2bb74/core/src/main/resources/oozie-default.xml
----------------------------------------------------------------------
diff --git a/core/src/main/resources/oozie-default.xml b/core/src/main/resources/oozie-default.xml
index 830893f..5b5e34f 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -1938,6 +1938,15 @@ will be the requeue interval for the actions which are waiting for a long time w
     </property>
 
     <property>
+        <name>oozie.action.shell.max-print-size-kb</name>
+        <value>128</value>
+        <description>
+            When an oozie shell action starts, the shell script will be printed. Scripts larger than the size configured here
+            (in KiB) will not be printed. If this value is less than or equal to zero, the script will not be printed.
+        </description>
+    </property>
+
+    <property>
         <name>oozie.action.launcher.yarn.timeline-service.enabled</name>
         <value>false</value>
         <description>

http://git-wip-us.apache.org/repos/asf/oozie/blob/c7b2bb74/core/src/test/java/org/apache/oozie/action/hadoop/TestShellContentWriter.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/action/hadoop/TestShellContentWriter.java b/core/src/test/java/org/apache/oozie/action/hadoop/TestShellContentWriter.java
new file mode 100644
index 0000000..75a98cb
--- /dev/null
+++ b/core/src/test/java/org/apache/oozie/action/hadoop/TestShellContentWriter.java
@@ -0,0 +1,132 @@
+/**
+ * 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.oozie.action.hadoop;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+
+public class TestShellContentWriter {
+
+    public static final int MAX_TEST_SCRIPT_SIZE_KB = 128;
+
+    ByteArrayOutputStream outputStream;
+    ByteArrayOutputStream errorStream;
+    File scriptFile;
+
+    @Before
+    public void setUp() throws IOException {
+        outputStream = new ByteArrayOutputStream(MAX_TEST_SCRIPT_SIZE_KB);
+        errorStream = new ByteArrayOutputStream(MAX_TEST_SCRIPT_SIZE_KB);
+        scriptFile = folder.newFile("shell_script.sh");
+    }
+
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    @Test
+    public void testPrintShellFile() throws Exception {
+        writeScript("echo Hello World");
+
+        Assert.assertTrue(outputStream.toString().contains("echo Hello World"));
+        Assert.assertTrue(errorStream.toString().isEmpty());
+    }
+
+    @Test
+    public void testPrintShellNullByte() throws Exception {
+        writeScript("echo Hello World\0");
+
+        Assert.assertFalse(outputStream.toString().contains("Hello World"));
+        Assert.assertTrue(errorStream.toString().contains("appears to be a binary file"));
+    }
+
+    @Test
+    public void testPrintControlCharacter() throws Exception {
+        writeScript("echo Hello World\011");
+
+        Assert.assertFalse(outputStream.toString().contains("Hello World"));
+        Assert.assertTrue(errorStream.toString().contains("appears to be a binary file"));
+    }
+
+    @Test
+    public void testEmptyFile() throws Exception {
+        writeScript("");
+
+        Assert.assertTrue(outputStream.toString().contains("---\n\n---"));
+        Assert.assertTrue(errorStream.toString().isEmpty());
+    }
+
+    @Test
+    public void testTooLargeFile() throws Exception {
+        byte[] arr = new byte[2048];
+        Arrays.fill(arr, (byte) Character.getNumericValue('x'));
+
+        writeScript(new String(arr));
+
+        Assert.assertTrue(outputStream.toString().isEmpty());
+        Assert.assertTrue(errorStream.toString().contains("content suppressed."));
+        Assert.assertTrue(errorStream.toString().contains("File size=2048b; max printable size=1024b"));
+    }
+
+    @Test
+    public void testNegativeMaxSize() throws Exception {
+        writeScript("test script", -1);
+
+        Assert.assertTrue(outputStream.toString().isEmpty());
+        Assert.assertTrue(errorStream.toString().contains("Not printing script file as configured, content suppressed."));
+    }
+
+    @Test
+    public void testMissingFile() throws Exception {
+        scriptFile.delete();
+
+        writeScript("");
+
+        Assert.assertTrue(outputStream.toString().isEmpty());
+        Assert.assertTrue(errorStream.toString().contains("doesn't appear to exist"));
+    }
+
+    private void writeScript(String content) throws IOException {
+        writeScript(content, 1);
+    }
+
+    // Write a stub script with the given content, and invoke the writer to print its content
+    private void writeScript(String content, int maxLen) throws IOException {
+        if (content != null && !content.isEmpty()) {
+            Files.write(scriptFile.toPath(), content.getBytes());
+        }
+
+        ShellContentWriter writer = new ShellContentWriter(
+                maxLen,
+                outputStream,
+                errorStream,
+                scriptFile.getAbsolutePath()
+        );
+
+        writer.print();
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/c7b2bb74/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 8f58eac..5829699 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.0.0 release (trunk - unreleased)
 
+OOZIE-2150 Shell launcher should print shell script (jtolar via andras.piros)
 OOZIE-3147 Misleading documentation of oozie.service.PurgeService.purge.limit configuration property (okalinin via andras.piros)
 OOZIE-3148 Rerun Failing Tests through Maven surefire (asasvari)
 OOZIE-3113 Retry for ZK lock release (satishsaley)

http://git-wip-us.apache.org/repos/asf/oozie/blob/c7b2bb74/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellContentWriter.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellContentWriter.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellContentWriter.java
new file mode 100644
index 0000000..fe833ac
--- /dev/null
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellContentWriter.java
@@ -0,0 +1,123 @@
+/**
+ * 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.oozie.action.hadoop;
+
+import org.apache.commons.io.Charsets;
+
+import java.io.BufferedInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.attribute.BasicFileAttributes;
+
+/**
+ * Dump the content of a shell script to output stream.
+ *
+ * If the file is binary or larger than a configurable size, the file content will not be printed.
+ */
+public class ShellContentWriter {
+
+    private final long maxLen;
+    private final OutputStream outputStream;
+    private final OutputStream errorStream;
+    private final String filename;
+
+    /**
+     * This class prints out the script file to the specified stream.
+     *
+     * @param maxLenKb max length of file to print, in KiB
+     * @param outputStream stream where script content is written
+     * @param errorStream stream for error messages are written
+     * @param filename filename of script to print
+     */
+    public ShellContentWriter(int maxLenKb, OutputStream outputStream, OutputStream errorStream, String filename) {
+        this.outputStream = outputStream;
+        this.errorStream = errorStream;
+        this.filename = filename;
+        this.maxLen = maxLenKb > 0 ? ((long) maxLenKb) * 1024L : 0L;
+    }
+
+    /**
+     * Print the script being executed.
+     *
+     * If the file can't be found, appears to be binary, or is too large, it isn't printed.
+     */
+    public void print() {
+
+        Path path = Paths.get(filename);
+
+        try {
+            if (Files.exists(path)) {
+                BasicFileAttributes attributes = Files.readAttributes(path, BasicFileAttributes.class);
+                long len = attributes.size();
+                if (len < maxLen) {
+                    byte[] buffer = new byte[(int)len];
+                    if (!readFile(path, buffer)) {
+                        return;
+                    }
+
+                    writeLine(outputStream, "Content of script " + filename + " (size = " + len + "b):");
+                    writeLine(outputStream, "------------------------------------");
+                    writeLine(outputStream, buffer);
+                    writeLine(outputStream, "------------------------------------");
+                } else {
+                    String message = "Not printing script file as configured, content suppressed.";
+                    if (maxLen > 0) {
+                        message = message + " File size=" + len + "b; max printable size=" + maxLen + "b";
+                    }
+                    writeLine(errorStream, message);
+                }
+            } else {
+                writeLine(errorStream, "Path " + filename + " doesn't appear to exist");
+            }
+        } catch (IOException ignored) { }
+    }
+
+    // Read the file into the given buffer.
+    // Return true if the file appears to be printable, false otherwise.
+    private boolean readFile(Path path,  byte[] buffer) throws IOException {
+        int index = 0;
+        try (InputStream stream = new BufferedInputStream(new FileInputStream(path.toFile()))) {
+            int c;
+            while ((c = stream.read()) >= 0 && index < buffer.length) {
+                // a byte less than 0x0a is a hint that the file is binary
+                if (c < 0x0a) {
+                    writeLine(errorStream, "File " + filename + " appears to be a binary file, content suppressed.");
+                    return false;
+                }
+                buffer[index++] = (byte)c;
+            }
+        }
+
+        return true;
+    }
+
+    private void writeLine(OutputStream stream, String s) throws IOException {
+        writeLine(stream, s.getBytes(Charsets.UTF_8));
+    }
+
+    private void writeLine(OutputStream stream, byte[] arr) throws IOException {
+        stream.write(arr);
+        stream.write('\n');
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/c7b2bb74/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
index 55d3d96..554652e 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/ShellMain.java
@@ -45,6 +45,10 @@ public class ShellMain extends LauncherMain {
             "oozie.action.shell.setup.hadoop.conf.dir.write.log4j.properties";
     public static final String CONF_OOZIE_SHELL_SETUP_HADOOP_CONF_DIR_LOG4J_CONTENT =
             "oozie.action.shell.setup.hadoop.conf.dir.log4j.content";
+
+    public static final String CONF_OOZIE_SHELL_MAX_SCRIPT_SIZE_TO_PRINT_KB = "oozie.action.shell.max-print-size-kb";
+    private static final int DEFAULT_MAX_SRIPT_SIZE_TO_PRINT_KB = 128;
+
     public static final String OOZIE_ACTION_CONF_XML = "OOZIE_ACTION_CONF_XML";
     private static final String HADOOP_CONF_DIR = "HADOOP_CONF_DIR";
     private static final String YARN_CONF_DIR = "YARN_CONF_DIR";
@@ -96,7 +100,7 @@ public class ShellMain extends LauncherMain {
         // Setup Hadoop *-site files in case the user runs a Hadoop-type program (e.g. hive)
         prepareHadoopConfigs(actionConf, envp, currDir);
 
-        printCommand(cmdArray, envp); // For debugging purpose
+        printCommand(actionConf, cmdArray, envp); // For debugging purpose
 
         System.out.println("=================================================================");
         System.out.println();
@@ -314,8 +318,9 @@ public class ShellMain extends LauncherMain {
      *
      * @param cmdArray :Command Array
      * @param envp :Environment array
+     * @param config :Hadoop configuration
      */
-    protected void printCommand(ArrayList<String> cmdArray, Map<String, String> envp) {
+    protected void printCommand(Configuration config, ArrayList<String> cmdArray, Map<String, String> envp) {
         int i = 0;
         System.out.println("Full Command .. ");
         System.out.println("-------------------------");
@@ -323,6 +328,16 @@ public class ShellMain extends LauncherMain {
             System.out.println(i++ + ":" + arg + ":");
         }
 
+        if (!cmdArray.isEmpty()) {
+            ShellContentWriter writer = new ShellContentWriter(
+                    config.getInt(CONF_OOZIE_SHELL_MAX_SCRIPT_SIZE_TO_PRINT_KB, DEFAULT_MAX_SRIPT_SIZE_TO_PRINT_KB),
+                    System.out,
+                    System.err,
+                    cmdArray.get(0)
+            );
+            writer.print();
+        }
+
         if (envp != null) {
             System.out.println("List of passing environment");
             System.out.println("-------------------------");
@@ -330,7 +345,6 @@ public class ShellMain extends LauncherMain {
                 System.out.println(entry.getKey() + "=" + entry.getValue() + ":");
             }
         }
-
     }
 
     /**