sqoop-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jarek Cecho" <jar...@apache.org>
Subject Re: Review Request 40625: SQOOP-2709 Sqoop2: HDFS: Impersonation on secured cluster doesn't work
Date Tue, 01 Dec 2015 00:01:50 GMT


> On Nov. 30, 2015, 5:58 p.m., Abraham Fine wrote:
> > connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsToInitializer.java,
line 62
> > <https://reviews.apache.org/r/40625/diff/1/?file=1137847#file1137847line62>
> >
> >     creating the proxy user and then generating the delegation tokens is a pattern
that we use twice. once in the hdfstoinitializer and again in the hdfsfrominitializer.
> >     
> >     perhaps, it could make sense to do something similar to what is being done when
we load the delegation tokens (`createProxyUserAndLoadDelegationTokens`) and have a method
`createProxyUserAndGenerateDelegationTokens`). that way we can make `generateDelegationTokens`
private, and never have to worry about it being called outside of a delegation block.

I've noticed that as well and I was thinking about creating a createProxyUserAndGenerateDelegationTokens.
The reasons why I did not do that eventually were:

1) The method generateDelegationTokens has to run under to doAs and I didn't want to have
two doAs blocks (one inside the createProxyUserAndGenerateDelegationTokens() and second in
the Initializers) - this one is kind of subjective, but the objective one is:

2) We're doing bunch of checks inside the initializer that should be done before we attemt
to create the delegation tokens (e.g. checking if given path exists and such). I don't want
us to generate error message while generating the delegation tokes if the HDFS URL or any
other related configs are invalid. That should fail as a first class failure and not hidden
in delegation token creation.


> On Nov. 30, 2015, 5:58 p.m., Abraham Fine wrote:
> > connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/security/SecurityUtils.java,
line 55
> > <https://reviews.apache.org/r/40625/diff/1/?file=1137848#file1137848line55>
> >
> >     i understand that we used `getLoginUser` previously. is this the correct way
to go instead of using `getCurrentUser`?

For our use case it doesn't matter because getLoginUser and getCurrentUser should return the
same username. I feel that getLoginUser() is a better and cleaner because that should always
gave us the username of a user who owns the kerberos ticket (the user that owns the process)
whereas getCurrentUser might be differnt if we're running under the doAs() already.


> On Nov. 30, 2015, 5:58 p.m., Abraham Fine wrote:
> > connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/security/SecurityUtils.java,
line 84
> > <https://reviews.apache.org/r/40625/diff/1/?file=1137848#file1137848line84>
> >
> >     how verbose is this, maybe it should be a debug?

I find it super helpful, so I would like to keep it at info.


> On Nov. 30, 2015, 5:58 p.m., Abraham Fine wrote:
> > connector/connector-hdfs/src/test/java/org/apache/sqoop/connector/hdfs/security/TestSecurityUtils.java,
line 27
> > <https://reviews.apache.org/r/40625/diff/1/?file=1137849#file1137849line27>
> >
> >     can we test writing the tokens to and reading them from the context?

I can add such test, but I feel that it won't add much value. We're already verifying that
we can reconstruct the token given serialized string and adding context in/out would be just
verifying that we will retrieve stored data which we already have tests for.

We should eventually add a real integration tests on kerberos that will test this functionality
end to end, but that is currently blocked by SQOOP-2704 that Dian is working on.


- Jarek


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


On Nov. 24, 2015, 12:58 a.m., Jarek Cecho wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/40625/
> -----------------------------------------------------------
> 
> (Updated Nov. 24, 2015, 12:58 a.m.)
> 
> 
> Review request for Sqoop.
> 
> 
> Bugs: SQOOP-2709
>     https://issues.apache.org/jira/browse/SQOOP-2709
> 
> 
> Repository: sqoop-sqoop2
> 
> 
> Description
> -------
> 
> I've provided util class that can retrieve delegation token for "current" user and store
it in our Context that is passed to execution engine.
> 
> 
> Diffs
> -----
> 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConstants.java
39ee4a3 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsExtractor.java
583acdd 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsFromInitializer.java
be837ca 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsLoader.java
04acd18 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsPartitioner.java
998b903 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsToDestroyer.java
2bad23a 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsToInitializer.java
5856371 
>   connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/security/SecurityUtils.java
PRE-CREATION 
>   connector/connector-hdfs/src/test/java/org/apache/sqoop/connector/hdfs/security/TestSecurityUtils.java
PRE-CREATION 
> 
> Diff: https://reviews.apache.org/r/40625/diff/
> 
> 
> Testing
> -------
> 
> I've tested the patch on secured real cluster to make sure that it's working. Sadly I
did not included any integration test as our suite currently doesn't have any support for
MiniKDC (this is something that we will add later).
> 
> 
> Thanks,
> 
> Jarek Cecho
> 
>


Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message