You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by th...@apache.org on 2011/07/25 07:46:18 UTC

svn commit: r1150552 - in /pig/trunk: ./ src/org/apache/pig/builtin/ test/org/apache/pig/test/ test/org/apache/pig/test/utils/

Author: thejas
Date: Mon Jul 25 05:46:17 2011
New Revision: 1150552

URL: http://svn.apache.org/viewvc?rev=1150552&view=rev
Log:
PIG-2186: PigStorage new warnings about missing schema file
 can be confusing

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/builtin/JsonMetadata.java
    pig/trunk/src/org/apache/pig/builtin/PigStorage.java
    pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java
    pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java
    pig/trunk/test/org/apache/pig/test/TestPigStorage.java
    pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidatorNewLP.java
    pig/trunk/test/org/apache/pig/test/utils/TypeCheckingTestUtil.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Mon Jul 25 05:46:17 2011
@@ -89,6 +89,9 @@ PIG-2011: Speed up TestTypedMap.java (dv
 
 BUG FIXES
 
+PIG-2186: PigStorage new warnings about missing schema file 
+ can be confusing (thejas)
+
 PIG-2179: tests in TestLoad are failing (thejas)
 
 PIG-2146: POStore.getSchema() returns null because of which PigOutputCommitter 

Modified: pig/trunk/src/org/apache/pig/builtin/JsonMetadata.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/JsonMetadata.java?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/JsonMetadata.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/JsonMetadata.java Mon Jul 25 05:46:17 2011
@@ -33,6 +33,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.pig.Expression;
 import org.apache.pig.LoadFunc;
 import org.apache.pig.LoadMetadata;
+import org.apache.pig.PigException;
 import org.apache.pig.StoreMetadata;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.ResourceStatistics;
@@ -44,6 +45,7 @@ import org.apache.pig.backend.hadoop.dat
 import org.apache.pig.backend.hadoop.datastorage.HDirectory;
 import org.apache.pig.backend.hadoop.datastorage.HFile;
 import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.JsonParseException;
 import org.codehaus.jackson.map.JsonMappingException;
@@ -154,6 +156,7 @@ public class JsonMetadata implements Loa
             throws IOException {
     }
 
+    
     /**
      * For JsonMetadata schema is considered optional
      * This method suppresses (and logs) errors if they are encountered.
@@ -161,13 +164,26 @@ public class JsonMetadata implements Loa
      */
     @Override
     public ResourceSchema getSchema(String location, Job job) throws IOException {
+        return getSchema(location, job, false);
+    }
+    
+    /**
+     * Read the schema from json metadata file
+     * If isSchemaOn parameter is false, the errors are suppressed and logged
+     * @param location
+     * @param job
+     * @param isSchemaOn
+     * @return schema
+     * @throws IOException
+     */
+    public ResourceSchema getSchema(String location, Job job, boolean isSchemaOn) throws IOException {
         Configuration conf = job.getConfiguration();
         Set<ElementDescriptor> schemaFileSet = null;
         try {
             schemaFileSet = findMetaFile(location, schemaFileName, conf);
         } catch (IOException e) {
-            log.warn("Could not find schema file for "+ location);
-            return null;
+            String msg = "Could not find schema file for "+ location;
+            return nullOrException(isSchemaOn, msg, e);
         }
 
         // TODO we assume that all schemas are the same. The question of merging schemas is left open for now.
@@ -175,26 +191,37 @@ public class JsonMetadata implements Loa
         if (!schemaFileSet.isEmpty()) {
             schemaFile = schemaFileSet.iterator().next();
         } else {
-            log.warn("Could not find schema file for "+location);
-            return null;
+            String msg = "Could not find schema file for "+location;
+            return nullOrException(isSchemaOn, msg, null);
         }
         log.debug("Found schema file: "+schemaFile.toString());
         ResourceSchema resourceSchema = null;
         try {
             resourceSchema = new ObjectMapper().readValue(schemaFile.open(), ResourceSchema.class);
         } catch (JsonParseException e) {
-            log.warn("Unable to load Resource Schema for "+location);
-            e.printStackTrace();
+            String msg = "Unable to load Resource Schema for "+location;
+            return nullOrException(isSchemaOn, msg, e);
         } catch (JsonMappingException e) {
-            log.warn("Unable to load Resource Schema for "+location);
-            e.printStackTrace();
+            String msg = "Unable to load Resource Schema for "+location;
+            return nullOrException(isSchemaOn, msg, e);
         } catch (IOException e) {
-            log.warn("Unable to load Resource Schema for "+location);
-            e.printStackTrace();
+            String msg = "Unable to load Resource Schema for "+location;
+            return nullOrException(isSchemaOn, msg, e);
         }
         return resourceSchema;
     }
 
+    private ResourceSchema nullOrException(boolean isSchemaOn, String msg,
+            IOException e) throws FrontendException {
+        if(isSchemaOn){
+            throw  new FrontendException(msg, 1131, PigException.INPUT, e);
+        }
+        //a valid schema file was probably not expected, so just log a 
+        //debug message and return null
+        log.debug(msg);
+        return null;
+    }
+
     /**
      * For JsonMetadata stats are considered optional
      * This method suppresses (and logs) errors if they are encountered.

Modified: pig/trunk/src/org/apache/pig/builtin/PigStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/PigStorage.java?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/PigStorage.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/PigStorage.java Mon Jul 25 05:46:17 2011
@@ -79,7 +79,8 @@ import org.apache.pig.parser.ParserExcep
  * An optional second constructor argument is provided that allows one to customize
  * advanced behaviors. A list of available options is below:
  * <ul>
- * <li><code>-schema</code> Stores the schema of the relation using a hidden JSON file.
+ * <li><code>-schema</code> Reads/Stores the schema of the relation using a 
+ *  hidden JSON file.
  * <li><code>-noschema</code> Ignores a stored schema during loading.
  * </ul>
  * <p>
@@ -89,6 +90,8 @@ import org.apache.pig.parser.ParserExcep
  * field names and types of the data without the need for a user to explicitly provide the schema in an
  * <code>as</code> clause, unless <code>-noschema</code> is specified. No attempt to merge conflicting
  * schemas is made during loading. The first schema encountered during a file system scan is used.
+ * If the schema file is not present while '-schema' option is used during loading, 
+ * it results in an error.
  * <p>
  * In addition, using <code>-schema</code> drops a ".pig_headers" file in the output directory.
  * This file simply lists the delimited aliases. This is intended to make export to tools that can read
@@ -121,7 +124,7 @@ LoadPushDown, LoadMetadata, StoreMetadat
     private TupleFactory mTupleFactory = TupleFactory.getInstance();
     private String loadLocation;
 
-    boolean storeSchema = false;
+    boolean isSchemaOn = false;
     boolean dontLoadSchema = false;
     protected ResourceSchema schema;
     protected LoadCaster caster;
@@ -172,7 +175,7 @@ LoadPushDown, LoadMetadata, StoreMetadat
         String[] optsArr = options.split(" ");
         try {
             configuredOptions = parser.parse(validOptions, optsArr);
-            storeSchema = configuredOptions.hasOption("schema");
+            isSchemaOn = configuredOptions.hasOption("schema");
             dontLoadSchema = configuredOptions.hasOption("noschema");
         } catch (ParseException e) {
             HelpFormatter formatter = new HelpFormatter();
@@ -382,6 +385,8 @@ LoadPushDown, LoadMetadata, StoreMetadat
         if (codec != null) {
             FileOutputFormat.setCompressOutput(job, true);
             FileOutputFormat.setOutputCompressorClass(job, codec.getClass());
+        }else {
+            FileOutputFormat.setCompressOutput(job, false);  
         }
     }
 
@@ -430,7 +435,7 @@ LoadPushDown, LoadMetadata, StoreMetadat
     public ResourceSchema getSchema(String location,
             Job job) throws IOException {
         if (!dontLoadSchema) {
-            schema = (new JsonMetadata()).getSchema(location, job);
+            schema = (new JsonMetadata()).getSchema(location, job, isSchemaOn);
 
             if (signature != null && schema != null) {
                 Properties p = UDFContext.getUDFContext().getUDFProperties(this.getClass(),
@@ -464,7 +469,7 @@ LoadPushDown, LoadMetadata, StoreMetadat
     @Override
     public void storeSchema(ResourceSchema schema, String location,
             Job job) throws IOException {
-        if (storeSchema) {
+        if (isSchemaOn) {
             JsonMetadata metadataWriter = new JsonMetadata();
             byte recordDel = '\n';
             metadataWriter.setFieldDel(fieldDel);

Modified: pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java Mon Jul 25 05:46:17 2011
@@ -17,11 +17,18 @@
  */
 package org.apache.pig.test;
 
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.Properties;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.impl.util.MultiMap;
 import org.apache.pig.newplan.logical.expression.AndExpression;
@@ -39,6 +46,7 @@ import org.apache.pig.newplan.logical.re
 import org.apache.pig.newplan.logical.relational.LogicalSchema;
 import org.apache.pig.newplan.logical.relational.LOJoin.JOINTYPE;
 import org.apache.pig.newplan.logical.relational.LogicalSchema.LogicalFieldSchema;
+import org.junit.Before;
 import org.junit.Test;
 
 import junit.framework.TestCase;
@@ -46,7 +54,18 @@ import junit.framework.TestCase;
 /**
  * Test end to end logical optimizations.
  */
-public class TestNewPlanLogicalOptimizer extends TestCase {
+public class TestNewPlanLogicalOptimizer {
+    
+    Configuration conf = null;
+    
+    @Before
+    public void setUp() throws Exception {
+        PigContext pc = new PigContext(ExecType.LOCAL, new Properties());
+        pc.connect();
+        conf = new Configuration(
+                ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration())
+                );
+    }
     
     @Test
     public void testFilterPushDown() throws IOException {
@@ -65,7 +84,7 @@ public class TestNewPlanLogicalOptimizer
             	"x", null, DataType.BYTEARRAY));
         	aschema.addField(new LogicalSchema.LogicalFieldSchema(
             	"y", null, DataType.BYTEARRAY));
-        	LOLoad A = new LOLoad(new FileSpec("bla", new FuncSpec("PigStorage", "\t")), aschema, lp, null);
+        	LOLoad A = new LOLoad(new FileSpec("bla", new FuncSpec("PigStorage", "\t")), aschema, lp, conf);
         	A.setAlias("A");
         	lp.add(A);
 	        
@@ -75,7 +94,7 @@ public class TestNewPlanLogicalOptimizer
             	"a", null, DataType.BYTEARRAY));
         	bschema.addField(new LogicalSchema.LogicalFieldSchema(
             	"b", null, DataType.BYTEARRAY));
