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 2016/11/09 00:30:57 UTC

svn commit: r1768807 - in /pig/trunk: CHANGES.txt src/docs/src/documentation/content/xdocs/udf.xml src/org/apache/pig/NonFSLoadFunc.java src/org/apache/pig/parser/LogicalPlanBuilder.java test/org/apache/pig/parser/TestQueryParserUtils.java

Author: daijy
Date: Wed Nov  9 00:30:57 2016
New Revision: 1768807

URL: http://svn.apache.org/viewvc?rev=1768807&view=rev
Log:
PIG-4939: QueryParserUtils.setHdfsServers(QueryParserUtils.java:104) should not be called for non-dfs methods

Added:
    pig/trunk/src/org/apache/pig/NonFSLoadFunc.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/docs/src/documentation/content/xdocs/udf.xml
    pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java
    pig/trunk/test/org/apache/pig/parser/TestQueryParserUtils.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1768807&r1=1768806&r2=1768807&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Wed Nov  9 00:30:57 2016
@@ -24,6 +24,9 @@ INCOMPATIBLE CHANGES
  
 IMPROVEMENTS
 
+PIG-4939: QueryParserUtils.setHdfsServers(QueryParserUtils.java:104) should not be called for non-dfs
+  methods (szita via daijy)
+
 PIG-5034: Remove org.apache.hadoop.hive.serde2.objectinspector.primitive package (nkollar via daijy)
 
 PIG-5036: Remove biggish from e2e input dataset (daijy)

Modified: pig/trunk/src/docs/src/documentation/content/xdocs/udf.xml
URL: http://svn.apache.org/viewvc/pig/trunk/src/docs/src/documentation/content/xdocs/udf.xml?rev=1768807&r1=1768806&r2=1768807&view=diff
==============================================================================
--- pig/trunk/src/docs/src/documentation/content/xdocs/udf.xml (original)
+++ pig/trunk/src/docs/src/documentation/content/xdocs/udf.xml Wed Nov  9 00:30:57 2016
@@ -1038,6 +1038,9 @@ has methods to convert byte arrays to sp
 
 <li id="loadpredicatepushdown"><a href="http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/LoadPredicatePushdown.java?view=markup">LoadPredicatePushdown</a> 
  has the methods to push predicates to the loader. It is different than LoadMetadata.setPartitionFilter in that loader may load records which does not satisfy the predicates. In other words, predicates is only a hint. Note this interface is still in development and might change in next version. Currently only OrcStorage implements this interface.</li>
+
+<li id="nonfsloadfunc"><a href="http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/NonFSLoadFunc.java?view=markup">NonFSLoadFunc</a> 
+ is a marker interface to indicate that a LoadFunc implementation is not a filesystem loader. This is useful for LoadFunc classes that for example supply queries instead of filesystem pathes to the LOAD operator.</li>
 </ul>
 
  <p>The LoadFunc abstract class is the main class to extend for implementing a loader. The methods which need to be overridden are explained below:</p>

Added: pig/trunk/src/org/apache/pig/NonFSLoadFunc.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/NonFSLoadFunc.java?rev=1768807&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/NonFSLoadFunc.java (added)
+++ pig/trunk/src/org/apache/pig/NonFSLoadFunc.java Wed Nov  9 00:30:57 2016
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/**
+ * Marker interface to distinguish LoadFunc implementations that don't use file system sources.
+ */
+public interface NonFSLoadFunc {
+
+}

Modified: pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java?rev=1768807&r1=1768806&r2=1768807&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java (original)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java Wed Nov  9 00:30:57 2016
@@ -34,6 +34,7 @@ import org.antlr.runtime.RecognitionExce
 import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.LoadFunc;
+import org.apache.pig.NonFSLoadFunc;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.StoreFuncInterface;
 import org.apache.pig.backend.executionengine.ExecException;
@@ -888,7 +889,7 @@ public class LogicalPlanBuilder {
             if (absolutePath == null) {
                 absolutePath = loFunc.relativeToAbsolutePath( filename, QueryParserUtils.getCurrentDir( pigContext ) );
 
-                if (absolutePath!=null) {
+                if (absolutePath!=null && !(loFunc instanceof NonFSLoadFunc)) {
                     QueryParserUtils.setHdfsServers( absolutePath, pigContext );
                 }
                 fileNameMap.put( fileNameKey, absolutePath );

Modified: pig/trunk/test/org/apache/pig/parser/TestQueryParserUtils.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/parser/TestQueryParserUtils.java?rev=1768807&r1=1768806&r2=1768807&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/TestQueryParserUtils.java (original)
+++ pig/trunk/test/org/apache/pig/parser/TestQueryParserUtils.java Wed Nov  9 00:30:57 2016
@@ -19,10 +19,20 @@ package org.apache.pig.parser;
 
 import static org.junit.Assert.assertEquals;
 
+import java.io.IOException;
 import java.util.Properties;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.pig.ExecType;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.NonFSLoadFunc;
+import org.apache.pig.PigServer;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
+import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.test.Util;
 import org.junit.Test;
@@ -107,8 +117,44 @@ public class TestQueryParserUtils {
 
 
         }
+    }
 
 
+    @Test
+    public void testNonFSLoadFunc() throws Exception {
+        PigServer pigServer = new PigServer(Util.getLocalTestMode(), new Properties());
+        pigServer.registerQuery("A =  load 'hbase://query/SELECT ID, NAME, DATE FROM HIRES WHERE DATE > TO_DATE(\"1990-12-21 05:55:00.000\")' using org.apache.pig.parser.TestQueryParserUtils$DummyNonFSLoader();");
+        pigServer.shutdown();
     }
 
+    /**
+     * Test class for testNonFSLoadFuncNoSetHdfsServersCall test case
+     */
+    public static class DummyNonFSLoader extends LoadFunc implements NonFSLoadFunc {
+
+        @Override
+        public void setLocation(String location, Job job) throws IOException {
+            throw new RuntimeException("Should not be called");
+        }
+
+        @Override
+        public InputFormat getInputFormat() throws IOException {
+            throw new RuntimeException("Should not be called");
+        }
+
+        @Override
+        public void prepareToRead(RecordReader reader, PigSplit split) throws IOException {
+            throw new RuntimeException("Should not be called");
+        }
+
+        @Override
+        public Tuple getNext() throws IOException {
+            throw new RuntimeException("Should not be called");
+        }
+
+        @Override
+        public String relativeToAbsolutePath(String location, Path curDir) throws IOException {
+            return location;
+        }
+    }
 }