You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ga...@apache.org on 2008/03/28 22:44:46 UTC

svn commit: r642398 [2/2] - in /incubator/pig/branches/types: ./ src/org/apache/pig/ src/org/apache/pig/builtin/ src/org/apache/pig/impl/io/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/schema/ src/org/apache/pig/impl/plan/

Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONot.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONot.java?rev=642398&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONot.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONot.java Fri Mar 28 14:44:40 2008
@@ -0,0 +1,60 @@
+/* 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.pig.impl.logicalLayer;
+
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.plan.PlanVisitor;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+public class LONot extends UnaryExpressionOperator {
+
+    private static final long serialVersionUID = 2L;
+
+    /**
+     * 
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     * @param operand
+     *            the only operand for a unary operator
+     */
+    public LONot(LogicalPlan plan, OperatorKey k, int rp,
+            ExpressionOperator operand) {
+        super(plan, k, rp, operand);
+    }
+
+    @Override
+    public Schema getSchema() {
+        return mSchema;
+    }
+
+    @Override
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
+    }
+
+    @Override
+    public String name() {
+        return "Not " + mKey.scope + "-" + mKey.id;
+    }
+}

Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONotEqual.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONotEqual.java?rev=642398&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONotEqual.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LONotEqual.java Fri Mar 28 14:44:40 2008
@@ -0,0 +1,62 @@
+/* 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.pig.impl.logicalLayer;
+
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.plan.PlanVisitor;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+public class LONotEqual extends BinaryExpressionOperator {
+
+    private static final long serialVersionUID = 2L;
+
+    /**
+     * 
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     * @param lhsOperand
+     *            the left hand side operand
+     * @param rhsOperand
+     *            the right hand side operand
+     */
+    public LONotEqual(LogicalPlan plan, OperatorKey k, int rp,
+            ExpressionOperator lhsOperand, ExpressionOperator rhsOperand) {
+        super(plan, k, rp, lhsOperand, rhsOperand);
+    }
+
+    @Override
+    public Schema getSchema() {
+        return mSchema;
+    }
+
+    @Override
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
+    }
+
+    @Override
+    public String name() {
+        return "NotEqual " + mKey.scope + "-" + mKey.id;
+    }
+}

Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOOr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOOr.java?rev=642398&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOOr.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOOr.java Fri Mar 28 14:44:40 2008
@@ -0,0 +1,62 @@
+/* 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.pig.impl.logicalLayer;
+
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.plan.PlanVisitor;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+public class LOOr extends BinaryExpressionOperator {
+
+    private static final long serialVersionUID = 2L;
+
+    /**
+     * 
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     * @param lhsOperand
+     *            the left hand side operand
+     * @param rhsOperand
+     *            the right hand side operand
+     */
+    public LOOr(LogicalPlan plan, OperatorKey k, int rp,
+            ExpressionOperator lhsOperand, ExpressionOperator rhsOperand) {
+        super(plan, k, rp, lhsOperand, rhsOperand);
+    }
+
+    @Override
+    public Schema getSchema() {
+        return mSchema;
+    }
+
+    @Override
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
+    }
+
+    @Override
+    public String name() {
+        return "Or " + mKey.scope + "-" + mKey.id;
+    }
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java Fri Mar 28 14:44:40 2008
@@ -22,9 +22,7 @@
 import java.util.Iterator;
 import java.io.PrintStream;
 