-        	LOLoad B = new LOLoad(new FileSpec("morebla", new FuncSpec("PigStorage", "\t")), bschema, lp, null);
+        	LOLoad B = new LOLoad(new FileSpec("morebla", new FuncSpec("PigStorage", "\t")), bschema, lp, conf);
         	B.setAlias("B");
         	lp.add(B);
 	        
@@ -141,7 +160,7 @@ public class TestNewPlanLogicalOptimizer
             	"x", null, DataType.BYTEARRAY));
         	aschema.addField(new LogicalSchema.LogicalFieldSchema(
             	"y", null, DataType.BYTEARRAY));
-        	LOLoad A = new LOLoad(new FileSpec("bla", new FuncSpec("PigStorage", "\t")), aschema, expected, null);
+        	LOLoad A = new LOLoad(new FileSpec("bla", new FuncSpec("PigStorage", "\t")), aschema, expected, conf);
         	expected.add(A);
         	
         	// DA = filter
@@ -167,7 +186,7 @@ public class TestNewPlanLogicalOptimizer
             	"a", null, DataType.BYTEARRAY));
         	bschema.addField(new LogicalSchema.LogicalFieldSchema(
             	"b", null, DataType.BYTEARRAY));
-        	LOLoad B = new LOLoad(new FileSpec("morebla", new FuncSpec("PigStorage", "\t")), bschema, expected, null);
+        	LOLoad B = new LOLoad(new FileSpec("morebla", new FuncSpec("PigStorage", "\t")), bschema, expected, conf);
         	expected.add(B);
         	
         	// DB = filter
