You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@oozie.apache.org by Ryota Egashira <eg...@yahoo-inc.com> on 2015/04/21 19:17:05 UTC

Review Request 33400: OOZIE-2215 Support glob in FS EL function

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/
-----------------------------------------------------------

Review request for oozie.


Repository: oozie-git


Description
-------

https://issues.apache.org/jira/browse/OOZIE-2215


Diffs
-----

  core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
  core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
  docs/src/site/twiki/WorkflowFunctionalSpec.twiki e3790a4 

Diff: https://reviews.apache.org/r/33400/diff/


Testing
-------


Thanks,

Ryota Egashira


Re: Review Request 33400: OOZIE-2215 Support glob in FS EL function

Posted by Rohini Palaniswamy <ro...@gmail.com>.

> On April 23, 2015, 7:34 p.m., Rohini Palaniswamy wrote:
> > core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java, line 81
> > <https://reviews.apache.org/r/33400/diff/1/?file=938254#file938254line81>
> >
> >     Please get rid of this check. You have already done the globbing. To throw an error after that is meaningless.  NN and DistributedFileSystem are already built to handle this by fetching in batches. Need to remove this check everywhere - https://issues.apache.org/jira/browse/OOZIE-1682 .

Actually more efficient one is to go with the PathFilter suggestion by Puru. Throw some specific exception in the accept() method and catch the exception and return true. You don't have to do any limit check. Even if you get one Path that means it exists and you need to return ture. If it did not go into catch block, you return false.

This would minimize the calls to namenode to one listStatus call for first pattern. It would be good to avoid the multiple batch calls fs.listStatus (DistributedFileSystem.listStatusInternal) makes to NN in case of big list of paths, but that is not possible. Filed https://issues.apache.org/jira/browse/HDFS-8234 to fix it in hdfs code.


- Rohini


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/#review81348
-----------------------------------------------------------


On April 21, 2015, 5:17 p.m., Ryota Egashira wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/33400/
> -----------------------------------------------------------
> 
> (Updated April 21, 2015, 5:17 p.m.)
> 
> 
> Review request for oozie.
> 
> 
> Repository: oozie-git
> 
> 
> Description
> -------
> 
> https://issues.apache.org/jira/browse/OOZIE-2215
> 
> 
> Diffs
> -----
> 
>   core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
>   core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
>   docs/src/site/twiki/WorkflowFunctionalSpec.twiki e3790a4 
> 
> Diff: https://reviews.apache.org/r/33400/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Ryota Egashira
> 
>


Re: Review Request 33400: OOZIE-2215 Support glob in FS EL function

Posted by Rohini Palaniswamy <ro...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/#review81348
-----------------------------------------------------------



core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java
<https://reviews.apache.org/r/33400/#comment131694>

    Please get rid of this check. You have already done the globbing. To throw an error after that is meaningless.  NN and DistributedFileSystem are already built to handle this by fetching in batches. Need to remove this check everywhere - https://issues.apache.org/jira/browse/OOZIE-1682 .



core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java
<https://reviews.apache.org/r/33400/#comment131720>

    Redundant code. If globbing returns a list of paths, then it means something exists. All you need is
    
    FileStatus[] fileStatuses = fs.globStatus(path);
    
    return (fileStatuses != null and fileStatuses.length > 0);



docs/src/site/twiki/WorkflowFunctionalSpec.twiki
<https://reviews.apache.org/r/33400/#comment131721>

    Only need below in documentation
    
    It returns =true= or =false= depending if the specified path URI exists or not. If the URI is a glob pattern, it returns true if there is atleast one matching path.


- Rohini Palaniswamy


On April 21, 2015, 5:17 p.m., Ryota Egashira wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/33400/
> -----------------------------------------------------------
> 
> (Updated April 21, 2015, 5:17 p.m.)
> 
> 
> Review request for oozie.
> 
> 
> Repository: oozie-git
> 
> 
> Description
> -------
> 
> https://issues.apache.org/jira/browse/OOZIE-2215
> 
> 
> Diffs
> -----
> 
>   core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
>   core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
>   docs/src/site/twiki/WorkflowFunctionalSpec.twiki e3790a4 
> 
> Diff: https://reviews.apache.org/r/33400/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Ryota Egashira
> 
>


Re: Review Request 33400: OOZIE-2215 Support glob in FS EL function

Posted by Purshotam Shah <pu...@yahoo-inc.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/#review81345
-----------------------------------------------------------