-import org.apache.pig.impl.eval.*;
-import org.apache.pig.impl.logicalLayer.schema.TupleSchema;
-import org.apache.pig.impl.logicalLayer.schema.AtomSchema;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 
 /**
@@ -32,6 +30,14 @@
  */
 public class LOPrinter extends LOVisitor {
 
+    public LOPrinter(LogicalPlan plan) {
+        super(plan);
+    }
+
+    public void visit() throws ParseException {}
+
+    // TODO FIX
+    /*
     private PrintStream mStream = null;
 
     public LOPrinter(PrintStream ps) {
@@ -41,6 +47,7 @@
     /**
      * Only LOCogroup.visit() should ever call this method.
      */
+    /*
     @Override
     public void visitCogroup(LOCogroup g) {
         print(g, g.name(), g.getSpecs());
@@ -50,6 +57,7 @@
     /**
      * Only LOEval.visit() should ever call this method.
      */
+    /*
     @Override
     public void visitEval(LOEval e) {
         List<EvalSpec> ls = new ArrayList<EvalSpec>();
@@ -61,6 +69,7 @@
     /**
      * Only LOUnion.visit() should ever call this method.
      */
+    /*
     @Override
     public void visitUnion(LOUnion u) {
         print(u, u.name());
@@ -70,6 +79,7 @@
     /**
      * Only LOLoad.visit() should ever call this method.
      */
+    /*
     @Override
     public void visitLoad(LOLoad l) {
         print(l, l.name());
@@ -79,6 +89,7 @@
     /**
      * Only LOSort.visit() should ever call this method.
      */
+    /*
     @Override
     public void visitSort(LOSort s) {
         List<EvalSpec> ls = new ArrayList<EvalSpec>();
@@ -90,6 +101,7 @@
     /**
      * Only LOSplit.visit() should ever call this method.
      */
+    /*
     @Override
     public void visitSplit(LOSplit s) {
         print(s, s.name());
@@ -99,6 +111,7 @@
     /**
      * Only LOStore.visit() should ever call this method.
      */
+    /*
     @Override
     public void visitStore(LOStore s) {
         print(s, s.name());
@@ -157,6 +170,7 @@
             throw new AssertionError("Unknown schema type.");
         }
     }
+    */
 }
 
         

Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOProject.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOProject.java?rev=642398&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOProject.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOProject.java Fri Mar 28 14:44:40 2008
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.impl.logicalLayer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.plan.PlanVisitor;
+
+public class LOProject extends ExpressionOperator {
+    private static final long serialVersionUID = 2L;
+
+    /**
+     * The expression and the column to be projected.
+     */
+    private ExpressionOperator mExp;
+    private List<String> mProjection;
+
+    /**
+     * 
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     * @param exp
+     *            the expression which might contain the column to project
+     * @param projection
+     *            the list of columns to project
+     */
+    public LOProject(LogicalPlan plan, OperatorKey key, int rp,
+            ExpressionOperator exp, List<String> projection) {
+        super(plan, key, rp);
+        mExp = exp;
+        mProjection = projection;
+    }
+
+    /**
+     * 
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     * @param exp
+     *            the expression which might contain the column to project
+     * @param projection
+     *            the column to project
+     */
+    public LOProject(LogicalPlan plan, OperatorKey key, int rp,
+            ExpressionOperator exp, String projection) {
+        super(plan, key, rp);
+        mExp = exp;
+        mProjection = new ArrayList<String>(1);
+        mProjection.add(projection);
+    }
+
+    public ExpressionOperator getExpression() {
+        return mExp;
+    }
+
+    public List<String> getProjection() {
+        return mProjection;
+    }
+
+    @Override
+    public String name() {
+        return "Project " + mKey.scope + "-" + mKey.id;
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return false;
+    }
+
+    @Override
+    public Schema getSchema() {
+        return mSchema;
+    }
+
+    @Override
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
+    }
+
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java Fri Mar 28 14:44:40 2008
@@ -17,6 +17,7 @@
  */
 package org.apache.pig.impl.logicalLayer;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -34,14 +35,20 @@
     private LOUserFunc mSortFunc;
 
     /**
-     * @param plan LogicalPlan this operator is a part of.
-     * @param key OperatorKey for this operator
-     * @param sortCols Array of column numbers that will be used for sorting
-     * data.
-     * @param ascCols Array of booleans.  Should be same size as sortCols.  True
-     * indicates sort ascending (default), false sort descending.  If this array
-     * is null, then all columns will be sorted ascending.
-     * @param rp Requested level of parallelism to be used in the sort.
+     * @param plan
+     *            LogicalPlan this operator is a part of.
+     * @param key
+     *            OperatorKey for this operator
+     * @param sortCols
+     *            Array of column numbers that will be used for sorting data.
+     * @param ascCols
+     *            Array of booleans. Should be same size as sortCols. True
+     *            indicates sort ascending (default), false sort descending. If
+     *            this array is null, then all columns will be sorted ascending.
+     * @param sorFunc
+     *            the user defined sorting function
+     * @param rp
+     *            Requested level of parallelism to be used in the sort.
      */
     public LOSort(LogicalPlan plan,
                   OperatorKey key,
@@ -78,11 +85,22 @@
     }
 
     @Override