@@ -246,7 +265,6 @@ public class TestNewPlanLogicalOptimizer
         
         
         assertTrue( lp.isEqual(expected) );
-        // assertEquals(lp, expected);
     }
 
 }

Modified: pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java Mon Jul 25 05:46:17 2011
@@ -20,10 +20,15 @@ package org.apache.pig.test;
 
 import java.util.Collection;
 import java.util.List;
+import java.util.Properties;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.MultiMap;
@@ -52,6 +57,7 @@ import org.apache.pig.newplan.logical.re
 import org.apache.pig.newplan.logical.relational.LogicalSchema;
 import org.apache.pig.newplan.logical.relational.LOJoin.JOINTYPE;
 import org.apache.pig.newplan.logical.relational.LogicalSchema.LogicalFieldSchema;
+import org.junit.Before;
 import org.junit.Test;
 
 import junit.framework.TestCase;
@@ -109,6 +115,16 @@ public class TestNewPlanOperatorPlan ext
         }
         
     }
+    Configuration conf = null;
+    
+    @Before
+    public void setUp() throws Exception {
+        PigContext pc = new PigContext(ExecType.LOCAL, new Properties());
+        pc.connect();
+        conf = new Configuration(
+                ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration())
+                );
+    }
     
     // Tests for PlanEdge
     
