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 2015/11/20 02:17:02 UTC

svn commit: r1715312 - in /pig/trunk: ./ conf/ ivy/ src/org/apache/pig/ src/org/apache/pig/parser/ src/org/apache/pig/tools/ src/org/apache/pig/tools/grunt/ test/org/apache/pig/test/ test/org/apache/pig/test/data/

Author: daijy
Date: Fri Nov 20 01:17:02 2015
New Revision: 1715312

URL: http://svn.apache.org/viewvc?rev=1715312&view=rev
Log:
PIG-4417: Pig's register command should support automatic fetching of jars from repo

Added:
    pig/trunk/conf/ivysettings.xml
    pig/trunk/src/org/apache/pig/parser/RegisterResolver.java
    pig/trunk/src/org/apache/pig/tools/DownloadResolver.java
    pig/trunk/test/org/apache/pig/test/TestRegisterParser.java
    pig/trunk/test/org/apache/pig/test/data/testivysettings.xml
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/build.xml
    pig/trunk/conf/pig.properties
    pig/trunk/ivy.xml
    pig/trunk/ivy/pig-template.xml
    pig/trunk/src/org/apache/pig/Main.java
    pig/trunk/src/org/apache/pig/PigConfiguration.java
    pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
    pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Fri Nov 20 01:17:02 2015
@@ -24,6 +24,8 @@ INCOMPATIBLE CHANGES
 
 IMPROVEMENTS
 
+PIG-4417: Pig's register command should support automatic fetching of jars from repo (akshayrai09 via daijy)
+
 PIG-4713: Document Bloom UDF (gliptak via daijy)
 
 PIG-3251: Bzip2TextInputFormat requires double the memory of maximum record size (knoguchi)

Modified: pig/trunk/build.xml
URL: http://svn.apache.org/viewvc/pig/trunk/build.xml?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/build.xml (original)
+++ pig/trunk/build.xml Fri Nov 20 01:17:02 2015
@@ -731,6 +731,7 @@
             <fileset dir="${ivy.lib.dir}" includes="kryo-*.jar"/>
             <fileset dir="${ivy.lib.dir}" includes="httpdlog-*-${basjes-httpdlog-pigloader.version}.jar"/>
             <fileset dir="${ivy.lib.dir}" includes="parser-core-${basjes-httpdlog-pigloader.version}.jar"/>
+            <fileset dir="${ivy.lib.dir}" includes="ivy-*.jar"/>
         </copy>
     </target>
 
@@ -813,6 +814,7 @@
                 <zipgroupfileset refid="@{includedJars}" />
                 <fileset file="${basedir}/src/pig-default.properties" />
                 <fileset file="${basedir}/src/main/jruby/pigudf.rb" />
+                <fileset file="${basedir}/conf/ivysettings.xml" />
                 <exclude name="hadoop-site.xml" />
             </jar>
         </sequential>

Added: pig/trunk/conf/ivysettings.xml
URL: http://svn.apache.org/viewvc/pig/trunk/conf/ivysettings.xml?rev=1715312&view=auto
==============================================================================
--- pig/trunk/conf/ivysettings.xml (added)
+++ pig/trunk/conf/ivysettings.xml Fri Nov 20 01:17:02 2015
@@ -0,0 +1,36 @@
+<!--
+  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.
+-->
+
+<!--
+  This file is used by grapes to download dependencies from a maven repository.
+  This is just a template and can be edited to add more repositories.
+-->
+<ivysettings>
+  <!--name of the defaultResolver should always be 'downloadGrapes'. -->
+  <settings defaultResolver="downloadGrapes"/>
+  <resolvers>
+    <!-- more resolvers can be added here -->
+    <chain name="downloadGrapes">
+      <!-- This resolver uses ibiblio to find artifacts, compatible with maven2 repository -->
+      <ibiblio name="central" m2compatible="true"/>
+      <!-- File resolver to add jars from the local system. -->
+      <filesystem name="test" checkmodified="true">
+        <artifact pattern="/tmp/[module]-[revision](-[classifier]).jar"/>
+      </filesystem>
+    </chain>
+  </resolvers>
+</ivysettings>

Modified: pig/trunk/conf/pig.properties
URL: http://svn.apache.org/viewvc/pig/trunk/conf/pig.properties?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/conf/pig.properties (original)
+++ pig/trunk/conf/pig.properties Fri Nov 20 01:17:02 2015
@@ -162,6 +162,12 @@
 #
 # pig.datetime.default.tz=
 
