MAPREDUCE-5231. Bring back a constructor in mapred's DBInputFormat.DBRecordReader for binary compatibility with 1.x mapred APIs. Contributed by Zhijie Shen.

svn merge --ignore-ancestry -c 1488436 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1488437 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-05-31 23:57:42 +00:00
parent 297e916f41
commit 75a96a140a
3 changed files with 26 additions and 13 deletions

View File

@ -118,6 +118,10 @@ Release 2.0.5-beta - UNRELEASED
MAPREDUCE-5275. Bring back a couple of APIs in mapreduce.security.TokenCache
for binary compatibility with 1.x mapreduce APIs. (Mayank Bansal via vinodkv)
MAPREDUCE-5231. Bring back a constructor in mapred's
DBInputFormat.DBRecordReader for binary compatibility with 1.x mapred APIs.
(Zhijie Shen via vinodkv)
OPTIMIZATIONS
MAPREDUCE-4974. Optimising the LineRecordReader initialize() method

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.mapreduce.Job;
@InterfaceAudience.Public
@InterfaceStability.Stable
@SuppressWarnings("deprecation")
public class DBInputFormat<T extends DBWritable>
extends org.apache.hadoop.mapreduce.lib.db.DBInputFormat<T>
implements InputFormat<LongWritable, T>, JobConfigurable {
@ -48,6 +49,17 @@ public class DBInputFormat<T extends DBWritable>
protected class DBRecordReader extends
org.apache.hadoop.mapreduce.lib.db.DBRecordReader<T>
implements RecordReader<LongWritable, T> {
/**
* The constructor is kept to be compatible with M/R 1.x
*
* @param split The InputSplit to read data for
* @throws SQLException
*/
protected DBRecordReader(DBInputSplit split, Class<T> inputClass,
JobConf job) throws SQLException {
super(split, inputClass, job, connection, dbConf, conditions, fieldNames, tableName);
}
/**
* @param split The InputSplit to read data for
* @throws SQLException
@ -152,7 +164,6 @@ public class DBInputFormat<T extends DBWritable>
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
public RecordReader<LongWritable, T> getRecordReader(InputSplit split,
JobConf job, Reporter reporter) throws IOException {

View File

@ -32,6 +32,10 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.InputFormat;
@ -41,11 +45,6 @@ import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
/**
* A InputFormat that reads input data from an SQL table.
* <p>
@ -62,7 +61,7 @@ public class DBInputFormat<T extends DBWritable>
private static final Log LOG = LogFactory.getLog(DBInputFormat.class);
private String dbProductName = "DEFAULT";
protected String dbProductName = "DEFAULT";
/**
* A Class that does nothing, implementing DBWritable
@ -144,15 +143,15 @@ public class DBInputFormat<T extends DBWritable>
}
}
private String conditions;
protected String conditions;
private Connection connection;
protected Connection connection;
private String tableName;
protected String tableName;
private String[] fieldNames;
protected String[] fieldNames;
private DBConfiguration dbConf;
protected DBConfiguration dbConf;
/** {@inheritDoc} */
public void setConf(Configuration conf) {
@ -230,7 +229,6 @@ public class DBInputFormat<T extends DBWritable>
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
public RecordReader<LongWritable, T> createRecordReader(InputSplit split,
TaskAttemptContext context) throws IOException, InterruptedException {