@@ -690,7 +706,7 @@ public class TestNewPlanOperatorPlan ext
     @Test
     public void testLogicalPlanVisitor() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
-        LOLoad load = new LOLoad(null, null, lp, null);
+        LOLoad load = new LOLoad(null, null, lp, conf);
         /*lp.add((LogicalRelationalOperator)null, load,
             (LogicalRelationalOperator)null);*/
         lp.add(load);
@@ -819,7 +835,7 @@ public class TestNewPlanOperatorPlan ext
             aschema.addField(new LogicalSchema.LogicalFieldSchema(
                 "x", null, DataType.INTEGER));
             LOLoad A = new LOLoad(new FileSpec("/abc",
-                new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema, lp, null);
+                new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema, lp, conf);
             lp.add(A);
         
             // B = load
@@ -827,7 +843,7 @@ public class TestNewPlanOperatorPlan ext
             bschema.addField(new LogicalSchema.LogicalFieldSchema(
                 "y", null, DataType.INTEGER));
             LOLoad B = new LOLoad(new FileSpec("/def",
-                new FuncSpec("PigStorage", "\t")), bschema, lp, null);
+                new FuncSpec("PigStorage", "\t")), bschema, lp, conf);
             lp.add(B);
         
             // C = join
@@ -869,7 +885,7 @@ public class TestNewPlanOperatorPlan ext
             aschema.addField(new LogicalSchema.LogicalFieldSchema(
                 "x", null, DataType.INTEGER));
             LOLoad A = new LOLoad(new FileSpec("/abc",
-                new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema, lp1, null);
+                new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema, lp1, conf);
             lp1.add(A);
             
             // B = load
@@ -877,7 +893,7 @@ public class TestNewPlanOperatorPlan ext
             bschema.addField(new LogicalSchema.LogicalFieldSchema(
                 "y", null, DataType.INTEGER));
             LOLoad B = new LOLoad(new FileSpec("/def",
-                new FuncSpec("PigStorage", "\t")), bschema, lp1, null);
+                new FuncSpec("PigStorage", "\t")), bschema, lp1, conf);
             lp1.add(B);
             
             // C = join
@@ -924,11 +940,11 @@ public class TestNewPlanOperatorPlan ext
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "x", null, DataType.INTEGER));
         LOLoad load1 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, conf);
         lp.add(load1);
         
         LOLoad load2 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema1, lp, conf);
         lp.add(load2);
         
         assertFalse(load1.isEqual(load2));