+# Path to download the artifacts when registering ivy coordinates. This defaults
+# to the directory grape uses for downloading libraries.
+# (default: ~/.groovy/grapes)
+#
+# pig.artifacts.download.location=
+
 ############################################################################
 #
 # Memory impacting properties

Modified: pig/trunk/ivy.xml
URL: http://svn.apache.org/viewvc/pig/trunk/ivy.xml?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/ivy.xml (original)
+++ pig/trunk/ivy.xml Fri Nov 20 01:17:02 2015
@@ -241,6 +241,8 @@
       conf="releaseaudit->default"/>
     <dependency org="org.codehaus.groovy" name="groovy-all" rev="${groovy.version}"
       conf="compile->master"/>
+    <dependency org="org.apache.ivy" name="ivy" rev="${ivy.version}"
+      conf="compile->master"/>
     <dependency org="org.codehaus.jackson" name="jackson-mapper-asl" rev="${jackson.version}"
       conf="compile->master"/>
     <dependency org="org.codehaus.jackson" name="jackson-core-asl" rev="${jackson.version}"

Modified: pig/trunk/ivy/pig-template.xml
URL: http://svn.apache.org/viewvc/pig/trunk/ivy/pig-template.xml?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/ivy/pig-template.xml (original)
+++ pig/trunk/ivy/pig-template.xml Fri Nov 20 01:17:02 2015
@@ -175,5 +175,15 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ivy</groupId>
+      <artifactId>ivy</artifactId>
+      <version>2.2.0</version>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.groovy</groupId>
+      <artifactId>groovy-all</artifactId>
+      <version>1.8.6</version>
+    </dependency>
   </dependencies>
 </project>

Modified: pig/trunk/src/org/apache/pig/Main.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/Main.java?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/Main.java (original)
+++ pig/trunk/src/org/apache/pig/Main.java Fri Nov 20 01:17:02 2015
@@ -966,6 +966,8 @@ public class Main {
             System.out.println("        stop.on.failure=true|false; default is false. Set to true to terminate on the first error.");
             System.out.println("        pig.datetime.default.tz=<UTC time offset>. e.g. +08:00. Default is the default timezone of the host.");
             System.out.println("            Determines the timezone used to handle datetime datatype and UDFs. ");
+            System.out.println("        pig.artifacts.download.location=<path to download artifacts>; default is ~/.groovy/grapes");
+            System.out.println("            Determines the location to download the artifacts when registering jars using ivy coordinates.");
             System.out.println("Additionally, any Hadoop property can be specified.");
     }
 

