HBASE-18649 Deprecate KV Usage in MR to move to Cells in 3.0 (Ram)
This commit is contained in:
parent
4475ba88c1
commit
66fb60d4a4
|
@ -24,22 +24,21 @@ import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.conf.Configured;
|
import org.apache.hadoop.conf.Configured;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
import org.apache.hadoop.hbase.mapreduce.CellSortReducer;
|
||||||
import org.apache.hadoop.hbase.mapreduce.HFileInputFormat;
|
import org.apache.hadoop.hbase.mapreduce.HFileInputFormat;
|
||||||
import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
|
import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
|
||||||
import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
|
|
||||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||||
import org.apache.hadoop.io.NullWritable;
|
import org.apache.hadoop.io.NullWritable;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
@ -47,6 +46,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
import org.apache.hadoop.util.Tool;
|
import org.apache.hadoop.util.Tool;
|
||||||
import org.apache.hadoop.util.ToolRunner;
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A tool to split HFiles into new region boundaries as a MapReduce job. The tool generates HFiles
|
* A tool to split HFiles into new region boundaries as a MapReduce job. The tool generates HFiles
|
||||||
|
@ -70,24 +70,15 @@ public class MapReduceHFileSplitterJob extends Configured implements Tool {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A mapper that just writes out cells. This one can be used together with
|
* A mapper that just writes out cells. This one can be used together with
|
||||||
* {@link KeyValueSortReducer}
|
* {@link CellSortReducer}
|
||||||
*/
|
*/
|
||||||
static class HFileCellMapper extends
|
static class HFileCellMapper extends
|
||||||
Mapper<NullWritable, KeyValue, ImmutableBytesWritable, KeyValue> {
|
Mapper<NullWritable, Cell, ImmutableBytesWritable, Cell> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void map(NullWritable key, KeyValue value, Context context) throws IOException,
|
public void map(NullWritable key, Cell value, Context context)
|
||||||
InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
// Convert value to KeyValue if subclass
|
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), new MapReduceCell(value));
|
||||||
if (!value.getClass().equals(KeyValue.class)) {
|
|
||||||
value =
|
|
||||||
new KeyValue(value.getRowArray(), value.getRowOffset(), value.getRowLength(),
|
|
||||||
value.getFamilyArray(), value.getFamilyOffset(), value.getFamilyLength(),
|
|
||||||
value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(),
|
|
||||||
value.getTimestamp(), Type.codeToType(value.getTypeByte()), value.getValueArray(),
|
|
||||||
value.getValueOffset(), value.getValueLength());
|
|
||||||
}
|
|
||||||
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -119,10 +110,10 @@ public class MapReduceHFileSplitterJob extends Configured implements Tool {
|
||||||
LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
|
LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
|
||||||
TableName tableName = TableName.valueOf(tabName);
|
TableName tableName = TableName.valueOf(tabName);
|
||||||
job.setMapperClass(HFileCellMapper.class);
|
job.setMapperClass(HFileCellMapper.class);
|
||||||
job.setReducerClass(KeyValueSortReducer.class);
|
job.setReducerClass(CellSortReducer.class);
|
||||||
Path outputDir = new Path(hfileOutPath);
|
Path outputDir = new Path(hfileOutPath);
|
||||||
FileOutputFormat.setOutputPath(job, outputDir);
|
FileOutputFormat.setOutputPath(job, outputDir);
|
||||||
job.setMapOutputValueClass(KeyValue.class);
|
job.setMapOutputValueClass(MapReduceCell.class);
|
||||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||||
Table table = conn.getTable(tableName);
|
Table table = conn.getTable(tableName);
|
||||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||||
|
|
|
@ -24,6 +24,7 @@ import static org.apache.hadoop.hbase.KeyValue.COLUMN_FAMILY_DELIMITER;
|
||||||
import static org.apache.hadoop.hbase.KeyValue.getDelimiter;
|
import static org.apache.hadoop.hbase.KeyValue.getDelimiter;
|
||||||
import static org.apache.hadoop.hbase.KeyValue.COLUMN_FAMILY_DELIM_ARRAY;
|
import static org.apache.hadoop.hbase.KeyValue.COLUMN_FAMILY_DELIM_ARRAY;
|
||||||
|
|
||||||
|
import java.io.DataOutput;
|
||||||
import java.io.DataOutputStream;
|
import java.io.DataOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
|
@ -1465,9 +1466,12 @@ public final class CellUtil {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Estimate based on keyvalue's serialization format.
|
* Estimate based on keyvalue's serialization format in the RPC layer. Note that there is an extra
|
||||||
|
* SIZEOF_INT added to the size here that indicates the actual length of the cell for cases where
|
||||||
|
* cell's are serialized in a contiguous format (For eg in RPCs).
|
||||||
* @param cell
|
* @param cell
|
||||||
* @return Estimate of the <code>cell</code> size in bytes.
|
* @return Estimate of the <code>cell</code> size in bytes plus an extra SIZEOF_INT indicating the
|
||||||
|
* actual cell length.
|
||||||
*/
|
*/
|
||||||
public static int estimatedSerializedSizeOf(final Cell cell) {
|
public static int estimatedSerializedSizeOf(final Cell cell) {
|
||||||
if (cell instanceof ExtendedCell) {
|
if (cell instanceof ExtendedCell) {
|
||||||
|
@ -1762,9 +1766,48 @@ public final class CellUtil {
|
||||||
* timestamp><1 byte type>
|
* timestamp><1 byte type>
|
||||||
* @param cell
|
* @param cell
|
||||||
* @param out
|
* @param out
|
||||||
|
* @deprecated Use {@link #writeFlatKey(Cell, DataOutput)}
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException {
|
public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException {
|
||||||
|
short rowLen = cell.getRowLength();
|
||||||
|
byte fLen = cell.getFamilyLength();
|
||||||
|
int qLen = cell.getQualifierLength();
|
||||||
|
// Using just one if/else loop instead of every time checking before writing every
|
||||||
|
// component of cell
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
out.writeShort(rowLen);
|
||||||
|
ByteBufferUtils.copyBufferToStream((DataOutput) out,
|
||||||
|
((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
|
||||||
|
rowLen);
|
||||||
|
out.writeByte(fLen);
|
||||||
|
ByteBufferUtils.copyBufferToStream((DataOutput) out,
|
||||||
|
((ByteBufferCell) cell).getFamilyByteBuffer(), ((ByteBufferCell) cell).getFamilyPosition(),
|
||||||
|
fLen);
|
||||||
|
ByteBufferUtils.copyBufferToStream((DataOutput) out,
|
||||||
|
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||||
|
((ByteBufferCell) cell).getQualifierPosition(), qLen);
|
||||||
|
} else {
|
||||||
|
out.writeShort(rowLen);
|
||||||
|
out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
|
||||||
|
out.writeByte(fLen);
|
||||||
|
out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
|
||||||
|
out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen);
|
||||||
|
}
|
||||||
|
out.writeLong(cell.getTimestamp());
|
||||||
|
out.writeByte(cell.getTypeByte());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Writes the Cell's key part as it would have serialized in a KeyValue. The format is <2 bytes
|
||||||
|
* rk len><rk><1 byte cf len><cf><qualifier><8 bytes
|
||||||
|
* timestamp><1 byte type>
|
||||||
|
* @param cell
|
||||||
|
* @param out
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static void writeFlatKey(Cell cell, DataOutput out) throws IOException {
|
||||||
short rowLen = cell.getRowLength();
|
short rowLen = cell.getRowLength();
|
||||||
byte fLen = cell.getFamilyLength();
|
byte fLen = cell.getFamilyLength();
|
||||||
int qLen = cell.getQualifierLength();
|
int qLen = cell.getQualifierLength();
|
||||||
|
@ -1790,6 +1833,69 @@ public final class CellUtil {
|
||||||
out.writeByte(cell.getTypeByte());
|
out.writeByte(cell.getTypeByte());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deep clones the given cell if the cell supports deep cloning
|
||||||
|
* @param cell the cell to be cloned
|
||||||
|
* @return the cloned cell
|
||||||
|
* @throws CloneNotSupportedException
|
||||||
|
*/
|
||||||
|
public static Cell deepClone(Cell cell) throws CloneNotSupportedException {
|
||||||
|
if (cell instanceof ExtendedCell) {
|
||||||
|
return ((ExtendedCell) cell).deepClone();
|
||||||
|
}
|
||||||
|
throw new CloneNotSupportedException();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Writes the cell to the given OutputStream
|
||||||
|
* @param cell the cell to be written
|
||||||
|
* @param out the outputstream
|
||||||
|
* @param withTags if tags are to be written or not
|
||||||
|
* @return the total bytes written
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static int writeCell(Cell cell, OutputStream out, boolean withTags) throws IOException {
|
||||||
|
if (cell instanceof ExtendedCell) {
|
||||||
|
return ((ExtendedCell) cell).write(out, withTags);
|
||||||
|
} else {
|
||||||
|
ByteBufferUtils.putInt(out, CellUtil.estimatedSerializedSizeOfKey(cell));
|
||||||
|
ByteBufferUtils.putInt(out, cell.getValueLength());
|
||||||
|
CellUtil.writeFlatKey(cell, out);
|
||||||
|
CellUtil.writeValue(out, cell, cell.getValueLength());
|
||||||
|
int tagsLength = cell.getTagsLength();
|
||||||
|
if (withTags) {
|
||||||
|
byte[] len = new byte[Bytes.SIZEOF_SHORT];
|
||||||
|
Bytes.putAsShort(len, 0, tagsLength);
|
||||||
|
out.write(len);
|
||||||
|
if (tagsLength > 0) {
|
||||||
|
CellUtil.writeTags(out, cell, tagsLength);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
int lenWritten = (2 * Bytes.SIZEOF_INT) + CellUtil.estimatedSerializedSizeOfKey(cell)
|
||||||
|
+ cell.getValueLength();
|
||||||
|
if (withTags) {
|
||||||
|
lenWritten += Bytes.SIZEOF_SHORT + tagsLength;
|
||||||
|
}
|
||||||
|
return lenWritten;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Writes a cell to the buffer at the given offset
|
||||||
|
* @param cell the cell to be written
|
||||||
|
* @param buf the buffer to which the cell has to be wrriten
|
||||||
|
* @param offset the offset at which the cell should be written
|
||||||
|
*/
|
||||||
|
public static void writeCellToBuffer(Cell cell, ByteBuffer buf, int offset) {
|
||||||
|
if (cell instanceof ExtendedCell) {
|
||||||
|
((ExtendedCell) cell).write(buf, offset);
|
||||||
|
} else {
|
||||||
|
// Using the KVUtil
|
||||||
|
byte[] bytes = KeyValueUtil.copyToNewByteArray(cell);
|
||||||
|
ByteBufferUtils.copyFromArrayToBuffer(buf, offset, bytes, 0, bytes.length);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static int writeFlatKey(Cell cell, OutputStream out) throws IOException {
|
public static int writeFlatKey(Cell cell, OutputStream out) throws IOException {
|
||||||
short rowLen = cell.getRowLength();
|
short rowLen = cell.getRowLength();
|
||||||
byte fLen = cell.getFamilyLength();
|
byte fLen = cell.getFamilyLength();
|
||||||
|
@ -1844,7 +1950,7 @@ public final class CellUtil {
|
||||||
public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
|
public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
|
||||||
int commonPrefix) throws IOException {
|
int commonPrefix) throws IOException {
|
||||||
if (cell instanceof ByteBufferCell) {
|
if (cell instanceof ByteBufferCell) {
|
||||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
ByteBufferUtils.copyBufferToStream((DataOutput)out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||||
((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
|
((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
|
||||||
} else {
|
} else {
|
||||||
out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
|
out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
|
||||||
|
@ -1894,7 +2000,7 @@ public final class CellUtil {
|
||||||
public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell,
|
public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell,
|
||||||
int qlength, int commonPrefix) throws IOException {
|
int qlength, int commonPrefix) throws IOException {
|
||||||
if (cell instanceof ByteBufferCell) {
|
if (cell instanceof ByteBufferCell) {
|
||||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
ByteBufferUtils.copyBufferToStream((DataOutput)out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||||
((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
|
((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
|
||||||
} else {
|
} else {
|
||||||
out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
|
out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.hadoop.hbase.io.encoding;
|
package org.apache.hadoop.hbase.io.encoding;
|
||||||
|
|
||||||
import java.io.DataInputStream;
|
import java.io.DataInputStream;
|
||||||
|
import java.io.DataOutput;
|
||||||
import java.io.DataOutputStream;
|
import java.io.DataOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -262,7 +263,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
||||||
ByteBufferUtils.putCompressedInt(out, kLength);
|
ByteBufferUtils.putCompressedInt(out, kLength);
|
||||||
ByteBufferUtils.putCompressedInt(out, vLength);
|
ByteBufferUtils.putCompressedInt(out, vLength);
|
||||||
ByteBufferUtils.putCompressedInt(out, 0);
|
ByteBufferUtils.putCompressedInt(out, 0);
|
||||||
CellUtil.writeFlatKey(cell, out);
|
CellUtil.writeFlatKey(cell, (DataOutput)out);
|
||||||
// Write the value part
|
// Write the value part
|
||||||
CellUtil.writeValue(out, cell, cell.getValueLength());
|
CellUtil.writeValue(out, cell, cell.getValueLength());
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.hadoop.hbase.io.encoding;
|
package org.apache.hadoop.hbase.io.encoding;
|
||||||
|
|
||||||
import java.io.DataInputStream;
|
import java.io.DataInputStream;
|
||||||
|
import java.io.DataOutput;
|
||||||
import java.io.DataOutputStream;
|
import java.io.DataOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -59,7 +60,7 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
||||||
ByteBufferUtils.putCompressedInt(out, klength);
|
ByteBufferUtils.putCompressedInt(out, klength);
|
||||||
ByteBufferUtils.putCompressedInt(out, vlength);
|
ByteBufferUtils.putCompressedInt(out, vlength);
|
||||||
ByteBufferUtils.putCompressedInt(out, 0);
|
ByteBufferUtils.putCompressedInt(out, 0);
|
||||||
CellUtil.writeFlatKey(cell, out);
|
CellUtil.writeFlatKey(cell, (DataOutput)out);
|
||||||
} else {
|
} else {
|
||||||
// find a common prefix and skip it
|
// find a common prefix and skip it
|
||||||
int common = CellUtil.findCommonPrefixInFlatKey(cell, state.prevCell, true, true);
|
int common = CellUtil.findCommonPrefixInFlatKey(cell, state.prevCell, true, true);
|
||||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
import java.io.DataInput;
|
import java.io.DataInput;
|
||||||
import java.io.DataInputStream;
|
import java.io.DataInputStream;
|
||||||
|
import java.io.DataOutput;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
|
@ -193,6 +194,28 @@ public final class ByteBufferUtils {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Copy data from a buffer to an output stream. Does not update the position
|
||||||
|
* in the buffer.
|
||||||
|
* @param out the output stream to write bytes to
|
||||||
|
* @param in the buffer to read bytes from
|
||||||
|
* @param offset the offset in the buffer (from the buffer's array offset)
|
||||||
|
* to start copying bytes from
|
||||||
|
* @param length the number of bytes to copy
|
||||||
|
*/
|
||||||
|
public static void copyBufferToStream(DataOutput out, ByteBuffer in, int offset, int length)
|
||||||
|
throws IOException {
|
||||||
|
if (out instanceof ByteBufferWriter) {
|
||||||
|
((ByteBufferWriter) out).write(in, offset, length);
|
||||||
|
} else if (in.hasArray()) {
|
||||||
|
out.write(in.array(), in.arrayOffset() + offset, length);
|
||||||
|
} else {
|
||||||
|
for (int i = 0; i < length; ++i) {
|
||||||
|
out.write(toByte(in, offset + i));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static int putLong(OutputStream out, final long value,
|
public static int putLong(OutputStream out, final long value,
|
||||||
final int fitInBytes) throws IOException {
|
final int fitInBytes) throws IOException {
|
||||||
long tmpValue = value;
|
long tmpValue = value;
|
||||||
|
|
|
@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.DataOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.math.BigDecimal;
|
import java.math.BigDecimal;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -488,4 +490,119 @@ public class TestCellUtil {
|
||||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||||
assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
|
assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWriteCell() throws IOException {
|
||||||
|
byte[] r = Bytes.toBytes("row1");
|
||||||
|
byte[] f = Bytes.toBytes("cf1");
|
||||||
|
byte[] q1 = Bytes.toBytes("qual1");
|
||||||
|
byte[] q2 = Bytes.toBytes("qual2");
|
||||||
|
byte[] v = Bytes.toBytes("val1");
|
||||||
|
byte[] tags = Bytes.toBytes("tag1");
|
||||||
|
KeyValue kv = new KeyValue(r, f, q1, 0, q1.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||||
|
NonExtendedCell nonExtCell = new NonExtendedCell(kv);
|
||||||
|
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
||||||
|
int writeCell = CellUtil.writeCell(nonExtCell, os, true);
|
||||||
|
byte[] byteArray = os.toByteArray();
|
||||||
|
KeyValue res = new KeyValue(byteArray);
|
||||||
|
assertTrue(CellUtil.equals(kv, res));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class NonExtendedCell implements Cell {
|
||||||
|
private KeyValue kv;
|
||||||
|
|
||||||
|
public NonExtendedCell(KeyValue kv) {
|
||||||
|
this.kv = kv;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getRowArray() {
|
||||||
|
return this.kv.getRowArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getRowOffset() {
|
||||||
|
return this.kv.getRowOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public short getRowLength() {
|
||||||
|
return this.kv.getRowLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getFamilyArray() {
|
||||||
|
return this.kv.getFamilyArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getFamilyOffset() {
|
||||||
|
return this.kv.getFamilyOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte getFamilyLength() {
|
||||||
|
return this.kv.getFamilyLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getQualifierArray() {
|
||||||
|
return this.kv.getQualifierArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getQualifierOffset() {
|
||||||
|
return this.kv.getQualifierOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getQualifierLength() {
|
||||||
|
return this.kv.getQualifierLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getTimestamp() {
|
||||||
|
return this.kv.getTimestamp();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte getTypeByte() {
|
||||||
|
return this.kv.getTypeByte();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSequenceId() {
|
||||||
|
return this.kv.getSequenceId();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getValueArray() {
|
||||||
|
return this.kv.getValueArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueOffset() {
|
||||||
|
return this.kv.getValueOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueLength() {
|
||||||
|
return this.kv.getValueLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getTagsArray() {
|
||||||
|
return this.kv.getTagsArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getTagsOffset() {
|
||||||
|
return this.kv.getTagsOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getTagsLength() {
|
||||||
|
return this.kv.getTagsLength();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,96 @@
|
||||||
|
/**
|
||||||
|
* 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.hadoop.hbase.mapreduce;
|
||||||
|
|
||||||
|
import java.io.DataInputStream;
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.Cell;
|
||||||
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
|
import org.apache.hadoop.hbase.ExtendedCell;
|
||||||
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.io.serializer.Deserializer;
|
||||||
|
import org.apache.hadoop.io.serializer.Serialization;
|
||||||
|
import org.apache.hadoop.io.serializer.Serializer;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use to specify the type of serialization for the mappers and reducers
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
public class CellSerialization implements Serialization<Cell> {
|
||||||
|
@Override
|
||||||
|
public boolean accept(Class<?> c) {
|
||||||
|
return Cell.class.isAssignableFrom(c);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CellDeserializer getDeserializer(Class<Cell> t) {
|
||||||
|
return new CellDeserializer();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CellSerializer getSerializer(Class<Cell> c) {
|
||||||
|
return new CellSerializer();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class CellDeserializer implements Deserializer<Cell> {
|
||||||
|
private DataInputStream dis;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
this.dis.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public KeyValue deserialize(Cell ignore) throws IOException {
|
||||||
|
// I can't overwrite the passed in KV, not from a proto kv, not just yet. TODO
|
||||||
|
return KeyValueUtil.create(this.dis);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void open(InputStream is) throws IOException {
|
||||||
|
this.dis = new DataInputStream(is);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class CellSerializer implements Serializer<Cell> {
|
||||||
|
private DataOutputStream dos;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
this.dos.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void open(OutputStream os) throws IOException {
|
||||||
|
this.dos = new DataOutputStream(os);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void serialize(Cell kv) throws IOException {
|
||||||
|
dos.writeInt(CellUtil.estimatedSerializedSizeOf(kv) - Bytes.SIZEOF_INT);
|
||||||
|
CellUtil.writeCell(kv, dos, true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,60 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.hadoop.hbase.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.TreeSet;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.Cell;
|
||||||
|
import org.apache.hadoop.hbase.CellComparator;
|
||||||
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||||
|
import org.apache.hadoop.mapreduce.Reducer;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Emits sorted Cells.
|
||||||
|
* Reads in all Cells from passed Iterator, sorts them, then emits
|
||||||
|
* Cells in sorted order. If lots of columns per row, it will use lots of
|
||||||
|
* memory sorting.
|
||||||
|
* @see HFileOutputFormat2
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
public class CellSortReducer
|
||||||
|
extends Reducer<ImmutableBytesWritable, Cell, ImmutableBytesWritable, Cell> {
|
||||||
|
protected void reduce(ImmutableBytesWritable row, Iterable<Cell> kvs,
|
||||||
|
Reducer<ImmutableBytesWritable, Cell, ImmutableBytesWritable, Cell>.Context context)
|
||||||
|
throws java.io.IOException, InterruptedException {
|
||||||
|
TreeSet<Cell> map = new TreeSet<>(CellComparator.COMPARATOR);
|
||||||
|
for (Cell kv : kvs) {
|
||||||
|
try {
|
||||||
|
map.add(CellUtil.deepClone(kv));
|
||||||
|
} catch (CloneNotSupportedException e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
context.setStatus("Read " + map.getClass());
|
||||||
|
int index = 0;
|
||||||
|
for (Cell kv: map) {
|
||||||
|
context.write(row, new MapReduceCell(kv));
|
||||||
|
if (++index % 100 == 0) context.setStatus("Wrote " + index);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -139,7 +139,7 @@ public class CopyTable extends Configured implements Tool {
|
||||||
job.setNumReduceTasks(0);
|
job.setNumReduceTasks(0);
|
||||||
|
|
||||||
if (bulkload) {
|
if (bulkload) {
|
||||||
TableMapReduceUtil.initTableMapperJob(tableName, scan, Import.KeyValueImporter.class, null,
|
TableMapReduceUtil.initTableMapperJob(tableName, scan, Import.CellImporter.class, null,
|
||||||
null, job);
|
null, job);
|
||||||
|
|
||||||
// We need to split the inputs by destination tables so that output of Map can be bulk-loaded.
|
// We need to split the inputs by destination tables so that output of Map can be bulk-loaded.
|
||||||
|
|
|
@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||||
import org.apache.hadoop.io.NullWritable;
|
import org.apache.hadoop.io.NullWritable;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
import org.apache.hadoop.io.SequenceFile;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
|
@ -90,7 +91,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
|
import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -233,14 +233,13 @@ public class HFileOutputFormat2
|
||||||
private final Map<byte[], WriterLength> writers =
|
private final Map<byte[], WriterLength> writers =
|
||||||
new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||||
private byte[] previousRow = HConstants.EMPTY_BYTE_ARRAY;
|
private byte[] previousRow = HConstants.EMPTY_BYTE_ARRAY;
|
||||||
private final byte[] now = Bytes.toBytes(EnvironmentEdgeManager.currentTime());
|
private final long now = EnvironmentEdgeManager.currentTime();
|
||||||
private boolean rollRequested = false;
|
private boolean rollRequested = false;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void write(ImmutableBytesWritable row, V cell)
|
public void write(ImmutableBytesWritable row, V cell)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
Cell kv = cell;
|
||||||
|
|
||||||
// null input == user explicitly wants to flush
|
// null input == user explicitly wants to flush
|
||||||
if (row == null && kv == null) {
|
if (row == null && kv == null) {
|
||||||
rollWriters();
|
rollWriters();
|
||||||
|
@ -248,7 +247,7 @@ public class HFileOutputFormat2
|
||||||
}
|
}
|
||||||
|
|
||||||
byte[] rowKey = CellUtil.cloneRow(kv);
|
byte[] rowKey = CellUtil.cloneRow(kv);
|
||||||
long length = kv.getLength();
|
int length = (CellUtil.estimatedSerializedSizeOf(kv)) - Bytes.SIZEOF_INT;
|
||||||
byte[] family = CellUtil.cloneFamily(kv);
|
byte[] family = CellUtil.cloneFamily(kv);
|
||||||
byte[] tableNameBytes = null;
|
byte[] tableNameBytes = null;
|
||||||
if (writeMultipleTables) {
|
if (writeMultipleTables) {
|
||||||
|
@ -337,7 +336,8 @@ public class HFileOutputFormat2
|
||||||
}
|
}
|
||||||
|
|
||||||
// we now have the proper WAL writer. full steam ahead
|
// we now have the proper WAL writer. full steam ahead
|
||||||
kv.updateLatestStamp(this.now);
|
// TODO : Currently in SettableTimeStamp but this will also move to ExtendedCell
|
||||||
|
CellUtil.updateLatestStamp(cell, this.now);
|
||||||
wl.writer.append(kv);
|
wl.writer.append(kv);
|
||||||
wl.written += length;
|
wl.written += length;
|
||||||
|
|
||||||
|
@ -578,10 +578,11 @@ public class HFileOutputFormat2
|
||||||
configureIncrementalLoad(job, singleTableInfo, HFileOutputFormat2.class);
|
configureIncrementalLoad(job, singleTableInfo, HFileOutputFormat2.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
static void configureIncrementalLoad(Job job, List<TableInfo> multiTableInfo, Class<? extends OutputFormat<?, ?>> cls) throws IOException {
|
static void configureIncrementalLoad(Job job, List<TableInfo> multiTableInfo,
|
||||||
|
Class<? extends OutputFormat<?, ?>> cls) throws IOException {
|
||||||
Configuration conf = job.getConfiguration();
|
Configuration conf = job.getConfiguration();
|
||||||
job.setOutputKeyClass(ImmutableBytesWritable.class);
|
job.setOutputKeyClass(ImmutableBytesWritable.class);
|
||||||
job.setOutputValueClass(KeyValue.class);
|
job.setOutputValueClass(MapReduceCell.class);
|
||||||
job.setOutputFormatClass(cls);
|
job.setOutputFormatClass(cls);
|
||||||
|
|
||||||
if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) {
|
if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) {
|
||||||
|
@ -595,8 +596,9 @@ public class HFileOutputFormat2
|
||||||
// Based on the configured map output class, set the correct reducer to properly
|
// Based on the configured map output class, set the correct reducer to properly
|
||||||
// sort the incoming values.
|
// sort the incoming values.
|
||||||
// TODO it would be nice to pick one or the other of these formats.
|
// TODO it would be nice to pick one or the other of these formats.
|
||||||
if (KeyValue.class.equals(job.getMapOutputValueClass())) {
|
if (KeyValue.class.equals(job.getMapOutputValueClass())
|
||||||
job.setReducerClass(KeyValueSortReducer.class);
|
|| MapReduceCell.class.equals(job.getMapOutputValueClass())) {
|
||||||
|
job.setReducerClass(CellSortReducer.class);
|
||||||
} else if (Put.class.equals(job.getMapOutputValueClass())) {
|
} else if (Put.class.equals(job.getMapOutputValueClass())) {
|
||||||
job.setReducerClass(PutSortReducer.class);
|
job.setReducerClass(PutSortReducer.class);
|
||||||
} else if (Text.class.equals(job.getMapOutputValueClass())) {
|
} else if (Text.class.equals(job.getMapOutputValueClass())) {
|
||||||
|
@ -607,7 +609,7 @@ public class HFileOutputFormat2
|
||||||
|
|
||||||
conf.setStrings("io.serializations", conf.get("io.serializations"),
|
conf.setStrings("io.serializations", conf.get("io.serializations"),
|
||||||
MutationSerialization.class.getName(), ResultSerialization.class.getName(),
|
MutationSerialization.class.getName(), ResultSerialization.class.getName(),
|
||||||
KeyValueSerialization.class.getName());
|
CellSerialization.class.getName());
|
||||||
|
|
||||||
if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {
|
if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {
|
||||||
LOG.info("bulkload locality sensitive enabled");
|
LOG.info("bulkload locality sensitive enabled");
|
||||||
|
@ -655,7 +657,7 @@ public class HFileOutputFormat2
|
||||||
Configuration conf = job.getConfiguration();
|
Configuration conf = job.getConfiguration();
|
||||||
|
|
||||||
job.setOutputKeyClass(ImmutableBytesWritable.class);
|
job.setOutputKeyClass(ImmutableBytesWritable.class);
|
||||||
job.setOutputValueClass(KeyValue.class);
|
job.setOutputValueClass(MapReduceCell.class);
|
||||||
job.setOutputFormatClass(HFileOutputFormat2.class);
|
job.setOutputFormatClass(HFileOutputFormat2.class);
|
||||||
|
|
||||||
ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);
|
ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);
|
||||||
|
|
|
@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.filter.Filter;
|
import org.apache.hadoop.hbase.filter.Filter;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
import org.apache.hadoop.io.RawComparator;
|
import org.apache.hadoop.io.RawComparator;
|
||||||
|
@ -96,11 +97,11 @@ public class Import extends Configured implements Tool {
|
||||||
|
|
||||||
private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
|
private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
|
||||||
|
|
||||||
public static class KeyValueWritableComparablePartitioner
|
public static class CellWritableComparablePartitioner
|
||||||
extends Partitioner<KeyValueWritableComparable, KeyValue> {
|
extends Partitioner<CellWritableComparable, Cell> {
|
||||||
private static KeyValueWritableComparable[] START_KEYS = null;
|
private static CellWritableComparable[] START_KEYS = null;
|
||||||
@Override
|
@Override
|
||||||
public int getPartition(KeyValueWritableComparable key, KeyValue value,
|
public int getPartition(CellWritableComparable key, Cell value,
|
||||||
int numPartitions) {
|
int numPartitions) {
|
||||||
for (int i = 0; i < START_KEYS.length; ++i) {
|
for (int i = 0; i < START_KEYS.length; ++i) {
|
||||||
if (key.compareTo(START_KEYS[i]) <= 0) {
|
if (key.compareTo(START_KEYS[i]) <= 0) {
|
||||||
|
@ -112,6 +113,26 @@ public class Import extends Configured implements Tool {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @deprecated Use {@link CellWritableComparablePartitioner}. Will be removed
|
||||||
|
* from 3.0 onwards
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
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
|
public static class KeyValueWritableComparable
|
||||||
implements WritableComparable<KeyValueWritableComparable> {
|
implements WritableComparable<KeyValueWritableComparable> {
|
||||||
|
|
||||||
|
@ -119,8 +140,7 @@ public class Import extends Configured implements Tool {
|
||||||
|
|
||||||
static {
|
static {
|
||||||
// register this comparator
|
// register this comparator
|
||||||
WritableComparator.define(KeyValueWritableComparable.class,
|
WritableComparator.define(KeyValueWritableComparable.class, new KeyValueWritableComparator());
|
||||||
new KeyValueWritableComparator());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public KeyValueWritableComparable() {
|
public KeyValueWritableComparable() {
|
||||||
|
@ -141,10 +161,10 @@ public class Import extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
|
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",
|
||||||
justification="This is wrong, yes, but we should be purging Writables, not fixing them")
|
justification = "This is wrong, yes, but we should be purging Writables, not fixing them")
|
||||||
public int compareTo(KeyValueWritableComparable o) {
|
public int compareTo(KeyValueWritableComparable o) {
|
||||||
return CellComparator.COMPARATOR.compare(this.kv, ((KeyValueWritableComparable)o).kv);
|
return CellComparator.COMPARATOR.compare(this.kv, ((KeyValueWritableComparable) o).kv);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class KeyValueWritableComparator extends WritableComparator {
|
public static class KeyValueWritableComparator extends WritableComparator {
|
||||||
|
@ -166,18 +186,93 @@ public class Import extends Configured implements Tool {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static class CellWritableComparable
|
||||||
|
implements WritableComparable<CellWritableComparable> {
|
||||||
|
|
||||||
|
private Cell kv = null;
|
||||||
|
|
||||||
|
static {
|
||||||
|
// register this comparator
|
||||||
|
WritableComparator.define(CellWritableComparable.class,
|
||||||
|
new CellWritableComparator());
|
||||||
|
}
|
||||||
|
|
||||||
|
public CellWritableComparable() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public CellWritableComparable(Cell kv) {
|
||||||
|
this.kv = kv;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void write(DataOutput out) throws IOException {
|
||||||
|
out.writeInt(CellUtil.estimatedSerializedSizeOfKey(kv));
|
||||||
|
out.writeInt(0);
|
||||||
|
CellUtil.writeFlatKey(kv, out);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFields(DataInput in) throws IOException {
|
||||||
|
kv = KeyValue.create(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
|
||||||
|
justification="This is wrong, yes, but we should be purging Writables, not fixing them")
|
||||||
|
public int compareTo(CellWritableComparable o) {
|
||||||
|
return CellComparator.COMPARATOR.compare(this.kv, ((CellWritableComparable)o).kv);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class CellWritableComparator extends WritableComparator {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
|
||||||
|
try {
|
||||||
|
CellWritableComparable kv1 = new CellWritableComparable();
|
||||||
|
kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1)));
|
||||||
|
CellWritableComparable kv2 = new CellWritableComparable();
|
||||||
|
kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2)));
|
||||||
|
return compare(kv1, kv2);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @deprecated Use {@link CellReducer}. Will be removed from 3.0 onwards
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
public static class KeyValueReducer
|
public static class KeyValueReducer
|
||||||
extends
|
extends Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue> {
|
||||||
Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue> {
|
protected void reduce(KeyValueWritableComparable row, Iterable<KeyValue> kvs,
|
||||||
protected void reduce(
|
Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue>.Context context)
|
||||||
KeyValueWritableComparable row,
|
|
||||||
Iterable<KeyValue> kvs,
|
|
||||||
Reducer<KeyValueWritableComparable,
|
|
||||||
KeyValue, ImmutableBytesWritable, KeyValue>.Context context)
|
|
||||||
throws java.io.IOException, InterruptedException {
|
throws java.io.IOException, InterruptedException {
|
||||||
int index = 0;
|
int index = 0;
|
||||||
for (KeyValue kv : kvs) {
|
for (KeyValue kv : kvs) {
|
||||||
context.write(new ImmutableBytesWritable(kv.getRowArray()), kv);
|
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 CellReducer
|
||||||
|
extends
|
||||||
|
Reducer<CellWritableComparable, Cell, ImmutableBytesWritable, Cell> {
|
||||||
|
protected void reduce(
|
||||||
|
CellWritableComparable row,
|
||||||
|
Iterable<Cell> kvs,
|
||||||
|
Reducer<CellWritableComparable,
|
||||||
|
Cell, ImmutableBytesWritable, Cell>.Context context)
|
||||||
|
throws java.io.IOException, InterruptedException {
|
||||||
|
int index = 0;
|
||||||
|
for (Cell kv : kvs) {
|
||||||
|
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)),
|
||||||
|
new MapReduceCell(kv));
|
||||||
if (++index % 100 == 0)
|
if (++index % 100 == 0)
|
||||||
context.setStatus("Wrote " + index + " KeyValues, "
|
context.setStatus("Wrote " + index + " KeyValues, "
|
||||||
+ "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
|
+ "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
|
||||||
|
@ -185,11 +280,15 @@ public class Import extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @deprecated Use {@link CellSortImporter}. Will be removed from 3.0 onwards
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
public static class KeyValueSortImporter
|
public static class KeyValueSortImporter
|
||||||
extends TableMapper<KeyValueWritableComparable, KeyValue> {
|
extends TableMapper<KeyValueWritableComparable, KeyValue> {
|
||||||
private Map<byte[], byte[]> cfRenameMap;
|
private Map<byte[], byte[]> cfRenameMap;
|
||||||
private Filter filter;
|
private Filter filter;
|
||||||
private static final Log LOG = LogFactory.getLog(KeyValueImporter.class);
|
private static final Log LOG = LogFactory.getLog(KeyValueSortImporter.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param row The current table row key.
|
* @param row The current table row key.
|
||||||
|
@ -198,17 +297,14 @@ public class Import extends Configured implements Tool {
|
||||||
* @throws IOException When something is broken with the data.
|
* @throws IOException When something is broken with the data.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void map(ImmutableBytesWritable row, Result value,
|
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException {
|
||||||
Context context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
try {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Considering the row."
|
LOG.trace(
|
||||||
+ Bytes.toString(row.get(), row.getOffset(), row.getLength()));
|
"Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
|
||||||
}
|
}
|
||||||
if (filter == null
|
if (filter == null || !filter.filterRowKey(
|
||||||
|| !filter.filterRowKey(CellUtil.createFirstOnRow(row.get(), row.getOffset(),
|
CellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
|
||||||
(short) row.getLength()))) {
|
|
||||||
for (Cell kv : value.rawCells()) {
|
for (Cell kv : value.rawCells()) {
|
||||||
kv = filterKv(filter, kv);
|
kv = filterKv(filter, kv);
|
||||||
// skip if we filtered it out
|
// skip if we filtered it out
|
||||||
|
@ -249,14 +345,57 @@ public class Import extends Configured implements Tool {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A mapper that just writes out KeyValues.
|
* A mapper that just writes out KeyValues.
|
||||||
|
* @deprecated Use {@link CellImporter}. Will be removed from 3.0 onwards
|
||||||
*/
|
*/
|
||||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
|
@Deprecated
|
||||||
justification="Writables are going away and this has been this way forever")
|
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",
|
||||||
|
justification = "Writables are going away and this has been this way forever")
|
||||||
public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> {
|
public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> {
|
||||||
private Map<byte[], byte[]> cfRenameMap;
|
private Map<byte[], byte[]> cfRenameMap;
|
||||||
private Filter filter;
|
private Filter filter;
|
||||||
private static final Log LOG = LogFactory.getLog(KeyValueImporter.class);
|
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(
|
||||||
|
CellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) 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
|
||||||
|
context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setup(Context context) {
|
||||||
|
cfRenameMap = createCfRenameMap(context.getConfiguration());
|
||||||
|
filter = instantiateFilter(context.getConfiguration());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class CellSortImporter
|
||||||
|
extends TableMapper<CellWritableComparable, Cell> {
|
||||||
|
private Map<byte[], byte[]> cfRenameMap;
|
||||||
|
private Filter filter;
|
||||||
|
private static final Log LOG = LogFactory.getLog(CellImporter.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param row The current table row key.
|
* @param row The current table row key.
|
||||||
* @param value The columns.
|
* @param value The columns.
|
||||||
|
@ -279,8 +418,72 @@ public class Import extends Configured implements Tool {
|
||||||
kv = filterKv(filter, kv);
|
kv = filterKv(filter, kv);
|
||||||
// skip if we filtered it out
|
// skip if we filtered it out
|
||||||
if (kv == null) continue;
|
if (kv == null) continue;
|
||||||
// TODO get rid of ensureKeyValue
|
Cell ret = convertKv(kv, cfRenameMap);
|
||||||
context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)));
|
context.write(new CellWritableComparable(ret), 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");
|
||||||
|
}
|
||||||
|
CellWritableComparable[] startKeyWraps =
|
||||||
|
new CellWritableComparable[startKeys.length - 1];
|
||||||
|
for (int i = 1; i < startKeys.length; ++i) {
|
||||||
|
startKeyWraps[i - 1] =
|
||||||
|
new CellWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i]));
|
||||||
|
}
|
||||||
|
CellWritableComparablePartitioner.START_KEYS = startKeyWraps;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A mapper that just writes out KeyValues.
|
||||||
|
*/
|
||||||
|
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
|
||||||
|
justification="Writables are going away and this has been this way forever")
|
||||||
|
public static class CellImporter extends TableMapper<ImmutableBytesWritable, Cell> {
|
||||||
|
private Map<byte[], byte[]> cfRenameMap;
|
||||||
|
private Filter filter;
|
||||||
|
private static final Log LOG = LogFactory.getLog(CellImporter.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(CellUtil.createFirstOnRow(row.get(), row.getOffset(),
|
||||||
|
(short) row.getLength()))) {
|
||||||
|
for (Cell kv : value.rawCells()) {
|
||||||
|
kv = filterKv(filter, kv);
|
||||||
|
// skip if we filtered it out
|
||||||
|
if (kv == null) continue;
|
||||||
|
context.write(row, new MapReduceCell(convertKv(kv, cfRenameMap)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
|
@ -505,7 +708,7 @@ public class Import extends Configured implements Tool {
|
||||||
if(cfRenameMap != null) {
|
if(cfRenameMap != null) {
|
||||||
// If there's a rename mapping for this CF, create a new KeyValue
|
// If there's a rename mapping for this CF, create a new KeyValue
|
||||||
byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
|
byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
|
||||||
if(newCfName != null) {
|
if (newCfName != null) {
|
||||||
kv = new KeyValue(kv.getRowArray(), // row buffer
|
kv = new KeyValue(kv.getRowArray(), // row buffer
|
||||||
kv.getRowOffset(), // row offset
|
kv.getRowOffset(), // row offset
|
||||||
kv.getRowLength(), // row length
|
kv.getRowLength(), // row length
|
||||||
|
@ -626,35 +829,35 @@ public class Import extends Configured implements Tool {
|
||||||
Table table = conn.getTable(tableName);
|
Table table = conn.getTable(tableName);
|
||||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||||
job.setMapperClass(KeyValueSortImporter.class);
|
job.setMapperClass(CellSortImporter.class);
|
||||||
job.setReducerClass(KeyValueReducer.class);
|
job.setReducerClass(CellReducer.class);
|
||||||
Path outputDir = new Path(hfileOutPath);
|
Path outputDir = new Path(hfileOutPath);
|
||||||
FileOutputFormat.setOutputPath(job, outputDir);
|
FileOutputFormat.setOutputPath(job, outputDir);
|
||||||
job.setMapOutputKeyClass(KeyValueWritableComparable.class);
|
job.setMapOutputKeyClass(CellWritableComparable.class);
|
||||||
job.setMapOutputValueClass(KeyValue.class);
|
job.setMapOutputValueClass(MapReduceCell.class);
|
||||||
job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
|
job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
|
||||||
KeyValueWritableComparable.KeyValueWritableComparator.class,
|
CellWritableComparable.CellWritableComparator.class,
|
||||||
RawComparator.class);
|
RawComparator.class);
|
||||||
Path partitionsPath =
|
Path partitionsPath =
|
||||||
new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));
|
new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));
|
||||||
FileSystem fs = FileSystem.get(job.getConfiguration());
|
FileSystem fs = FileSystem.get(job.getConfiguration());
|
||||||
fs.deleteOnExit(partitionsPath);
|
fs.deleteOnExit(partitionsPath);
|
||||||
job.setPartitionerClass(KeyValueWritableComparablePartitioner.class);
|
job.setPartitionerClass(CellWritableComparablePartitioner.class);
|
||||||
job.setNumReduceTasks(regionLocator.getStartKeys().length);
|
job.setNumReduceTasks(regionLocator.getStartKeys().length);
|
||||||
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
||||||
org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);
|
org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);
|
||||||
}
|
}
|
||||||
} else if (hfileOutPath != null) {
|
} else if (hfileOutPath != null) {
|
||||||
LOG.info("writing to hfiles for bulk load.");
|
LOG.info("writing to hfiles for bulk load.");
|
||||||
job.setMapperClass(KeyValueImporter.class);
|
job.setMapperClass(CellImporter.class);
|
||||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||||
Table table = conn.getTable(tableName);
|
Table table = conn.getTable(tableName);
|
||||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)){
|
RegionLocator regionLocator = conn.getRegionLocator(tableName)){
|
||||||
job.setReducerClass(KeyValueSortReducer.class);
|
job.setReducerClass(CellSortReducer.class);
|
||||||
Path outputDir = new Path(hfileOutPath);
|
Path outputDir = new Path(hfileOutPath);
|
||||||
FileOutputFormat.setOutputPath(job, outputDir);
|
FileOutputFormat.setOutputPath(job, outputDir);
|
||||||
job.setMapOutputKeyClass(ImmutableBytesWritable.class);
|
job.setMapOutputKeyClass(ImmutableBytesWritable.class);
|
||||||
job.setMapOutputValueClass(KeyValue.class);
|
job.setMapOutputValueClass(MapReduceCell.class);
|
||||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||||
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
||||||
org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);
|
org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);
|
||||||
|
|
|
@ -584,7 +584,7 @@ public class ImportTsv extends Configured implements Tool {
|
||||||
job.getConfiguration().setStrings("io.serializations",
|
job.getConfiguration().setStrings("io.serializations",
|
||||||
job.getConfiguration().get("io.serializations"),
|
job.getConfiguration().get("io.serializations"),
|
||||||
MutationSerialization.class.getName(), ResultSerialization.class.getName(),
|
MutationSerialization.class.getName(), ResultSerialization.class.getName(),
|
||||||
KeyValueSerialization.class.getName());
|
CellSerialization.class.getName());
|
||||||
}
|
}
|
||||||
TableMapReduceUtil.addDependencyJars(job);
|
TableMapReduceUtil.addDependencyJars(job);
|
||||||
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
||||||
|
|
|
@ -29,7 +29,13 @@ import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.hadoop.io.serializer.Deserializer;
|
import org.apache.hadoop.io.serializer.Deserializer;
|
||||||
import org.apache.hadoop.io.serializer.Serialization;
|
import org.apache.hadoop.io.serializer.Serialization;
|
||||||
import org.apache.hadoop.io.serializer.Serializer;
|
import org.apache.hadoop.io.serializer.Serializer;
|
||||||
|
/**
|
||||||
|
* Use to specify the type of serialization for the mappers
|
||||||
|
* and reducers
|
||||||
|
* @deprecated Use {@link CellSerialization}. Will be
|
||||||
|
* removed from 3.0 onwards
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
@InterfaceAudience.Public
|
@InterfaceAudience.Public
|
||||||
public class KeyValueSerialization implements Serialization<KeyValue> {
|
public class KeyValueSerialization implements Serialization<KeyValue> {
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -32,7 +32,10 @@ import org.apache.hadoop.mapreduce.Reducer;
|
||||||
* KeyValues in sorted order. If lots of columns per row, it will use lots of
|
* KeyValues in sorted order. If lots of columns per row, it will use lots of
|
||||||
* memory sorting.
|
* memory sorting.
|
||||||
* @see HFileOutputFormat2
|
* @see HFileOutputFormat2
|
||||||
|
* @deprecated Use {@link CellSortReducer}. Will be removed from
|
||||||
|
* 3.0 onwards
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
@InterfaceAudience.Public
|
@InterfaceAudience.Public
|
||||||
public class KeyValueSortReducer
|
public class KeyValueSortReducer
|
||||||
extends Reducer<ImmutableBytesWritable, KeyValue, ImmutableBytesWritable, KeyValue> {
|
extends Reducer<ImmutableBytesWritable, KeyValue, ImmutableBytesWritable, KeyValue> {
|
||||||
|
|
|
@ -48,7 +48,7 @@ import org.apache.hadoop.util.StringUtils;
|
||||||
* Puts in sorted order. If lots of columns per row, it will use lots of
|
* Puts in sorted order. If lots of columns per row, it will use lots of
|
||||||
* memory sorting.
|
* memory sorting.
|
||||||
* @see HFileOutputFormat2
|
* @see HFileOutputFormat2
|
||||||
* @see KeyValueSortReducer
|
* @see CellSortReducer
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Public
|
@InterfaceAudience.Public
|
||||||
public class PutSortReducer extends
|
public class PutSortReducer extends
|
||||||
|
|
|
@ -208,7 +208,7 @@ public class TableMapReduceUtil {
|
||||||
conf.set(TableInputFormat.SCAN, convertScanToString(scan));
|
conf.set(TableInputFormat.SCAN, convertScanToString(scan));
|
||||||
conf.setStrings("io.serializations", conf.get("io.serializations"),
|
conf.setStrings("io.serializations", conf.get("io.serializations"),
|
||||||
MutationSerialization.class.getName(), ResultSerialization.class.getName(),
|
MutationSerialization.class.getName(), ResultSerialization.class.getName(),
|
||||||
KeyValueSerialization.class.getName());
|
CellSerialization.class.getName());
|
||||||
if (addDependencyJars) {
|
if (addDependencyJars) {
|
||||||
addDependencyJars(job);
|
addDependencyJars(job);
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,7 +45,7 @@ import org.apache.hadoop.util.StringUtils;
|
||||||
/**
|
/**
|
||||||
* Emits Sorted KeyValues. Parse the passed text and creates KeyValues. Sorts them before emit.
|
* Emits Sorted KeyValues. Parse the passed text and creates KeyValues. Sorts them before emit.
|
||||||
* @see HFileOutputFormat2
|
* @see HFileOutputFormat2
|
||||||
* @see KeyValueSortReducer
|
* @see CellSortReducer
|
||||||
* @see PutSortReducer
|
* @see PutSortReducer
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Public
|
@InterfaceAudience.Public
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
import org.apache.hadoop.hbase.client.Delete;
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
|
@ -44,8 +43,9 @@ import org.apache.hadoop.hbase.client.RegionLocator;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
||||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||||
|
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||||
import org.apache.hadoop.hbase.wal.WALKey;
|
import org.apache.hadoop.hbase.wal.WALKey;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
@ -53,6 +53,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
import org.apache.hadoop.util.Tool;
|
import org.apache.hadoop.util.Tool;
|
||||||
import org.apache.hadoop.util.ToolRunner;
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A tool to replay WAL files as a M/R job.
|
* A tool to replay WAL files as a M/R job.
|
||||||
|
@ -95,7 +96,9 @@ public class WALPlayer extends Configured implements Tool {
|
||||||
/**
|
/**
|
||||||
* A mapper that just writes out KeyValues.
|
* A mapper that just writes out KeyValues.
|
||||||
* This one can be used together with {@link KeyValueSortReducer}
|
* This one can be used together with {@link KeyValueSortReducer}
|
||||||
|
* @deprecated Use {@link WALCellMapper}. Will be removed from 3.0 onwards
|
||||||
*/
|
*/
|
||||||
|
@Deprecated
|
||||||
static class WALKeyValueMapper
|
static class WALKeyValueMapper
|
||||||
extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
|
extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
|
||||||
private byte[] table;
|
private byte[] table;
|
||||||
|
@ -132,6 +135,47 @@ public class WALPlayer extends Configured implements Tool {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* A mapper that just writes out Cells.
|
||||||
|
* This one can be used together with {@link CellSortReducer}
|
||||||
|
*/
|
||||||
|
static class WALCellMapper
|
||||||
|
extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Cell> {
|
||||||
|
private byte[] table;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void map(WALKey key, WALEdit value,
|
||||||
|
Context context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
// skip all other tables
|
||||||
|
if (Bytes.equals(table, key.getTablename().getName())) {
|
||||||
|
for (Cell cell : value.getCells()) {
|
||||||
|
if (WALEdit.isMetaEditFamily(cell)) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(cell)),
|
||||||
|
new MapReduceCell(cell));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setup(Context context) throws IOException {
|
||||||
|
// only a single table is supported when HFiles are generated with HFileOutputFormat
|
||||||
|
String[] tables = context.getConfiguration().getStrings(TABLES_KEY);
|
||||||
|
if (tables == null || tables.length != 1) {
|
||||||
|
// this can only happen when WALMapper is used directly by a class other than WALPlayer
|
||||||
|
throw new IOException("Exactly one table must be specified for bulk HFile case.");
|
||||||
|
}
|
||||||
|
table = Bytes.toBytes(tables[0]);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -299,11 +343,11 @@ public class WALPlayer extends Configured implements Tool {
|
||||||
throw new IOException("Exactly one table must be specified for the bulk export option");
|
throw new IOException("Exactly one table must be specified for the bulk export option");
|
||||||
}
|
}
|
||||||
TableName tableName = TableName.valueOf(tables[0]);
|
TableName tableName = TableName.valueOf(tables[0]);
|
||||||
job.setMapperClass(WALKeyValueMapper.class);
|
job.setMapperClass(WALCellMapper.class);
|
||||||
job.setReducerClass(KeyValueSortReducer.class);
|
job.setReducerClass(CellSortReducer.class);
|
||||||
Path outputDir = new Path(hfileOutPath);
|
Path outputDir = new Path(hfileOutPath);
|
||||||
FileOutputFormat.setOutputPath(job, outputDir);
|
FileOutputFormat.setOutputPath(job, outputDir);
|
||||||
job.setMapOutputValueClass(KeyValue.class);
|
job.setMapOutputValueClass(MapReduceCell.class);
|
||||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||||
Table table = conn.getTable(tableName);
|
Table table = conn.getTable(tableName);
|
||||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||||
|
|
|
@ -0,0 +1,270 @@
|
||||||
|
/*
|
||||||
|
* 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.hadoop.hbase.util;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||||
|
import org.apache.hadoop.hbase.Cell;
|
||||||
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
|
import org.apache.hadoop.hbase.ExtendedCell;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A wrapper for a cell to be used with mapreduce, as the output value class for mappers/reducers.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
||||||
|
|
||||||
|
private final Cell cell;
|
||||||
|
|
||||||
|
public MapReduceCell(Cell cell) {
|
||||||
|
this.cell = cell;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getRowArray() {
|
||||||
|
return this.cell.getRowArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getRowOffset() {
|
||||||
|
return this.cell.getRowOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public short getRowLength() {
|
||||||
|
return this.cell.getRowLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getFamilyArray() {
|
||||||
|
return this.cell.getFamilyArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getFamilyOffset() {
|
||||||
|
return this.cell.getFamilyOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte getFamilyLength() {
|
||||||
|
return this.cell.getFamilyLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getQualifierArray() {
|
||||||
|
return this.cell.getQualifierArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getQualifierOffset() {
|
||||||
|
return this.cell.getQualifierOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getQualifierLength() {
|
||||||
|
return this.cell.getQualifierLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getTimestamp() {
|
||||||
|
return this.cell.getTimestamp();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte getTypeByte() {
|
||||||
|
return this.cell.getTypeByte();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSequenceId() {
|
||||||
|
return this.cell.getSequenceId();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getValueArray() {
|
||||||
|
return this.cell.getValueArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueOffset() {
|
||||||
|
return this.cell.getValueOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueLength() {
|
||||||
|
return this.cell.getValueLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getTagsArray() {
|
||||||
|
return this.cell.getTagsArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getTagsOffset() {
|
||||||
|
return this.cell.getTagsOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getTagsLength() {
|
||||||
|
return this.cell.getTagsLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ByteBuffer getRowByteBuffer() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getRowByteBuffer();
|
||||||
|
} else {
|
||||||
|
return ByteBuffer.wrap(CellUtil.cloneRow(this.cell));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getRowPosition() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getRowPosition();
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ByteBuffer getFamilyByteBuffer() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getFamilyByteBuffer();
|
||||||
|
} else {
|
||||||
|
return ByteBuffer.wrap(CellUtil.cloneFamily(this.cell));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getFamilyPosition() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getFamilyPosition();
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ByteBuffer getQualifierByteBuffer() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getQualifierByteBuffer();
|
||||||
|
} else {
|
||||||
|
return ByteBuffer.wrap(CellUtil.cloneQualifier(this.cell));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getQualifierPosition() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getQualifierPosition();
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ByteBuffer getValueByteBuffer() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getValueByteBuffer();
|
||||||
|
} else {
|
||||||
|
return ByteBuffer.wrap(CellUtil.cloneValue(this.cell));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValuePosition() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getValuePosition();
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ByteBuffer getTagsByteBuffer() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getTagsByteBuffer();
|
||||||
|
} else {
|
||||||
|
return ByteBuffer.wrap(CellUtil.cloneTags(this.cell));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getTagsPosition() {
|
||||||
|
if (cell instanceof ByteBufferCell) {
|
||||||
|
return ((ByteBufferCell) this.cell).getTagsPosition();
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return this.cell.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setSequenceId(long seqId) throws IOException {
|
||||||
|
CellUtil.setSequenceId(cell, seqId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setTimestamp(long ts) throws IOException {
|
||||||
|
CellUtil.setTimestamp(cell, ts);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setTimestamp(byte[] ts, int tsOffset) throws IOException {
|
||||||
|
CellUtil.setTimestamp(cell, ts, tsOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long heapSize() {
|
||||||
|
return CellUtil.estimatedHeapSizeOf(cell);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int write(OutputStream out, boolean withTags) throws IOException {
|
||||||
|
return CellUtil.writeCell(cell, out, withTags);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getSerializedSize(boolean withTags) {
|
||||||
|
return CellUtil.estimatedSerializedSizeOf(cell) - Bytes.SIZEOF_INT;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void write(ByteBuffer buf, int offset) {
|
||||||
|
CellUtil.writeCellToBuffer(cell, buf, offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ExtendedCell deepClone() {
|
||||||
|
try {
|
||||||
|
return (ExtendedCell) CellUtil.deepClone(cell);
|
||||||
|
} catch (CloneNotSupportedException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,801 @@
|
||||||
|
/**
|
||||||
|
* 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.hadoop.hbase.mapreduce;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
import static org.mockito.Matchers.any;
|
||||||
|
import static org.mockito.Mockito.doAnswer;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.PrintStream;
|
||||||
|
import java.net.URL;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
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.CellUtil;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||||
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||||
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
|
import org.apache.hadoop.hbase.client.Durability;
|
||||||
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||||
|
import org.apache.hadoop.hbase.filter.Filter;
|
||||||
|
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||||
|
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||||
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
|
||||||
|
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||||
|
import org.apache.hadoop.hbase.wal.WAL;
|
||||||
|
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||||
|
import org.apache.hadoop.hbase.wal.WALKey;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper.Context;
|
||||||
|
import org.apache.hadoop.util.GenericOptionsParser;
|
||||||
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.rules.TestName;
|
||||||
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests the table import and table export MR job functionality
|
||||||
|
*/
|
||||||
|
@Category({VerySlowMapReduceTests.class, MediumTests.class})
|
||||||
|
public class TestCellBasedImportExport2 {
|
||||||
|
|
||||||
|
private static final Log LOG = LogFactory.getLog(TestCellBasedImportExport2.class);
|
||||||
|
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||||
|
private static final byte[] ROW1 = Bytes.toBytesBinary("\\x32row1");
|
||||||
|
private static final byte[] ROW2 = Bytes.toBytesBinary("\\x32row2");
|
||||||
|
private static final byte[] ROW3 = Bytes.toBytesBinary("\\x32row3");
|
||||||
|
private static final String FAMILYA_STRING = "a";
|
||||||
|
private static final String FAMILYB_STRING = "b";
|
||||||
|
private static final byte[] FAMILYA = Bytes.toBytes(FAMILYA_STRING);
|
||||||
|
private static final byte[] FAMILYB = Bytes.toBytes(FAMILYB_STRING);
|
||||||
|
private static final byte[] QUAL = Bytes.toBytes("q");
|
||||||
|
private static final String OUTPUT_DIR = "outputdir";
|
||||||
|
private static String FQ_OUTPUT_DIR;
|
||||||
|
private static final String EXPORT_BATCH_SIZE = "100";
|
||||||
|
|
||||||
|
private static final long now = System.currentTimeMillis();
|
||||||
|
private final TableName EXPORT_TABLE = TableName.valueOf("export_table");
|
||||||
|
private final TableName IMPORT_TABLE = TableName.valueOf("import_table");
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void beforeClass() throws Throwable {
|
||||||
|
// Up the handlers; this test needs more than usual.
|
||||||
|
UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
|
||||||
|
UTIL.startMiniCluster();
|
||||||
|
FQ_OUTPUT_DIR =
|
||||||
|
new Path(OUTPUT_DIR).makeQualified(FileSystem.get(UTIL.getConfiguration())).toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void afterClass() throws Throwable {
|
||||||
|
UTIL.shutdownMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final TestName name = new TestName();
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void announce() {
|
||||||
|
LOG.info("Running " + name.getMethodName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void cleanup() throws Throwable {
|
||||||
|
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||||
|
fs.delete(new Path(OUTPUT_DIR), true);
|
||||||
|
if (UTIL.getAdmin().tableExists(EXPORT_TABLE)) {
|
||||||
|
UTIL.deleteTable(EXPORT_TABLE);
|
||||||
|
}
|
||||||
|
if (UTIL.getAdmin().tableExists(IMPORT_TABLE)) {
|
||||||
|
UTIL.deleteTable(IMPORT_TABLE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Runs an export job with the specified command line args
|
||||||
|
* @param args
|
||||||
|
* @return true if job completed successfully
|
||||||
|
* @throws IOException
|
||||||
|
* @throws InterruptedException
|
||||||
|
* @throws ClassNotFoundException
|
||||||
|
*/
|
||||||
|
protected boolean runExport(String[] args) throws Throwable {
|
||||||
|
// need to make a copy of the configuration because to make sure different temp dirs are used.
|
||||||
|
int status = ToolRunner.run(new Configuration(UTIL.getConfiguration()), new Export(), args);
|
||||||
|
return status == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void runExportMain(String[] args) throws Throwable {
|
||||||
|
Export.main(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Runs an import job with the specified command line args
|
||||||
|
* @param args
|
||||||
|
* @return true if job completed successfully
|
||||||
|
* @throws IOException
|
||||||
|
* @throws InterruptedException
|
||||||
|
* @throws ClassNotFoundException
|
||||||
|
*/
|
||||||
|
boolean runImport(String[] args) throws Throwable {
|
||||||
|
// need to make a copy of the configuration because to make sure different temp dirs are used.
|
||||||
|
int status = ToolRunner.run(new Configuration(UTIL.getConfiguration()), new Import(), args);
|
||||||
|
return status == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test simple replication case with column mapping
|
||||||
|
* @throws Exception
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testSimpleCase() throws Throwable {
|
||||||
|
try (Table t = UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILYA, 3);) {
|
||||||
|
Put p = new Put(ROW1);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
t.put(p);
|
||||||
|
p = new Put(ROW2);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
t.put(p);
|
||||||
|
p = new Put(ROW3);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
t.put(p);
|
||||||
|
}
|
||||||
|
|
||||||
|
String[] args = new String[] {
|
||||||
|
// Only export row1 & row2.
|
||||||
|
"-D" + TableInputFormat.SCAN_ROW_START + "=\\x32row1",
|
||||||
|
"-D" + TableInputFormat.SCAN_ROW_STOP + "=\\x32row3",
|
||||||
|
name.getMethodName(),
|
||||||
|
FQ_OUTPUT_DIR,
|
||||||
|
"1000", // max number of key versions per key to export
|
||||||
|
};
|
||||||
|
assertTrue(runExport(args));
|
||||||
|
|
||||||
|
final String IMPORT_TABLE = name.getMethodName() + "import";
|
||||||
|
try (Table t = UTIL.createTable(TableName.valueOf(IMPORT_TABLE), FAMILYB, 3);) {
|
||||||
|
args = new String[] {
|
||||||
|
"-D" + Import.CF_RENAME_PROP + "="+FAMILYA_STRING+":"+FAMILYB_STRING,
|
||||||
|
IMPORT_TABLE,
|
||||||
|
FQ_OUTPUT_DIR
|
||||||
|
};
|
||||||
|
assertTrue(runImport(args));
|
||||||
|
|
||||||
|
Get g = new Get(ROW1);
|
||||||
|
g.setMaxVersions();
|
||||||
|
Result r = t.get(g);
|
||||||
|
assertEquals(3, r.size());
|
||||||
|
g = new Get(ROW2);
|
||||||
|
g.setMaxVersions();
|
||||||
|
r = t.get(g);
|
||||||
|
assertEquals(3, r.size());
|
||||||
|
g = new Get(ROW3);
|
||||||
|
r = t.get(g);
|
||||||
|
assertEquals(0, r.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test export hbase:meta table
|
||||||
|
*
|
||||||
|
* @throws Throwable
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMetaExport() throws Throwable {
|
||||||
|
String[] args = new String[] { TableName.META_TABLE_NAME.getNameAsString(),
|
||||||
|
FQ_OUTPUT_DIR, "1", "0", "0" };
|
||||||
|
assertTrue(runExport(args));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test import data from 0.94 exported file
|
||||||
|
* @throws Throwable
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testImport94Table() throws Throwable {
|
||||||
|
final String name = "exportedTableIn94Format";
|
||||||
|
URL url = TestCellBasedImportExport2.class.getResource(name);
|
||||||
|
File f = new File(url.toURI());
|
||||||
|
if (!f.exists()) {
|
||||||
|
LOG.warn("FAILED TO FIND " + f + "; skipping out on test");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
assertTrue(f.exists());
|
||||||
|
LOG.info("FILE=" + f);
|
||||||
|
Path importPath = new Path(f.toURI());
|
||||||
|
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||||
|
fs.copyFromLocalFile(importPath, new Path(FQ_OUTPUT_DIR + Path.SEPARATOR + name));
|
||||||
|
String IMPORT_TABLE = name;
|
||||||
|
try (Table t = UTIL.createTable(TableName.valueOf(IMPORT_TABLE), Bytes.toBytes("f1"), 3);) {
|
||||||
|
String[] args = new String[] {
|
||||||
|
"-Dhbase.import.version=0.94" ,
|
||||||
|
IMPORT_TABLE, FQ_OUTPUT_DIR
|
||||||
|
};
|
||||||
|
assertTrue(runImport(args));
|
||||||
|
/* exportedTableIn94Format contains 5 rows
|
||||||
|
ROW COLUMN+CELL
|
||||||
|
r1 column=f1:c1, timestamp=1383766761171, value=val1
|
||||||
|
r2 column=f1:c1, timestamp=1383766771642, value=val2
|
||||||
|
r3 column=f1:c1, timestamp=1383766777615, value=val3
|
||||||
|
r4 column=f1:c1, timestamp=1383766785146, value=val4
|
||||||
|
r5 column=f1:c1, timestamp=1383766791506, value=val5
|
||||||
|
*/
|
||||||
|
assertEquals(5, UTIL.countRows(t));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test export scanner batching
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testExportScannerBatching() throws Throwable {
|
||||||
|
TableDescriptor desc = TableDescriptorBuilder
|
||||||
|
.newBuilder(TableName.valueOf(name.getMethodName()))
|
||||||
|
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
|
||||||
|
.setMaxVersions(1)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
UTIL.getAdmin().createTable(desc);
|
||||||
|
try (Table t = UTIL.getConnection().getTable(desc.getTableName());) {
|
||||||
|
|
||||||
|
Put p = new Put(ROW1);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 3, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 4, QUAL);
|
||||||
|
t.put(p);
|
||||||
|
|
||||||
|
String[] args = new String[] {
|
||||||
|
"-D" + ExportUtils.EXPORT_BATCHING + "=" + EXPORT_BATCH_SIZE, // added scanner batching arg.
|
||||||
|
name.getMethodName(),
|
||||||
|
FQ_OUTPUT_DIR
|
||||||
|
};
|
||||||
|
assertTrue(runExport(args));
|
||||||
|
|
||||||
|
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
|
||||||
|
fs.delete(new Path(FQ_OUTPUT_DIR), true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWithDeletes() throws Throwable {
|
||||||
|
TableDescriptor desc = TableDescriptorBuilder
|
||||||
|
.newBuilder(TableName.valueOf(name.getMethodName()))
|
||||||
|
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
|
||||||
|
.setMaxVersions(5)
|
||||||
|
.setKeepDeletedCells(KeepDeletedCells.TRUE)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
UTIL.getAdmin().createTable(desc);
|
||||||
|
try (Table t = UTIL.getConnection().getTable(desc.getTableName());) {
|
||||||
|
|
||||||
|
Put p = new Put(ROW1);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 3, QUAL);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 4, QUAL);
|
||||||
|
t.put(p);
|
||||||
|
|
||||||
|
Delete d = new Delete(ROW1, now+3);
|
||||||
|
t.delete(d);
|
||||||
|
d = new Delete(ROW1);
|
||||||
|
d.addColumns(FAMILYA, QUAL, now+2);
|
||||||
|
t.delete(d);
|
||||||
|
}
|
||||||
|
|
||||||
|
String[] args = new String[] {
|
||||||
|
"-D" + ExportUtils.RAW_SCAN + "=true",
|
||||||
|
name.getMethodName(),
|
||||||
|
FQ_OUTPUT_DIR,
|
||||||
|
"1000", // max number of key versions per key to export
|
||||||
|
};
|
||||||
|
assertTrue(runExport(args));
|
||||||
|
|
||||||
|
final String IMPORT_TABLE = name.getMethodName() + "import";
|
||||||
|
desc = TableDescriptorBuilder
|
||||||
|
.newBuilder(TableName.valueOf(IMPORT_TABLE))
|
||||||
|
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
|
||||||
|
.setMaxVersions(5)
|
||||||
|
.setKeepDeletedCells(KeepDeletedCells.TRUE)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
UTIL.getAdmin().createTable(desc);
|
||||||
|
try (Table t = UTIL.getConnection().getTable(desc.getTableName());) {
|
||||||
|
args = new String[] {
|
||||||
|
IMPORT_TABLE,
|
||||||
|
FQ_OUTPUT_DIR
|
||||||
|
};
|
||||||
|
assertTrue(runImport(args));
|
||||||
|
|
||||||
|
Scan s = new Scan();
|
||||||
|
s.setMaxVersions();
|
||||||
|
s.setRaw(true);
|
||||||
|
ResultScanner scanner = t.getScanner(s);
|
||||||
|
Result r = scanner.next();
|
||||||
|
Cell[] res = r.rawCells();
|
||||||
|
assertTrue(CellUtil.isDeleteFamily(res[0]));
|
||||||
|
assertEquals(now+4, res[1].getTimestamp());
|
||||||
|
assertEquals(now+3, res[2].getTimestamp());
|
||||||
|
assertTrue(CellUtil.isDelete(res[3]));
|
||||||
|
assertEquals(now+2, res[4].getTimestamp());
|
||||||
|
assertEquals(now+1, res[5].getTimestamp());
|
||||||
|
assertEquals(now, res[6].getTimestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWithMultipleDeleteFamilyMarkersOfSameRowSameFamily() throws Throwable {
|
||||||
|
final TableName exportTable = TableName.valueOf(name.getMethodName());
|
||||||
|
TableDescriptor desc = TableDescriptorBuilder
|
||||||
|
.newBuilder(TableName.valueOf(name.getMethodName()))
|
||||||
|
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
|
||||||
|
.setMaxVersions(5)
|
||||||
|
.setKeepDeletedCells(KeepDeletedCells.TRUE)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
UTIL.getAdmin().createTable(desc);
|
||||||
|
|
||||||
|
Table exportT = UTIL.getConnection().getTable(exportTable);
|
||||||
|
|
||||||
|
//Add first version of QUAL
|
||||||
|
Put p = new Put(ROW1);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
exportT.put(p);
|
||||||
|
|
||||||
|
//Add Delete family marker
|
||||||
|
Delete d = new Delete(ROW1, now+3);
|
||||||
|
exportT.delete(d);
|
||||||
|
|
||||||
|
//Add second version of QUAL
|
||||||
|
p = new Put(ROW1);
|
||||||
|
p.addColumn(FAMILYA, QUAL, now + 5, "s".getBytes());
|
||||||
|
exportT.put(p);
|
||||||
|
|
||||||
|
//Add second Delete family marker
|
||||||
|
d = new Delete(ROW1, now+7);
|
||||||
|
exportT.delete(d);
|
||||||
|
|
||||||
|
|
||||||
|
String[] args = new String[] {
|
||||||
|
"-D" + ExportUtils.RAW_SCAN + "=true", exportTable.getNameAsString(),
|
||||||
|
FQ_OUTPUT_DIR,
|
||||||
|
"1000", // max number of key versions per key to export
|
||||||
|
};
|
||||||
|
assertTrue(runExport(args));
|
||||||
|
|
||||||
|
final String importTable = name.getMethodName() + "import";
|
||||||
|
desc = TableDescriptorBuilder
|
||||||
|
.newBuilder(TableName.valueOf(importTable))
|
||||||
|
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
|
||||||
|
.setMaxVersions(5)
|
||||||
|
.setKeepDeletedCells(KeepDeletedCells.TRUE)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
UTIL.getAdmin().createTable(desc);
|
||||||
|
|
||||||
|
Table importT = UTIL.getConnection().getTable(TableName.valueOf(importTable));
|
||||||
|
args = new String[] {
|
||||||
|
importTable,
|
||||||
|
FQ_OUTPUT_DIR
|
||||||
|
};
|
||||||
|
assertTrue(runImport(args));
|
||||||
|
|
||||||
|
Scan s = new Scan();
|
||||||
|
s.setMaxVersions();
|
||||||
|
s.setRaw(true);
|
||||||
|
|
||||||
|
ResultScanner importedTScanner = importT.getScanner(s);
|
||||||
|
Result importedTResult = importedTScanner.next();
|
||||||
|
|
||||||
|
ResultScanner exportedTScanner = exportT.getScanner(s);
|
||||||
|
Result exportedTResult = exportedTScanner.next();
|
||||||
|
try {
|
||||||
|
Result.compareResults(exportedTResult, importedTResult);
|
||||||
|
} catch (Throwable e) {
|
||||||
|
fail("Original and imported tables data comparision failed with error:"+e.getMessage());
|
||||||
|
} finally {
|
||||||
|
exportT.close();
|
||||||
|
importT.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a simple table, run an Export Job on it, Import with filtering on, verify counts,
|
||||||
|
* attempt with invalid values.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testWithFilter() throws Throwable {
|
||||||
|
// Create simple table to export
|
||||||
|
TableDescriptor desc = TableDescriptorBuilder
|
||||||
|
.newBuilder(TableName.valueOf(name.getMethodName()))
|
||||||
|
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
|
||||||
|
.setMaxVersions(5)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
UTIL.getAdmin().createTable(desc);
|
||||||
|
Table exportTable = UTIL.getConnection().getTable(desc.getTableName());
|
||||||
|
|
||||||
|
Put p1 = new Put(ROW1);
|
||||||
|
p1.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
p1.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
p1.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
p1.addColumn(FAMILYA, QUAL, now + 3, QUAL);
|
||||||
|
p1.addColumn(FAMILYA, QUAL, now + 4, QUAL);
|
||||||
|
|
||||||
|
// Having another row would actually test the filter.
|
||||||
|
Put p2 = new Put(ROW2);
|
||||||
|
p2.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
|
||||||
|
exportTable.put(Arrays.asList(p1, p2));
|
||||||
|
|
||||||
|
// Export the simple table
|
||||||
|
String[] args = new String[] { name.getMethodName(), FQ_OUTPUT_DIR, "1000" };
|
||||||
|
assertTrue(runExport(args));
|
||||||
|
|
||||||
|
// Import to a new table
|
||||||
|
final String IMPORT_TABLE = name.getMethodName() + "import";
|
||||||
|
desc = TableDescriptorBuilder
|
||||||
|
.newBuilder(TableName.valueOf(IMPORT_TABLE))
|
||||||
|
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILYA)
|
||||||
|
.setMaxVersions(5)
|
||||||
|
.build())
|
||||||
|
.build();
|
||||||
|
UTIL.getAdmin().createTable(desc);
|
||||||
|
|
||||||
|
Table importTable = UTIL.getConnection().getTable(desc.getTableName());
|
||||||
|
args = new String[] { "-D" + Import.FILTER_CLASS_CONF_KEY + "=" + PrefixFilter.class.getName(),
|
||||||
|
"-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1), IMPORT_TABLE,
|
||||||
|
FQ_OUTPUT_DIR,
|
||||||
|
"1000" };
|
||||||
|
assertTrue(runImport(args));
|
||||||
|
|
||||||
|
// get the count of the source table for that time range
|
||||||
|
PrefixFilter filter = new PrefixFilter(ROW1);
|
||||||
|
int count = getCount(exportTable, filter);
|
||||||
|
|
||||||
|
Assert.assertEquals("Unexpected row count between export and import tables", count,
|
||||||
|
getCount(importTable, null));
|
||||||
|
|
||||||
|
// and then test that a broken command doesn't bork everything - easier here because we don't
|
||||||
|
// need to re-run the export job
|
||||||
|
|
||||||
|
args = new String[] { "-D" + Import.FILTER_CLASS_CONF_KEY + "=" + Filter.class.getName(),
|
||||||
|
"-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1) + "", name.getMethodName(),
|
||||||
|
FQ_OUTPUT_DIR, "1000" };
|
||||||
|
assertFalse(runImport(args));
|
||||||
|
|
||||||
|
// cleanup
|
||||||
|
exportTable.close();
|
||||||
|
importTable.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Count the number of keyvalues in the specified table for the given timerange
|
||||||
|
* @param table
|
||||||
|
* @return
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private int getCount(Table table, Filter filter) throws IOException {
|
||||||
|
Scan scan = new Scan();
|
||||||
|
scan.setFilter(filter);
|
||||||
|
ResultScanner results = table.getScanner(scan);
|
||||||
|
int count = 0;
|
||||||
|
for (Result res : results) {
|
||||||
|
count += res.size();
|
||||||
|
}
|
||||||
|
results.close();
|
||||||
|
return count;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* test main method. Import should print help and call System.exit
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testImportMain() throws Throwable {
|
||||||
|
PrintStream oldPrintStream = System.err;
|
||||||
|
SecurityManager SECURITY_MANAGER = System.getSecurityManager();
|
||||||
|
LauncherSecurityManager newSecurityManager= new LauncherSecurityManager();
|
||||||
|
System.setSecurityManager(newSecurityManager);
|
||||||
|
ByteArrayOutputStream data = new ByteArrayOutputStream();
|
||||||
|
String[] args = {};
|
||||||
|
System.setErr(new PrintStream(data));
|
||||||
|
try {
|
||||||
|
System.setErr(new PrintStream(data));
|
||||||
|
Import.main(args);
|
||||||
|
fail("should be SecurityException");
|
||||||
|
} catch (SecurityException e) {
|
||||||
|
assertEquals(-1, newSecurityManager.getExitCode());
|
||||||
|
assertTrue(data.toString().contains("Wrong number of arguments:"));
|
||||||
|
assertTrue(data.toString().contains("-Dimport.bulk.output=/path/for/output"));
|
||||||
|
assertTrue(data.toString().contains("-Dimport.filter.class=<name of filter class>"));
|
||||||
|
assertTrue(data.toString().contains("-Dimport.bulk.output=/path/for/output"));
|
||||||
|
assertTrue(data.toString().contains("-Dmapreduce.reduce.speculative=false"));
|
||||||
|
} finally {
|
||||||
|
System.setErr(oldPrintStream);
|
||||||
|
System.setSecurityManager(SECURITY_MANAGER);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testExportScan() throws Exception {
|
||||||
|
int version = 100;
|
||||||
|
long startTime = System.currentTimeMillis();
|
||||||
|
long endTime = startTime + 1;
|
||||||
|
String prefix = "row";
|
||||||
|
String label_0 = "label_0";
|
||||||
|
String label_1 = "label_1";
|
||||||
|
String[] args = {
|
||||||
|
"table",
|
||||||
|
"outputDir",
|
||||||
|
String.valueOf(version),
|
||||||
|
String.valueOf(startTime),
|
||||||
|
String.valueOf(endTime),
|
||||||
|
prefix
|
||||||
|
};
|
||||||
|
Scan scan = ExportUtils.getScanFromCommandLine(UTIL.getConfiguration(), args);
|
||||||
|
assertEquals(version, scan.getMaxVersions());
|
||||||
|
assertEquals(startTime, scan.getTimeRange().getMin());
|
||||||
|
assertEquals(endTime, scan.getTimeRange().getMax());
|
||||||
|
assertEquals(true, (scan.getFilter() instanceof PrefixFilter));
|
||||||
|
assertEquals(0, Bytes.compareTo(((PrefixFilter) scan.getFilter()).getPrefix(), Bytes.toBytesBinary(prefix)));
|
||||||
|
String[] argsWithLabels = {
|
||||||
|
"-D " + ExportUtils.EXPORT_VISIBILITY_LABELS + "=" + label_0 + "," + label_1,
|
||||||
|
"table",
|
||||||
|
"outputDir",
|
||||||
|
String.valueOf(version),
|
||||||
|
String.valueOf(startTime),
|
||||||
|
String.valueOf(endTime),
|
||||||
|
prefix
|
||||||
|
};
|
||||||
|
Configuration conf = new Configuration(UTIL.getConfiguration());
|
||||||
|
// parse the "-D" options
|
||||||
|
String[] otherArgs = new GenericOptionsParser(conf, argsWithLabels).getRemainingArgs();
|
||||||
|
Scan scanWithLabels = ExportUtils.getScanFromCommandLine(conf, otherArgs);
|
||||||
|
assertEquals(version, scanWithLabels.getMaxVersions());
|
||||||
|
assertEquals(startTime, scanWithLabels.getTimeRange().getMin());
|
||||||
|
assertEquals(endTime, scanWithLabels.getTimeRange().getMax());
|
||||||
|
assertEquals(true, (scanWithLabels.getFilter() instanceof PrefixFilter));
|
||||||
|
assertEquals(0, Bytes.compareTo(((PrefixFilter) scanWithLabels.getFilter()).getPrefix(), Bytes.toBytesBinary(prefix)));
|
||||||
|
assertEquals(2, scanWithLabels.getAuthorizations().getLabels().size());
|
||||||
|
assertEquals(label_0, scanWithLabels.getAuthorizations().getLabels().get(0));
|
||||||
|
assertEquals(label_1, scanWithLabels.getAuthorizations().getLabels().get(1));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* test main method. Export should print help and call System.exit
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testExportMain() throws Throwable {
|
||||||
|
PrintStream oldPrintStream = System.err;
|
||||||
|
SecurityManager SECURITY_MANAGER = System.getSecurityManager();
|
||||||
|
LauncherSecurityManager newSecurityManager= new LauncherSecurityManager();
|
||||||
|
System.setSecurityManager(newSecurityManager);
|
||||||
|
ByteArrayOutputStream data = new ByteArrayOutputStream();
|
||||||
|
String[] args = {};
|
||||||
|
System.setErr(new PrintStream(data));
|
||||||
|
try {
|
||||||
|
System.setErr(new PrintStream(data));
|
||||||
|
runExportMain(args);
|
||||||
|
fail("should be SecurityException");
|
||||||
|
} catch (SecurityException e) {
|
||||||
|
assertEquals(-1, newSecurityManager.getExitCode());
|
||||||
|
String errMsg = data.toString();
|
||||||
|
assertTrue(errMsg.contains("Wrong number of arguments:"));
|
||||||
|
assertTrue(errMsg.contains(
|
||||||
|
"Usage: Export [-D <property=value>]* <tablename> <outputdir> [<versions> " +
|
||||||
|
"[<starttime> [<endtime>]] [^[regex pattern] or [Prefix] to filter]]"));
|
||||||
|
assertTrue(
|
||||||
|
errMsg.contains("-D hbase.mapreduce.scan.column.family=<family1>,<family2>, ..."));
|
||||||
|
assertTrue(errMsg.contains("-D hbase.mapreduce.include.deleted.rows=true"));
|
||||||
|
assertTrue(errMsg.contains("-D hbase.client.scanner.caching=100"));
|
||||||
|
assertTrue(errMsg.contains("-D hbase.export.scanner.batch=10"));
|
||||||
|
assertTrue(errMsg.contains("-D hbase.export.scanner.caching=100"));
|
||||||
|
} finally {
|
||||||
|
System.setErr(oldPrintStream);
|
||||||
|
System.setSecurityManager(SECURITY_MANAGER);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test map method of Importer
|
||||||
|
*/
|
||||||
|
@SuppressWarnings({ "unchecked", "rawtypes" })
|
||||||
|
@Test
|
||||||
|
public void testKeyValueImporter() throws Throwable {
|
||||||
|
CellImporter importer = new CellImporter();
|
||||||
|
Configuration configuration = new Configuration();
|
||||||
|
Context ctx = mock(Context.class);
|
||||||
|
when(ctx.getConfiguration()).thenReturn(configuration);
|
||||||
|
|
||||||
|
doAnswer(new Answer<Void>() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||||
|
ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArguments()[0];
|
||||||
|
MapReduceCell key = (MapReduceCell) invocation.getArguments()[1];
|
||||||
|
assertEquals("Key", Bytes.toString(writer.get()));
|
||||||
|
assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}).when(ctx).write(any(ImmutableBytesWritable.class), any(MapReduceCell.class));
|
||||||
|
|
||||||
|
importer.setup(ctx);
|
||||||
|
Result value = mock(Result.class);
|
||||||
|
KeyValue[] keys = {
|
||||||
|
new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
|
||||||
|
Bytes.toBytes("value")),
|
||||||
|
new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
|
||||||
|
Bytes.toBytes("value1")) };
|
||||||
|
when(value.rawCells()).thenReturn(keys);
|
||||||
|
importer.map(new ImmutableBytesWritable(Bytes.toBytes("Key")), value, ctx);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test addFilterAndArguments method of Import This method set couple
|
||||||
|
* parameters into Configuration
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testAddFilterAndArguments() throws IOException {
|
||||||
|
Configuration configuration = new Configuration();
|
||||||
|
|
||||||
|
List<String> args = new ArrayList<>();
|
||||||
|
args.add("param1");
|
||||||
|
args.add("param2");
|
||||||
|
|
||||||
|
Import.addFilterAndArguments(configuration, FilterBase.class, args);
|
||||||
|
assertEquals("org.apache.hadoop.hbase.filter.FilterBase",
|
||||||
|
configuration.get(Import.FILTER_CLASS_CONF_KEY));
|
||||||
|
assertEquals("param1,param2", configuration.get(Import.FILTER_ARGS_CONF_KEY));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDurability() throws Throwable {
|
||||||
|
// Create an export table.
|
||||||
|
String exportTableName = name.getMethodName() + "export";
|
||||||
|
try (Table exportTable = UTIL.createTable(TableName.valueOf(exportTableName), FAMILYA, 3);) {
|
||||||
|
|
||||||
|
// Insert some data
|
||||||
|
Put put = new Put(ROW1);
|
||||||
|
put.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
put.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
put.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
exportTable.put(put);
|
||||||
|
|
||||||
|
put = new Put(ROW2);
|
||||||
|
put.addColumn(FAMILYA, QUAL, now, QUAL);
|
||||||
|
put.addColumn(FAMILYA, QUAL, now + 1, QUAL);
|
||||||
|
put.addColumn(FAMILYA, QUAL, now + 2, QUAL);
|
||||||
|
exportTable.put(put);
|
||||||
|
|
||||||
|
// Run the export
|
||||||
|
String[] args = new String[] { exportTableName, FQ_OUTPUT_DIR, "1000"};
|
||||||
|
assertTrue(runExport(args));
|
||||||
|
|
||||||
|
// Create the table for import
|
||||||
|
String importTableName = name.getMethodName() + "import1";
|
||||||
|
Table importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3);
|
||||||
|
|
||||||
|
// Register the wal listener for the import table
|
||||||
|
RegionInfo region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
|
||||||
|
.getRegions(importTable.getName()).get(0).getRegionInfo();
|
||||||
|
TableWALActionListener walListener = new TableWALActionListener(region);
|
||||||
|
WAL wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
|
||||||
|
wal.registerWALActionsListener(walListener);
|
||||||
|
|
||||||
|
// Run the import with SKIP_WAL
|
||||||
|
args =
|
||||||
|
new String[] { "-D" + Import.WAL_DURABILITY + "=" + Durability.SKIP_WAL.name(),
|
||||||
|
importTableName, FQ_OUTPUT_DIR };
|
||||||
|
assertTrue(runImport(args));
|
||||||
|
//Assert that the wal is not visisted
|
||||||
|
assertTrue(!walListener.isWALVisited());
|
||||||
|
//Ensure that the count is 2 (only one version of key value is obtained)
|
||||||
|
assertTrue(getCount(importTable, null) == 2);
|
||||||
|
|
||||||
|
// Run the import with the default durability option
|
||||||
|
importTableName = name.getMethodName() + "import2";
|
||||||
|
importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3);
|
||||||
|
region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
|
||||||
|
.getRegions(importTable.getName()).get(0).getRegionInfo();
|
||||||
|
wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
|
||||||
|
walListener = new TableWALActionListener(region);
|
||||||
|
wal.registerWALActionsListener(walListener);
|
||||||
|
args = new String[] { importTableName, FQ_OUTPUT_DIR };
|
||||||
|
assertTrue(runImport(args));
|
||||||
|
//Assert that the wal is visisted
|
||||||
|
assertTrue(walListener.isWALVisited());
|
||||||
|
//Ensure that the count is 2 (only one version of key value is obtained)
|
||||||
|
assertTrue(getCount(importTable, null) == 2);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This listens to the {@link #visitLogEntryBeforeWrite(RegionInfo, WALKey, WALEdit)} to
|
||||||
|
* identify that an entry is written to the Write Ahead Log for the given table.
|
||||||
|
*/
|
||||||
|
private static class TableWALActionListener extends WALActionsListener.Base {
|
||||||
|
|
||||||
|
private RegionInfo regionInfo;
|
||||||
|
private boolean isVisited = false;
|
||||||
|
|
||||||
|
public TableWALActionListener(RegionInfo region) {
|
||||||
|
this.regionInfo = region;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
|
||||||
|
if (logKey.getTablename().getNameAsString().equalsIgnoreCase(
|
||||||
|
this.regionInfo.getTable().getNameAsString()) && (!logEdit.isMetaEdit())) {
|
||||||
|
isVisited = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isWALVisited() {
|
||||||
|
return isVisited;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,232 @@
|
||||||
|
/**
|
||||||
|
* 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.hadoop.hbase.mapreduce;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
import static org.mockito.Matchers.any;
|
||||||
|
import static org.mockito.Mockito.doAnswer;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.PrintStream;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
|
||||||
|
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.CellUtil;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALCellMapper;
|
||||||
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
|
||||||
|
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||||
|
import org.apache.hadoop.hbase.wal.WAL;
|
||||||
|
import org.apache.hadoop.hbase.wal.WALKey;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper.Context;
|
||||||
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.rules.TestName;
|
||||||
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Basic test for the WALPlayer M/R tool
|
||||||
|
*/
|
||||||
|
@Category({MapReduceTests.class, LargeTests.class})
|
||||||
|
public class TestCellBasedWALPlayer2 {
|
||||||
|
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||||
|
private static MiniHBaseCluster cluster;
|
||||||
|
private static Path rootDir;
|
||||||
|
private static Path walRootDir;
|
||||||
|
private static FileSystem fs;
|
||||||
|
private static FileSystem logFs;
|
||||||
|
private static Configuration conf;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public TestName name = new TestName();
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void beforeClass() throws Exception {
|
||||||
|
conf= TEST_UTIL.getConfiguration();
|
||||||
|
rootDir = TEST_UTIL.createRootDir();
|
||||||
|
walRootDir = TEST_UTIL.createWALRootDir();
|
||||||
|
fs = FSUtils.getRootDirFileSystem(conf);
|
||||||
|
logFs = FSUtils.getWALFileSystem(conf);
|
||||||
|
cluster = TEST_UTIL.startMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void afterClass() throws Exception {
|
||||||
|
TEST_UTIL.shutdownMiniCluster();
|
||||||
|
fs.delete(rootDir, true);
|
||||||
|
logFs.delete(walRootDir, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Simple end-to-end test
|
||||||
|
* @throws Exception
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testWALPlayer() throws Exception {
|
||||||
|
final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1");
|
||||||
|
final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
|
||||||
|
final byte[] FAMILY = Bytes.toBytes("family");
|
||||||
|
final byte[] COLUMN1 = Bytes.toBytes("c1");
|
||||||
|
final byte[] COLUMN2 = Bytes.toBytes("c2");
|
||||||
|
final byte[] ROW = Bytes.toBytes("row");
|
||||||
|
Table t1 = TEST_UTIL.createTable(tableName1, FAMILY);
|
||||||
|
Table t2 = TEST_UTIL.createTable(tableName2, FAMILY);
|
||||||
|
|
||||||
|
// put a row into the first table
|
||||||
|
Put p = new Put(ROW);
|
||||||
|
p.addColumn(FAMILY, COLUMN1, COLUMN1);
|
||||||
|
p.addColumn(FAMILY, COLUMN2, COLUMN2);
|
||||||
|
t1.put(p);
|
||||||
|
// delete one column
|
||||||
|
Delete d = new Delete(ROW);
|
||||||
|
d.addColumns(FAMILY, COLUMN1);
|
||||||
|
t1.delete(d);
|
||||||
|
|
||||||
|
// replay the WAL, map table 1 to table 2
|
||||||
|
WAL log = cluster.getRegionServer(0).getWAL(null);
|
||||||
|
log.rollWriter();
|
||||||
|
String walInputDir = new Path(cluster.getMaster().getMasterFileSystem()
|
||||||
|
.getWALRootDir(), HConstants.HREGION_LOGDIR_NAME).toString();
|
||||||
|
|
||||||
|
Configuration configuration= TEST_UTIL.getConfiguration();
|
||||||
|
WALPlayer player = new WALPlayer(configuration);
|
||||||
|
String optionName="_test_.name";
|
||||||
|
configuration.set(optionName, "1000");
|
||||||
|
player.setupTime(configuration, optionName);
|
||||||
|
assertEquals(1000,configuration.getLong(optionName,0));
|
||||||
|
assertEquals(0, ToolRunner.run(configuration, player,
|
||||||
|
new String[] {walInputDir, tableName1.getNameAsString(),
|
||||||
|
tableName2.getNameAsString() }));
|
||||||
|
|
||||||
|
|
||||||
|
// verify the WAL was player into table 2
|
||||||
|
Get g = new Get(ROW);
|
||||||
|
Result r = t2.get(g);
|
||||||
|
assertEquals(1, r.size());
|
||||||
|
assertTrue(CellUtil.matchingQualifier(r.rawCells()[0], COLUMN2));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test WALKeyValueMapper setup and map
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testWALKeyValueMapper() throws Exception {
|
||||||
|
testWALKeyValueMapper(WALPlayer.TABLES_KEY);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWALKeyValueMapperWithDeprecatedConfig() throws Exception {
|
||||||
|
testWALKeyValueMapper("hlog.input.tables");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testWALKeyValueMapper(final String tableConfigKey) throws Exception {
|
||||||
|
Configuration configuration = new Configuration();
|
||||||
|
configuration.set(tableConfigKey, "table");
|
||||||
|
WALCellMapper mapper = new WALCellMapper();
|
||||||
|
WALKey key = mock(WALKey.class);
|
||||||
|
when(key.getTablename()).thenReturn(TableName.valueOf("table"));
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
Mapper<WALKey, WALEdit, ImmutableBytesWritable, Cell>.Context context = mock(Context.class);
|
||||||
|
when(context.getConfiguration()).thenReturn(configuration);
|
||||||
|
|
||||||
|
WALEdit value = mock(WALEdit.class);
|
||||||
|
ArrayList<Cell> values = new ArrayList<>();
|
||||||
|
KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"), null);
|
||||||
|
|
||||||
|
values.add(kv1);
|
||||||
|
when(value.getCells()).thenReturn(values);
|
||||||
|
mapper.setup(context);
|
||||||
|
|
||||||
|
doAnswer(new Answer<Void>() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||||
|
ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArguments()[0];
|
||||||
|
MapReduceCell key = (MapReduceCell) invocation.getArguments()[1];
|
||||||
|
assertEquals("row", Bytes.toString(writer.get()));
|
||||||
|
assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}).when(context).write(any(ImmutableBytesWritable.class), any(MapReduceCell.class));
|
||||||
|
|
||||||
|
mapper.map(key, value, context);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test main method
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMainMethod() throws Exception {
|
||||||
|
|
||||||
|
PrintStream oldPrintStream = System.err;
|
||||||
|
SecurityManager SECURITY_MANAGER = System.getSecurityManager();
|
||||||
|
LauncherSecurityManager newSecurityManager= new LauncherSecurityManager();
|
||||||
|
System.setSecurityManager(newSecurityManager);
|
||||||
|
ByteArrayOutputStream data = new ByteArrayOutputStream();
|
||||||
|
String[] args = {};
|
||||||
|
System.setErr(new PrintStream(data));
|
||||||
|
try {
|
||||||
|
System.setErr(new PrintStream(data));
|
||||||
|
try {
|
||||||
|
WALPlayer.main(args);
|
||||||
|
fail("should be SecurityException");
|
||||||
|
} catch (SecurityException e) {
|
||||||
|
assertEquals(-1, newSecurityManager.getExitCode());
|
||||||
|
assertTrue(data.toString().contains("ERROR: Wrong number of arguments:"));
|
||||||
|
assertTrue(data.toString().contains("Usage: WALPlayer [options] <wal inputdir>" +
|
||||||
|
" <tables> [<tableMappings>]"));
|
||||||
|
assertTrue(data.toString().contains("-Dwal.bulk.output=/path/for/output"));
|
||||||
|
}
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
System.setErr(oldPrintStream);
|
||||||
|
System.setSecurityManager(SECURITY_MANAGER);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -119,6 +119,7 @@ import org.mockito.Mockito;
|
||||||
* emits keys and values like those of {@link PerformanceEvaluation}.
|
* emits keys and values like those of {@link PerformanceEvaluation}.
|
||||||
*/
|
*/
|
||||||
@Category({VerySlowMapReduceTests.class, LargeTests.class})
|
@Category({VerySlowMapReduceTests.class, LargeTests.class})
|
||||||
|
//TODO : Remove this in 3.0
|
||||||
public class TestHFileOutputFormat2 {
|
public class TestHFileOutputFormat2 {
|
||||||
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
||||||
withTimeout(this.getClass()).withLookingForStuckThread(true).build();
|
withTimeout(this.getClass()).withLookingForStuckThread(true).build();
|
||||||
|
|
|
@ -91,6 +91,7 @@ import org.mockito.stubbing.Answer;
|
||||||
* Tests the table import and table export MR job functionality
|
* Tests the table import and table export MR job functionality
|
||||||
*/
|
*/
|
||||||
@Category({VerySlowMapReduceTests.class, MediumTests.class})
|
@Category({VerySlowMapReduceTests.class, MediumTests.class})
|
||||||
|
//TODO : Remove this in 3.0
|
||||||
public class TestImportExport {
|
public class TestImportExport {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(TestImportExport.class);
|
private static final Log LOG = LogFactory.getLog(TestImportExport.class);
|
||||||
|
|
|
@ -70,6 +70,7 @@ import org.mockito.stubbing.Answer;
|
||||||
* Basic test for the WALPlayer M/R tool
|
* Basic test for the WALPlayer M/R tool
|
||||||
*/
|
*/
|
||||||
@Category({MapReduceTests.class, LargeTests.class})
|
@Category({MapReduceTests.class, LargeTests.class})
|
||||||
|
//TODO : Remove this in 3.0
|
||||||
public class TestWALPlayer {
|
public class TestWALPlayer {
|
||||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||||
private static MiniHBaseCluster cluster;
|
private static MiniHBaseCluster cluster;
|
||||||
|
|
Loading…
Reference in New Issue