You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2013/09/21 00:36:36 UTC

svn commit: r1525162 - in /pig/trunk: CHANGES.txt src/org/apache/pig/PigServer.java src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java test/org/apache/pig/test/TestLogicalPlanBuilder.java

Author: daijy
Date: Fri Sep 20 22:36:35 2013
New Revision: 1525162

URL: http://svn.apache.org/r1525162
Log:
PIG-3199: Provide a method to retriever name of loader/storer in PigServer

Added:
    pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/PigServer.java
    pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1525162&r1=1525161&r2=1525162&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Fri Sep 20 22:36:35 2013
@@ -30,6 +30,8 @@ PIG-3174: Remove rpm and deb artifacts f
 
 IMPROVEMENTS
 
+PIG-3199: Provide a method to retriever name of loader/storer in PigServer (prkommireddi via daijy)
+
 PIG-3367: Add assert keyword (operator) in pig (aniket486)
 
 PIG-3235: Avoid extra byte array copies in streaming (rohini)

Modified: pig/trunk/src/org/apache/pig/PigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigServer.java?rev=1525162&r1=1525161&r2=1525162&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigServer.java (original)
+++ pig/trunk/src/org/apache/pig/PigServer.java Fri Sep 20 22:36:35 2013
@@ -82,6 +82,7 @@ import org.apache.pig.newplan.logical.re
 import org.apache.pig.newplan.logical.relational.LOLoad;
 import org.apache.pig.newplan.logical.relational.LOStore;
 import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.newplan.logical.relational.LogicalPlanData;
 import org.apache.pig.newplan.logical.relational.LogicalRelationalOperator;
 import org.apache.pig.newplan.logical.relational.LogicalSchema;
 import org.apache.pig.newplan.logical.visitor.CastLineageSetter;
@@ -1357,6 +1358,18 @@ public class PigServer {
         }
         return op;
     }
+    
+    /**
+     * Returns data associated with LogicalPlan. It makes
+     * sense to call this method only after a query/script
+     * has been registered with one of the {@link #registerQuery(String)}
+     * or {@link #registerScript(InputStream)} methods.
+     * 
+     * @return LogicalPlanData
+     */
+    public LogicalPlanData getLogicalPlanData() {
+	return new LogicalPlanData(currDAG.lp);
+    }
 
     /*
      * This class holds the internal states of a grunt shell session.

Added: pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java?rev=1525162&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java (added)
+++ pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java Fri Sep 20 22:36:35 2013
@@ -0,0 +1,156 @@
+/*
+ * 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.newplan.logical.relational;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.pig.newplan.Operator;
+
+/**
+ * 
+ * This class provides information regarding the LogicalPlan. Make sure to
+ * avoid exposing LogicalPlan itself. Only data regarding the logical plan
+ * could be exposed but none of Pig internals (plans, operators etc) should
+ * be.
+ *
+ */
+public class LogicalPlanData {
+
+    // Never expose LogicalPlan
+    private final LogicalPlan lp;
+
+    public LogicalPlanData(LogicalPlan lp) {
+        if(lp == null) {
+            throw new RuntimeException("LogicalPlan is null.");
+        }
+        this.lp = lp; 
+    }
+
+    /**
+     * 
+     * @return This method return the list of source paths defined 
+     *         in the script/query.
+     */
+    public List<String> getSources() {
+        List<LOLoad> sources = getLOLoads();
+        if (sources == null) {
+            return null;
+        }
+
+        List<String> result = new ArrayList<String>();
+        for (LOLoad load : sources) {
+            result.add(load.getFileSpec().getFileName());
+        }
+
+        return result;
+    }
+
+    /**
+     * 
+     * @return This method returns the list of store paths in the script/query.
+     */
+    public List<String> getSinks() {
+        List<LOStore> sinks = getLOStores();
+        if (sinks == null) {
+            return null;
+        }
+        List<String> result = new ArrayList<String>();
+        for (LOStore sink : sinks) {
+            result.add(sink.getFileSpec().getFileName());
+        }
+
+        return result;
+    }
+
+    /**
+     * 
+     * @return This method returns the list of LoadFunc(s) used.
+     */
+    public List<String> getLoadFuncs() {
+        List<LOLoad> sources = getLOLoads();
+        if (sources == null) {
+            return null;
+        }
+        List<String> result = new ArrayList<String>();
+        for (LOLoad load : sources) {
+            result.add(load.getFileSpec().getFuncName());
+        }
+
+        return result;
+    }
+
+    /**
+     * 
+     * @return This method returns the list of StoreFunc(s) used.
+     */
+    public List<String> getStoreFuncs() {
+        List<LOStore> sinks = getLOStores();
+        if (sinks == null) {
+            return null;
+        }
+        List<String> storeFuncs = new ArrayList<String>();
+        for (LOStore sink : sinks) {
+            storeFuncs.add(sink.getFileSpec().getFuncName());
+        }
+
+        return storeFuncs;
+    }
+
+    /**
+     * Internal to Pig. Do not expose this method
+     * @return
+     */
+    private List<LOLoad> getLOLoads() {
+        List<Operator> sources = lp.getSources();
+        if (sources == null) {
+            return null;
+        }
+        List<LOLoad> result = new ArrayList<LOLoad>();
+        for (Operator source : sources) {
+            if (source instanceof LOLoad) {
+                LOLoad load = (LOLoad) source;
+                result.add(load);
+            }
+        }
+
+        return result;
+    }
+    
+    /**
+     * Internal to Pig. Do not expose this method
+     * @return
+     */
+    private List<LOStore> getLOStores() {
+        List<Operator> sinks = lp.getSinks();
+        if (sinks == null) {
+            return null;
+        }
+        List<LOStore> result = new ArrayList<LOStore>();
+        for (Operator sink : sinks) {
+            if (sink instanceof LOStore) {
+                LOStore store = (LOStore) sink;
+                result.add(store);
+            }
+        }
+
+        return result;
+    }
+
+}