@@ -942,11 +958,11 @@ public class TestNewPlanOperatorPlan ext
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "x", null, DataType.INTEGER));
         LOLoad load1 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, conf);
         lp.add(load1);
         
         LOLoad load3 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), "x")), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), "x")), aschema1, lp, conf);
         lp.add(load3);
         
         assertFalse(load1.isEqual(load3));
@@ -960,12 +976,12 @@ public class TestNewPlanOperatorPlan ext
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "x", null, DataType.INTEGER));
         LOLoad load1 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, conf);
         lp.add(load1);
         
          // Different function names in FuncSpec
         LOLoad load4 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema1, lp, conf);
         lp.add(load4);
         
         assertFalse(load1.isEqual(load4));
@@ -978,12 +994,12 @@ public class TestNewPlanOperatorPlan ext
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "x", null, DataType.INTEGER));
         LOLoad load1 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, conf);
         lp.add(load1);
     
         // Different file name
         LOLoad load5 = new LOLoad(new FileSpec("/def",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema1, lp, conf);
         lp.add(load5);
         
         assertFalse(load1.isEqual(load5));
@@ -996,7 +1012,7 @@ public class TestNewPlanOperatorPlan ext
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "x", null, DataType.INTEGER));
         LOLoad load1 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, conf);
         lp.add(load1);
         
         // Different schema
@@ -1005,7 +1021,7 @@ public class TestNewPlanOperatorPlan ext
             "x", null, DataType.CHARARRAY));
         
         LOLoad load6 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema2, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), aschema2, lp, conf);
         lp.add(load6);
             
         assertFalse(load1.isEqual(load6));
@@ -1016,11 +1032,11 @@ public class TestNewPlanOperatorPlan ext
         LogicalPlan lp = new LogicalPlan();
         // Test that two loads with no schema are still equal
         LOLoad load7 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), null, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), null, lp, conf);
         lp.add(load7);
         
         LOLoad load8 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), null, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), null, lp, conf);
         lp.add(load8);
         
         assertTrue(load7.isEqual(load8));
@@ -1033,12 +1049,12 @@ public class TestNewPlanOperatorPlan ext
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "x", null, DataType.INTEGER));
         LOLoad load1 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "y"})), aschema1, lp, conf);
         lp.add(load1);
         
         // Test that one with schema and one without breaks equality
         LOLoad load9 = new LOLoad(new FileSpec("/abc",
-            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), null, lp, null);
+            new FuncSpec(DummyLoad.class.getName(), new String[] {"x", "z"})), null, lp, conf);
         lp.add(load9);
         
         assertFalse(load1.isEqual(load9));
@@ -1084,7 +1100,7 @@ public class TestNewPlanOperatorPlan ext
        jaschema1.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
        LOLoad A1 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema1, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema1, lp, conf);
        lp.add(A1);
         
         // B = load
@@ -1092,7 +1108,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "y", null, DataType.INTEGER));
         LOLoad B1 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema1, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema1, lp, conf);
         lp.add(B1);
         
         // C = join
@@ -1120,7 +1136,7 @@ public class TestNewPlanOperatorPlan ext
         jaschema2.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
         LOLoad A2 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema2, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema2, lp, conf);
         lp.add(A2);
         
         // B = load
@@ -1128,7 +1144,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema2.addField(new LogicalSchema.LogicalFieldSchema(
             "y", null, DataType.INTEGER));
         LOLoad B2 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema2, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema2, lp, conf);
         lp.add(B2);
         
         // C = join
@@ -1161,7 +1177,7 @@ public class TestNewPlanOperatorPlan ext
        jaschema1.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
        LOLoad A1 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema1, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema1, lp, conf);
        lp.add(A1);
         
         // B = load
@@ -1169,7 +1185,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "y", null, DataType.INTEGER));
         LOLoad B1 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema1, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema1, lp, conf);
         lp.add(B1);
         
         // C = join
