You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/06/11 17:13:01 UTC

[jira] [Commented] (NIFI-4906) Add GetHdfsFileInfo Processor

    [ https://issues.apache.org/jira/browse/NIFI-4906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16508394#comment-16508394 ] 

ASF GitHub Bot commented on NIFI-4906:
--------------------------------------

Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2639#discussion_r194427937
  
    --- Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSFileInfo.java ---
    @@ -0,0 +1,803 @@
    +/*
    + * 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.nifi.processors.hadoop;
    +
    +import java.io.IOException;
    +import java.security.PrivilegedExceptionAction;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +import java.util.regex.Pattern;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.hadoop.fs.FileStatus;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.permission.FsAction;
    +import org.apache.hadoop.fs.permission.FsPermission;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.hadoop.GetHDFSFileInfo.HDFSFileInfoRequest.Groupping;
    +
    +@TriggerSerially
    +@TriggerWhenEmpty
    +@InputRequirement(Requirement.INPUT_ALLOWED)
    +@Tags({"hadoop", "HDFS", "get", "list", "ingest", "source", "filesystem"})
    +@CapabilityDescription("Retrieves a listing of files and directories from HDFS. "
    +        + "This processor creates a FlowFile(s) that represents the HDFS file/dir with relevant information. "
    +        + "Main purpose of this processor to provide functionality similar to HDFS Client, i.e. count, du, ls, test, etc. "
    +        + "Unlike ListHDFS, this processor is stateless, supports incoming connections and provides information on a dir level. "
    +        )
    +@WritesAttributes({
    +    @WritesAttribute(attribute="hdfs.objectName", description="The name of the file/dir found on HDFS."),
    +    @WritesAttribute(attribute="hdfs.path", description="The path is set to the absolute path of the object's parent directory on HDFS. "
    +            + "For example, if an object is a directory 'foo', under directory '/bar' then 'hdfs.objectName' will have value 'foo', and 'hdfs.path' will be '/bar'"),
    +    @WritesAttribute(attribute="hdfs.type", description="The type of an object. Possible values: directory, file, link"),
    +    @WritesAttribute(attribute="hdfs.owner", description="The user that owns the object in HDFS"),
    +    @WritesAttribute(attribute="hdfs.group", description="The group that owns the object in HDFS"),
    +    @WritesAttribute(attribute="hdfs.lastModified", description="The timestamp of when the object in HDFS was last modified, as milliseconds since midnight Jan 1, 1970 UTC"),
    +    @WritesAttribute(attribute="hdfs.length", description=""
    +            + "In case of files: The number of bytes in the file in HDFS.  "
    +            + "In case of dirs: Retuns storage space consumed by directory. "
    +            + ""),
    +    @WritesAttribute(attribute="hdfs.count.files", description="In case of type='directory' will represent total count of files under this dir. "
    +            + "Won't be populated to other types of HDFS objects. "),
    +    @WritesAttribute(attribute="hdfs.count.dirs", description="In case of type='directory' will represent total count of directories under this dir (including itself). "
    +            + "Won't be populated to other types of HDFS objects. "),
    +    @WritesAttribute(attribute="hdfs.replication", description="The number of HDFS replicas for the file"),
    +    @WritesAttribute(attribute="hdfs.permissions", description="The permissions for the object in HDFS. This is formatted as 3 characters for the owner, "
    +            + "3 for the group, and 3 for other users. For example rw-rw-r--"),
    +    @WritesAttribute(attribute="hdfs.status", description="The status contains comma separated list of file/dir paths, which couldn't be listed/accessed. "
    +            + "Status won't be set if no errors occured."),
    +    @WritesAttribute(attribute="hdfs.full.tree", description="When destination is 'attribute', will be populated with full tree of HDFS directory in JSON format.")
    +})
    +@SeeAlso({ListHDFS.class, GetHDFS.class, FetchHDFS.class, PutHDFS.class})
    +public class GetHDFSFileInfo extends AbstractHadoopProcessor {
    +    static final long LISTING_LAG_NANOS = TimeUnit.MILLISECONDS.toNanos(100L);
    --- End diff --
    
    Nitpick, but this is unused


> Add GetHdfsFileInfo Processor
> -----------------------------
>
>                 Key: NIFI-4906
>                 URL: https://issues.apache.org/jira/browse/NIFI-4906
>             Project: Apache NiFi
>          Issue Type: New Feature
>          Components: Extensions
>            Reporter: Ed Berezitsky
>            Assignee: Ed Berezitsky
>            Priority: Major
>              Labels: patch, pull-request-available
>         Attachments: NiFi-GetHDFSFileInfo.pdf, gethdfsfileinfo.patch
>
>
> Add *GetHdfsFileInfo* Processor to be able to get stats from a file system.
> This processor should support recursive scan, getting information of directories and files.
> _File-level info required_: name, path, length, modified timestamp, last access timestamp, owner, group, permissions.
> _Directory-level info required_: name, path, sum of lengths of files under a dir, count of files under a dir, modified timestamp, last access timestamp, owner, group, permissions.
>  
> The result returned:
>  * in single flow file (in content - a json line per file/dir info);
>  * flow file per each file/dir info (in content as json obj or in set of attributes by the choice).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)