-    public Schema getSchema() {
-        if (mSchema == null) {
+    public Schema getSchema() throws IOException {
+        if (!mIsSchemaComputed && (null == mSchema)) {
             // get our parent's schema
             Collection<LogicalOperator> s = mPlan.getSuccessors(this);
-            mSchema = s.iterator().next().getSchema();
+            try {
+                LogicalOperator op = s.iterator().next();
+                if(null == op) {
+                    throw new IOException("Could not find operator in plan");
+                }
+                mSchema = op.getSchema();
+                mIsSchemaComputed = true;
+            } catch (IOException ioe) {
+                mSchema = null;
+                mIsSchemaComputed = false;
+                throw ioe;
+            }
         }
         return mSchema;
     }
@@ -97,11 +115,7 @@
         return false;
     }
 
-    public void visit(PlanVisitor v) throws ParseException {
-        if (!(v instanceof LOVisitor)) {
-            throw new RuntimeException("You can only visit LogicalOperators "
-                + "with an LOVisitor!");
-        }
-        ((LOVisitor)v).visitSort(this);
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSplit.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSplit.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSplit.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSplit.java Fri Mar 28 14:44:40 2008
@@ -18,49 +18,88 @@
 package org.apache.pig.impl.logicalLayer;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.io.IOException;
+import java.util.Set;
 
-import org.apache.pig.impl.eval.cond.Cond;
-import org.apache.pig.impl.logicalLayer.schema.TupleSchema;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.plan.PlanVisitor;
 
 public class LOSplit extends LogicalOperator {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
-      List<Cond> conditions = new ArrayList<Cond>();
+    private Map<String, ExpressionOperator> mOutputs;
 
-    public LOSplit(Map<OperatorKey, LogicalOperator> opTable,
-                   String scope, 
-                   long id, 
-                   OperatorKey input) {
-        super(opTable, scope, id, input);
-    }
-    
-    public void addCond(Cond cond) {
-        conditions.add(cond);
+    /**
+     * @param plan
+     *            LogicalPlan this operator is a part of.
+     * @param key
+     *            OperatorKey for this operator
+     * @param rp
+     *            Requested level of parallelism to be used in the sort.
+     * @param aliases
+     *            list of aliases that are the output of the split
+     * @param conditions
+     *            list of conditions for the split
+     */
+    public LOSplit(LogicalPlan plan, OperatorKey key, int rp,
+            Map<String, ExpressionOperator> outputs) {
+        super(plan, key, rp);
+        mOutputs = outputs;
+    }
+
+    public Collection<ExpressionOperator> getConditions() {
+        return mOutputs.values();
     }
 
-    @Override
-    public int getOutputType() {
-        return opTable.get(getInputs().get(0)).getOutputType();
+    public Set<String> getOutputAliases() {
+        return mOutputs.keySet();
     }
 
-    public ArrayList<Cond> getConditions() {
-        return new ArrayList<Cond> (conditions);
+    public void addOutputAlias(String output, ExpressionOperator cond) {
+        mOutputs.put(output, cond);
     }
 
     @Override
-    public TupleSchema outputSchema() {
-        return opTable.get(getInputs().get(0)).outputSchema().copy();
+    public String name() {
+        return "Split " + mKey.scope + "-" + mKey.id;
     }
 
     @Override
-    public String name() {
-        return "Split " + scope + "-" + id;
+    public Schema getSchema() throws IOException {
+        if (!mIsSchemaComputed && (null == mSchema)) {
+            // get our parent's schema
+            Collection<LogicalOperator> s = mPlan.getSuccessors(this);
+            try {
+                LogicalOperator op = s.iterator().next();
+                if (null == op) {
+                    throw new IOException("Could not find operator in plan");
+                }
+                mSchema = s.iterator().next().getSchema();
+                mIsSchemaComputed = true;
+            } catch (IOException ioe) {
+                mSchema = null;
+                mIsSchemaComputed = false;
+                throw ioe;
+            }
+        }
+        return mSchema;
     }
-    
-    public void visit(LOVisitor v) {
-        v.visitSplit(this);
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return false;
     }
 
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return true;
+    }
+
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
+    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOStore.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOStore.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOStore.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOStore.java Fri Mar 28 14:44:40 2008
@@ -18,98 +18,85 @@
 package org.apache.pig.impl.logicalLayer;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.pig.StoreFunc;
-import org.apache.pig.impl.PigContext;
+import org.apache.pig.StoreFunc; // import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileSpec;
-import org.apache.pig.impl.logicalLayer.schema.TupleSchema;
-
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.plan.PlanVisitor;
 
 public class LOStore extends LogicalOperator {
-    private static final long serialVersionUID = 1L;
-
-    protected FileSpec outputFileSpec;
-
-    protected boolean append;
+    private static final long serialVersionUID = 2L;
 
+    private FileSpec mOutputFile;
+    private StoreFunc mStoreFunc;
 
-    public LOStore(Map<OperatorKey, LogicalOperator> opTable,
-                   String scope,
-                   long id,
-                   OperatorKey input,
-                   FileSpec fileSpec,
-                   boolean append) throws IOException {
-        super(opTable, scope, id, input);
-        this.outputFileSpec = fileSpec;
-        this.append = append;
-
-        //See if the store function spec is valid
-        try {
-            StoreFunc StoreFunc =
-                (StoreFunc) PigContext.instantiateFuncFromSpec(
-                    fileSpec.getFuncSpec());
-        } catch(Exception e) {
-            IOException ioe = new IOException(e.getMessage());
-            ioe.setStackTrace(e.getStackTrace());
-            throw ioe;
-        } getOutputType();
+    /**
+     * @param plan
+     *            LogicalPlan this operator is a part of.
+     * @param key
+     *            OperatorKey for this operator
+     * @param rp
+     *            Requested level of parallelism to be used in the sort.
+     * @param outputFileSpec
+     *            the file to be stored
+     * @param storeFunc
+     *            the store function, pre-defined or user defined
+     */
+    public LOStore(LogicalPlan plan, OperatorKey key, int rp,
+            FileSpec outputFileSpec) throws IOException {
+        super(plan, key, rp);
+
+        mOutputFile = outputFileSpec;
+
+        // TODO
+        // The code below is commented out as PigContext pulls in
+        // HExecutionEngine which in turn is completely commented out
+        // Also remove the commented out import org.apache.pig.impl.PigContext
+
+        /*
+         * try { mStoreFunc = (StoreFunc)
+         * PigContext.instantiateFuncFromSpec(outputFileSpec.getFuncSpec()); }
+         * catch (Exception e) { IOException ioe = new
+         * IOException(e.getMessage()); ioe.setStackTrace(e.getStackTrace());
+         * throw ioe; }
+         */
     }
 
-
-    public FileSpec getOutputFileSpec() {
-        return outputFileSpec;
+    public FileSpec getOutputFile() {
+        return mOutputFile;
     }
 
-
-    @Override
-    public String toString() {
-        StringBuffer result = new StringBuffer(super.toString());
-        result.append(" (append: ");
-        result.append(append);
-        result.append(')');
-        return result.toString();
+    public StoreFunc getStoreFunc() {
+        return mStoreFunc;
     }
 
-
     @Override
     public String name() {
-        return "Store " + scope + "-" + id;
+        return "Store " + mKey.scope + "-" + mKey.id;
     }
 
     @Override
-    public TupleSchema outputSchema() {
-        throw new
-            RuntimeException
-            ("Internal error: Asking for schema of a store operator.");
+    public Schema getSchema() throws RuntimeException {
+        throw new RuntimeException("Internal error: Requested schema of a "
+                + "store operator.");
     }
 
     @Override
-    public int getOutputType() {
-        switch (opTable.get(getInputs().get(0)).getOutputType()) {
-        case FIXED:
-            return FIXED;
-        case MONOTONE:
-            return MONOTONE;
-        default:
-            throw new RuntimeException("Illegal input type for store operator");
-        }
+    public boolean supportsMultipleInputs() {
+        return false;
     }
 
     @Override
-    public List<String> getFuncs() {
-        List<String> funcs = super.getFuncs();
-        funcs.add(outputFileSpec.getFuncName());
-        return funcs;
-    }
-
-
-    public boolean isAppend() {
-        return append;
+    public boolean supportsMultipleOutputs() {
+        return false;
     }
 
-    public void visit(LOVisitor v) {
-        v.visitStore(this);
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
     }
 }

Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSubtract.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSubtract.java?rev=642398&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSubtract.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSubtract.java Fri Mar 28 14:44:40 2008
@@ -0,0 +1,66 @@
+/* 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.pig.impl.logicalLayer;
+
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.plan.PlanVisitor;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+public class LOSubtract extends BinaryExpressionOperator {
+
+    private static final long serialVersionUID = 2L;
+
+    /**
+     * 
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     * @param lhsOperand
+     *            the left hand side operand
+     * @param rhsOperand
+     *            the right hand side operand
+     */
+    public LOSubtract(LogicalPlan plan, OperatorKey k, int rp,
+            ExpressionOperator lhsOperand, ExpressionOperator rhsOperand) {
+        super(plan, k, rp, lhsOperand, rhsOperand);
+    }
+
+    @Override
+    public Schema getSchema() {
+        // TODO When tuple subtraction is implemented, getSchema should
+        // compute the schema, store the computed schema and return
+        // the computed schema
+
+        return mSchema;
+    }
+
+    @Override
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
+    }
+
+    @Override
+    public String name() {
+        return "Subtract " + mKey.scope + "-" + mKey.id;
+    }
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUnion.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUnion.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUnion.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUnion.java Fri Mar 28 14:44:40 2008
@@ -17,72 +17,57 @@
  */
 package org.apache.pig.impl.logicalLayer;
 
+import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-
-import org.apache.pig.impl.logicalLayer.schema.TupleSchema;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.plan.PlanVisitor;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
 
+public class LOUnion extends LogicalOperator {
 
+    private static final long serialVersionUID = 2L;
 
-public class LOUnion extends LogicalOperator {
-    private static final long serialVersionUID = 1L;
+    /**
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     */
+    public LOUnion(LogicalPlan plan, OperatorKey k, int rp) {
+        super(plan, k, rp);
+    }
 
-    public LOUnion(Map<OperatorKey, LogicalOperator> opTable,
-                   String scope, 
-                   long id, 
-                   List<OperatorKey> inputsIn) {
-        super(opTable, scope, id, inputsIn);
+    @Override
+    public Schema getSchema() {
+        if (null == mSchema) {
+            // TODO FIX
+            // The schema merge operation needs to be implemented in
+            // order to compute the schema of the union
+        }
+        return mSchema;
     }
-    
+
     @Override
     public String name() {
-        return "Union " + scope + "-" + id;
+        return "Union " + mKey.scope + "-" + mKey.id;
     }
 
     @Override
-    public TupleSchema outputSchema() {
-        if (schema == null) {
-            TupleSchema longest = opTable.get(getInputs().get(0)).outputSchema();
-            int current = 0;
-          for (OperatorKey opKey: getInputs()) {
-              LogicalOperator lo = opTable.get(opKey);
-              
-              if (lo != null && lo.outputSchema() != null && 
-                  lo.outputSchema().numFields() > current) {
-                  longest = lo.outputSchema();
-                  current = longest.numFields();
-                }
-            }
-            schema = longest.copy();
-        }
-
-        schema.setAlias(alias);
-        return schema;
+    public boolean supportsMultipleInputs() {
+        return true;
     }
 
     @Override
-    public int getOutputType() {
-        int outputType = FIXED;
-        for (int i = 0; i < getInputs().size(); i++) {
-            switch (opTable.get(getInputs().get(i)).getOutputType()) {
-            case FIXED:
-                continue;
-            case MONOTONE:
-                if (outputType == FIXED)
-                    outputType = MONOTONE;
-                continue;
-            case AMENDABLE:
-                outputType = AMENDABLE;
-            default:
-                throw new
-                    RuntimeException
-                    ("Invalid input type to the UNION operator");
-            }
-        }
-        return outputType;
+    public boolean supportsMultipleOutputs() {
+        return false;
     }
 
-    public void visit(LOVisitor v) {
-        v.visitUnion(this);
+    @Override
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
     }
+
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUserFunc.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUserFunc.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUserFunc.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUserFunc.java Fri Mar 28 14:44:40 2008
@@ -29,16 +29,17 @@
     private List<ExpressionOperator> mArgs;
 
     /**
-     * @param plan LogicalPlan this operator is a part of.
-     * @param k OperatorKey for this operator.
-     * @param args List of expressions that form the arguments for this
-     * function.
-     * @param returnType return type of this function.
+     * @param plan
+     *            LogicalPlan this operator is a part of.
+     * @param k
+     *            OperatorKey for this operator.
+     * @param args
+     *            List of expressions that form the arguments for this function.
+     * @param returnType
+     *            return type of this function.
      */
-    public LOUserFunc(LogicalPlan plan,
-                      OperatorKey k,
-                      List<ExpressionOperator> args,
-                      byte returnType) {
+    public LOUserFunc(LogicalPlan plan, OperatorKey k,
+            List<ExpressionOperator> args, byte returnType) {
         super(plan, k, -1);
         mArgs = args;
         mType = returnType;
@@ -54,21 +55,11 @@
     }
 
     @Override
-    public boolean supportsMultipleOutputs() {
-        return false;
-    }
-    
-    @Override
     public String name() {
         return "UserFunc " + mKey.scope + "-" + mKey.id;
     }
 
     @Override
-    public String typeName() {
-        return "LOUserFunc";
-    }
-
-    @Override
     public Schema getSchema() {
         if (mSchema == null) {
             mSchema = new Schema(new Schema.FieldSchema(null, mType));
@@ -77,11 +68,7 @@
     }
 
     @Override
-    public void visit(PlanVisitor v) throws ParseException {
-        if (!(v instanceof LOVisitor)) {
-            throw new RuntimeException("You can only visit LogicalOperators "
-                + "with an LOVisitor!");
-        }
-        ((LOVisitor)v).visitUserFunc(this);
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOVisitor.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOVisitor.java Fri Mar 28 14:44:40 2008
@@ -19,64 +19,205 @@
 
 import java.util.List;
 import java.util.Iterator;
+import java.util.Set;
 
 import org.apache.pig.impl.plan.PlanVisitor;
 import org.apache.pig.impl.logicalLayer.parser.ParseException;
 
-
 /**
  * A visitor mechanism for navigating and operating on a tree of Logical
- * Operators.  This class contains the logic to navigate thre tree, but does
- * not do anything with or to the tree.  In order to operate on or extract
- * information from the tree, extend this class.  You only need to implement
- * the methods dealing with the logical operators you are concerned
- * with.  For example, if you wish to find every LOEval in a logical plan
- * and perform some operation on it, your visitor would look like:
- * class MyLOVisitor extends LOVisitor {
- *     public void visitEval(LOEval e) { you're logic here }
- * }
- * Any operators that you do not implement the visitX method for will then
- * be navigated through by this class.
- *
- * *NOTE* When envoking a visitor, you should never call one of the
- * methods in this class.  You should pass your visitor as an argument to
- * visit() on the object you want to visit.  So:
- * RIGHT:  LOEval myEval; MyVisitor v; myEval.visit(v);
- * WRONG:  LOEval myEval; MyVisitor v; v.visitEval(myEval);
+ * Operators. This class contains the logic to navigate the tree, but does not
+ * do anything with or to the tree. In order to operate on or extract
+ * information from the tree, extend this class. You only need to implement the
+ * methods dealing with the logical operators you are concerned with. For
+ * example, if you wish to find every LOEval in a logical plan and perform some
+ * operation on it, your visitor would look like: class MyLOVisitor extends
+ * LOVisitor { public void visitEval(LOEval e) { you're logic here } } Any
+ * operators that you do not implement the visitX method for will then be
+ * navigated through by this class.
+ * 
+ * *NOTE* When invoking a visitor, you should never call one of the methods in
+ * this class. You should pass your visitor as an argument to visit() on the
+ * object you want to visit. So: RIGHT: LOEval myEval; MyVisitor v;
+ * myEval.visit(v); WRONG: LOEval myEval; MyVisitor v; v.visitEval(myEval);
  * These methods are only public to make them accessible to the LO* objects.
  */
-abstract public class LOVisitor extends PlanVisitor {
+abstract public class LOVisitor extends PlanVisitor<LogicalOperator, LogicalPlan> {
 
     public LOVisitor(LogicalPlan plan) {
         super(plan);
     }
 
-    void visitCogroup(LOCogroup cg) throws ParseException {
+    /**
+     * @param lOp
+     *            the logical operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LogicalOperator lOp)
+            throws ParseException {
+        //
+        // Do Nothing
+        //
+    }
+
+    /**
+     * @param eOp
+     *            the logical expression operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(ExpressionOperator eOp)
+            throws ParseException {
+        //
+        // Do Nothing
+        //
+    }
+
+    /**
+     * @param binOp
+     *            the logical binary expression operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(BinaryExpressionOperator binOp)
+            throws ParseException {
+        //
+        // Visit the left hand side operand followed by the right hand side
+        // operand
+        //
+
+        binOp.getLhsOperand().visit(this);
+        binOp.getRhsOperand().visit(this);
+    }
+
+    /**
+     * 
+     * @param uniOp
+     *            the logical unary operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(UnaryExpressionOperator uniOp) throws ParseException {
+        // Visit the operand
+
+        uniOp.getOperand().visit(this);
+    }
+
+    /**
+     * 
+     * @param cg
+     *            the logical cogroup operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOCogroup cg) throws ParseException {
+        // Visit each of the inputs of cogroup.
+        Iterator<ExpressionOperator> i = cg.getGroupByCols().iterator();
+        while (i.hasNext()) {
+            i.next().visit(this);
+        }
     }
 
-    void visitEval(LOEval e) throws ParseException {
-        // Don't worry about visiting the contained logical operators, as the logical
-        // operators in it are already contained in the outer plan.
+    /**
+     * 
+     * @param g
+     *            the logical generate operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOGenerate g) throws ParseException {
+        // Visit each of generates projection elements.
+        Iterator<ExpressionOperator> i = g.getProjections().iterator();
+        while (i.hasNext()) {
+            i.next().visit(this);
+        }
     }
 
-    void visitGenerate(LOGenerate g) throws ParseException {
-        // Visit each of generates projection elements.
-        Iterator<LogicalOperator> i = g.getProjections().iterator();
+    /**
+     * 
+     * @param s
+     *            the logical sort operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOSort s) throws ParseException {
+        // Visit the sort function
+        s.getUserFunc().visit(this);
+    }
+
+    /**
+     * 
+     * @param filter
+     *            the logical filter operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOFilter filter) throws ParseException {
+        // Visit the condition for the filter followed by the input
+        filter.getCondition().visit(this);
+    }
+
+    /**
+     * 
+     * @param split
+     *            the logical split operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOSplit split) throws ParseException {
+        // Visit each of split's conditions
+        Iterator<ExpressionOperator> i = split.getConditions().iterator();
         while (i.hasNext()) {
             i.next().visit(this);
         }
     }
-        
-    void visitSort(LOSort s) throws ParseException {
+
+    /**
+     * 
+     * @param forEach
+     *            the logical foreach operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOForEach forEach) throws ParseException {
+        // Visit the operators that are part of the foreach
+        Iterator<LogicalOperator> i = forEach.getOperators().iterator();
+        while (i.hasNext()) {
+            i.next().visit(this);
+        }
     }
 
-    void visitUserFunc(LOUserFunc func) throws ParseException {
+    /**
+     * Iterate over each expression that is part of the function argument list
+     * 
+     * @param func
+     *            the user defined function
+     * @throws ParseException
+     */
+    void visit(LOUserFunc func) throws ParseException {
         // Visit each of the arguments
         Iterator<ExpressionOperator> i = func.getArguments().iterator();
         while (i.hasNext()) {
             i.next().visit(this);
         }
     }
-}
 
-        
+    /**
+     * @param binCond the logical binCond operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOBinCond binCond) throws ParseException {
+        /**
+         * Visit the conditional expression followed by the left hand operator
+         * and the right hand operator respectively
+         */
+
+        binCond.getCond().visit(this);
+        binCond.getLhsOp().visit(this);
+        binCond.getRhsOp().visit(this);
+    }
+
+    /**
+     * 
+     * @param cast
+     *            the logical cast operator that has to be visited
+     * @throws ParseException
+     */
+    void visit(LOCast cast) throws ParseException {
+        // Visit the expression to be cast
+
+        cast.getExpression().visit(this);
+    }
+
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java Fri Mar 28 14:44:40 2008
@@ -20,17 +20,17 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.io.IOException;
 
-import org.apache.pig.data.DataType;
 import org.apache.pig.impl.logicalLayer.parser.ParseException;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.plan.Operator;
-
+import org.apache.pig.data.DataType;
 
 /**
  * Parent for all Logical operators.
  */
-abstract public class LogicalOperator extends Operator {
+abstract public class LogicalOperator extends Operator<LOVisitor> {
     private static final long serialVersionUID = 2L;
 
     /**
@@ -39,6 +39,11 @@
     protected Schema mSchema = null;
 
     /**
+     * A boolean variable to remember if the schema has been computed
+     */
+    protected boolean mIsSchemaComputed = false;
+  
+    /**
      * Datatype of this output of this operator.  Operators start out with data type
      * set to UNKNOWN, and have it set for them by the type checker.
      */
@@ -97,7 +102,11 @@
     public final void setSchema(Schema schema) throws ParseException {
         // In general, operators don't generate their schema until they're
         // asked, so ask them to do it.
-        getSchema();
+        try {
+            getSchema();
+        } catch (IOException ioe) {
+            // It's fine, it just means we don't have a schema yet.
+        }
         if (mSchema == null) mSchema = schema;
         else mSchema.reconcile(schema);
     }
@@ -105,7 +114,7 @@
     /**
      * Get a copy of the schema for the output of this operator.
      */
-    public abstract Schema getSchema();
+    public abstract Schema getSchema() throws IOException;
 
     /**
      * Set the type of this operator.  This should only be called by the type
@@ -161,5 +170,16 @@
 
         // TODO
     }
+
+    /**
+     * Visit this node with the provided visitor. This should only be called by
+     * the visitor class itself, never directly.
+     * 
+     * @param v
+     *            Visitor to visit with.
+     * @throws ParseException
+     *             if the visitor has a problem.
+     */
+    public abstract void visit(LOVisitor v) throws ParseException;
 
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlan.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlan.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlan.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlan.java Fri Mar 28 14:44:40 2008
@@ -36,7 +36,7 @@
     }
 
     public void explain(OutputStream out) {
-        // TODO
+        // TODO FIX
         /*
         LOVisitor lprinter = new LOPrinter(new PrintStream(out));
         

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java Fri Mar 28 14:44:40 2008
@@ -17,6 +17,8 @@
  */
 package org.apache.pig.impl.logicalLayer;
 
+// TODO FIX
+/*
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.Map;
@@ -24,6 +26,7 @@
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.logicalLayer.parser.ParseException;
 import org.apache.pig.impl.logicalLayer.parser.QueryParser;
+*/
 
 
 /**
@@ -31,6 +34,8 @@
  * 
  */
 public class LogicalPlanBuilder {
+    // TODO FIX
+    /*
     public static ClassLoader classloader = LogicalPlanBuilder.class.getClassLoader();
     private PigContext pigContext;
     public LogicalPlanBuilder(PigContext pigContext) {
@@ -46,4 +51,5 @@
         QueryParser parser = new QueryParser(in, pigContext, scope, aliases, opTable);
         return parser.Parse();        
     }
+    */
 }

Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/UnaryExpressionOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/UnaryExpressionOperator.java?rev=642398&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/UnaryExpressionOperator.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/UnaryExpressionOperator.java Fri Mar 28 14:44:40 2008
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.pig.impl.logicalLayer;
+
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.plan.PlanVisitor;
+import org.apache.pig.data.DataType;
+
+/**
+ * This abstract class represents the logical Unary Expression Operator The
+ * unary operator has an operand and an operator. The format of the expression
+ * is operator operand. The operator is implicit and not recorded in the class
+ */
+public abstract class UnaryExpressionOperator extends ExpressionOperator {
+    private static final long serialVersionUID = 2L;
+    private ExpressionOperator mOperand; // operand
+
+    /**
+     * @param plan
+     *            Logical plan this operator is a part of.
+     * @param k
+     *            Operator key to assign to this node.
+     * @param rp
+     *            degree of requested parallelism with which to execute this
+     *            node.
+     * @param operand
+     *            ExpressionOperator the left hand side operand
+     * @param operator
+     *            LogicalExperssion the actual operator
+     */
+    public UnaryExpressionOperator(LogicalPlan plan, OperatorKey k, int rp,
+            ExpressionOperator operand) {
+        super(plan, k, rp);
+        mOperand = operand;
+    }
+
+    public ExpressionOperator getOperand() {
+        return mOperand;
+    }
+
+    @Override
+    public void visit(LOVisitor v) throws ParseException {
+        v.visit(this);
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return false;
+    }
+
+}
\ No newline at end of file

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java Fri Mar 28 14:44:40 2008
@@ -25,21 +25,21 @@
 import org.apache.pig.impl.logicalLayer.OperatorKey;
 import org.apache.pig.impl.logicalLayer.parser.ParseException;
 
-
 /**
  * Base class for all types of operators.
  */
-abstract public class Operator {
+abstract public class Operator<V extends PlanVisitor> {
     private static final long serialVersionUID = 1L;
 
     /**
-     * OperatorKey associated with this operator.  This key is used to find the
+     * OperatorKey associated with this operator. This key is used to find the
      * operator in an OperatorPlan.
      */
     protected OperatorKey mKey;
 
     /**
-     * @param - k Operator key to assign to this node.
+     * @param -
+     *            k Operator key to assign to this node.
      */
     public Operator(OperatorKey k) {
         mKey = k;
@@ -53,33 +53,40 @@
     }
 
     /**
-     * Visit this node with the provided visitor.  This should only be called by
+     * Visit this node with the provided visitor. This should only be called by
      * the visitor class itself, never directly.
-     * @param v Visitor to visit with.
-     * @throws ParseException if the visitor has a problem.
+     * 
+     * @param v
+     *            Visitor to visit with.
+     * @throws ParseException
+     *             if the visitor has a problem.
      */
-    public abstract void visit(PlanVisitor v) throws ParseException;
+    public abstract void visit(V v) throws ParseException;
 
     /**
      * Indicates whether this operator supports multiple inputs.
+     * 
      * @return true if it does, otherwise false.
      */
     public abstract boolean supportsMultipleInputs();
 
     /**
      * Indicates whether this operator supports multiple outputs.
+     * 
      * @return true if it does, otherwise false.
      */
     public abstract boolean supportsMultipleOutputs();
-    
+
     public abstract String name();
 
-    public abstract String typeName();
-    
+    public String typeName() {
+        return this.getClass().getName();
+    }
+
     @Override
     public String toString() {
         StringBuilder msg = new StringBuilder();
-        
+
         msg.append("(Name: " + name() + " Operator Key: " + mKey + ")");
 
         return msg.toString();

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanVisitor.java?rev=642398&r1=642397&r2=642398&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanVisitor.java Fri Mar 28 14:44:40 2008
@@ -33,9 +33,9 @@
  * LOVisitor).
  *
  */
-abstract public class PlanVisitor {
+abstract public class PlanVisitor <O extends Operator, P extends OperatorPlan<O>> {
 
-    protected OperatorPlan mPlan;
+    protected P mPlan;
 
     /**
      * Entry point for visiting the plan.
@@ -46,7 +46,7 @@
     /**
      * @param plan OperatorPlan this visitor will visit.
      */
-    protected PlanVisitor(OperatorPlan plan) {
+    protected PlanVisitor(P plan) {
         mPlan = plan;
     }
 
@@ -55,21 +55,21 @@
      * @throws ParseException if the underlying visitor has a problem.
      */
     protected void depthFirst() throws ParseException {
-        List<Operator> roots = mPlan.getRoots();
-        Set<Operator> seen = new HashSet<Operator>();
+        List<O> roots = mPlan.getRoots();
+        Set<O> seen = new HashSet<O>();
 
         depthFirst(null, roots, seen);
     }
 
-    private void depthFirst(Operator node,
-                            Collection<Operator> successors,
-                            Set<Operator> seen) throws ParseException {
+    private void depthFirst(O node,
+                            Collection<O> successors,
+                            Set<O> seen) throws ParseException {
         if (successors == null) return;
 
-        for (Operator suc : successors) {
+        for (O suc : successors) {
             if (seen.add(suc)) {
                 suc.visit(this);
-                Collection<Operator> newSuccessors = mPlan.getSuccessors(suc);
+                Collection<O> newSuccessors = mPlan.getSuccessors(suc);
                 depthFirst(suc, newSuccessors, seen);
             }
         }
@@ -82,7 +82,7 @@
      * @throws ParseException if the underlying visitor has a problem.
      */
     protected void dependencyOrder() throws ParseException {
-        // This is highly inneficient, but our graphs are small so it should be okay.
+        // This is highly inefficient, but our graphs are small so it should be okay.
         // The algorithm works by starting at any node in the graph, finding it's
         // predecessors and calling itself for each of those predecessors.  When it
         // finds a node that has no unfinished predecessors it puts that node in the
@@ -90,28 +90,28 @@
         // It keeps track of what nodes it's seen as it goes so it doesn't put any
         // nodes in the graph twice.
 
-        List<Operator> fifo = new ArrayList<Operator>();
-        Set<Operator> seen = new HashSet<Operator>();
-        List<Operator> leaves = mPlan.getLeaves();
+        List<O> fifo = new ArrayList<O>();
+        Set<O> seen = new HashSet<O>();
+        List<O> leaves = mPlan.getLeaves();
         if (leaves == null) return;
-        for (Operator op : leaves) {
+        for (O op : leaves) {
             doAllPredecessors(op, seen, fifo);
         }
 
-        for (Operator op: fifo) {
+        for (O op: fifo) {
             op.visit(this);
         }
     }
 
-    private void doAllPredecessors(Operator node,
-                                   Set<Operator> seen,
-                                   Collection<Operator> fifo) throws ParseException {
+    private void doAllPredecessors(O node,
+                                   Set<O> seen,
+                                   Collection<O> fifo) throws ParseException {
         if (!seen.contains(node)) {
             // We haven't seen this one before.
-            Collection<Operator> preds = mPlan.getPredecessors(node);
+            Collection<O> preds = mPlan.getPredecessors(node);
             if (preds != null && preds.size() > 0) {
                 // Do all our predecessors before ourself
-                for (Operator op : preds) {
+                for (O op : preds) {
                     doAllPredecessors(op, seen, fifo);
                 }
             }