You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ro...@apache.org on 2014/01/23 06:52:39 UTC

svn commit: r1560587 - in /pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/datastorage/HPath.java src/org/apache/pig/impl/io/FileLocalizer.java test/org/apache/pig/test/TestPigServer.java

Author: rohini
Date: Thu Jan 23 05:52:39 2014
New Revision: 1560587

URL: http://svn.apache.org/r1560587
Log:
PIG-3511: Security: Pig temporary directories might have world readable permissions (rohini)

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/HPath.java
    pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java
    pig/trunk/test/org/apache/pig/test/TestPigServer.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1560587&r1=1560586&r2=1560587&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Jan 23 05:52:39 2014
@@ -74,6 +74,8 @@ OPTIMIZATIONS
  
 BUG FIXES
 
+PIG-3511: Security: Pig temporary directories might have world readable permissions (rohini)
+
 PIG-3664: Piggy Bank XPath UDF can't be called (nezihyigitbasi via daijy)
 
 PIG-3662: Static loadcaster in BinStorage can cause exception (lbendig via rohini)

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/HPath.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/HPath.java?rev=1560587&r1=1560586&r2=1560587&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/HPath.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/HPath.java Thu Jan 23 05:52:39 2014
@@ -29,8 +29,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.conf.Configuration;
-
 import org.apache.pig.PigException;
 import org.apache.pig.backend.datastorage.*;
 import org.apache.pig.backend.executionengine.ExecException;
@@ -48,7 +48,7 @@ public abstract class HPath implements E
     public HPath(HDataStorage fs, String parent, String child) {
         this(fs, new Path(parent), new Path(child));
     }
-    
+
     public HPath(HDataStorage fs, Path parent, String child) {
         this(fs, parent, new Path(child));
     }
@@ -56,11 +56,11 @@ public abstract class HPath implements E
     public HPath(HDataStorage fs, String parent, Path child) {
         this(fs, new Path(parent), child);
     }
-        
+
     public HPath(HDataStorage fs, String pathString) {
         this(fs, new Path(pathString));
     }
-        
+
     public HPath(HDataStorage fs, Path path) {
         this.path = path;
         this.fs = fs;
@@ -69,27 +69,27 @@ public abstract class HPath implements E
     public DataStorage getDataStorage() {
         return fs;
     }
-    
-    public abstract OutputStream create(Properties configuration) 
+
+    public abstract OutputStream create(Properties configuration)
              throws IOException;
-    
+
     public void copy(ElementDescriptor dstName,
                      Properties dstConfiguration,
                      boolean removeSrc)
             throws IOException {
         FileSystem srcFS = this.fs.getHFS();
         FileSystem dstFS = ((HPath)dstName).fs.getHFS();
-        
+
         Path srcPath = this.path;
         Path dstPath = ((HPath)dstName).path;
-        
+
         boolean result = FileUtil.copy(srcFS,
                                        srcPath,
                                        dstFS,
                                        dstPath,
                                        false,
                                        new Configuration());
-        
+
         if (!result) {
             int errCode = 2097;
             String msg = "Failed to copy from: " + this.toString() +
@@ -97,7 +97,7 @@ public abstract class HPath implements E
             throw new ExecException(msg, errCode, PigException.BUG);
         }
     }
-    
+
     public abstract InputStream open() throws IOException;
 
     public abstract SeekableInputStream sopen() throws IOException;
@@ -105,8 +105,8 @@ public abstract class HPath implements E
     public boolean exists() throws IOException {
         return fs.getHFS().exists(path);
     }
-    
-    public void rename(ElementDescriptor newName) 
+
+    public void rename(ElementDescriptor newName)
              throws IOException {
         if (newName != null) {
             fs.getHFS().rename(path, ((HPath)newName).path);
@@ -118,16 +118,20 @@ public abstract class HPath implements E
         fs.getHFS().delete(path, true);
     }
 
+    public void setPermission(FsPermission permission) throws IOException {
+        fs.getHFS().setPermission(path, permission);
+    }
+
     public Properties getConfiguration() throws IOException {
         HConfiguration props = new HConfiguration();
 
         long blockSize = fs.getHFS().getFileStatus(path).getBlockSize();
 
         short replication = fs.getHFS().getFileStatus(path).getReplication();
-        
+
         props.setProperty(BLOCK_SIZE_KEY, (Long.valueOf(blockSize)).toString());
         props.setProperty(BLOCK_REPLICATION_KEY, (Short.valueOf(replication)).toString());
-        
+
         return props;
     }
 
@@ -135,23 +139,23 @@ public abstract class HPath implements E
         if (newConfig == null) {
             return;
         }
-        
+
         String blkReplStr = newConfig.getProperty(BLOCK_REPLICATION_KEY);
-        
-        fs.getHFS().setReplication(path, 
-                                   new Short(blkReplStr).shortValue());    
+
+        fs.getHFS().setReplication(path,
+                                   new Short(blkReplStr).shortValue());
     }
 
     public Map<String, Object> getStatistics() throws IOException {
         HashMap<String, Object> props = new HashMap<String, Object>();
-        
+
         FileStatus fileStatus = fs.getHFS().getFileStatus(path);
 
         props.put(BLOCK_SIZE_KEY, fileStatus.getBlockSize());
         props.put(BLOCK_REPLICATION_KEY, fileStatus.getReplication());
         props.put(LENGTH_KEY, fileStatus.getLen());
         props.put(MODIFICATION_TIME_KEY, fileStatus.getModificationTime());
-        
+
         return props;
     }
 
@@ -160,43 +164,43 @@ public abstract class HPath implements E
     }
 
     public void copy(ElementDescriptor dstName,
-                     boolean removeSrc) 
+                     boolean removeSrc)
             throws IOException {
         copy(dstName, null, removeSrc);
     }
