-
Couldn't load subscription status.
- Fork 13.7k
[FLINK-2555] Properly pass security credentials in the Hadoop Input/Output format wrappers #1038
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
Conversation
|
Looks good. The HadoopFormatBase and similar classes could use a line or two more in comments, but otherwise, this seems well. Any way to test this? There does not seem to be any test for the format wrappers, yet... |
|
I actually think that there is no need for the There are some tests for the non secure case in |
|
There might actually be a way of testing against a secured cluster: https://issues.apache.org/jira/browse/HADOOP-9848 / https://github.com/apache/hadoop/blob/master/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java |
|
The Bases exist because there is a java-specific and a scals-specific version of each HadoopInputFormat. |
|
Okay, that makes sense. |
…utput format wrappers This is needed because the Hadoop IF/OF's are using Hadoop's FileSystem stack, which is using the security credentials passed in the JobConf / Job class in the getSplits() method. Note that access to secured Hadoop 1.x using Hadoop IF/OF's is not possible with this change. This limitation is due to missing methods in the old APIs.
|
@mxm: I removed the comment. |
|
It would be great if we implemented a test case against the MiniKDC server. |
|
I agree. Lets file a JIRA and do it separately, as this is probably a bigger task. |
|
I've opened another issue for that: https://issues.apache.org/jira/browse/FLINK-2573 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
whitespace
|
I'll address my trivial comment and merge this. Thanks! |
|
Thanks alot! |
…utput format wrappers This is needed because the Hadoop IF/OF's are using Hadoop's FileSystem stack, which is using the security credentials passed in the JobConf / Job class in the getSplits() method. Note that access to secured Hadoop 1.x using Hadoop IF/OF's is not possible with this change. This limitation is due to missing methods in the old APIs. - Add some comments & change dependency scope to test This closes #1038.
|
I'm manually closing this pull request. It has been merged by @uce. |
…utput format wrappers This is needed because the Hadoop IF/OF's are using Hadoop's FileSystem stack, which is using the security credentials passed in the JobConf / Job class in the getSplits() method. Note that access to secured Hadoop 1.x using Hadoop IF/OF's is not possible with this change. This limitation is due to missing methods in the old APIs. - Add some comments & change dependency scope to test This closes apache#1038.
This is needed because the Hadoop IF/OF's are using Hadoop's FileSystem stack, which is using the security credentials passed in the JobConf / Job class in the getSplits() method.
Note that access to secured Hadoop 1.x using Hadoop IF/OF's is not possible with this change. This limitation is due to missing methods in the old APIs.
I've also updated the version of the "de.javakaffee.kryo-serializers" from 0.27 to 0.36 because a user on the ML recently needed a specific Kryo serializer which was not available in the old dependency.
For the Java and Scala API, I renamed the first argument's name:
readHadoopFile(org.apache.hadoop.mapreduce.lib.input.FileInputFormat<K,V> mapreduceInputFormat, Class<K> key, Class<V> value, String inputPath, Job job)This makes it easier in IDE completions to distinguish between the mapreduce and the mapred variant. (before the argument was always called
mapredInputFormatnow, we have themapreduceInputFormatvariant where applicable)