core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java
<https://reviews.apache.org/r/33400/#comment131691>

    This can hog the NN if user uses a broad wildcard like **/**.
    
    You can use globStatus(Path pathPattern, PathFilter filter) in place of  globStatus(Path pathPattern)
    
    with custome filter.
    
    In customer filter, keep on checking the count of files it matches and throw exception if you are reaching the limit.


- Purshotam Shah


On April 21, 2015, 5:17 p.m., Ryota Egashira wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/33400/
> -----------------------------------------------------------
> 
> (Updated April 21, 2015, 5:17 p.m.)
> 
> 
> Review request for oozie.
> 
> 
> Repository: oozie-git
> 
> 
> Description
> -------
> 
> https://issues.apache.org/jira/browse/OOZIE-2215
> 
> 
> Diffs
> -----
> 
>   core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
>   core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
>   docs/src/site/twiki/WorkflowFunctionalSpec.twiki e3790a4 
> 
> Diff: https://reviews.apache.org/r/33400/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Ryota Egashira
> 
>


Re: Review Request 33400: OOZIE-2215 Support glob in FS EL function

Posted by Rohini Palaniswamy <ro...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/#review87085
-----------------------------------------------------------



core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java
<https://reviews.apache.org/r/33400/#comment139363>

    exists variable not required
    
    return (pathArr != null && pathArr.length > 0);



docs/src/site/twiki/WorkflowFunctionalSpec.twiki
<https://reviews.apache.org/r/33400/#comment139362>

    ...when there is at least one ...


- Rohini Palaniswamy


On June 1, 2015, 5:49 p.m., Ryota Egashira wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/33400/
> -----------------------------------------------------------
> 
> (Updated June 1, 2015, 5:49 p.m.)
> 
> 
> Review request for oozie.
> 
> 
> Repository: oozie-git
> 
> 
> Description
> -------
> 
> https://issues.apache.org/jira/browse/OOZIE-2215
> 
> 
> Diffs
> -----
> 
>   core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
>   core/src/main/resources/oozie-default.xml 8960073 
>   core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
>   docs/src/site/twiki/WorkflowFunctionalSpec.twiki 02dc65b 
> 
> Diff: https://reviews.apache.org/r/33400/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Ryota Egashira
> 
>


Re: Review Request 33400: OOZIE-2215 Support glob in FS EL function

Posted by Ryota Egashira <eg...@yahoo-inc.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/#review86855
-----------------------------------------------------------


this behavior is observed in hadoop-1 only,  fixed in hadoop-2

- Ryota Egashira


On June 1, 2015, 5:49 p.m., Ryota Egashira wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/33400/
> -----------------------------------------------------------
> 
> (Updated June 1, 2015, 5:49 p.m.)
> 
> 
> Review request for oozie.
> 
> 
> Repository: oozie-git
> 
> 
> Description
> -------
> 
> https://issues.apache.org/jira/browse/OOZIE-2215
> 
> 
> Diffs
> -----
> 
>   core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
>   core/src/main/resources/oozie-default.xml 8960073 
>   core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
>   docs/src/site/twiki/WorkflowFunctionalSpec.twiki 02dc65b 
> 
> Diff: https://reviews.apache.org/r/33400/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Ryota Egashira
> 
>


Re: Review Request 33400: OOZIE-2215 Support glob in FS EL function

Posted by Ryota Egashira <eg...@yahoo-inc.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/
-----------------------------------------------------------

(Updated June 8, 2015, 10:14 p.m.)


Review request for oozie.


Changes
-------

final one


Repository: oozie-git


Description
-------

https://issues.apache.org/jira/browse/OOZIE-2215


Diffs (updated)
-----

  core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
  core/src/main/resources/oozie-default.xml 8960073 
  core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
  docs/src/site/twiki/WorkflowFunctionalSpec.twiki 02dc65b 

Diff: https://reviews.apache.org/r/33400/diff/


Testing
-------


Thanks,

Ryota Egashira


Re: Review Request 33400: OOZIE-2215 Support glob in FS EL function

Posted by Ryota Egashira <eg...@yahoo-inc.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/33400/
-----------------------------------------------------------

(Updated June 1, 2015, 5:49 p.m.)


Review request for oozie.


Changes
-------

one weird thing in Pathfilter behavior is, 
fs.globStatus(path, new FSPathFilter());
when path is some non-existent single path (no glob expression),  PathFilter.accept() is called even though it does not exist. 
if thrwoing exception at first discovery, it breaks test case..  

this version is, not throwing excpetion at first discovery, but avoid duplicated check done in the orignal verison.  
also added glob max check, and to support normal datapipeline application, raised fs glob max to 50K.


Repository: oozie-git


Description
-------

https://issues.apache.org/jira/browse/OOZIE-2215


Diffs (updated)
-----

  core/src/main/java/org/apache/oozie/action/hadoop/FsELFunctions.java f8af61c 
  core/src/main/resources/oozie-default.xml 8960073 
  core/src/test/java/org/apache/oozie/action/hadoop/TestFsELFunctions.java c81fc61 
  docs/src/site/twiki/WorkflowFunctionalSpec.twiki 02dc65b 

Diff: https://reviews.apache.org/r/33400/diff/


Testing
-------


Thanks,

Ryota Egashira