-    
+
     public Path getPath() {
         return path;
     }
-    
+
     public FileSystem getHFS() {
         return fs.getHFS();
     }
 
     public boolean systemElement() {
-        return (path != null && 
+        return (path != null &&
                 (path.getName().startsWith("_") ||
                  path.getName().startsWith(".")));
     }
-   
+
     @Override
     public String toString() {
         return path.makeQualified(getHFS()).toString();
     }
-    
+
     @Override
     public boolean equals(Object obj) {
         if (! (obj instanceof HPath)) {
             return false;
         }
-        
-        return this.path.equals(((HPath)obj).path);  
+
+        return this.path.equals(((HPath)obj).path);
     }
-    
+
     public int compareTo(ElementDescriptor other) {
         return path.compareTo(((HPath)other).path);
     }
-    
+
     @Override
     public int hashCode() {
         return this.path.hashCode();

Modified: pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java?rev=1560587&r1=1560586&r2=1560587&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java (original)
+++ pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java Thu Jan 23 05:52:39 2014
@@ -39,6 +39,8 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Shell;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigException;
@@ -51,6 +53,7 @@ import org.apache.pig.backend.datastorag
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.hadoop.datastorage.HDataStorage;
+import org.apache.pig.backend.hadoop.datastorage.HDirectory;
 import org.apache.pig.backend.hadoop.datastorage.HPath;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
 import org.apache.pig.impl.PigContext;
@@ -59,16 +62,19 @@ import com.google.common.annotations.Vis
 
 public class FileLocalizer {
     private static final Log log = LogFactory.getLog(FileLocalizer.class);
-    
+
     static public final String LOCAL_PREFIX  = "file:";
     static public final int STYLE_UNIX = 0;
     static public final int STYLE_WINDOWS = 1;
 
+    private static FsPermission rootPerm = new FsPermission(FsAction.ALL, FsAction.NONE,
+            FsAction.NONE); // rwx------
+
     public static class DataStorageInputStreamIterator extends InputStream {
         InputStream current;
         ElementDescriptor[] elements;
         int currentElement;
-        
+
         public DataStorageInputStreamIterator(ElementDescriptor[] elements) {
             this.elements = elements;
         }
@@ -164,7 +170,7 @@ public class FileLocalizer {
             throw new RuntimeException(
                     "can't open DFS file while executing locally");
         }
-        
+
         return openDFSFile(fileName, ConfigurationUtil.toProperties(conf));
 
     }
@@ -174,7 +180,7 @@ public class FileLocalizer {
         ElementDescriptor elem = dds.asElement(fileName);
         return openDFSFile(elem);
     }
-    
+
     public static long getSize(String fileName) throws IOException {
     	Configuration conf = PigMapReduce.sJobConfInternal.get();
 
@@ -185,28 +191,28 @@ public class FileLocalizer {
 
         return getSize(fileName, ConfigurationUtil.toProperties(conf));
     }
-    
+
     public static long getSize(String fileName, Properties properties) throws IOException {
     	DataStorage dds = new HDataStorage(properties);
         ElementDescriptor elem = dds.asElement(fileName);
-       
+
         // recursively get all the files under this path
         ElementDescriptor[] allElems = getFileElementDescriptors(elem);
-        
+
         long size = 0;
-        
+
         // add up the sizes of all files found
         for (int i=0; i<allElems.length; i++) {
         	Map<String, Object> stats = allElems[i].getStatistics();
         	size += (Long) (stats.get(ElementDescriptor.LENGTH_KEY));
         }
-        
+
         return size;
     }
-    
+
     private static InputStream openDFSFile(ElementDescriptor elem) throws IOException{
         ElementDescriptor[] elements = null;
-        
+
         if (elem.exists()) {
             try {
                 if(! elem.getDataStorage().isContainer(elem.toString())) {
@@ -218,7 +224,7 @@ public class FileLocalizer {
             catch (DataStorageException e) {
                 throw new IOException("Failed to determine if elem=" + elem + " is container", e);
             }
-            
+
             // elem is a directory - recursively get all files in it
             elements = getFileElementDescriptors(elem);
         } else {
@@ -226,15 +232,15 @@ public class FileLocalizer {
             if (!globMatchesFiles(elem, elem.getDataStorage())) {
                 throw new IOException(elem.toString() + " does not exist");
             } else {
-                elements = getFileElementDescriptors(elem); 
+                elements = getFileElementDescriptors(elem);
                 return new DataStorageInputStreamIterator(elements);
-                
+
             }
         }
-        
+
         return new DataStorageInputStreamIterator(elements);
     }
-    
+
     /**
      * recursively get all "File" element descriptors present in the input element descriptor
      * @param elem input element descriptor
@@ -259,7 +265,7 @@ public class FileLocalizer {
             if (fullPath.systemElement()) {
                 continue;
             }
-            
+
             if (fullPath instanceof ContainerDescriptor) {
                 for (ElementDescriptor child : ((ContainerDescriptor) fullPath)) {
                     paths.add(child);
@@ -274,7 +280,7 @@ public class FileLocalizer {
         filePaths.toArray(elems);
         return elems;
     }
-    
+
     private static InputStream openLFSFile(ElementDescriptor elem) throws IOException{
         // IMPORTANT NOTE: Currently we use HXXX classes to represent
         // files and dirs in local mode - so we can just delegate this
@@ -282,7 +288,7 @@ public class FileLocalizer {
         // and dirs THIS WILL NEED TO CHANGE
         return openDFSFile(elem);
     }
-    
+
     /**
      * This function returns an input stream to a local file system file or
      * a file residing on Hadoop's DFS
@@ -306,7 +312,7 @@ public class FileLocalizer {
             return openLFSFile(elem);
         }
     }
-    
+
     /**
      * @deprecated Use {@link #fullPath(String, PigContext)} instead
      */
@@ -317,7 +323,7 @@ public class FileLocalizer {
             if (fileName.charAt(0) != '/') {
                 ElementDescriptor currentDir = storage.getActiveContainer();
                 ElementDescriptor elem = storage.asElement(currentDir.toString(), fileName);
-                
+
                 fullPath = elem.toString();
             } else {
                 fullPath = fileName;
@@ -327,7 +333,7 @@ public class FileLocalizer {
         }
         return fullPath;
     }
-    
+
     static public InputStream open(String fileSpec, PigContext pigContext) throws IOException {
         fileSpec = checkDefaultPrefix(pigContext.getExecType(), fileSpec);
         if (!fileSpec.startsWith(LOCAL_PREFIX)) {
@@ -342,35 +348,35 @@ public class FileLocalizer {
             return openLFSFile(elem);
         }
     }
-    
+
     /**
      * @param fileSpec
      * @param offset
      * @param pigContext
      * @return SeekableInputStream
      * @throws IOException
-     * 
+     *
      * This is an overloaded version of open where there is a need to seek in stream. Currently seek is supported
      * only in file, not in directory or glob.
      */
     static public SeekableInputStream open(String fileSpec, long offset, PigContext pigContext) throws IOException {
-        
+
         fileSpec = checkDefaultPrefix(pigContext.getExecType(), fileSpec);
-        
+
         ElementDescriptor elem;
-        if (!fileSpec.startsWith(LOCAL_PREFIX)) 
+        if (!fileSpec.startsWith(LOCAL_PREFIX))
             elem = pigContext.getDfs().asElement(fullPath(fileSpec, pigContext));
-                
+
         else{
             fileSpec = fileSpec.substring(LOCAL_PREFIX.length());
-            elem = pigContext.getLfs().asElement(fullPath(fileSpec, pigContext));            
+            elem = pigContext.getLfs().asElement(fullPath(fileSpec, pigContext));
         }
-        
+
         if (elem.exists() && (!elem.getDataStorage().isContainer(elem.toString()))) {
             try {
                 if (elem.systemElement())
                     throw new IOException ("Attempt is made to open system file " + elem.toString());
-                
+
                 SeekableInputStream sis = elem.sopen();
                 sis.seek(offset, FLAGS.SEEK_SET);
                 return sis;
@@ -383,7 +389,7 @@ public class FileLocalizer {
         else
             throw new IOException("Currently seek is supported only in a file, not in glob or directory.");
     }
-    
+
     static public OutputStream create(String fileSpec, PigContext pigContext) throws IOException{
         return create(fileSpec,false,pigContext);
     }
@@ -404,11 +410,11 @@ public class FileLocalizer {
                 if (!res)
                     log.warn("FileLocalizer.create: failed to create " + f);
             }
-            
+
             return new FileOutputStream(fileSpec,append);
         }
     }
-    
+
     static public boolean delete(String fileSpec, PigContext pigContext) throws IOException{
         fileSpec = checkDefaultPrefix(pigContext.getExecType(), fileSpec);
         ElementDescriptor elem = null;
@@ -456,12 +462,27 @@ public class FileLocalizer {
 
         if (relativeRoot.get() == null) {
             String tdir= pigContext.getProperties().getProperty("pig.temp.dir", "/tmp");
-            relativeRoot.set(pigContext.getDfs().asContainer(tdir + "/temp" + r.nextInt()));
+            ContainerDescriptor relative = pigContext.getDfs().asContainer(tdir + "/temp" + r.nextInt());
+            relativeRoot.set(relative);
+            try {
+                if (!relative.exists()) {
+                    createRelativeRoot(relative);
+                }
+            } catch (IOException e) {
+                throw new DataStorageException(e);
+            }
         }
 
         return relativeRoot.get();
     }
 
+    private static void createRelativeRoot(ContainerDescriptor relativeRoot) throws IOException {
+        relativeRoot.create();
+        if (relativeRoot instanceof HDirectory) {
+            ((HDirectory) relativeRoot).setPermission(rootPerm);
+        }
+    }
+
     public static void deleteTempFiles() {
         if (relativeRoot.get() != null) {
             try {
@@ -480,9 +501,6 @@ public class FileLocalizer {
     public static Path getTemporaryPath(PigContext pigContext, String suffix) throws IOException {
       ElementDescriptor relative = relativeRoot(pigContext);
 
-      if (!relativeRoot(pigContext).exists()) {
-          relativeRoot(pigContext).create();
-      }
       ElementDescriptor elem=
           pigContext.getDfs().asElement(relative.toString(), "tmp" + r.nextInt() + suffix);
       return ((HPath)elem).getPath();
@@ -495,30 +513,30 @@ public class FileLocalizer {
         if (filename.startsWith(LOCAL_PREFIX)) {
             filename = filename.substring(LOCAL_PREFIX.length());
         }
-        
+
         ElementDescriptor localElem =
             pigContext.getLfs().asElement(filename);
-            
+
         if (!localElem.exists()) {
             throw new FileNotFoundException(filename);
         }
-            
+
         ElementDescriptor distribElem = pigContext.getDfs().asElement(
                 getTemporaryPath(pigContext).toString());
-    
+
         int suffixStart = filename.lastIndexOf('.');
         if (suffixStart != -1) {
             distribElem = pigContext.getDfs().asElement(distribElem.toString() +
                     filename.substring(suffixStart));
         }
-            
+
         // TODO: currently the copy method in Data Storage does not allow to specify overwrite
         //       so the work around is to delete the dst file first, if it exists
         if (distribElem.exists()) {
             distribElem.delete();
         }
         localElem.copy(distribElem, null, false);
-            
+
         return distribElem.toString();
     }
 
@@ -528,7 +546,7 @@ public class FileLocalizer {
                 ElementDescriptor currentDir = pigContext.getDfs().getActiveContainer();
                 ElementDescriptor elem = pigContext.getDfs().asElement(currentDir.toString(),
                                                                                   filename);
-                
+
                 return elem.toString();
             }
             return filename;
@@ -546,7 +564,7 @@ public class FileLocalizer {
     /**
      * @deprecated Use {@link #fileExists(String, PigContext)} instead
      */
-    @Deprecated 
+    @Deprecated
     public static boolean fileExists(String filename, DataStorage store)
             throws IOException {
         ElementDescriptor elem = store.asElement(filename);
@@ -559,7 +577,7 @@ public class FileLocalizer {
     }
 
     /**
-     * @deprecated Use {@link #isFile(String, PigContext)} instead 
+     * @deprecated Use {@link #isFile(String, PigContext)} instead
      */
     @Deprecated
     public static boolean isFile(String filename, DataStorage store)
@@ -593,10 +611,10 @@ public class FileLocalizer {
             switch (elems.length) {
             case 0:
                 return false;
-    
+
             case 1:
                 return !elems[0].equals(elem);
-    
+
             default:
                 return true;
             }
@@ -610,17 +628,17 @@ public class FileLocalizer {
     public static void setR(Random r) {
         FileLocalizer.r = r;
     }
-    
+
     /**
      * Convert path from Windows convention to Unix convention. Invoked under
      * cygwin.
-     * 
+     *
      * @param path
      *            path in Windows convention
      * @return path in Unix convention, null if fail
      */
     static public String parseCygPath(String path, int style) {
-        String[] command; 
+        String[] command;
         if (style==STYLE_WINDOWS)
             command = new String[] { "cygpath", "-w", path };
         else
@@ -653,7 +671,7 @@ public class FileLocalizer {
         }
         return line;
     }
-    
+
     static File localTempDir = null;
     static {
         File f;
@@ -669,8 +687,8 @@ public class FileLocalizer {
         if (!success) {
           throw new RuntimeException("Error creating FileLocalizer temp directory.");
         }
-    }    
-    
+    }
+
     public static class FetchFileRet {
         public FetchFileRet(File file, boolean didFetch) {
             this.file = file;
@@ -681,9 +699,9 @@ public class FileLocalizer {
     }
 
     /**
-     * Ensures that the passed path is on the local file system, fetching it 
+     * Ensures that the passed path is on the local file system, fetching it
      * to the java.io.tmpdir if necessary. If pig.jars.relative.to.dfs is true
-     * and dfs is not null, then a relative path is assumed to be relative to the passed 
+     * and dfs is not null, then a relative path is assumed to be relative to the passed
      * dfs active directory. Else they are assumed to be relative to the local working
      * directory.
      */
@@ -776,21 +794,21 @@ public class FileLocalizer {
 
         return fetchFiles;
     }
-    
+
     /**
      * Ensures that the passed resource is available from the local file system, fetching
      * it to a temporary directory.
-     * 
-     * @throws ResourceNotFoundException 
+     *
+     * @throws ResourceNotFoundException
      */
     public static FetchFileRet fetchResource(String name) throws IOException, ResourceNotFoundException {
       FetchFileRet localFileRet = null;
       InputStream resourceStream = PigContext.getClassLoader().getResourceAsStream(name);
-      if (resourceStream != null) {        
+      if (resourceStream != null) {
         File dest = new File(localTempDir, name);
-        dest.getParentFile().mkdirs();        
+        dest.getParentFile().mkdirs();
         dest.deleteOnExit();
-                
+
         OutputStream outputStream = new BufferedOutputStream(new FileOutputStream(dest));
         byte[] buffer = new byte[1024];
         int len;
@@ -798,14 +816,14 @@ public class FileLocalizer {
           outputStream.write(buffer,0,len);
         }
         outputStream.close();
-        
+
         localFileRet = new FetchFileRet(dest,false);
       }
       else
       {
         throw new ResourceNotFoundException(name);
       }
-      
+
       return localFileRet;
     }
 }

Modified: pig/trunk/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigServer.java?rev=1560587&r1=1560586&r2=1560587&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServer.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigServer.java Thu Jan 23 05:52:39 2014
@@ -50,6 +50,7 @@ import java.util.Properties;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
@@ -57,6 +58,7 @@ import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.builtin.mock.Storage;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.DataType;
@@ -535,16 +537,16 @@ public class TestPigServer {
         pig.registerQuery("c = group b by site;");
         pig.registerQuery("d = foreach c generate FLATTEN($1);");
         pig.registerQuery("e = group d by $2;");
-        
+
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
         pig.explain("e", "xml", true, false, ps, ps, null, null);
-        
+
         ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
         DocumentBuilderFactory dbFactory = DocumentBuilderFactory.newInstance();
         DocumentBuilder dBuilder = dbFactory.newDocumentBuilder();
         Document doc = dBuilder.parse(bais);
-        
+
         //Verify Logical and Physical Plans aren't supported.
         NodeList logicalPlan = doc.getElementsByTagName("logicalPlan");
         assertEquals(1, logicalPlan.getLength());
@@ -552,20 +554,20 @@ public class TestPigServer {
         NodeList physicalPlan = doc.getElementsByTagName("physicalPlan");
         assertEquals(1, physicalPlan.getLength());
         assertTrue(physicalPlan.item(0).getTextContent().contains("Not Supported"));
-        
+
         //Verify we have two loads and one is temporary
         NodeList loads = doc.getElementsByTagName("POLoad");
         assertEquals(2, loads.getLength());
-        
+
         boolean sawTempLoad = false;
         boolean sawNonTempLoad = false;
         for (int i = 0; i < loads.getLength(); i++) {
             Boolean isTempLoad = null;
             boolean hasAlias = false;
-            
+
             Node poLoad = loads.item(i);
             NodeList children = poLoad.getChildNodes();
-            
+
             for (int j = 0; j < children.getLength(); j++) {
                 Node child = children.item(j);
                 if (child.getNodeName().equals("alias")) {
@@ -579,7 +581,7 @@ public class TestPigServer {
                     }
                 }
             }
-            
+
             if (isTempLoad == null) {
                 fail("POLoad elements should have isTmpLoad child node.");
             } else if (isTempLoad && hasAlias) {
@@ -587,11 +589,11 @@ public class TestPigServer {
             } else if (!isTempLoad && !hasAlias) {
                 fail("Non temporary loads should be associated with alias.");
             }
-            
+
             sawTempLoad = sawTempLoad || isTempLoad;
             sawNonTempLoad = sawNonTempLoad || !isTempLoad;
         }
-        
+
         assertTrue(sawTempLoad && sawNonTempLoad);
     }
 
@@ -775,14 +777,20 @@ public class TestPigServer {
         File propertyFile = new File(tempDir, "pig.properties");
         propertyFile.deleteOnExit();
         PrintWriter out = new PrintWriter(new FileWriter(propertyFile));
-        out.println("pig.temp.dir=/opt/temp");
+        out.println("pig.temp.dir=/tmp/test");
         out.close();
         Properties properties = PropertiesUtil.loadDefaultProperties();
         PigContext pigContext=new PigContext(ExecType.LOCAL, properties);
         pigContext.connect();
         FileLocalizer.setInitialized(false);
         String tempPath= FileLocalizer.getTemporaryPath(pigContext).toString();
-        assertTrue(tempPath.startsWith("file:/opt/temp"));
+        assertTrue(tempPath.startsWith("file:/tmp/test/"));
+        Path path = new Path(tempPath);
+        FileSystem fs = FileSystem.get(path.toUri(),
+                ConfigurationUtil.toConfiguration(pigContext.getProperties()));
+        FileStatus status = fs.getFileStatus(path.getParent());
+        // Temporary root dir should have 700 as permission
+        assertEquals("rwx------", status.getPermission().toString());
         propertyFile.delete();
         FileLocalizer.setInitialized(false);
     }