[ 
https://issues.apache.org/jira/browse/FLINK-4315?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15622479#comment-15622479
 ] 

ASF GitHub Bot commented on FLINK-4315:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2637#discussion_r85764470
  
    --- Diff: 
flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopInputs.java
 ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.hadoopcompatibility;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapreduce.Job;
    +
    +import java.io.IOException;
    +
    +/**
    + * The HadoopInputs is the utility class for create {@link 
HadoopInputFormat}.
    --- End diff --
    
    I would make the purpose of the class a bit more explicit.
    
    > HadoopInputs is a utility class to use Apache Hadoop InputFormats with 
Apache Flink.
    
    > It provides methods to create Flink `InputFormat` wrappers for Hadoop 
`org.apache.hadoop.mapred.InputFormat` and 
`org.apache.hadoop.mapreduce.InputFormat`.
    > Key value pairs produced by the Hadoop InputFormats are converted into 
Flink Tuple2 objects where the first field (`Tuple2.f0`) is the key and the 
second field (`Tuple2.f1`) is the value.


> Deprecate Hadoop dependent methods in flink-java
> ------------------------------------------------
>
>                 Key: FLINK-4315
>                 URL: https://issues.apache.org/jira/browse/FLINK-4315
>             Project: Flink
>          Issue Type: Task
>          Components: Java API
>            Reporter: Stephan Ewen
>            Assignee: Evgeny Kincharov
>             Fix For: 2.0.0
>
>
> The API projects should be independent of Hadoop, because Hadoop is not an 
> integral part of the Flink stack, and we should have the option to offer 
> Flink without Hadoop dependencies.
> The current batch APIs have a hard dependency on Hadoop, mainly because the 
> API has utility methods like `readHadoopFile(...)`.
> I suggest to deprecate those methods and add helpers in the 
> `flink-hadoop-compatibility` project.
> FLINK-4048 will later remove the deprecated methods.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to