You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ol...@apache.org on 2008/10/22 20:37:14 UTC

svn commit: r707158 [1/2] - in /incubator/pig/branches/types: ./ src/org/apache/pig/builtin/ src/org/apache/pig/data/ src/org/apache/pig/impl/io/ src/org/apache/pig/impl/logicalLayer/schema/ src/org/apache/pig/impl/streaming/ test/org/apache/pig/test/

Author: olga
Date: Wed Oct 22 11:37:13 2008
New Revision: 707158

URL: http://svn.apache.org/viewvc?rev=707158&view=rev
Log:
PIG-504: illustrate cleanup; PIG-469: describe cleanup

Modified:
    incubator/pig/branches/types/CHANGES.txt
    incubator/pig/branches/types/src/org/apache/pig/builtin/PigStorage.java
    incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java
    incubator/pig/branches/types/src/org/apache/pig/data/DataType.java
    incubator/pig/branches/types/src/org/apache/pig/impl/io/FileLocalizer.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java
    incubator/pig/branches/types/src/org/apache/pig/impl/streaming/ExecutableManager.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestAlgebraicEval.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestBZip.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestBestFitCast.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestCompressedFiles.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestFilterUDF.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestForEachNestedPlan.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestImplicitSplit.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestInputOutputFileValidator.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestJobSubmission.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestLocalJobSubmission.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestLocalPOSplit.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce2.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestParamSubPreproc.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestPigContext.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestPigScriptParser.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestPigServer.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestPigSplit.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestSplitStore.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java
    incubator/pig/branches/types/test/org/apache/pig/test/Util.java

Modified: incubator/pig/branches/types/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/CHANGES.txt?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/CHANGES.txt (original)
+++ incubator/pig/branches/types/CHANGES.txt Wed Oct 22 11:37:13 2008
@@ -287,6 +287,12 @@
 
     PIG-494: invalid handling of UTF-8 data in PigStorage (pradeepk via olgan)
 
+    PIG-501: Make branches/types work under cygwin (daijy via olgan)
+
+    PIG-504: cleanup illustrate not to produce cn= (shubham via olgan)
+
+    PIG-469: make sure that describe says "int" not "integer" (sms via olgan)
+
     PIG-495: projecting of bags only give 1 field (olgan)
 
     PIG-500: Load Func for POCast is not being set in some cases (sms via

Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/PigStorage.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/PigStorage.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/builtin/PigStorage.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/builtin/PigStorage.java Wed Oct 22 11:37:13 2008
@@ -22,6 +22,7 @@
 import java.io.OutputStream;
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Map;
 
 import org.apache.commons.logging.LogFactory;
@@ -55,8 +56,14 @@
     private byte fieldDel = '\t';
     private ByteArrayOutputStream mBuf = null;
     private ArrayList<Object> mProtoTuple = null;
+    private int os;
+    private static final int OS_UNIX = 0;
+    private static final int OS_WINDOWS = 1;
     
     public PigStorage() {
+        os = OS_UNIX;
+        if (System.getProperty("os.name").toUpperCase().startsWith("WINDOWS"))
+            os = OS_WINDOWS;
     }
 
     /**
@@ -67,6 +74,7 @@
      *            ("\t" is the default.)
      */
     public PigStorage(String delimiter) {
+        this();
         if (delimiter.length() == 1) {
             this.fieldDel = (byte)delimiter.charAt(0);
         } else if (delimiter.length() > 1 && delimiter.charAt(0) == '\\') {
@@ -214,7 +222,13 @@
             // TODO, once this can take schemas, we need to figure out
             // if the user requested this to be viewed as a certain
             // type, and if so, then construct it appropriately.
-            mProtoTuple.add(new DataByteArray(mBuf.toByteArray()));
+            byte[] array = mBuf.toByteArray();
+            if (array[array.length-1]=='\r' && os==OS_WINDOWS)
+                array = Arrays.copyOf(array, array.length-1);
+            if (array.length==0)
+                mProtoTuple.add(null);
+            else
+                mProtoTuple.add(new DataByteArray(array));
         }
         mBuf.reset();
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java Wed Oct 22 11:37:13 2008
@@ -59,6 +59,8 @@
             return null;
         String line;
         if ((line = in.readLine(utf8, (byte)'\n')) != null) {
+            if (line.length()>0 && line.charAt(line.length()-1)=='\r' && System.getProperty("os.name").toUpperCase().startsWith("WINDOWS"))
+                line = line.substring(0, line.length()-1);
             return mTupleFactory.newTuple(new DataByteArray(line.getBytes()));
         }
         return null;

Modified: incubator/pig/branches/types/src/org/apache/pig/data/DataType.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DataType.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/data/DataType.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/data/DataType.java Wed Oct 22 11:37:13 2008
@@ -192,7 +192,7 @@
         case NULL:      return "NULL";
         case BOOLEAN:   return "boolean";
         case BYTE:      return "byte";
-        case INTEGER:   return "integer";
+        case INTEGER:   return "int";
         case LONG:      return "long";
         case FLOAT:     return "float";
         case DOUBLE:    return "double";

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/io/FileLocalizer.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/io/FileLocalizer.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/io/FileLocalizer.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/io/FileLocalizer.java Wed Oct 22 11:37:13 2008
@@ -17,13 +17,13 @@
  */
 package org.apache.pig.impl.io;
 
-import java.io.BufferedInputStream;
+import java.io.BufferedReader;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -51,6 +51,8 @@
     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;
 
     public static class DataStorageInputStreamIterator extends InputStream {
         InputStream current;
@@ -535,4 +537,42 @@
             }
     	}
     }
