You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by bbende <gi...@git.apache.org> on 2016/03/15 20:00:48 UTC

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

GitHub user bbende opened a pull request:

    https://github.com/apache/nifi/pull/281

    NIFI-1488 Refactoring HBase Kerberos support

    - Storing UGI so we can support multiple HBaseClientServices with different configs
    - Creating nifi-hadoop-utils to hold utility code shared between HDFS and HBase processors
    - Incorporating KerberosProperties into existing hadoop processors

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/bbende/nifi NIFI-1488

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/281.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #281
    
----
commit 42766156c67542dc02d9ea9d86666418c7f86bd7
Author: Bryan Bende <bb...@apache.org>
Date:   2016-03-15T18:58:03Z

    NIFI-1488 Refactoring HBase Kerberos support
    - Storing UGI so we can support multiple HBaseClientServices with different configs
    - Creating nifi-hadoop-utils to hold utility code shared between HDFS and HBase processors
    - Incorporating KerberosProperties into existing hadoop processors

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56410960
  
    --- Diff: nifi-commons/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosTicketRenewer.java ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.hadoop;
    +
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.nifi.logging.ComponentLog;
    +
    +import java.io.IOException;
    +
    +/**
    + * Periodically attempts to renew the Kerberos user's ticket for the given UGI.
    + */
    +public class KerberosTicketRenewer implements Runnable {
    +
    +    private final UserGroupInformation ugi;
    +    private final long renewalPeriod;
    +    private final ComponentLog logger;
    +
    +    private volatile boolean stopped = false;
    +
    +    public KerberosTicketRenewer(final UserGroupInformation ugi, final long renewalPeriod, final ComponentLog logger) {
    +        this.ugi = ugi;
    +        this.renewalPeriod = renewalPeriod;
    +        this.logger = logger;
    +    }
    +
    +    @Override
    +    public void run() {
    +        stopped = false;
    +        while (!stopped) {
    +            try {
    +                logger.debug("Invoking renewal attempt for Kerberos ticket");
    +                // While we run this "frequently", the Hadoop implementation will only perform the login at 80% of ticket lifetime.
    +                ugi.checkTGTAndReloginFromKeytab();
    +            } catch (IOException e) {
    +                // Should failures to renew the ticket be retried more quickly?
    --- End diff --
    
    I definitely think they should. I believe 24 hours is a fairly common renewal period, but we don't want to wait 24 hours on failure. Perhaps provide a new constructor arg for retryPeriod?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56505883
  
    --- Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/resources/core-site.xml ---
    @@ -1,4 +1,5 @@
     <?xml version="1.0"?>
    +<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
    --- End diff --
    
    Any reason this was moved above the license header? I think the license header should be first.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56520899
  
    --- Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/resources/core-site.xml ---
    @@ -1,4 +1,5 @@
     <?xml version="1.0"?>
    +<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
    --- End diff --
    
    I only did this because I noticed the IDE said that "XML Declaration should proceed all documenting content", so it seemed like xml-stylesheet is another declaration like <?xml version> and should come before anything else.
    
    Technically I don't think it stops anything from working to put it back how it was, but just seemed like an easy thing to change if tools are going to recognize it as being invalid.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56412875
  
    --- Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java ---
    @@ -262,17 +250,15 @@ HdfsResources resetHDFSResources(String configResources, String dir, ProcessCont
                 UserGroupInformation ugi;
                 synchronized (RESOURCES_LOCK) {
                     if (config.get("hadoop.security.authentication").equalsIgnoreCase("kerberos")) {
    --- End diff --
    
    should this use the isSecurityEnabled method of SecurityUtil?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56506078
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/test/resources/hbase-site-security.xml ---
    @@ -0,0 +1,30 @@
    +<?xml version="1.0"?>
    +<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
    --- End diff --
    
    I think we should keep the License header above the xml-stylesheet directive, no?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the pull request:

    https://github.com/apache/nifi/pull/281#issuecomment-197888150
  
    @bbende code looks good. Just a few minor comments in-line (stylistic for the most part). There were a lot of files that were changed due to just white space and Java imports being reorganized, which made the review a bit harder but otherwise, the code all looks good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56506093
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/test/resources/hbase-site.xml ---
    @@ -0,0 +1,22 @@
    +<?xml version="1.0"?>
    +<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
    --- End diff --
    
    And here as well


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/281


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56411887
  
    --- Diff: nifi-commons/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/SecurityUtil.java ---
    @@ -0,0 +1,113 @@
    +/*
    + * 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.hadoop;
    +
    +import org.apache.commons.lang3.Validate;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.nifi.logging.ComponentLog;
    +
    +import java.io.IOException;
    +
    +/**
    + * Provides synchronized access to UserGroupInformation to avoid multiple processors/services from
    + * interfering with each other.
    + */
    +public class SecurityUtil {
    +
    +    /**
    +     * Initializes UserGroupInformation with the given Configuration and performs the login for the given principal
    +     * and keytab. All logins should happen through this class to ensure other threads are not concurrently modifying
    +     * UserGroupInformation.
    +     *
    +     * @param config the configuration instance
    +     * @param principal the principal to authenticate as
    +     * @param keyTab the keytab to authenticate with
    +     *
    +     * @return the UGI for the given principal
    +     *
    +     * @throws IOException if login failed
    +     */
    +    public static synchronized UserGroupInformation loginKerberos(final Configuration config, final String principal, final String keyTab)
    +            throws IOException {
    +        Validate.notNull(config);
    +        Validate.notNull(principal);
    +        Validate.notNull(keyTab);
    +
    +        UserGroupInformation.setConfiguration(config);
    +        return UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal.trim(), keyTab.trim());
    +    }
    +
    +    /**
    +     * Initializes UserGroupInformation with the given Configuration and returns UserGroupInformation.getLoginUser().
    +     * All logins should happen through this class to ensure other threads are not concurrently modifying
    +     * UserGroupInformation.
    +     *
    +     * @param config the configuration instance
    +     *
    +     * @return the UGI for the given principal
    +     *
    +     * @throws IOException if login failed
    +     */
    +    public static synchronized UserGroupInformation loginSimple(final Configuration config) throws IOException {
    +        Validate.notNull(config);
    +        UserGroupInformation.setConfiguration(config);
    +        return UserGroupInformation.getLoginUser();
    +    }
    +
    +    /**
    +     * Initializes UserGroupInformation with the given Configuration and returns UserGroupInformation.isSecurityEnabled().
    +     *
    +     * All checks for isSecurityEnabled() should happen through this method.
    +     *
    +     * @param config the given configuration
    +     *
    +     * @return true if kerberos is enabled on the given configuration, false otherwise
    +     *
    +     */
    +    public static synchronized boolean isSecurityEnabled(final Configuration config) {
    +        Validate.notNull(config);
    +        return "kerberos".equalsIgnoreCase(config.get("hadoop.security.authentication"));
    +    }
    +
    +    /**
    +     * Start a thread that periodically attempts to renew the current Kerberos user's ticket.
    +     *
    +     * Callers of this method should store the reference to the KerberosTicketRenewer and call stop() to stop the thread.
    +     *
    +     * @param clazz
    +     *          The class that this renewal is for (i.e. PutHDFS, etc)
    +     * @param ugi
    +     *          The current Kerberos user.
    +     * @param renewalPeriod
    +     *          The amount of time between attempting renewals.
    +     * @param logger
    +     *          The logger to use with in the renewer
    +     *
    +     * @return the KerberosTicketRenewer Runnable
    +     */
    +    public static KerberosTicketRenewer startTicketRenewalThread(final Class clazz, final UserGroupInformation ugi, final long renewalPeriod, final ComponentLog logger) {
    --- End diff --
    
    Given that the class is used only for the thread name, I think it would make more sense to just pass in a String for the name, no?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the pull request:

    https://github.com/apache/nifi/pull/281#issuecomment-198075844
  
    @bbende code looks good, thanks for addressing those few issues. +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1488 Refactoring HBase Kerberos support

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/281#discussion_r56410644
  
    --- Diff: nifi-commons/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java ---
    @@ -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.nifi.hadoop;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.util.NiFiProperties;
    +import org.apache.nifi.util.StringUtils;
    +
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/**
    + * All processors and controller services that need properties for Kerberos Principal and Keytab
    + * should obtain them through this class by calling:
    + *
    + * KerberosProperties props = KerberosProperties.create(NiFiProperties.getInstance())
    + *
    + * The properties can be accessed from the resulting KerberosProperties instance.
    + */
    +public class KerberosProperties {
    +
    +    private final File kerberosConfigFile;
    +    private final Validator kerberosConfigValidator;
    +    private final PropertyDescriptor kerberosPrincipal;
    +    private final PropertyDescriptor kerberosKeytab;
    +
    +    private KerberosProperties(final File kerberosConfigFile) {
    +        this.kerberosConfigFile = kerberosConfigFile;
    +
    +        if (this.kerberosConfigFile != null) {
    +            System.setProperty("java.security.krb5.conf", kerberosConfigFile.getAbsolutePath());
    +        }
    +
    +        this.kerberosConfigValidator = new Validator() {
    +            @Override
    +            public ValidationResult validate(String subject, String input, ValidationContext context) {
    +                // Check that the Kerberos configuration is set
    +                if (kerberosConfigFile == null) {
    +                    return new ValidationResult.Builder()
    +                            .subject(subject).input(input).valid(false)
    +                            .explanation("you are missing the nifi.kerberos.krb5.file property which "
    +                                    + "must be set in order to use Kerberos")
    +                            .build();
    +                }
    +
    +                // Check that the Kerberos configuration is readable
    +                if (!kerberosConfigFile.canRead()) {
    +                    return new ValidationResult.Builder().subject(subject).input(input).valid(false)
    +                            .explanation(String.format("unable to read Kerberos config [%s], please make sure the path is valid "
    +                                    + "and nifi has adequate permissions", kerberosConfigFile.getAbsoluteFile()))
    +                            .build();
    +                }
    +
    +                return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
    +            }
    +        };
    +
    +        this.kerberosPrincipal = new PropertyDescriptor.Builder()
    +                .name("Kerberos Principal")
    +                .required(false)
    +                .description("Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in your nifi.properties")
    +                .addValidator(kerberosConfigValidator)
    +                .build();
    +
    +        this.kerberosKeytab = new PropertyDescriptor.Builder()
    +                .name("Kerberos Keytab").required(false)
    +                .description("Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be set in your nifi.properties")
    +                .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +                .addValidator(kerberosConfigValidator)
    +                .build();
    +    }
    +
    +    public static KerberosProperties create(final NiFiProperties niFiProperties) {
    +        if (niFiProperties == null) {
    +            throw new IllegalArgumentException("NiFiProperties can not be null");
    +        }
    +        return new KerberosProperties(niFiProperties.getKerberosConfigurationFile());
    +    }
    +
    +    public File getKerberosConfigFile() {
    +        return kerberosConfigFile;
    +    }
    +
    +    public Validator getKerberosConfigValidator() {
    +        return kerberosConfigValidator;
    +    }
    +
    +    public PropertyDescriptor getKerberosPrincipal() {
    +        return kerberosPrincipal;
    +    }
    +
    +    public PropertyDescriptor getKerberosKeytab() {
    +        return kerberosKeytab;
    +    }
    +
    +    public static List<ValidationResult> validatePrincipalAndKeytab(final String subject, final Configuration config, final String principal, final String keytab, final ComponentLog logger) {
    +        final List<ValidationResult> results = new ArrayList<>();
    +
    +        // if security is enabled then the keytab and principal are required
    +        final boolean isSecurityEnabled = SecurityUtil.isSecurityEnabled(config);
    +
    +        if (isSecurityEnabled && StringUtils.isBlank(principal)) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject(subject)
    +                    .explanation("Kerberos Principal must be provided when using a secure HBase")
    --- End diff --
    
    I'm guessing this is due to some refactoring, but we should probably not mention using a secure HBase in a general KerberosProperties class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---