@@ -1199,7 +1215,7 @@ public class TestNewPlanOperatorPlan ext
         jaschema3.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
         LOLoad A3 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema3, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema3, lp, conf);
         lp.add(A3);
         
         // B = load
@@ -1207,7 +1223,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema3.addField(new LogicalSchema.LogicalFieldSchema(
             "y", null, DataType.INTEGER));
         LOLoad B3 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema3, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema3, lp, conf);
         lp.add(B3);
         
         // C = join
@@ -1241,7 +1257,7 @@ public class TestNewPlanOperatorPlan ext
        jaschema1.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
        LOLoad A1 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema1, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema1, lp, conf);
        lp.add(A1);
         
         // B = load
@@ -1249,7 +1265,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema1.addField(new LogicalSchema.LogicalFieldSchema(
             "y", null, DataType.INTEGER));
         LOLoad B1 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema1, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema1, lp, conf);
         lp.add(B1);
         
         // C = join
@@ -1277,7 +1293,7 @@ public class TestNewPlanOperatorPlan ext
         jaschema5.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
         LOLoad A5 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema5, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema5, lp, conf);
         lp.add(A5);
         
         // B = load
@@ -1285,7 +1301,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema5.addField(new LogicalSchema.LogicalFieldSchema(
             "y", null, DataType.INTEGER));
         LOLoad B5 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema5, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema5, lp, conf);
         lp.add(B5);
         
         // Beta = load
@@ -1293,7 +1309,7 @@ public class TestNewPlanOperatorPlan ext
         jbetaschema5.addField(new LogicalSchema.LogicalFieldSchema(
             "y", null, DataType.INTEGER));
         LOLoad Beta5 = new LOLoad(new FileSpec("/ghi",
-            new FuncSpec("PigStorage", "\t")), jbetaschema5, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbetaschema5, lp, conf);
         lp.add(Beta5);
         
         // C = join
@@ -1332,7 +1348,7 @@ public class TestNewPlanOperatorPlan ext
         jaschema6.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
         LOLoad A6 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema6, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema6, lp, conf);
         lp.add(A6);
         
         // B = load
@@ -1342,7 +1358,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema6.addField(new LogicalSchema.LogicalFieldSchema(
             "z", null, DataType.LONG));
         LOLoad B6 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema6, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema6, lp, conf);
         lp.add(B6);
         
         // C = join
@@ -1373,7 +1389,7 @@ public class TestNewPlanOperatorPlan ext
         jaschema7.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
         LOLoad A7 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema7, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema7, lp, conf);
         lp.add(A7);
         
         // B = load
@@ -1383,7 +1399,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema7.addField(new LogicalSchema.LogicalFieldSchema(
             "z", null, DataType.LONG));
         LOLoad B7 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema7, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema7, lp, conf);
         lp.add(B7);
         
         // C = join
@@ -1421,7 +1437,7 @@ public class TestNewPlanOperatorPlan ext
         jaschema6.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
         LOLoad A6 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema6, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema6, lp, conf);
         lp.add(A6);
         
         // B = load
@@ -1431,7 +1447,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema6.addField(new LogicalSchema.LogicalFieldSchema(
             "z", null, DataType.LONG));
         LOLoad B6 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema6, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema6, lp, conf);
         lp.add(B6);
         
         // C = join
@@ -1462,7 +1478,7 @@ public class TestNewPlanOperatorPlan ext
         jaschema8.addField(new LogicalSchema.LogicalFieldSchema(
            "x", null, DataType.INTEGER));
         LOLoad A8 = new LOLoad(new FileSpec("/abc",
-           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema8, lp, null);
+           new FuncSpec("/fooload", new String[] {"x", "y"})), jaschema8, lp, conf);
         lp.add(A8);
         
         // B = load