Modified: pig/trunk/src/org/apache/pig/PigConfiguration.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigConfiguration.java?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigConfiguration.java (original)
+++ pig/trunk/src/org/apache/pig/PigConfiguration.java Fri Nov 20 01:17:02 2015
@@ -363,6 +363,10 @@ public class PigConfiguration {
      */
     public static final String PIG_BZIP_USE_HADOOP_INPUTFORMAT = "pig.bzip.use.hadoop.inputformat";
 
+    /**
+     * This key is used to set the download location when registering an artifact using ivy coordinate
+     */
+    public static final String PIG_ARTIFACTS_DOWNLOAD_LOCATION = "pig.artifacts.download.location";
 
     // Pig on Tez runtime settings
     /**

Modified: pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java Fri Nov 20 01:17:02 2015
@@ -315,32 +315,14 @@ public class QueryParserDriver {
         if (t.getText().equalsIgnoreCase(REGISTER_DEF)) {
             String path = t.getChild(0).getText();
             path = path.substring(1, path.length()-1);
-
-            if (path.endsWith(".jar")) {
-                if (t.getChildCount() != 1) {
-                    throw new ParserException("REGISTER statement refers to JAR but has a USING..AS scripting engine clause. " +
-                                              "Statement: " + t.toStringTree());
-                }
-
-                try {
-                    getPigServer().registerJar(path);
-                } catch (IOException ioe) {
-                    throw new ParserException(ioe.getMessage());
-                }
-            } else {
-                if (t.getChildCount() != 5) {
-                    throw new ParserException("REGISTER statement for non-JAR file requires a USING scripting_lang AS namespace clause. " +
-                                              "Ex. REGISTER 'my_file.py' USING jython AS my_jython_udfs;");
-                }
-
-                String scriptingLang = t.getChild(2).getText();
-                String namespace = t.getChild(4).getText();
-
-                try {
-                    getPigServer().registerCode(path, scriptingLang, namespace);
-                } catch (IOException ioe) {
-                    throw new ParserException(ioe.getMessage());
+            try {
+                if (t.getChildCount() == 5) {
+                    new RegisterResolver(getPigServer()).parseRegister(path, t.getChild(2).getText(), t.getChild(4).getText());
+                } else {
+                    new RegisterResolver(getPigServer()).parseRegister(path, null, null);
                 }
+            } catch (IOException ioe) {
+                throw new ParserException(ioe.getMessage());
             }
         } else {
             for (int i = 0; i < t.getChildCount(); i++) {

Added: pig/trunk/src/org/apache/pig/parser/RegisterResolver.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/RegisterResolver.java?rev=1715312&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/RegisterResolver.java (added)
+++ pig/trunk/src/org/apache/pig/parser/RegisterResolver.java Fri Nov 20 01:17:02 2015
@@ -0,0 +1,89 @@
+/*
+ * 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.parser;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.pig.PigServer;
+import org.apache.pig.tools.DownloadResolver;
+
+public class RegisterResolver {
+
+    private PigServer pigServer;
+
+    public RegisterResolver(PigServer pigServer) {
+	this.pigServer = pigServer;
+    }
+
+    /**
+     * @param path
+     * @param scriptingLang
+     * @param namespace
+     * @throws IOException
+     */
+    public void parseRegister(String path, String scriptingLang, String namespace) throws IOException {
+        try {
+            URI uri = new URI(path);
+            if (resolvesToJar(uri)) {
+                if (scriptingLang != null || namespace != null) {
+                    throw new ParserException("Cannot register a jar with a scripting language or namespace");
+                }
+                URI[] uriList = resolve(uri);
+                for (URI jarUri : uriList) {
+                    pigServer.registerJar(jarUri.toString());
+                }
+            } else {
+                pigServer.registerCode(path, scriptingLang, namespace);
+            }
+        } catch (URISyntaxException e) {
+            throw new ParserException("URI " + path + " is incorrect.", e);
+        }
+    }
+
+    /**
+     * @param uri
+     * @return List of URIs
+     * @throws IOException
+     */
+    public URI[] resolve(URI uri) throws IOException {
+        String scheme = uri.getScheme();
+        if (scheme != null) {
+            scheme = scheme.toLowerCase();
+        }
+        if (scheme == null || scheme.equals("file") || scheme.equals("hdfs")) {
+            return new URI[] { uri };
+        } else if (scheme.equals("ivy")) {
+            DownloadResolver downloadResolver = DownloadResolver.getInstance();
+            return downloadResolver.downloadArtifact(uri, pigServer);
+        } else {
+            throw new ParserException("Invalid Scheme: " + uri.getScheme());
+        }
+    }
+
+    /**
+     * @param uri
+     * @return True if the uri is a jar or an ivy coordinate
+     */
+    private boolean resolvesToJar(URI uri) {
+        String scheme = uri.getScheme();
+	return (uri.toString().endsWith("jar") || scheme != null && scheme.toLowerCase().equals("ivy"));
+    }
+
+}

