HBASE-13897 OOM may occur when Import imports a row with too many KeyValues (Liu Junhong)

This commit is contained in:
tedyu 2015-07-08 10:21:00 -07:00
parent 4e84ac7924
commit 7ab78d9ddf
1 changed files with 189 additions and 3 deletions

View File

@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.mapreduce;
import java.io.ByteArrayInputStream;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
@ -30,11 +34,11 @@ import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
@ -58,11 +62,17 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.zookeeper.KeeperException;
@ -81,6 +91,155 @@ public class Import {
public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
public final static String TABLE_NAME = "import.table.name";
public final static String WAL_DURABILITY = "import.wal.durability";
public final static String HAS_LARGE_RESULT= "import.bulk.hasLargeResult";
public static class KeyValueWritableComparablePartitioner
extends Partitioner<KeyValueWritableComparable, KeyValue> {
private static KeyValueWritableComparable[] START_KEYS = null;
@Override
public int getPartition(KeyValueWritableComparable key, KeyValue value,
int numPartitions) {
for (int i = 0; i < START_KEYS.length; ++i) {
if (key.compareTo(START_KEYS[i]) <= 0) {
return i;
}
}
return START_KEYS.length;
}
}
public static class KeyValueWritableComparable
implements WritableComparable<KeyValueWritableComparable> {
private KeyValue kv = null;
private CellComparator cellComparator = new CellComparator();
static {
// register this comparator
WritableComparator.define(KeyValueWritableComparable.class,
new KeyValueWritableComparator());
}
public KeyValueWritableComparable() {
}
public KeyValueWritableComparable(KeyValue kv) {
this.kv = kv;
}
@Override
public void write(DataOutput out) throws IOException {
KeyValue.write(kv, out);
}
@Override
public void readFields(DataInput in) throws IOException {
kv = KeyValue.create(in);
}
@Override
public int compareTo(KeyValueWritableComparable o) {
return cellComparator.compare(this.kv, ((KeyValueWritableComparable)o).kv);
}
public static class KeyValueWritableComparator extends WritableComparator {
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
try {
KeyValueWritableComparable kv1 = new KeyValueWritableComparable();
kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1)));
KeyValueWritableComparable kv2 = new KeyValueWritableComparable();
kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2)));
return compare(kv1, kv2);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
}
public static class KeyValueReducer
extends
Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue> {
protected void reduce(
KeyValueWritableComparable row,
Iterable<KeyValue> kvs,
Reducer<KeyValueWritableComparable,
KeyValue, ImmutableBytesWritable, KeyValue>.Context context)
throws java.io.IOException, InterruptedException {
int index = 0;
for (KeyValue kv : kvs) {
context.write(new ImmutableBytesWritable(kv.getRowArray()), kv);
if (++index % 100 == 0)
context.setStatus("Wrote " + index + " KeyValues, "
+ "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
}
}
}
public static class KeyValueSortImporter
extends TableMapper<KeyValueWritableComparable, KeyValue> {
private Map<byte[], byte[]> cfRenameMap;
private Filter filter;
private static final Log LOG = LogFactory.getLog(KeyValueImporter.class);
/**
* @param row The current table row key.
* @param value The columns.
* @param context The current context.
* @throws IOException When something is broken with the data.
*/
@Override
public void map(ImmutableBytesWritable row, Result value, Context context)
throws IOException {
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Considering the row."
+ Bytes.toString(row.get(), row.getOffset(), row.getLength()));
}
if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
for (Cell kv : value.rawCells()) {
kv = filterKv(filter, kv);
// skip if we filtered it out
if (kv == null) continue;
// TODO get rid of ensureKeyValue
KeyValue ret = KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap));
context.write(new KeyValueWritableComparable(ret.createKeyOnly(false)), ret);
}
}
} catch (InterruptedException e) {
e.printStackTrace();
}
}
@Override
public void setup(Context context) throws IOException {
cfRenameMap = createCfRenameMap(context.getConfiguration());
filter = instantiateFilter(context.getConfiguration());
int reduceNum = context.getNumReduceTasks();
Configuration conf = context.getConfiguration();
TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME));
try (Connection conn = ConnectionFactory.createConnection(conf);
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
byte[][] startKeys = regionLocator.getStartKeys();
if (startKeys.length != reduceNum) {
throw new IOException("Region split after job initialization");
}
KeyValueWritableComparable[] startKeyWraps =
new KeyValueWritableComparable[startKeys.length - 1];
for (int i = 1; i < startKeys.length; ++i) {
startKeyWraps[i - 1] =
new KeyValueWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i]));
}
KeyValueWritableComparablePartitioner.START_KEYS = startKeyWraps;
}
}
}
/**
* A mapper that just writes out KeyValues.
@ -445,7 +604,31 @@ public class Import {
throw new IOException(e);
}
if (hfileOutPath != null) {
if (hfileOutPath != null && conf.getBoolean(HAS_LARGE_RESULT, false)) {
LOG.info("Use Large Result!!");
try (Connection conn = ConnectionFactory.createConnection(conf);
Table table = conn.getTable(tableName);
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
job.setMapperClass(KeyValueSortImporter.class);
job.setReducerClass(KeyValueReducer.class);
Path outputDir = new Path(hfileOutPath);
FileOutputFormat.setOutputPath(job, outputDir);
job.setMapOutputKeyClass(KeyValueWritableComparable.class);
job.setMapOutputValueClass(KeyValue.class);
job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
KeyValueWritableComparable.KeyValueWritableComparator.class,
RawComparator.class);
Path partitionsPath =
new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));
FileSystem fs = FileSystem.get(job.getConfiguration());
fs.deleteOnExit(partitionsPath);
job.setPartitionerClass(KeyValueWritableComparablePartitioner.class);
job.setNumReduceTasks(regionLocator.getStartKeys().length);
TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
com.google.common.base.Preconditions.class);
}
} else if (hfileOutPath != null) {
job.setMapperClass(KeyValueImporter.class);
try (Connection conn = ConnectionFactory.createConnection(conf);
Table table = conn.getTable(tableName);
@ -480,6 +663,9 @@ public class Import {
System.err.println("By default Import will load data directly into HBase. To instead generate");
System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
System.err.println("If there is a large result that includes too much KeyValue "
+ "whitch can occur OOME caused by the memery sort in reducer, pass the option:");
System.err.println(" -D" + HAS_LARGE_RESULT + "=true");
System.err
.println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");