@@ -1472,7 +1488,7 @@ public class TestNewPlanOperatorPlan ext
         jbschema8.addField(new LogicalSchema.LogicalFieldSchema(
             "z", null, DataType.LONG));
         LOLoad B8 = new LOLoad(new FileSpec("/def",
-            new FuncSpec("PigStorage", "\t")), jbschema8, lp, null);
+            new FuncSpec("PigStorage", "\t")), jbschema8, lp, conf);
         lp.add(B8);
         
         // C = join

Modified: pig/trunk/test/org/apache/pig/test/TestPigStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigStorage.java?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigStorage.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigStorage.java Mon Jul 25 05:46:17 2011
@@ -22,6 +22,7 @@ import static org.apache.pig.ExecType.MA
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -45,6 +46,7 @@ import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.test.utils.TypeCheckingTestUtil;
 import org.junit.After;
@@ -182,11 +184,27 @@ public class TestPigStorage  {
         assertFalse(it.hasNext());
 
     }
+    
+    @Test
+    public void testPigStorageNoSchema() throws Exception {
+        //if the schema file does not exist, and '-schema' option is used
+        // it should result in an error
+        pigContext.connect();
+        String query = "a = LOAD '" + datadir + "originput' using PigStorage('\\t', '-schema') " +
+        "as (f1:chararray, f2:int);";
+        pig.registerQuery(query);
+        try{
+            pig.dumpSchema("a");
+        }catch(FrontendException ex){
+            return;
+        }
+        fail("no exception caught");
+    }
 
     @Test
     public void testPigStorageSchema() throws Exception {
         pigContext.connect();
-        String query = "a = LOAD '" + datadir + "originput' using PigStorage('\\t', '-schema') " +
+        String query = "a = LOAD '" + datadir + "originput' using PigStorage('\\t') " +
         "as (f1:chararray, f2:int);";
         pig.registerQuery(query);
         Schema origSchema = pig.dumpSchema("a");
@@ -227,7 +245,7 @@ public class TestPigStorage  {
                               "5", "5", "8", "8",
                               "8", "9"});
 