Added: pig/trunk/src/org/apache/pig/tools/DownloadResolver.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/DownloadResolver.java?rev=1715312&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/DownloadResolver.java (added)
+++ pig/trunk/src/org/apache/pig/tools/DownloadResolver.java Fri Nov 20 01:17:02 2015
@@ -0,0 +1,202 @@
+/*
+ * 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.tools;
+
+import groovy.grape.Grape;
+import groovy.lang.GroovyClassLoader;
+
+import java.io.File;
+import java.net.URI;
+import java.util.LinkedList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.pig.PigConfiguration;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.parser.ParserException;
+
+public class DownloadResolver {
+
+    private static final String IVY_FILE_NAME = "ivysettings.xml";
+    private static final Log LOG = LogFactory.getLog(DownloadResolver.class);
+    private static DownloadResolver downloadResolver = new DownloadResolver();
+
+    private DownloadResolver() {
+        System.setProperty("groovy.grape.report.downloads", "true");
+
+        if (System.getProperty("grape.config") != null) {
+            LOG.info("Using ivysettings file from " + System.getProperty("grape.config"));
+        } else {
+            // Retrieve the ivysettings configuration file
+            Map<String, String> envMap = System.getenv();
+            File confFile = null;
+            // Check for configuration file in PIG_CONF_DIR
+            if (envMap.containsKey("PIG_CONF_DIR")) {
+                confFile = new File(new File(envMap.get("PIG_CONF_DIR")).getPath(), IVY_FILE_NAME);
+            }
+
+            // Check for configuration file in PIG_HOME if not found in PIG_CONF_DIR
+            if (confFile == null || !confFile.exists()) {
+                confFile = new File(new File(envMap.get("PIG_HOME"), "conf").getPath(), IVY_FILE_NAME);
+            }
+
+            // Check for configuration file in Classloader if not found in PIG_CONF_DIR and PIG_HOME
+            if (confFile == null || !confFile.exists()) {
+                ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+                if (classLoader.getResource(IVY_FILE_NAME) != null) {
+                    LOG.info("Found ivysettings file in classpath");
+                    confFile = new File(classLoader.getResource(IVY_FILE_NAME).getFile());
+
+                    if (!confFile.exists()) {
+                        // ivysettings file resides inside a jar
+                        try {
+                            List<String> ivyLines = IOUtils.readLines(classLoader.getResourceAsStream(IVY_FILE_NAME));
+                            confFile = File.createTempFile("ivysettings", ".xml");
+                            confFile.deleteOnExit();
+                            for(String str: ivyLines) {
+                                FileUtils.writeStringToFile(confFile, str, true);
+                            }
+                        } catch (Exception e) {
+                            LOG.warn("Could not create an ivysettings file from resource");
+                        }
+                    }
+                }
+            }
+
+            // Set the Configuration file
+            if (confFile != null && confFile.exists()) {
+                LOG.info("Using ivysettings file from " + confFile.toString());
+                System.setProperty("grape.config", confFile.toString());
+            } else {
+                LOG.warn("Could not find custom ivysettings file in PIG_CONF_DIR or PIG_HOME or classpath.");
+            }
+        }
+    }
+
+    /**
+     * @return Singleton Object
+     */
+    public static DownloadResolver getInstance() {
+        return downloadResolver;
+    }
+
+    /**
+     * @param uri
+     * @return A map of all Query String Parameters
+     */
+    private Map<String, String> parseQueryString(URI uri) {
+        Map<String, String> paramMap = new HashMap<String, String>();
+        String queryString = uri.getQuery();
+        if (queryString != null) {
+            String queryParams[] = queryString.split("&");
+            for (String param : queryParams) {
+                String[] parts = param.split("=");
+                if (parts.length == 2) {
+                    String name = parts[0].toLowerCase();
+                    String value = parts[1];
+                    paramMap.put(name, value);
+                }
+            }
+        }
+        return paramMap;
+    }
+
+    /**
+     * @param uri
+     * @return Returns a Map containing the organization, module, version and
+     *         all the query string parameters
+     * @throws ParserException
+     */
+    private Map<String, Object> parseUri(URI uri) throws ParserException {
+        // Parse uri for artifact organization, module and version
+        Map<String, Object> uriMap = new HashMap<String, Object>();
+        String authority = uri.getAuthority();
+        if (authority != null) {
+            String[] tokens = authority.split(":", -1);
+            if (tokens.length == 3) {
+                uriMap.put("org", tokens[0]);
+                if (tokens[1].isEmpty()) {
+                    throw new ParserException("Please specify the artifact module.");
+                }
+                uriMap.put("module", tokens[1]);
+                uriMap.put("version", tokens[2]);
+            } else {
+                throw new ParserException("Invalid Artifact. Please specify organization, module and version");
+            }
+        } else {
+            throw new ParserException("Invalid Artifact. Please specify organization, module and version");
+        }
+
+        // Parse query string for exclude list and other parameters
+        uriMap.putAll(parseQueryString(uri));
+        if (uriMap.containsKey("transitive")) {
+            uriMap.put("transitive", Boolean.parseBoolean(uriMap.get("transitive").toString()));
+        }
+        List<Map<String, Object>> excludeList = new LinkedList<Map<String, Object>>();
+        if (uriMap.containsKey("exclude")) {
+            for (String exclude : uriMap.get("exclude").toString().split(",")) {
+                Map<String, Object> excludeMap = new HashMap<String, Object>();
+                String parts[] = exclude.split(":", -1);
+                if (parts.length == 2) {
+                    excludeMap.put("group", parts[0]);
+                    excludeMap.put("module", parts[1]);
+                } else {
+                    throw new ParserException("Exclude must contain organization and module separated by a colon.");
+                }
+                excludeList.add(excludeMap);
+            }
+        }
+        uriMap.put("excludes", excludeList);
+
+        return uriMap;
+    }
+
+    /**
+     * @param uri
+     * @return List of URIs of the downloaded jars
+     * @throws ParserException
+     */
+    public URI[] downloadArtifact(URI uri, PigServer pigServer) throws ParserException {
+        Configuration conf = ConfigurationUtil.toConfiguration(pigServer.getPigContext().getProperties());
+
+        String artDownloadLocation = conf.get(PigConfiguration.PIG_ARTIFACTS_DOWNLOAD_LOCATION);
+
+        if (artDownloadLocation != null) {
+            LOG.info("Artifacts will be downloaded to " + artDownloadLocation);
+            System.setProperty("grape.root", artDownloadLocation);
+        } else if (System.getProperty("grape.root") != null) {
+            LOG.info("Artifacts will be downloaded to " + System.getProperty("grape.root"));
+        } else {
+            LOG.info("Artifacts will be downloaded to default location. Please check ~/.groovy/grapes");
+        }
+
+        Map<String, Object> uriMap = parseUri(uri);
+        Map<String, Object> args = new HashMap<String, Object>();
+        args.put("classLoader", new GroovyClassLoader());
+        args.put("excludes", uriMap.get("excludes"));
+        return Grape.resolve(args, uriMap);
+    }
+
+}