+    /**
+     * 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; 
+        if (style==STYLE_WINDOWS)
+            command = new String[] { "cygpath", "-w", path };
+        else
+            command = new String[] { "cygpath", "-u", path };
+        Process p = null;
+        try {
+            p = Runtime.getRuntime().exec(command);
+        } catch (IOException e) {
+            return null;
+        }
+        int exitVal = 0;
+        try {
+            exitVal = p.waitFor();
+        } catch (InterruptedException e) {
+            return null;
+        }
+        if (exitVal != 0)
+            return null;
+        String line = null;
+        try {
+            InputStreamReader isr = new InputStreamReader(p.getInputStream());
+            BufferedReader br = new BufferedReader(isr);
+            line = br.readLine();
+        } catch (IOException e) {
+            return null;
+        }
+        return line;
+    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java Wed Oct 22 11:37:13 2008
@@ -386,10 +386,10 @@
                 sb.append(")");
             }
 
-            if (canonicalName != null) {
-                sb.append(" cn: ");
-                sb.append(canonicalName);
-            }
+//            if (canonicalName != null) {
+//                sb.append(" cn: ");
+//                sb.append(canonicalName);
+//            }
 
             return sb.toString();
         }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/streaming/ExecutableManager.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/streaming/ExecutableManager.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/streaming/ExecutableManager.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/streaming/ExecutableManager.java Wed Oct 22 11:37:13 2008
@@ -40,6 +40,7 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.BufferedPositionedInputStream;
+import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.streaming.InputHandler.InputType;
 import org.apache.pig.impl.streaming.OutputHandler.OutputType;
 
@@ -199,41 +200,6 @@
     }
 
     /**
-     * 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
-     */
-    private String parseCygPath(String path) {
-        String[] command = new String[] { "cygpath", "-u", path };
-        Process p = null;
-        try {
-            p = Runtime.getRuntime().exec(command);
-        } catch (IOException e) {
-            return null;
-        }
-        int exitVal = 0;
-        try {
-            exitVal = p.waitFor();
-        } catch (InterruptedException e) {
-            return null;
-        }
-        if (exitVal != 0)
-            return null;
-        String line = null;
-        try {
-            InputStreamReader isr = new InputStreamReader(p.getInputStream());
-            BufferedReader br = new BufferedReader(isr);
-            line = br.readLine();
-        } catch (IOException e) {
-            return null;
-        }
-        return line;
-    }
-
-    /**
      * Set up the run-time environment of the managed process.
      * 
      * @param pb
@@ -249,7 +215,7 @@
                 .getProperty("user.dir");
 
         if (System.getProperty("os.name").toUpperCase().startsWith("WINDOWS")) {
-            String unixCwd = parseCygPath(cwd);
+            String unixCwd = FileLocalizer.parseCygPath(cwd, FileLocalizer.STYLE_UNIX);
             if (unixCwd == null)
                 throw new RuntimeException(
                         "Can not convert Windows path to Unix path under cygwin");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestAlgebraicEval.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestAlgebraicEval.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestAlgebraicEval.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestAlgebraicEval.java Wed Oct 22 11:37:13 2008
@@ -95,7 +95,7 @@
                 }
                 ps.close();                
             }
-            pig.registerQuery(" a = group (load 'file:" + tmpFile + "') by ($0,$1);");
+            pig.registerQuery(" a = group (load '" + Util.generateURI(tmpFile.toString()) + "') by ($0,$1);");
             pig.registerQuery("b = foreach a generate flatten(group), SUM($1.$2);");
             Iterator<Tuple> it = pig.openIterator("b");
             int count = 0;
@@ -134,7 +134,7 @@
         
             PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
             int numNulls = generateInput(ps, nullFlags[i]);
-            String query = "myid =  foreach (group (load 'file:" + tmpFile + "') all) generate COUNT($1);";
+            String query = "myid =  foreach (group (load '" + Util.generateURI(tmpFile.toString()) + "') all) generate COUNT($1);";
             System.out.println(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("myid");
@@ -153,7 +153,7 @@
         
             PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
             int numNulls = generateInput(ps, nullFlags[i]);
-            String query = "myid = foreach (group (load 'file:" + tmpFile + "') all) generate group, COUNT($1) ;";
+            String query = "myid = foreach (group (load '" + Util.generateURI(tmpFile.toString()) + "') all) generate group, COUNT($1) ;";
             System.out.println(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("myid");
@@ -171,7 +171,7 @@
             System.err.println("Testing testGroupCount with null flag:" + nullFlags[i]);
             PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
             int numNulls = generateInput(ps, nullFlags[i]);
-            String query = "myid = foreach (group (load 'file:" + tmpFile + "') all) generate COUNT($1), group ;";
+            String query = "myid = foreach (group (load '" + Util.generateURI(tmpFile.toString()) + "') all) generate COUNT($1), group ;";
             System.out.println(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("myid");
@@ -210,7 +210,7 @@
                 }
             }         
             ps.close();
-            String query = "myid = foreach (group (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) by $0) generate group, COUNT($1.$1) ;";
+            String query = "myid = foreach (group (load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':')) by $0) generate group, COUNT($1.$1) ;";
             System.out.println(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("myid");
@@ -257,7 +257,7 @@
                 }
             }
             ps.close();
-            String query = "myid = foreach (group (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) by $0) generate group, COUNT($1.$1), COUNT($1.$0) ;";
+            String query = "myid = foreach (group (load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':')) by $0) generate group, COUNT($1.$1), COUNT($1.$0) ;";
             System.out.println(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("myid");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestBZip.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestBZip.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestBZip.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestBZip.java Wed Oct 22 11:37:13 2008
@@ -43,16 +43,16 @@
             cos.write((-i + "\n").getBytes());
         }
         cos.close();
-        pig.registerQuery("AA=load 'file:" + in.getAbsolutePath() + "';");
+        pig.registerQuery("AA=load '" + Util.generateURI(in.getAbsolutePath()) + "';");
         pig.registerQuery("A=foreach (group (filter AA by $0 > 0) all) generate flatten($1);");
-        pig.store("A", "file:" + out.getAbsolutePath());
+        pig.store("A", Util.generateURI(out.getAbsolutePath()));
         CBZip2InputStream cis = new CBZip2InputStream(
                 new LocalSeekableInputStream(new File(out, "part-00000.bz")));
         // Just a sanity check, to make sure it was a bzip file; we
         // will do the value verification later
         assertEquals(100, cis.read(new byte[100]));
         cis.close();
-        pig.registerQuery("B=load 'file:" + out.getAbsolutePath() + "';");
+        pig.registerQuery("B=load '" + Util.generateURI(out.getAbsolutePath()) + "';");
         Iterator<Tuple> i = pig.openIterator("B");
         HashMap<Integer, Integer> map = new HashMap<Integer, Integer>();
         while (i.hasNext()) {
@@ -87,15 +87,15 @@
         fos.write("55\n".getBytes());
         fos.close();
         System.out.println(in.getAbsolutePath());
-        pig.registerQuery("AA=load 'file:" + in.getAbsolutePath() + "';");
+        pig.registerQuery("AA=load '" + Util.generateURI(in.getAbsolutePath()) + "';");
         pig
                 .registerQuery("A=foreach (group (filter AA by $0 < '0') all) generate flatten($1);");
-        pig.store("A", "file:" + out.getAbsolutePath());
+        pig.store("A", Util.generateURI(out.getAbsolutePath()));
         CBZip2InputStream cis = new CBZip2InputStream(
                 new LocalSeekableInputStream(new File(out, "part-00000.bz")));
         assertEquals(-1, cis.read(new byte[100]));
         cis.close();
-        pig.registerQuery("B=load 'file:" + out.getAbsolutePath() + "';");
+        pig.registerQuery("B=load '" + Util.generateURI(out.getAbsolutePath()) + "';");
         pig.openIterator("B");
         in.delete();
         out.delete();

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestBestFitCast.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestBestFitCast.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestBestFitCast.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestBestFitCast.java Wed Oct 22 11:37:13 2008
@@ -91,7 +91,7 @@
         //Passing (long, int)
         //Possible matches: (float, float) , (long, double)
         //Chooses (long, double) as it has only one cast compared to two for (float, float)
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "' as (x:long, y:int);");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "' as (x:long, y:int);");
         pigServer.registerQuery("B = FOREACH A generate x, " + UDF1.class.getName() + "(x,y);");
         Iterator<Tuple> iter = pigServer.openIterator("B");
         if(!iter.hasNext()) fail("No Output received");
@@ -110,7 +110,7 @@
         //Passing (int, int)
         //Possible matches: (float, float) , (long, double)
         //Throws Exception as ambiguous definitions found
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "' as (x:long, y:int);");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "' as (x:long, y:int);");
         pigServer.registerQuery("B = FOREACH A generate x, " + UDF1.class.getName() + "(y,y);");
         try{
             pigServer.openIterator("B");
@@ -126,7 +126,7 @@
         //Passing (int, int)
         //Possible matches: (float, float) , (long, double)
         //Chooses (float, float) as both options lead to same score and (float, float) occurs first.
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "' as (x:long, y:int);");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "' as (x:long, y:int);");
         pigServer.registerQuery("B = FOREACH A generate x, " + UDF1.class.getName() + "((float)y,(float)y);");
         Iterator<Tuple> iter = pigServer.openIterator("B");
         if(!iter.hasNext()) fail("No Output received");
@@ -145,7 +145,7 @@
         //Passing (long)
         //Possible matches: (float), (integer), (double)
         //Chooses (float) as it leads to a better score that to (double)
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "' as (x:long, y:int);");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "' as (x:long, y:int);");
         pigServer.registerQuery("B = FOREACH A generate x, " + UDF1.class.getName() + "(x);");
         Iterator<Tuple> iter = pigServer.openIterator("B");
         if(!iter.hasNext()) fail("No Output received");
@@ -163,7 +163,7 @@
         //Passing bytearrays
         //Possible matches: (float, float) , (long, double)
         //Throws exception since more than one funcSpec and inp is bytearray
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
         pigServer.registerQuery("B = FOREACH A generate $0, " + UDF1.class.getName() + "($1,$1);");
         try{
             pigServer.openIterator("B");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java Wed Oct 22 11:37:13 2008
@@ -78,8 +78,8 @@
             ps.println(line);
         }
         ps.close();
-        pig.registerQuery(loadAlias + " = load 'file:"
-                + inputFile + "' using "
+        pig.registerQuery(loadAlias + " = load '"
+                + Util.generateURI(inputFile.toString()) + "' using "
                 + PigStorage.class.getName() + "(',');");
     }
 

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestCompressedFiles.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestCompressedFiles.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestCompressedFiles.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestCompressedFiles.java Wed Oct 22 11:37:13 2008
@@ -74,7 +74,7 @@
     @Test
     public void testCompressed1() throws Throwable {
         PigServer pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-        pig.registerQuery("A = foreach (cogroup (load 'file:"+gzFile+"') by $1, (load 'file:"+datFile + "') by $1) generate flatten( " + DIFF.class.getName() + "($1.$1,$2.$1)) ;");
+        pig.registerQuery("A = foreach (cogroup (load '"+Util.generateURI(gzFile.toString())+"') by $1, (load '"+Util.generateURI(datFile.toString()) + "') by $1) generate flatten( " + DIFF.class.getName() + "($1.$1,$2.$1)) ;");
         Iterator it = pig.openIterator("A");
         boolean success = true;
         while(it.hasNext()) {

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java Wed Oct 22 11:37:13 2008
@@ -96,7 +96,7 @@
         
         File f1 = createFile(new String[]{"a:1","b:1","a:1"});
 
-        pigServer.registerQuery("a = load 'file:" + f1 + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("a = load '" + Util.generateURI(f1.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         pigServer.registerQuery("b = foreach a generate 1-1/1;");
         Iterator<Tuple> iter  = pigServer.openIterator("b");
         
@@ -112,8 +112,8 @@
         File f1 = createFile(new String[]{"a:1","b:1","a:1"});
         File f2 = createFile(new String[]{"b","b","a"});
         
-        pigServer.registerQuery("a = load 'file:" + f1 + "' using " + PigStorage.class.getName() + "(':');");
-        pigServer.registerQuery("b = load 'file:" + f2 + "';");
+        pigServer.registerQuery("a = load '" + Util.generateURI(f1.toString()) + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("b = load '" + Util.generateURI(f2.toString()) + "';");
         pigServer.registerQuery("c = cogroup a by $0, b by $0;");        
         pigServer.registerQuery("d = foreach c generate flatten($1),flatten($2);");
         
@@ -134,7 +134,7 @@
         pw.println("a");
         pw.println("a");
         pw.close();
-        pigServer.registerQuery("a = foreach (load 'file:" + f + "') generate 1, flatten(" + MyBagFunction.class.getName() + "(*));");
+        pigServer.registerQuery("a = foreach (load '" + Util.generateURI(f.toString()) + "') generate 1, flatten(" + MyBagFunction.class.getName() + "(*));");
 //        pigServer.registerQuery("b = foreach a generate $0, flatten($1);");
         Iterator<Tuple> iter = pigServer.openIterator("a");
         int count = 0;
@@ -170,7 +170,7 @@
         f.store(b, new BinStorage(), pigServer.getPigContext());
         
         
-        pigServer.registerQuery("a = load '" + fileName + "' using BinStorage();");
+        pigServer.registerQuery("a = load '" + Util.encodeEscape(fileName) + "' using BinStorage();");
         pigServer.registerQuery("b = foreach a generate $0#'apple',flatten($1#'orange');");
         Iterator<Tuple> iter = pigServer.openIterator("b");
         t = iter.next();
@@ -291,8 +291,8 @@
         expectedResults.put("world", 1);
         expectedResults.put("conference", 1);
         
-        pigServer.registerQuery("newsArticles = LOAD 'file:" + newsFile + "' USING " + TextLoader.class.getName() + "();");
-        pigServer.registerQuery("queryLog = LOAD 'file:" + queryLogFile + "';");
+        pigServer.registerQuery("newsArticles = LOAD '" + Util.generateURI(newsFile.toString()) + "' USING " + TextLoader.class.getName() + "();");
+        pigServer.registerQuery("queryLog = LOAD '" + Util.generateURI(queryLogFile.toString()) + "';");
 
         pigServer.registerQuery("titleNGrams = FOREACH newsArticles GENERATE flatten(" + TitleNGrams.class.getName() + "(*));");
         pigServer.registerQuery("cogrouped = COGROUP titleNGrams BY $0 INNER, queryLog BY $0 INNER;");
@@ -345,7 +345,7 @@
         ps.close(); 
         
         String tmpOutputFile = FileLocalizer.getTemporaryPath(null, pigServer.getPigContext()).toString();
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
         if (eliminateDuplicates){
             pigServer.registerQuery("B = DISTINCT (FOREACH A GENERATE $0) PARALLEL 10;");
         }else{
@@ -393,7 +393,7 @@
 
         String tmpOutputFile = FileLocalizer.getTemporaryPath(null, 
         pigServer.getPigContext()).toString();
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
         pigServer.registerQuery("B = group A by $0;");
         String query = "C = foreach B {"
         + "C1 = filter A by $0 > -1;"
@@ -434,7 +434,7 @@
 
         String tmpOutputFile = FileLocalizer.getTemporaryPath(null, 
         pigServer.getPigContext()).toString();
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
         pigServer.registerQuery("B = group A by $0;");
         String query = "C = foreach B {"
         + "C1 = filter A by $0 > -1;"
@@ -477,7 +477,7 @@
 
         String tmpOutputFile = FileLocalizer.getTemporaryPath(null, 
         pigServer.getPigContext()).toString();
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
         pigServer.registerQuery("B = limit A 5;");
         Iterator<Tuple> iter = pigServer.openIterator("B");
         if(!iter.hasNext()) fail("No output found");
@@ -495,7 +495,7 @@
         File input = Util.createInputFile("tmp", "", 
                 new String[] {"{(f1, f2),(f3, f4)}\t(1,2)\t[key1#value1,key2#value2]"});
         
-        pigServer.registerQuery("a = load 'file:" + Util.encodeEscape(input.toString()) + "' using PigStorage() " +
+        pigServer.registerQuery("a = load '" + Util.generateURI(input.toString()) + "' using PigStorage() " +
                 "as (b:bag{t:tuple(x,y)}, t2:tuple(a,b), m:map[]);");
         pigServer.registerQuery("b = foreach a generate COUNT(b), t2.a, t2.b, m#'key1', m#'key2';");
         Iterator<Tuple> it = pigServer.openIterator("b");
@@ -507,7 +507,7 @@
         assertEquals("value2", t.get(4).toString());
         
         //test with BinStorage
-        pigServer.registerQuery("a = load 'file:" + Util.encodeEscape(input.toString()) + "' using PigStorage() " +
+        pigServer.registerQuery("a = load '" + Util.generateURI(input.toString()) + "' using PigStorage() " +
                 "as (b:bag{t:tuple(x,y)}, t2:tuple(a,b), m:map[]);");
         String output = "/pig/out/TestEvalPipeline-testComplexData";
         pigServer.deleteFile(output);

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java Wed Oct 22 11:37:13 2008
@@ -62,7 +62,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using "+PigStorage.class.getName() +"(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using "+PigStorage.class.getName() +"(':');");
         String query = "A = filter A by $0 == $1;";
         log.info(query);
         pig.registerQuery(query);
@@ -92,7 +92,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 != $1;";
         log.info(query);
         pig.registerQuery(query);
@@ -118,7 +118,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " +
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " +
                 PigStorage.class.getName() +
                 "(':') as (f1: double, f2:double);");
         String query = "A = filter A by $0 > $1;";
@@ -143,7 +143,7 @@
             ps.println(i + "\t" + i + "\t1");            
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "';");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "';");
         String query = "A = foreach A generate ($1 >= "+ LOOP_COUNT+"-10?'1':'0');";
         log.info(query);
         pig.registerQuery(query);
@@ -171,7 +171,7 @@
             ps.println(i + "\t" + i + "\t1");            
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "';");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "';");
         String query = "A = foreach A generate ($0 < 10?($1 >= 5 ? 2: 1) : 0);";
         log.info(query);
         pig.registerQuery(query);
@@ -200,7 +200,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':') as (a: double, b:double);");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':') as (a: double, b:double);");
         String query = "A = filter A by $0 < $1;";
 
         log.info(query);
@@ -231,7 +231,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 >= $1;";
 
         log.info(query);
@@ -260,7 +260,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':') as (a: double, b:double);");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':') as (a: double, b:double);");
         String query = "A = filter A by $0 <= $1;";
 
         log.info(query);

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java Wed Oct 22 11:37:13 2008
@@ -68,7 +68,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 eq $1;";
 
         log.info(query);
@@ -105,7 +105,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 neq $1;";
 
         log.info(query);
@@ -142,7 +142,7 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 gt $1;";
 
         log.info(query);
@@ -184,7 +184,7 @@
         }
         ps.close();
         
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 gte $1;";
 
         log.info(query);
@@ -221,7 +221,7 @@
         }
         ps.close();
         
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 lt $1;";
 
         log.info(query);
@@ -262,7 +262,7 @@
         }
         ps.close();
         
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pig.registerQuery("A=load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 lte $1;";
 
         log.info(query);

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestFilterUDF.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestFilterUDF.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestFilterUDF.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestFilterUDF.java Wed Oct 22 11:37:13 2008
@@ -67,7 +67,7 @@
     @Test
     public void testFilterUDF() throws Exception{
         
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "' as (x:int);");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "' as (x:int);");
         pigServer.registerQuery("B = filter A by " + MyFilterFunction.class.getName() + "();");
         Iterator<Tuple> iter = pigServer.openIterator("B");
         if(!iter.hasNext()) fail("No Output received");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestForEachNestedPlan.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestForEachNestedPlan.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestForEachNestedPlan.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestForEachNestedPlan.java Wed Oct 22 11:37:13 2008
@@ -49,7 +49,7 @@
         for (int i = 0; i < nullFlags.length; i++) {
             System.err.println("Running testInnerOrderBy with nullFlags set to :" + nullFlags[i]);
             File tmpFile = genDataSetFile1(nullFlags[i]);
-            pig.registerQuery("a = load 'file:" + tmpFile + "'; ");
+            pig.registerQuery("a = load '" + Util.generateURI(tmpFile.toString()) + "'; ");
             pig.registerQuery("b = group a by $0; ");
             pig.registerQuery("c = foreach b { " + "     c1 = order $1 by *; "
                     + "    generate flatten(c1); " + "};");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestImplicitSplit.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestImplicitSplit.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestImplicitSplit.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestImplicitSplit.java Wed Oct 22 11:37:13 2008
@@ -36,7 +36,7 @@
             ps.println(i);
         }
         ps.close();
-        pigServer.registerQuery("A = LOAD 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
         pigServer.registerQuery("B = filter A by $0<=10;");
         pigServer.registerQuery("C = filter A by $0>10;");
         pigServer.registerQuery("D = union B,C;");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java Wed Oct 22 11:37:13 2008
@@ -63,7 +63,7 @@
             System.err.println("Testing with nulls: " + nullFlags[i]);
             PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
             generateInput(ps, nullFlags[i]);
-            String query = "A = foreach (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 + $1, $1;";
+            String query = "A = foreach (load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 + $1, $1;";
             log.info(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("A");
@@ -92,7 +92,7 @@
             System.err.println("Testing with nulls: " + nullFlags[i]);
             PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
             generateInput(ps, nullFlags[i]);
-            String query = "A = foreach (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 - $1, $1 ;";
+            String query = "A = foreach (load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 - $1, $1 ;";
             log.info(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("A");
@@ -119,7 +119,7 @@
             System.err.println("Testing with nulls: " + nullFlags[i]);
             PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
             generateInput(ps, nullFlags[i]);
-            String query = "A = foreach (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 * $1, $1 ;";
+            String query = "A = foreach (load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 * $1, $1 ;";
             log.info(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("A");
@@ -146,7 +146,7 @@
             System.err.println("Testing with nulls: " + nullFlags[i]);
             PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
             generateInput(ps, nullFlags[i]);
-            String query = "A = foreach (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 / $1, $1 ;";
+            String query = "A = foreach (load '" + Util.generateURI(tmpFile.toString()) + "' using " + PigStorage.class.getName() + "(':')) generate $0, $0 / $1, $1 ;";
             log.info(query);
             pig.registerQuery(query);
             Iterator it = pig.openIterator("A");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestInputOutputFileValidator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestInputOutputFileValidator.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestInputOutputFileValidator.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestInputOutputFileValidator.java Wed Oct 22 11:37:13 2008
@@ -7,6 +7,7 @@
 import org.apache.pig.FuncSpec;
 import org.apache.pig.backend.datastorage.ElementDescriptor;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.impl.logicalLayer.LOLoad;
 import org.apache.pig.impl.logicalLayer.LOStore;
@@ -153,7 +154,11 @@
         ElementDescriptor localElem =
             ctx.getLfs().asElement(fp1.getAbsolutePath());           
             
-        ElementDescriptor distribElem = ctx.getDfs().asElement(fp1.getAbsolutePath()) ;
+        String path = fp1.getAbsolutePath();
+        if (System.getProperty("os.name").toUpperCase().startsWith("WINDOWS"))
+            path = FileLocalizer.parseCygPath(path, FileLocalizer.STYLE_UNIX);
+            
+        ElementDescriptor distribElem = ctx.getDfs().asElement(path) ;
     
         localElem.copy(distribElem, null, false);
             
@@ -163,8 +168,12 @@
     private String createHadoopNonExistenceTempFile(PigContext ctx) throws Throwable {
         
         File fp1 = generateTempFile() ;         
-            
-        ElementDescriptor distribElem = ctx.getDfs().asElement(fp1.getAbsolutePath()) ;
+         
+        String path = fp1.getAbsolutePath();
+        if (System.getProperty("os.name").toUpperCase().startsWith("WINDOWS"))
+            path = FileLocalizer.parseCygPath(path, FileLocalizer.STYLE_UNIX);
+        
+        ElementDescriptor distribElem = ctx.getDfs().asElement(path) ;
         
         if (distribElem.exists()) {
             distribElem.delete() ;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestJobSubmission.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestJobSubmission.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestJobSubmission.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestJobSubmission.java Wed Oct 22 11:37:13 2008
@@ -105,7 +105,11 @@
     public void setUp() throws Exception{
         curDir = System.getProperty("user.dir");
         inpDir = curDir + File.separatorChar + "test/org/apache/pig/test/data/InputFiles/";
+        if ((System.getProperty("os.name").toUpperCase().startsWith("WINDOWS")))
+            inpDir="/"+FileLocalizer.parseCygPath(inpDir, FileLocalizer.STYLE_WINDOWS);
         golDir = curDir + File.separatorChar + "test/org/apache/pig/test/data/GoldenFiles/";
+        if ((System.getProperty("os.name").toUpperCase().startsWith("WINDOWS")))
+            golDir="/"+FileLocalizer.parseCygPath(golDir, FileLocalizer.STYLE_WINDOWS);
     }
     
     @After

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLocalJobSubmission.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLocalJobSubmission.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestLocalJobSubmission.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestLocalJobSubmission.java Wed Oct 22 11:37:13 2008
@@ -93,7 +93,19 @@
         inpDir = curDir + File.separatorChar + "test/org/apache/pig/test/data/InputFiles/";
         outDir = curDir + File.separatorChar + "test/org/apache/pig/test/data/OutputFiles/";
         golDir = curDir + File.separatorChar + "test/org/apache/pig/test/data/GoldenFiles/";
+        inpDir = curDir + File.separatorChar + "test/org/apache/pig/test/data/InputFiles/";
+        
+        if ((System.getProperty("os.name").toUpperCase().startsWith("WINDOWS")))
+            inpDir="/"+FileLocalizer.parseCygPath(inpDir, FileLocalizer.STYLE_WINDOWS);
+        
+        if ((System.getProperty("os.name").toUpperCase().startsWith("WINDOWS")))
+            golDir="/"+FileLocalizer.parseCygPath(golDir, FileLocalizer.STYLE_WINDOWS);
+        
         File f = new File(outDir);
+        
+        if ((System.getProperty("os.name").toUpperCase().startsWith("WINDOWS")))
+            outDir="/"+FileLocalizer.parseCygPath(outDir, FileLocalizer.STYLE_WINDOWS);
+        
         boolean didMakeDir = f.mkdirs();
         /*if(!didMakeDir)
             throw new Exception("Could not Create Directory " + outDir);*/

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLocalPOSplit.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLocalPOSplit.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestLocalPOSplit.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestLocalPOSplit.java Wed Oct 22 11:37:13 2008
@@ -80,7 +80,7 @@
 
         dat.close();
 
