Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[STORM-2482] Refactor the Storm auto credential plugins to be more usable #2081

Merged
merged 2 commits into from May 2, 2017

Conversation

arunmahadevan
Copy link
Contributor

@arunmahadevan arunmahadevan commented Apr 21, 2017

  1. Create a new storm module storm-autocreds
  2. Move AutoHDFS and AutoHBase to storm-autocreds
  3. Refactor code and accepts config keys for customizing the hadoop configuration for the plugins
  4. Package the auto cred and dependency jars in the storm binary and deploy to external/storm-autocreds

…able

1. Create a new storm module storm-autocreds
2. Move AutoHDFS and AutoHBase to storm-autocreds
3. Refactor code and accepts config keys for customizing the hadoop configuration for the plugins
4. Package the auto cred and dependency jars in the storm binary and deploy to lib-autocreds
Copy link
Contributor

@harshach harshach left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1. LGTM.

nimbus.credential.renewers.classes should also be set to this value so that nimbus can periodically renew the TGT on behalf of the user.
To hide this from them in the common case plugins can be used to populate the credentials, unpack them on the other side into a java Subject, and also allow Nimbus to renew the credentials if needed. These are controlled by the following configs.

`topology.auto-credentials` is a list of java plugins, all of which must implement the `IAutoCredentials` interface, that populate the credentials on gateway
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is gateway here nimbus or namenode like entity or machine on which storm client will be run? If its nimbus, should we rename it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Gateway is the host from where the topology is submitted.


Storm will download the tokens separately for each of the clusters and populate it into the subject and also renew the tokens periodically.
This way it would be possible to run multiple bolts connecting to separate HBase cluster within the same topology.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just a few lines below it says "As nimbus is impersonating..... and it mentions storm.kerberos.principal. Is that right or should we change it to hbase.kerberos.principal ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it should be hbase.kerberos.principal. Will update.

}
}
LOG.info("Initializing UGI with config {}", configuration);
UserGroupInformation.setConfiguration(configuration);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will this work if tow hbase or hdfs bolts connecting to different clusters are running in the same jvm? I think this was always the problem but re-factoring in this PR aims to be able to connect to two different clusters. Just checking if issue still exists.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Never mind. It seems that this will always be executed in sequence for hdfs and hbase. So it will be overwriting the configuration and then login and get the tokens.

@priyank5485
Copy link
Contributor

+1(non-binding)

@Override
public void populateCredentials(Map<String, String> credentials, Map conf) {
try {
if (configKeys != null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

configKeys is initialised to ArrayList. we need to use configKeys.isEmpty() check here.


private Map<String, Object> updateConfigs(Map topologyConf) {
Map<String, Object> res = new HashMap<>(topologyConf);
if (configKeys != null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above


protected Set<Pair<String, Credentials>> getCredentials(Map<String, String> credentials) {
Set<Pair<String, Credentials>> res = new HashSet<>();
if (configKeys != null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above

@harshach
Copy link
Contributor

harshach commented May 1, 2017

still +1 after the above comments.

@asfgit asfgit merged commit 6d08e29 into apache:1.x-branch May 2, 2017
@harshach
Copy link
Contributor

harshach commented May 2, 2017

@arunmahadevan we need a PR for master

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
5 participants