Modified: pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java?rev=1715312&r1=1715311&r2=1715312&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java (original)
+++ pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java Fri Nov 20 01:17:02 2015
@@ -66,6 +66,7 @@ import org.apache.pig.impl.io.FileLocali
 import org.apache.pig.impl.io.FileLocalizer.FetchFileRet;
 import org.apache.pig.impl.util.LogUtils;
 import org.apache.pig.impl.util.TupleFormat;
+import org.apache.pig.parser.RegisterResolver;
 import org.apache.pig.tools.pigscript.parser.ParseException;
 import org.apache.pig.tools.pigscript.parser.PigScriptParser;
 import org.apache.pig.tools.pigscript.parser.PigScriptParserTokenManager;
@@ -461,15 +462,7 @@ public class GruntParser extends PigScri
         path = parameterSubstitutionInGrunt(path);
         scriptingLang = parameterSubstitutionInGrunt(scriptingLang);
         namespace = parameterSubstitutionInGrunt(namespace);
-        if(path.endsWith(".jar")) {
-            if(scriptingLang != null || namespace != null) {
-                throw new ParseException("Cannot register a jar with a scripting language or namespace");
-            }
-            mPigServer.registerJar(path);
-        }
-        else {
-            mPigServer.registerCode(path, scriptingLang, namespace);
-        }
+        new RegisterResolver(mPigServer).parseRegister(path, scriptingLang, namespace);
     }
 
     private String runPreprocessor(String scriptPath, List<String> params, List<String> paramFiles)