-        String query = "split (load '" + datFile.getAbsolutePath()
+        String query = "split (load '" + Util.encodeEscape(datFile.getAbsolutePath())
                 + "') into a if $0 == 2, b if $0 == 9, c if $0 == 7 ;";
 
         LogicalPlan plan = buildPlan(query);

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java Wed Oct 22 11:37:13 2008
@@ -113,7 +113,7 @@
     @Test
     public Double bigGroupAll( File tmpFile ) throws Throwable {
 
-        String query = "foreach (group (load 'file:" + tmpFile + "') all) generate " + COUNT.class.getName() + "($1) ;";
+        String query = "foreach (group (load '" + Util.generateURI(tmpFile.toString()) + "') all) generate " + COUNT.class.getName() + "($1) ;";
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
         Iterator it = pig.openIterator("asdf_id");
@@ -301,7 +301,7 @@
         ps.close();
 
 	//Load, Execute and Store query
-        String query = "foreach (load 'file:"+tmpFile+"') generate $0,$1;";
+        String query = "foreach (load '"+Util.generateURI(tmpFile.toString())+"') generate $0,$1;";
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
         try {
@@ -350,7 +350,7 @@
         File tmpFile=TestHelper.createTempFile(data) ;
 
 	//Load, Execute and Store query
-        String query = "foreach (load 'file:"+tmpFile+"') generate $0,$1;";
+        String query = "foreach (load '"+Util.generateURI(tmpFile.toString())+"') generate $0,$1;";
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
         try {
@@ -393,7 +393,7 @@
         ps.close();
 
         // execute query
-        String query = "foreach (group (load 'file:"+tmpFile+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(" + MyApply.class.getName() + "($1)) ;";
+        String query = "foreach (group (load '"+Util.generateURI(tmpFile.toString())+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(" + MyApply.class.getName() + "($1)) ;";
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
 
@@ -427,7 +427,7 @@
         ps.close();
 
         // execute query
-        String query = "foreach (group (load 'file:"+tmpFile+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(" + MyApply.class.getName() + "($1)) ;";
+        String query = "foreach (group (load '"+Util.generateURI(tmpFile.toString())+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(" + MyApply.class.getName() + "($1)) ;";
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
 
@@ -457,7 +457,7 @@
         ps.close();
         pig.registerFunction("foo",
             new FuncSpec(MyApply.class.getName()+"('foo')"));
-        String query = "foreach (group (load 'file:"+tmpFile+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(foo($1)) ;";
+        String query = "foreach (group (load '"+Util.generateURI(tmpFile.toString())+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(foo($1)) ;";
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
         Iterator it = pig.openIterator("asdf_id");
@@ -502,7 +502,7 @@
         ps.close();
         pig.registerFunction("foo",
             new FuncSpec(MyApply.class.getName()+"('foo')"));
-        String query = "foreach (group (load 'file:"+tmpFile+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(foo($1)) ;";
+        String query = "foreach (group (load '"+Util.generateURI(tmpFile.toString())+"' using " + MyStorage.class.getName() + "()) by " + MyGroup.class.getName() + "('all')) generate flatten(foo($1)) ;";
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
         Iterator it = pig.openIterator("asdf_id");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce2.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce2.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce2.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce2.java Wed Oct 22 11:37:13 2008
@@ -46,8 +46,8 @@
     public void testUnion1() throws Exception {
         File tmpFile1 = genDataSetFile(false, 30 ) ;
         File tmpFile2 = genDataSetFile(false, 50 ) ;
-        pig.registerQuery("a = load 'file:" + tmpFile1 + "'; ") ;
-        pig.registerQuery("b = load 'file:" + tmpFile2 + "'; ") ;
+        pig.registerQuery("a = load '" + Util.generateURI(tmpFile1.toString()) + "'; ") ;
+        pig.registerQuery("b = load '" + Util.generateURI(tmpFile2.toString()) + "'; ") ;
         pig.registerQuery("c = union a, b; ") ;
         
         verifyUnion( "c", 30 + 50 );
@@ -58,8 +58,8 @@
 
         File tmpFile1 = genDataSetFile(true, 30 ) ;
         File tmpFile2 = genDataSetFile(true, 50 ) ;
-        pig.registerQuery("a = load 'file:" + tmpFile1 + "'; ") ;
-        pig.registerQuery("b = load 'file:" + tmpFile2 + "'; ") ;
+        pig.registerQuery("a = load '" + Util.generateURI(tmpFile1.toString()) + "'; ") ;
+        pig.registerQuery("b = load '" + Util.generateURI(tmpFile2.toString()) + "'; ") ;
         pig.registerQuery("c = union a, b; ") ;
 
         verifyUnion( "c", 30 + 50 );
@@ -70,8 +70,8 @@
 
         File tmpFile1 = genDataSetFile(false, 30) ;
         File tmpFile2 = genDataSetFile(false, 50) ;
-        pig.registerQuery("a = load 'file:" + tmpFile1 + "'; ") ;
-        pig.registerQuery("b = load 'file:" + tmpFile2 + "'; ") ;
+        pig.registerQuery("a = load '" + Util.generateURI(tmpFile1.toString()) + "'; ") ;
+        pig.registerQuery("b = load '" + Util.generateURI(tmpFile2.toString()) + "'; ") ;
         pig.registerQuery("a1 = foreach a generate $0, $1; ") ;
         pig.registerQuery("b1 = foreach b generate $0, $1; ") ;
         pig.registerQuery("c = union a1, b1; ") ;
@@ -83,8 +83,8 @@
     public void testUnion2WithNulls() throws Exception {
         File tmpFile1 = genDataSetFile(true, 30) ;
         File tmpFile2 = genDataSetFile(true, 50) ;
-        pig.registerQuery("a = load 'file:" + tmpFile1 + "'; ") ;
-        pig.registerQuery("b = load 'file:" + tmpFile2 + "'; ") ;
+        pig.registerQuery("a = load '" + Util.generateURI(tmpFile1.toString()) + "'; ") ;
+        pig.registerQuery("b = load '" + Util.generateURI(tmpFile2.toString()) + "'; ") ;
         pig.registerQuery("a1 = foreach a generate $0, $1; ") ;
         pig.registerQuery("b1 = foreach b generate $0, $1; ") ;
         pig.registerQuery("c = union a1, b1; ") ;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestParamSubPreproc.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestParamSubPreproc.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestParamSubPreproc.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestParamSubPreproc.java Wed Oct 22 11:37:13 2008
@@ -1197,6 +1197,8 @@
             pigOStream = new FileWriter(basedir + "/output1.pig");
 
             String[] arg = {"date=`perl -e 'print \"20080228\n20070101\"' | head -n 1`"};
+            if (System.getProperty("os.name").toUpperCase().startsWith("WINDOWS"))
+                arg[0] = "date=`perl -e 'print \\\"20080228\n20070101\\\"' | head -n 1`";
             String[] argFiles = null;
             ps.genSubstitutedFile(pigIStream , pigOStream , arg , argFiles);
 

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPigContext.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPigContext.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPigContext.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPigContext.java Wed Oct 22 11:37:13 2008
@@ -100,7 +100,7 @@
 
     private List<String> getCommands() {
         List<String> commands = new ArrayList<String>();
-        commands.add("my_input = LOAD '" + input.getAbsolutePath() + "' USING PigStorage();");
+        commands.add("my_input = LOAD '" + Util.encodeEscape(input.getAbsolutePath()) + "' USING PigStorage();");
         commands.add("words = FOREACH my_input GENERATE FLATTEN(TOKENIZE(*));");
         commands.add("grouped = GROUP words BY $0;");
         commands.add("counts = FOREACH grouped GENERATE group, COUNT(words);");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPigScriptParser.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPigScriptParser.java?rev=707158&r1=707157&r2=707158&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPigScriptParser.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPigScriptParser.java Wed Oct 22 11:37:13 2008
@@ -36,7 +36,7 @@
         // Start the real parsing job
         {
         	// Initial statement
-        	String query = String.format("A = LOAD '%s' ;", tempFile) ;
+        	String query = String.format("A = LOAD '%s' ;", Util.encodeEscape(tempFile)) ;
         	ByteArrayInputStream in = new ByteArrayInputStream(query.getBytes()); 
         	QueryParser parser = new QueryParser(in, pigContext, "scope", aliases, opTable, aliasOp) ;
         	LogicalPlan lp = parser.Parse() ;