-        pig.registerQuery("A = LOAD '" + datadir + "originput2' using PigStorage('\\t', '-schema') " +
+        pig.registerQuery("A = LOAD '" + datadir + "originput2' using PigStorage('\\t') " +
         "as (f:int);");
         pig.registerQuery("B = group A by f;");
         Schema origSchema = pig.dumpSchema("B");
@@ -264,7 +282,7 @@ public class TestPigStorage  {
     @Test
     public void testSchemaConversion2() throws Exception {
 
-        pig.registerQuery("A = LOAD '" + datadir + "originput' using PigStorage(',', '-schema') " +
+        pig.registerQuery("A = LOAD '" + datadir + "originput' using PigStorage(',') " +
         "as (f1:chararray, f2:int);");
         pig.registerQuery("B = group A by f1;");
         Schema origSchema = pig.dumpSchema("B");
@@ -355,7 +373,7 @@ public class TestPigStorage  {
     @Test
     public void testPigStorageSchemaHeaderDelimiter() throws Exception {
         pigContext.connect();
-        String query = "a = LOAD '" + datadir + "originput' using PigStorage(',', '-schema') " +
+        String query = "a = LOAD '" + datadir + "originput' using PigStorage(',') " +
                 "as (foo:chararray, bar:int);";
         pig.registerQuery(query);
         pig.registerQuery("STORE a into '" + datadir + "dout' using PigStorage('#', '-schema');");

Modified: pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidatorNewLP.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidatorNewLP.java?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidatorNewLP.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidatorNewLP.java Mon Jul 25 05:46:17 2011
@@ -42,10 +42,12 @@ import java.util.Properties;
 
 import junit.framework.Assert;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.pig.EvalFunc;
 import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
@@ -131,6 +133,7 @@ public class TestTypeCheckingValidatorNe
      */
     @Before
     public void setUp() throws Exception {
+        pc.connect();
     }
 
     private static final String simpleEchoStreamingCommand;
@@ -793,12 +796,12 @@ public class TestTypeCheckingValidatorNe
             
             LOLoad load1 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
                 
             LOLoad load2 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
 //                
 //                new LOLoad(plan,
@@ -928,12 +931,12 @@ public class TestTypeCheckingValidatorNe
             
             LOLoad load1 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
                 
             LOLoad load2 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
   
             // schema for input#1
@@ -1036,7 +1039,7 @@ public class TestTypeCheckingValidatorNe
     
             LOLoad load1 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
     
             // schema for input#1
@@ -1104,7 +1107,7 @@ public class TestTypeCheckingValidatorNe
 
             LOLoad load1 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
             // schema for input#1
             Schema inputSchema1 = null ;
@@ -1167,7 +1170,7 @@ public class TestTypeCheckingValidatorNe
 
             LOLoad load1 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
             
             // schema for input#1
@@ -1229,7 +1232,7 @@ public class TestTypeCheckingValidatorNe
     
             LOLoad load1 = new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
 
     
@@ -1319,7 +1322,7 @@ public class TestTypeCheckingValidatorNe
     
             LOLoad load1 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
 
             // schema for input#1
@@ -1404,7 +1407,7 @@ public class TestTypeCheckingValidatorNe
     
             LOLoad load1 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
 
     
@@ -1487,7 +1490,7 @@ public class TestTypeCheckingValidatorNe
 
             LOLoad load1 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
 
             // schema for input#1
@@ -1594,7 +1597,7 @@ public class TestTypeCheckingValidatorNe
 
             LOLoad load1 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
 
             // schema for input#1
@@ -1679,12 +1682,12 @@ public class TestTypeCheckingValidatorNe
     
             LOLoad load1 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
             
             LOLoad load2 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
               
             // schema for input#1
@@ -1817,12 +1820,12 @@ public class TestTypeCheckingValidatorNe
             String pigStorage = PigStorage.class.getName() ;
             LOLoad load1 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
             
             LOLoad load2 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
             // schema for input#1
             Schema inputSchema1 = null ;
@@ -1932,16 +1935,16 @@ public class TestTypeCheckingValidatorNe
             printCurrentMethodName();
             LogicalPlan plan = new LogicalPlan() ;
     
-            String pigStorage = PigStorage.class.getName() ;
+            String pigStorage = PigStorage.class.getName() + "('\\t','-noschema')" ;
     
             LOLoad load1 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
             
             LOLoad load2 =  new LOLoad(
                     new FileSpec("pi", new FuncSpec(pigStorage)),
-                    null, plan, null
+                    null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
             );
     
             // schema for input#1

Modified: pig/trunk/test/org/apache/pig/test/utils/TypeCheckingTestUtil.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/utils/TypeCheckingTestUtil.java?rev=1150552&r1=1150551&r2=1150552&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/utils/TypeCheckingTestUtil.java (original)
+++ pig/trunk/test/org/apache/pig/test/utils/TypeCheckingTestUtil.java Mon Jul 25 05:46:17 2011
@@ -21,9 +21,15 @@ package org.apache.pig.test.utils;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Properties;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.plan.CompilationMessageCollector;
@@ -34,12 +40,14 @@ import org.apache.pig.newplan.logical.re
 public class TypeCheckingTestUtil {
 
     public static org.apache.pig.newplan.logical.relational.LOLoad 
-    genDummyLOLoadNewLP( org.apache.pig.newplan.logical.relational.LogicalPlan plan)  {
+    genDummyLOLoadNewLP( org.apache.pig.newplan.logical.relational.LogicalPlan plan) throws ExecException  {
         String pigStorage = PigStorage.class.getName() ;
+        PigContext pc = new PigContext(ExecType.LOCAL, new Properties());
+        pc.connect();
         org.apache.pig.newplan.logical.relational.LOLoad load =
         new org.apache.pig.newplan.logical.relational.LOLoad(
                 new FileSpec("pi", new FuncSpec(pigStorage)),
-                null, plan, null
+                null, plan, new Configuration(ConfigurationUtil.toConfiguration(pc.getFs().getConfiguration()))
         );
         return load ;
     }