lvhu created MAPREDUCE-7450:
-------------------------------

             Summary: Set the record delimiter for the input file based on its 
path
                 Key: MAPREDUCE-7450
                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7450
             Project: Hadoop Map/Reduce
          Issue Type: Improvement
          Components: client
    Affects Versions: 3.3.6
         Environment: Any
            Reporter: lvhu
             Fix For: MR-3902


In the mapreduce program, when reading files, we can easily set the record 
delimiter based on the parameter textinputformat.record.delimiter.
This parameter can also be easily set, including Spark, for example:
spark.sparkContext.hadoopConfiguration.set("textinputformat.record.delimiter", 
"|@|")
val rdd = spark.sparkContext.newAPIHadoopFile(...)
But once the textinputformat.record.delimiter parameter is modified, it will 
take effect for all files. In actual scenarios, different files often have 
different delimiters.

In Hive, as Hive does not support programming, we cannot modify the record 
delimiter through the above methods. If modified through a configuration file, 
it will take effect on all Hive tables.
The only way to modify record delimiter in hive is to rewrite a TextInputFormat 
class.
The current method of hive is as follows:
package abc.hive.MyFstTextInputFormat
public class MyFstTextInputFormat extends FileInputFormat<LongWritable, Text> 
implements JobConfigurable {
 ...
}
create table test  (  
    id string,  
    name string  
)  stored as  
INPUTFORMAT 'abc.hive.MyFstTextInputFormat'  
If there are multiple different record delimiters, multiple TextInputFormats 
need to be rewritten.

My idea is to modify TextInputFormat class to support setting record delimiter 
for input files based on the prefix of the file path.
The specific idea is to make the following modifications to TextInputFormat:
public class TextInputFormat extends FileInputFormat<LongWritable, Text>
  implements JobConfigurable {
  ....
  public RecordReader<LongWritable, Text> getRecordReader(
                                          InputSplit genericSplit, JobConf job,
                                          Reporter reporter)
    throws IOException {
    
    reporter.setStatus(genericSplit.toString());
    // default delimiter
    String delimiter = job.get("textinputformat.record.delimiter");
    //Obtain the path of the file
    String filePath = genericSplit.getPath().toUri().getPath();
    //Obtain a list of file paths and delimiter relationships based on 
configuration file parameters
    Map pathToDelimiterMap = //Obtain by parsing the configuration file
    for(Map.Entry<String, String> entry: pathToDelimiterMap.entrySet()){
       //config path
       String configPath = entry.getKey();
       //if configPath is the prefix of filePath
       if(filePath.startsWith(configPath)){
         //Set delimiter corresponding to the file path
         delimiter = entry.getValue();
       }   
    });
    byte[] recordDelimiterBytes = null;
    if (null != delimiter) {
      recordDelimiterBytes = delimiter.getBytes(Charsets.UTF_8);
    }
    return new LineRecordReader(job, (FileSplit) genericSplit,
        recordDelimiterBytes);
  }
}

After implementing the record delimiter function of setting input files 
according to the path, not only does it save code to modify the delimiter, but 
it is also very convenient for Hadoop and Spark, without frequent parameter 
configuration modifications.

If you accept my idea, I hope you can assign the task to me. My Github account 
is: lvhu-goodluck
I really hope to contribute code to the community.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: mapreduce-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: mapreduce-dev-h...@hadoop.apache.org

Reply via email to