Details

    • Type: New Feature
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.6.0
    • Component/s: Sinks+Sources
    • Labels:
      None

      Description

      1. The idea is to enable authentication primarily by using SASL/GSSAPI/Kerberos with Thrift RPC. [Thrift already has support for SASL api that supports kerberos, so implementing right now for Thrift. For Avro RPC kerberos support, Avro needs to support SASL first for its Netty Server, before we can use it in flume]

      2. Authentication will happen hop to hop[Client to source, intermediate sources to sinks, final sink to destination].

      3. As per the initial model, the user principals won’t be carried forward. The flume client[ThriftRpcClient] will authenticate itself to the KDC. All the intermediate agents [Thrift Sources/Sinks] will authenticate as principal ‘flume’ (typically, but this can be any valid principal that KDC can autenticate) to each other and the final agent will authenticate to the destination as the principal it wishes to identify to the destination

      1. FLUME-2631.patch
        44 kB
        Johny Rufus
      2. FLUME-2631-1.patch
        80 kB
        Johny Rufus
      3. FLUME-2631-2.patch
        99 kB
        Johny Rufus
      4. FLUME-2631-5.patch
        106 kB
        Johny Rufus
      5. FLUME-2631-7.patch
        106 kB
        Johny Rufus

        Activity

        Hide
        jrufus Johny Rufus added a comment -

        Roshan Naik, The documentation is part of the Doc issue - https://issues.apache.org/jira/browse/FLUME-2630

        Show
        jrufus Johny Rufus added a comment - Roshan Naik , The documentation is part of the Doc issue - https://issues.apache.org/jira/browse/FLUME-2630
        Hide
        roshan_naik Roshan Naik added a comment -

        Johny Rufus
        Was there any documentation provided for this ? Unable to find in the patch.

        Show
        roshan_naik Roshan Naik added a comment - Johny Rufus Was there any documentation provided for this ? Unable to find in the patch.
        Hide
        hudson Hudson added a comment -

        FAILURE: Integrated in flume-trunk #719 (See https://builds.apache.org/job/flume-trunk/719/)
        FLUME-2631. End to End authentication in Flume (hshreedharan: http://git-wip-us.apache.org/repos/asf/flume/repo?p=flume.git&a=commit&h=542b1695033d330eb00ae81713fdc838b88332b6)

        • flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java
        • flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java
        • flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java
        • flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java
        • flume-ng-auth/pom.xml
        • flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java
        • flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java
        • pom.xml
        • flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java
        • flume-ng-dist/pom.xml
        • flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java
        • flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java
        • flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java
        • flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java
        • flume-ng-dist/src/main/assembly/bin.xml
        • flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java
        • flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java
        • flume-ng-core/pom.xml
        • flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java
        • flume-ng-dist/src/main/assembly/src.xml
        • flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java
        • flume-ng-sinks/flume-dataset-sink/pom.xml
        • flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java
        • flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java
        • flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java
        • flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java
        Show
        hudson Hudson added a comment - FAILURE: Integrated in flume-trunk #719 (See https://builds.apache.org/job/flume-trunk/719/ ) FLUME-2631 . End to End authentication in Flume (hshreedharan: http://git-wip-us.apache.org/repos/asf/flume/repo?p=flume.git&a=commit&h=542b1695033d330eb00ae81713fdc838b88332b6 ) flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java flume-ng-auth/pom.xml flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java pom.xml flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java flume-ng-dist/pom.xml flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java flume-ng-dist/src/main/assembly/bin.xml flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java flume-ng-core/pom.xml flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java flume-ng-dist/src/main/assembly/src.xml flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java flume-ng-sinks/flume-dataset-sink/pom.xml flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java
        Hide
        hudson Hudson added a comment -

        SUCCESS: Integrated in Flume-trunk-hbase-98 #76 (See https://builds.apache.org/job/Flume-trunk-hbase-98/76/)
        FLUME-2631. End to End authentication in Flume (hshreedharan: http://git-wip-us.apache.org/repos/asf/flume/repo?p=flume.git&a=commit&h=542b1695033d330eb00ae81713fdc838b88332b6)

        • flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java
        • flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java
        • flume-ng-auth/pom.xml
        • flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java
        • flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java
        • flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java
        • flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java
        • flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java
        • flume-ng-sinks/flume-dataset-sink/pom.xml
        • pom.xml
        • flume-ng-dist/pom.xml
        • flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java
        • flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java
        • flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java
        • flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java
        • flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java
        • flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java
        • flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java
        • flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java
        • flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java
        • flume-ng-core/pom.xml
        • flume-ng-dist/src/main/assembly/bin.xml
        • flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java
        • flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java
        • flume-ng-dist/src/main/assembly/src.xml
        • flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java
        Show
        hudson Hudson added a comment - SUCCESS: Integrated in Flume-trunk-hbase-98 #76 (See https://builds.apache.org/job/Flume-trunk-hbase-98/76/ ) FLUME-2631 . End to End authentication in Flume (hshreedharan: http://git-wip-us.apache.org/repos/asf/flume/repo?p=flume.git&a=commit&h=542b1695033d330eb00ae81713fdc838b88332b6 ) flume-ng-auth/src/main/java/org/apache/flume/api/SecureThriftRpcClient.java flume-ng-auth/src/main/java/org/apache/flume/api/SecureRpcClientFactory.java flume-ng-auth/pom.xml flume-ng-core/src/main/java/org/apache/flume/source/ThriftSource.java flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticator.java flume-ng-sinks/flume-dataset-sink/src/test/java/org/apache/flume/sink/kite/TestKerberosUtil.java flume-ng-sdk/src/main/java/org/apache/flume/api/RpcClientConfigurationConstants.java flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/HDFSEventSink.java flume-ng-auth/src/main/java/org/apache/flume/auth/SecurityException.java flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/DatasetSink.java flume-ng-auth/src/main/java/org/apache/flume/auth/PrivilegedExecutor.java flume-ng-auth/src/main/java/org/apache/flume/auth/UGIExecutor.java flume-ng-sinks/flume-dataset-sink/pom.xml pom.xml flume-ng-dist/pom.xml flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java flume-ng-sinks/flume-dataset-sink/src/main/java/org/apache/flume/sink/kite/KerberosUtil.java flume-ng-sinks/flume-hdfs-sink/src/main/java/org/apache/flume/sink/hdfs/BucketWriter.java flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSinkSecurityManager.java flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestHDFSEventSink.java flume-ng-core/src/main/java/org/apache/flume/sink/ThriftSink.java flume-ng-sinks/flume-ng-hbase-sink/src/main/java/org/apache/flume/sink/hbase/HBaseSink.java flume-ng-core/pom.xml flume-ng-dist/src/main/assembly/bin.xml flume-ng-sdk/src/main/java/org/apache/flume/api/ThriftRpcClient.java flume-ng-sinks/flume-hdfs-sink/src/test/java/org/apache/flume/sink/hdfs/TestBucketWriter.java flume-ng-dist/src/main/assembly/src.xml flume-ng-auth/src/main/java/org/apache/flume/auth/SimpleAuthenticator.java
        Hide
        hshreedharan Hari Shreedharan added a comment -

        Committed! Great work, Johny!

        Show
        hshreedharan Hari Shreedharan added a comment - Committed! Great work, Johny!
        Hide
        jira-bot ASF subversion and git services added a comment -

        Commit 1b0f051b610b5a102c869a9d06254258a3de898f in flume's branch refs/heads/flume-1.6 from Hari Shreedharan
        [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=1b0f051 ]

        FLUME-2631. End to End authentication in Flume

        (Johny Rufus via Hari)

        Show
        jira-bot ASF subversion and git services added a comment - Commit 1b0f051b610b5a102c869a9d06254258a3de898f in flume's branch refs/heads/flume-1.6 from Hari Shreedharan [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=1b0f051 ] FLUME-2631 . End to End authentication in Flume (Johny Rufus via Hari)
        Hide
        jira-bot ASF subversion and git services added a comment -

        Commit 542b1695033d330eb00ae81713fdc838b88332b6 in flume's branch refs/heads/trunk from Hari Shreedharan
        [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=542b169 ]

        FLUME-2631. End to End authentication in Flume

        (Johny Rufus via Hari)

        Show
        jira-bot ASF subversion and git services added a comment - Commit 542b1695033d330eb00ae81713fdc838b88332b6 in flume's branch refs/heads/trunk from Hari Shreedharan [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=542b169 ] FLUME-2631 . End to End authentication in Flume (Johny Rufus via Hari)
        Hide
        hshreedharan Hari Shreedharan added a comment -

        +1. Running tests now. Will commit once they pass.

        Show
        hshreedharan Hari Shreedharan added a comment - +1. Running tests now. Will commit once they pass.
        Hide
        jrufus Johny Rufus added a comment -

        Attached the latest version, with minor changes based on Ryan's comments

        Show
        jrufus Johny Rufus added a comment - Attached the latest version, with minor changes based on Ryan's comments
        Hide
        jrufus Johny Rufus added a comment -

        Updated the file in the review request, uploading here as well

        Show
        jrufus Johny Rufus added a comment - Updated the file in the review request, uploading here as well
        Hide
        jrufus Johny Rufus added a comment -

        the authenticators we return are singletons, so in order to support proxyAs, then we to include proxyAs as part of authenticators, and every time the user executes, he would have to use proxyAs, if there is a proxy username
        I think our initial idea was to avoid this, and get an executor in the beginning based on principal, keytab and proxyuser, and just use the executor from there on.

        Show
        jrufus Johny Rufus added a comment - the authenticators we return are singletons, so in order to support proxyAs, then we to include proxyAs as part of authenticators, and every time the user executes, he would have to use proxyAs, if there is a proxy username I think our initial idea was to avoid this, and get an executor in the beginning based on principal, keytab and proxyuser, and just use the executor from there on.
        Hide
        rdblue Ryan Blue added a comment -

        Can't we do both? If the Authenticator that is returned implements the FlumeExecutor interface then you can avoid the extra method and still make other authenticator methods public later.

        Show
        rdblue Ryan Blue added a comment - Can't we do both? If the Authenticator that is returned implements the FlumeExecutor interface then you can avoid the extra method and still make other authenticator methods public later.
        Hide
        jrufus Johny Rufus added a comment -

        @Ryan, I thought about having directly returning an executor, I still followed the original approach, if we want to support some other methods in the authenticator which the client needs access to, other than executor. But for now, its not needed, I agree.
        I named the Executor as FlumeExecutor, PriviligedExecutor also makes sense.

        Show
        jrufus Johny Rufus added a comment - @Ryan, I thought about having directly returning an executor, I still followed the original approach, if we want to support some other methods in the authenticator which the client needs access to, other than executor. But for now, its not needed, I agree. I named the Executor as FlumeExecutor, PriviligedExecutor also makes sense.
        Hide
        jrufus Johny Rufus added a comment -

        Comments from Ryan :

        Authenticator.get() - static

        (This returns a "not authenticated" executor instance?)

        Authenticator getExecutor(), getExecutorAs() - instance methods

        (Why not make the authenticator implement PrivilegedExecutor so you don't need getExecutor()?)

        Executor executePrivileged (action + exception action) - instance methods

        (Would it make sense to be a PrivilegedExecutor with an execute method? There is already a java.util.concurrent.Executor class, so the name should be different. You could even extend the concurrent one if you want.)

        SimpleAuthenticator extends Authenticator (private)

        (If you return only initialized PrivilegedExecutor objects, then you don't need this class. You can just return a the SimpleExecutor.)

        Show
        jrufus Johny Rufus added a comment - Comments from Ryan : Authenticator.get() - static (This returns a "not authenticated" executor instance?) Authenticator getExecutor(), getExecutorAs() - instance methods (Why not make the authenticator implement PrivilegedExecutor so you don't need getExecutor()?) Executor executePrivileged (action + exception action) - instance methods (Would it make sense to be a PrivilegedExecutor with an execute method? There is already a java.util.concurrent.Executor class, so the name should be different. You could even extend the concurrent one if you want.) SimpleAuthenticator extends Authenticator (private) (If you return only initialized PrivilegedExecutor objects, then you don't need this class. You can just return a the SimpleExecutor.)
        Hide
        jrufus Johny Rufus added a comment -

        Since the client api will be exposed via flume-ng-auth, we need more detailed specifics as to what should be exposed.
        Based on some internal discussions with Hari, Ryan we came up with the following structure for the client api.

        Only Exposed public Class + Api
        -------------------------------------

        Authenticator.get() - static
        Authenticator getExecutor(), getExecutorAs() - instance methods

        Executor executePrivileged (action + exception action) - instance methods

        Authenticator:
        -------------------
        static method:
        1. Authenticator.get(principal, keytab) : Authenticator
        Returns either of the two singletons SimpleAuthenticator or KerberosAuthenticator based on inputs
        Instance methods:
        2. getExecutor() : Executor
        returns SimpleExecutor or UgiExecutor , based on which Authenticator it was called on

        getExecutorAs(proxyUserName) : Executor
        returns UgiExecutor

        Executor
        ------------
        executePriviliged (one for action , one for exceptionaction)

        Class Hierarchy:
        ----------------------

        Authenticator (Exposed abstract base class)
        SimpleAuthenticator extends Authenticator (private)
        KerberosAuthenticator extends Authenticator (private)

        Executor(Exposed i/f)
        SimpleExecutor extends Executor (private)
        UgiExecutor extends Executor (private)

        Show
        jrufus Johny Rufus added a comment - Since the client api will be exposed via flume-ng-auth, we need more detailed specifics as to what should be exposed. Based on some internal discussions with Hari, Ryan we came up with the following structure for the client api. Only Exposed public Class + Api ------------------------------------- Authenticator.get() - static Authenticator getExecutor(), getExecutorAs() - instance methods Executor executePrivileged (action + exception action) - instance methods Authenticator: ------------------- static method: 1. Authenticator.get(principal, keytab) : Authenticator Returns either of the two singletons SimpleAuthenticator or KerberosAuthenticator based on inputs Instance methods: 2. getExecutor() : Executor returns SimpleExecutor or UgiExecutor , based on which Authenticator it was called on getExecutorAs(proxyUserName) : Executor returns UgiExecutor Executor ------------ executePriviliged (one for action , one for exceptionaction) Class Hierarchy: ---------------------- Authenticator (Exposed abstract base class) SimpleAuthenticator extends Authenticator (private) KerberosAuthenticator extends Authenticator (private) Executor(Exposed i/f) SimpleExecutor extends Executor (private) UgiExecutor extends Executor (private)
        Hide
        jrufus Johny Rufus added a comment -

        Attached patch with review suggestions

        Show
        jrufus Johny Rufus added a comment - Attached patch with review suggestions
        Hide
        hshreedharan Hari Shreedharan added a comment -

        You could add the secure RPC client to the same flume-ng-auth package itself and change it something else.

        Show
        hshreedharan Hari Shreedharan added a comment - You could add the secure RPC client to the same flume-ng-auth package itself and change it something else.
        Hide
        jrufus Johny Rufus added a comment -

        Thanks Hari/Jarcec. Will upload a patch with the secure sdk changes + hbase sink and kit sink changes to depend on flume-ng-auth

        Show
        jrufus Johny Rufus added a comment - Thanks Hari/Jarcec. Will upload a patch with the secure sdk changes + hbase sink and kit sink changes to depend on flume-ng-auth
        Hide
        jarcec Jarek Jarcec Cecho added a comment -

        That seems reasonable to me Hari Shreedharan!

        Show
        jarcec Jarek Jarcec Cecho added a comment - That seems reasonable to me Hari Shreedharan !
        Hide
        hshreedharan Hari Shreedharan added a comment -

        So here is one thought - we add a new module - flume-ng-secure-sdk (or something), which will have the secure thrift RPC client, which will extend from the normal thrift RPC client which will live in the flume-ng-sdk. This way we can keep the flume-ng-sdk from depending on hadoop, while the secure one will. This way, we can avoid users having to pull in hadoop if they don't care about security.

        We'd still be adding hadoop as a dependency for the flume-ng-core package, but that is still OK, I think - since that is meant to operate in its own JVM anyway.

        Show
        hshreedharan Hari Shreedharan added a comment - So here is one thought - we add a new module - flume-ng-secure-sdk (or something), which will have the secure thrift RPC client, which will extend from the normal thrift RPC client which will live in the flume-ng-sdk. This way we can keep the flume-ng-sdk from depending on hadoop, while the secure one will. This way, we can avoid users having to pull in hadoop if they don't care about security. We'd still be adding hadoop as a dependency for the flume-ng-core package, but that is still OK, I think - since that is meant to operate in its own JVM anyway.
        Hide
        jarcec Jarek Jarcec Cecho added a comment -

        (For the record, I don't particularly like depending on Hadoop common in Sqoop client either, I'm just saying that we did that)

        Show
        jarcec Jarek Jarcec Cecho added a comment - (For the record, I don't particularly like depending on Hadoop common in Sqoop client either, I'm just saying that we did that)
        Hide
        hshreedharan Hari Shreedharan added a comment -

        OK, I am still not convinced that pulling in Hadoop to the sdk is actually a good idea.

        Show
        hshreedharan Hari Shreedharan added a comment - OK, I am still not convinced that pulling in Hadoop to the sdk is actually a good idea.
        Hide
        jarcec Jarek Jarcec Cecho added a comment -

        Sqoop client is actually a dependency of the shell that exposes public Java APIs that users can use to talk to Sqoop 2 server from their Java applications I don't think that we have any active users yet, but considering how many requests we've seen for public Java API, I'm assuming that it will be used a lot.

        Show
        jarcec Jarek Jarcec Cecho added a comment - Sqoop client is actually a dependency of the shell that exposes public Java APIs that users can use to talk to Sqoop 2 server from their Java applications I don't think that we have any active users yet, but considering how many requests we've seen for public Java API, I'm assuming that it will be used a lot.
        Hide
        hshreedharan Hari Shreedharan added a comment -

        There is one fundamental difference though - Sqoop client (at least the shell) is a standalone client, while the flume sdk gets embedded in user app, so whatever gets pulled in is in user's classpath, not in a standalone application for flume.

        Show
        hshreedharan Hari Shreedharan added a comment - There is one fundamental difference though - Sqoop client (at least the shell) is a standalone client, while the flume sdk gets embedded in user app, so whatever gets pulled in is in user's classpath, not in a standalone application for flume.
        Hide
        jarcec Jarek Jarcec Cecho added a comment -

        We've recently added dependency on hadoop-common on Sqoop client side because hadoop-auth didn't contain all required code. Abraham Elmahrek have more context there and perhaps can help to see if we can limit the hadoop dependency here?

        Show
        jarcec Jarek Jarcec Cecho added a comment - We've recently added dependency on hadoop-common on Sqoop client side because hadoop-auth didn't contain all required code. Abraham Elmahrek have more context there and perhaps can help to see if we can limit the hadoop dependency here?
        Hide
        hshreedharan Hari Shreedharan added a comment -

        /cc Mike Percy, Jarek Jarcec Cecho - I'd like it if you guys took a look as well. The implementation seems reasonable, but what I worry about is the hadoop-common dependency (and its dependencies) getting pulled in.

        Show
        hshreedharan Hari Shreedharan added a comment - /cc Mike Percy , Jarek Jarcec Cecho - I'd like it if you guys took a look as well. The implementation seems reasonable, but what I worry about is the hadoop-common dependency (and its dependencies) getting pulled in.
        Hide
        jrufus Johny Rufus added a comment -

        Review request - https://reviews.apache.org/r/31339/
        Thanks.

        Show
        jrufus Johny Rufus added a comment - Review request - https://reviews.apache.org/r/31339/ Thanks.
        Hide
        hshreedharan Hari Shreedharan added a comment -

        Can you also upload it to reviewboard?

        Show
        hshreedharan Hari Shreedharan added a comment - Can you also upload it to reviewboard?
        Hide
        jrufus Johny Rufus added a comment -

        Attaching a patch enabling Thrift Src/Sink kerberos authentication

        Show
        jrufus Johny Rufus added a comment - Attaching a patch enabling Thrift Src/Sink kerberos authentication

          People

          • Assignee:
            jrufus Johny Rufus
            Reporter:
            jrufus Johny Rufus
          • Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development