GitHub user rmetzger opened a pull request:

    https://github.com/apache/flink/pull/1038

    [FLINK-2555] Properly pass security credentials in the Hadoop Input/Output 
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.
    
    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 
`mapredInputFormat` now, we have the `mapreduceInputFormat` variant where 
applicable)

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

    $ git pull https://github.com/rmetzger/flink flink2555

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

    https://github.com/apache/flink/pull/1038.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 #1038
    
----
commit bac21bf5d77c8e15c608ecbf006d29e7af1dd68a
Author: Aljoscha Krettek <[email protected]>
Date:   2015-07-23T13:12:38Z

    [FLINK-2398][api-breaking] Introduce StreamGraphGenerator
    
    This decouples the building of the StreamGraph from the API methods.
    Before the methods would build the StreamGraph as they go. Now the API
    methods build a hierachy of StreamTransformation nodes. From these a
    StreamGraph is generated upon execution.
    
    This also introduces some API breaking changes:
    
     - The result of methods that create sinks is now DataStreamSink instead
       of DataStream
     - Iterations cannot have feedback edges with differing parallelism
     - "Preserve partitioning" is not the default for feedback edges. The
       previous option for this is removed.
     - You can close an iteration several times, no need for a union.
     - Strict checking of whether partitioning and parallelism work
       together. I.e. if upstream and downstream parallelism don't match it
       is not legal to have Forward partitioning anymore. This was not very
       transparent: When you went from low parallelism to high dop some
       downstream  operators would never get any input. When you went from high
       parallelism to low dop you would get skew in the downstream operators
       because all elements that would be forwarded to an operator that is not
       "there" go to another operator. This requires insertion of global()
       or rebalance() in some places. For example with most sources which
       have parallelism one.
    
    This also makes StreamExecutionEnvironment.execute() behave consistently
    across different execution environments (local, remote ...): The list of
    operators to be executed are cleared after execute is called.

commit e4b72e6d0148d071a97d2dab5c3bd97b81ee97a5
Author: Robert Metzger <[email protected]>
Date:   2015-08-20T16:43:04Z

    [FLINK-2555] Properly pass security credentials in the Hadoop Input/Output 
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.

----


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to