Modified: pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java?rev=1525162&r1=1525161&r2=1525162&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java Fri Sep 20 22:36:35 2013
@@ -18,6 +18,7 @@
 package org.apache.pig.test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -25,6 +26,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import junit.framework.Assert;
+import junit.framework.AssertionFailedError;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
@@ -59,14 +63,12 @@ import org.apache.pig.newplan.logical.re
 import org.apache.pig.newplan.logical.relational.LOSort;
 import org.apache.pig.newplan.logical.relational.LOStore;
 import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.newplan.logical.relational.LogicalPlanData;
 import org.apache.pig.newplan.logical.relational.LogicalSchema;
 import org.apache.pig.test.utils.Identity;
 import org.junit.Before;
 import org.junit.Test;
 
-import junit.framework.Assert;
-import junit.framework.AssertionFailedError;
-
 public class TestLogicalPlanBuilder {
     PigContext pigContext = new PigContext(ExecType.LOCAL, new Properties());
     private PigServer pigServer = null;
@@ -2156,6 +2158,21 @@ public class TestLogicalPlanBuilder {
         pigServer.getPigContext().getProperties().remove(PigConfiguration.PIG_DEFAULT_STORE_FUNC);      
     }
     
+    @Test
+    public void testLogicalPlanData() throws Exception {
+        String query = "a = load 'input.txt'; b = load 'anotherinput.txt'; c = join a by $0, b by $1;" +
+                "store c into 'output' using org.apache.pig.test.PigStorageWithSchema();";
+        // Set batch on so the query is not executed
+        pigServer.setBatchOn();
+        pigServer.registerQuery(query);
+        LogicalPlanData lData = pigServer.getLogicalPlanData();
+        assertEquals("LoadFunc must be PigStorage", "org.apache.pig.builtin.PigStorage", lData.getLoadFuncs().get(0));
+        assertEquals("StoreFunc must be PigStorageWithSchema", "org.apache.pig.test.PigStorageWithSchema", lData.getStoreFuncs().get(0));
+        assertEquals("Number of sources must be 2", lData.getSources().size(), 2);
+        assertTrue("Source must end with input.txt", lData.getSources().get(0).endsWith("input.txt"));
+        assertTrue("Source must end with input.txt", lData.getSinks().get(0).endsWith("output"));
+    }
+    
     /**
      * This method is not generic. Expects logical plan to have atleast
      * 1 source and returns the corresponding FuncSpec.