Added: pig/trunk/test/org/apache/pig/test/TestRegisterParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestRegisterParser.java?rev=1715312&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestRegisterParser.java (added)
+++ pig/trunk/test/org/apache/pig/test/TestRegisterParser.java Fri Nov 20 01:17:02 2015
@@ -0,0 +1,146 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.InputStreamReader;
+import java.io.IOException;
+import java.io.Writer;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.pig.ExecType;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.PigServer;
+import org.apache.pig.parser.ParserException;
+import org.apache.pig.parser.RegisterResolver;
+import org.apache.pig.tools.grunt.Grunt;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestRegisterParser {
+    private PigServer pigServer;
+    private static final String TEST_JAR_DIR = "/tmp/";
+
+    @Before
+    public void setUp() throws Exception {
+	pigServer = new PigServer(ExecType.LOCAL);
+
+	// Generate test jar files
+	for (int i = 1; i <= 5; i++) {
+	    Writer output = null;
+	    String dataFile = TEST_JAR_DIR + "testjar-" + i + ".jar";
+	    File file = new File(dataFile);
+	    output = new BufferedWriter(new FileWriter(file));
+	    output.write("sample");
+	    output.close();
+	}
+    }
+
+    // Test to check if registering a jar using a file resolver adds the right jar to the classpath.
+    @Test
+    public void testRegisterArtifactWithFileResolver() throws Throwable {
+	PigContext context = pigServer.getPigContext();
+
+	File confFile = new File("test/org/apache/pig/test/data/testivysettings.xml");
+	System.setProperty("grape.config", confFile.toString());
+
+	// 'changing=true' tells Grape to re-fetch the jar rather than picking it from the groovy cache.
+	String strCmd = "register ivy://testgroup:testjar:1?changing=true\n";
+	ByteArrayInputStream cmd = new ByteArrayInputStream(strCmd.getBytes());
+	InputStreamReader reader = new InputStreamReader(cmd);
+
+	Grunt grunt = new Grunt(new BufferedReader(reader), context);
+	grunt.exec();
+
+	assertEquals(context.extraJars + " of size 1", 1, context.extraJars.size());
+	assertTrue(context.extraJars.get(0) + " ends with /testjar-1.jar",
+		context.extraJars.get(0).toString().endsWith("/testjar-1.jar"));
+    }
+
+    // Test to check if all dependencies are successfully added to the classpath
+    @Test
+    public void testRegisterArtifact() throws URISyntaxException, IOException, ParserException {
+	URI[] list = new URI[5];
+	list[0] = new URI(TEST_JAR_DIR + "testjar-1.jar");
+	list[1] = new URI(TEST_JAR_DIR + "testjar-2.jar");
+	list[2] = new URI(TEST_JAR_DIR + "testjar-3.jar");
+	list[3] = new URI(TEST_JAR_DIR + "testjar-4.jar");
+	list[4] = new URI(TEST_JAR_DIR + "testjar-5.jar");
+
+	// Make sure that the jars are not in the classpath
+	for (URI dependency : list) {
+	    Assert.assertFalse(pigServer.getPigContext().hasJar(dependency.toString()));
+	}
+
+	RegisterResolver registerResolver = Mockito.spy(new RegisterResolver(pigServer));
+	Mockito.doReturn(list).when(registerResolver).resolve(new URI("ivy://testQuery"));
+	registerResolver.parseRegister("ivy://testQuery", null, null);
+
+	for (URI dependency : list) {
+	    Assert.assertTrue(pigServer.getPigContext().hasJar(dependency.toString()));
+	}
+    }
+
+    // Throw error when a scripting language and namespace is specified for a jar
+    @Test(expected = ParserException.class)
+    public void testRegisterJarException1() throws IOException, ParserException {
+	new RegisterResolver(pigServer).parseRegister("test.jar", "jython", "myfunc");
+    }
+
+    // Throw error when a scripting language and namespace is specified for an ivy coordinate
+    @Test(expected = ParserException.class)
+    public void testRegisterJarException2() throws IOException, ParserException {
+	new RegisterResolver(pigServer).parseRegister("ivy://org:mod:ver", "jython", "myfunc");
+    }
+
+    // Throw error when a scripting language is specified for a jar
+    @Test(expected = ParserException.class)
+    public void testRegisterJarException3() throws IOException, ParserException {
+	new RegisterResolver(pigServer).parseRegister("test.jar", "jython", null);
+    }
+
+    // Throw error when an Illegal URI is passed
+    @Test(expected = ParserException.class)
+    public void testIllegalUriException() throws IOException, ParserException {
+	new RegisterResolver(pigServer).parseRegister("ivy:||org:mod:ver", null, null);
+    }
+
+    @After
+    public void close() {
+	// delete sample jars
+	for (int i = 1; i <= 5; i++) {
+	    String dataFile = TEST_JAR_DIR + "testjar-" + i + ".jar";
+
+	    File f = new File(dataFile);
+	    if (!f.delete()) {
+		throw new RuntimeException("Could not delete the data file");
+	    }
+	}
+    }
+}

Added: pig/trunk/test/org/apache/pig/test/data/testivysettings.xml
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/testivysettings.xml?rev=1715312&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/testivysettings.xml (added)
+++ pig/trunk/test/org/apache/pig/test/data/testivysettings.xml Fri Nov 20 01:17:02 2015
@@ -0,0 +1,31 @@
+<!--
+  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.
+-->
+
+<!--
+  This is a test ivy settings file with a file resolver.
+-->
+<ivysettings>
+  <settings defaultResolver="downloadGrapes"/>
+  <resolvers>
+    <chain name="downloadGrapes">
+      <!-- File resolver to add jars from the local system. -->
+      <filesystem name="test" checkmodified="true">
+        <artifact pattern="/tmp/[module]-[revision](-[classifier]).jar"/>
+      </filesystem>
+    </chain>
+  </resolvers>
+</ivysettings>