From 66fb60d4a4828d181c4ebb7fd908f188e65141a4 Mon Sep 17 00:00:00 2001 From: Ramkrishna Date: Wed, 4 Oct 2017 14:44:19 +0530 Subject: [PATCH] HBASE-18649 Deprecate KV Usage in MR to move to Cells in 3.0 (Ram) --- .../mapreduce/MapReduceHFileSplitterJob.java | 33 +- .../org/apache/hadoop/hbase/CellUtil.java | 114 +- .../io/encoding/FastDiffDeltaEncoder.java | 3 +- .../io/encoding/PrefixKeyDeltaEncoder.java | 3 +- .../hadoop/hbase/util/ByteBufferUtils.java | 23 + .../org/apache/hadoop/hbase/TestCellUtil.java | 117 ++ .../hbase/mapreduce/CellSerialization.java | 96 ++ .../hbase/mapreduce/CellSortReducer.java | 60 + .../hadoop/hbase/mapreduce/CopyTable.java | 2 +- .../hbase/mapreduce/HFileOutputFormat2.java | 26 +- .../apache/hadoop/hbase/mapreduce/Import.java | 315 +++- .../hadoop/hbase/mapreduce/ImportTsv.java | 2 +- .../mapreduce/KeyValueSerialization.java | 8 +- .../hbase/mapreduce/KeyValueSortReducer.java | 3 + .../hbase/mapreduce/PutSortReducer.java | 2 +- .../hbase/mapreduce/TableMapReduceUtil.java | 2 +- .../hbase/mapreduce/TextSortReducer.java | 2 +- .../hadoop/hbase/mapreduce/WALPlayer.java | 54 +- .../hadoop/hbase/util/MapReduceCell.java | 270 +++ .../TestCellBasedHFileOutputFormat2.java | 1496 +++++++++++++++++ .../mapreduce/TestCellBasedImportExport2.java | 801 +++++++++ .../mapreduce/TestCellBasedWALPlayer2.java | 232 +++ .../mapreduce/TestHFileOutputFormat2.java | 1 + .../hbase/mapreduce/TestImportExport.java | 1 + .../hadoop/hbase/mapreduce/TestWALPlayer.java | 1 + 25 files changed, 3561 insertions(+), 106 deletions(-) create mode 100644 hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSerialization.java create mode 100644 hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java create mode 100644 hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java create mode 100644 hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java create mode 100644 hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java create mode 100644 hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java index 97ece3d964c..51a6b1ddd6a 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java @@ -24,22 +24,21 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; 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.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; 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.HFileOutputFormat2; -import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.MapReduceCell; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.Job; 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.util.Tool; 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 @@ -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 - * {@link KeyValueSortReducer} + * {@link CellSortReducer} */ static class HFileCellMapper extends - Mapper { + Mapper { @Override - public void map(NullWritable key, KeyValue value, Context context) throws IOException, - InterruptedException { - // Convert value to KeyValue if subclass - 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); + public void map(NullWritable key, Cell value, Context context) + throws IOException, InterruptedException { + context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), new MapReduceCell(value)); } @Override @@ -119,14 +110,14 @@ public class MapReduceHFileSplitterJob extends Configured implements Tool { LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs); TableName tableName = TableName.valueOf(tabName); job.setMapperClass(HFileCellMapper.class); - job.setReducerClass(KeyValueSortReducer.class); + job.setReducerClass(CellSortReducer.class); Path outputDir = new Path(hfileOutPath); FileOutputFormat.setOutputPath(job, outputDir); - job.setMapOutputValueClass(KeyValue.class); + job.setMapOutputValueClass(MapReduceCell.class); try (Connection conn = ConnectionFactory.createConnection(conf); Table table = conn.getTable(tableName); RegionLocator regionLocator = conn.getRegionLocator(tableName)) { - HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); + HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); } LOG.debug("success configuring load incremental job"); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java index dc5df304044..7a3695fcd01 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -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.COLUMN_FAMILY_DELIM_ARRAY; +import java.io.DataOutput; import java.io.DataOutputStream; import java.io.IOException; 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 - * @return Estimate of the cell size in bytes. + * @return Estimate of the cell size in bytes plus an extra SIZEOF_INT indicating the + * actual cell length. */ public static int estimatedSerializedSizeOf(final Cell cell) { if (cell instanceof ExtendedCell) { @@ -1762,9 +1766,48 @@ public final class CellUtil { * timestamp><1 byte type> * @param cell * @param out + * @deprecated Use {@link #writeFlatKey(Cell, DataOutput)} * @throws IOException */ + @Deprecated 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(); byte fLen = cell.getFamilyLength(); int qLen = cell.getQualifierLength(); @@ -1790,6 +1833,69 @@ public final class CellUtil { 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 { short rowLen = cell.getRowLength(); byte fLen = cell.getFamilyLength(); @@ -1844,7 +1950,7 @@ public final class CellUtil { public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength, int commonPrefix) throws IOException { if (cell instanceof ByteBufferCell) { - ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(), + ByteBufferUtils.copyBufferToStream((DataOutput)out, ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix); } else { 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, int qlength, int commonPrefix) throws IOException { if (cell instanceof ByteBufferCell) { - ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(), + ByteBufferUtils.copyBufferToStream((DataOutput)out, ((ByteBufferCell) cell).getQualifierByteBuffer(), ((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix); } else { out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix, diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java index ac810490d5e..03cf7680c72 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java @@ -17,6 +17,7 @@ package org.apache.hadoop.hbase.io.encoding; import java.io.DataInputStream; +import java.io.DataOutput; import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -262,7 +263,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { ByteBufferUtils.putCompressedInt(out, kLength); ByteBufferUtils.putCompressedInt(out, vLength); ByteBufferUtils.putCompressedInt(out, 0); - CellUtil.writeFlatKey(cell, out); + CellUtil.writeFlatKey(cell, (DataOutput)out); // Write the value part CellUtil.writeValue(out, cell, cell.getValueLength()); } else { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java index 6f529db32a6..8edb305b929 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java @@ -17,6 +17,7 @@ package org.apache.hadoop.hbase.io.encoding; import java.io.DataInputStream; +import java.io.DataOutput; import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -59,7 +60,7 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder { ByteBufferUtils.putCompressedInt(out, klength); ByteBufferUtils.putCompressedInt(out, vlength); ByteBufferUtils.putCompressedInt(out, 0); - CellUtil.writeFlatKey(cell, out); + CellUtil.writeFlatKey(cell, (DataOutput)out); } else { // find a common prefix and skip it int common = CellUtil.findCommonPrefixInFlatKey(cell, state.prevCell, true, true); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java index 7ef578dca03..3fc1a7bdacf 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java @@ -20,6 +20,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe import java.io.ByteArrayOutputStream; import java.io.DataInput; import java.io.DataInputStream; +import java.io.DataOutput; import java.io.IOException; import java.io.InputStream; 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, final int fitInBytes) throws IOException { long tmpValue = value; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java index ac9fc45a1e7..3bd1b66eeed 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -488,4 +490,119 @@ public class TestCellUtil { bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining()); 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(); + } + } } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSerialization.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSerialization.java new file mode 100644 index 00000000000..6f4419e462a --- /dev/null +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSerialization.java @@ -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 { + @Override + public boolean accept(Class c) { + return Cell.class.isAssignableFrom(c); + } + + @Override + public CellDeserializer getDeserializer(Class t) { + return new CellDeserializer(); + } + + @Override + public CellSerializer getSerializer(Class c) { + return new CellSerializer(); + } + + public static class CellDeserializer implements Deserializer { + 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 { + 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); + } + } +} diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java new file mode 100644 index 00000000000..c33ee15537f --- /dev/null +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java @@ -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 { + protected void reduce(ImmutableBytesWritable row, Iterable kvs, + Reducer.Context context) + throws java.io.IOException, InterruptedException { + TreeSet 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); + } + } +} diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java index 679d991887e..81af16580c7 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java @@ -139,7 +139,7 @@ public class CopyTable extends Configured implements Tool { job.setNumReduceTasks(0); if (bulkload) { - TableMapReduceUtil.initTableMapperJob(tableName, scan, Import.KeyValueImporter.class, null, + TableMapReduceUtil.initTableMapperJob(tableName, scan, Import.CellImporter.class, null, null, job); // We need to split the inputs by destination tables so that output of Map can be bulk-loaded. diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java index e75774260e6..20b2d42d733 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java @@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 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.SequenceFile; 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.partition.TotalOrderPartitioner; import org.apache.yetus.audience.InterfaceAudience; - import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; /** @@ -233,14 +233,13 @@ public class HFileOutputFormat2 private final Map writers = new TreeMap<>(Bytes.BYTES_COMPARATOR); 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; @Override public void write(ImmutableBytesWritable row, V cell) throws IOException { - KeyValue kv = KeyValueUtil.ensureKeyValue(cell); - + Cell kv = cell; // null input == user explicitly wants to flush if (row == null && kv == null) { rollWriters(); @@ -248,7 +247,7 @@ public class HFileOutputFormat2 } byte[] rowKey = CellUtil.cloneRow(kv); - long length = kv.getLength(); + int length = (CellUtil.estimatedSerializedSizeOf(kv)) - Bytes.SIZEOF_INT; byte[] family = CellUtil.cloneFamily(kv); byte[] tableNameBytes = null; if (writeMultipleTables) { @@ -337,7 +336,8 @@ public class HFileOutputFormat2 } // 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.written += length; @@ -578,10 +578,11 @@ public class HFileOutputFormat2 configureIncrementalLoad(job, singleTableInfo, HFileOutputFormat2.class); } - static void configureIncrementalLoad(Job job, List multiTableInfo, Class> cls) throws IOException { + static void configureIncrementalLoad(Job job, List multiTableInfo, + Class> cls) throws IOException { Configuration conf = job.getConfiguration(); job.setOutputKeyClass(ImmutableBytesWritable.class); - job.setOutputValueClass(KeyValue.class); + job.setOutputValueClass(MapReduceCell.class); job.setOutputFormatClass(cls); 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 // sort the incoming values. // TODO it would be nice to pick one or the other of these formats. - if (KeyValue.class.equals(job.getMapOutputValueClass())) { - job.setReducerClass(KeyValueSortReducer.class); + if (KeyValue.class.equals(job.getMapOutputValueClass()) + || MapReduceCell.class.equals(job.getMapOutputValueClass())) { + job.setReducerClass(CellSortReducer.class); } else if (Put.class.equals(job.getMapOutputValueClass())) { job.setReducerClass(PutSortReducer.class); } else if (Text.class.equals(job.getMapOutputValueClass())) { @@ -607,7 +609,7 @@ public class HFileOutputFormat2 conf.setStrings("io.serializations", conf.get("io.serializations"), MutationSerialization.class.getName(), ResultSerialization.class.getName(), - KeyValueSerialization.class.getName()); + CellSerialization.class.getName()); if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) { LOG.info("bulkload locality sensitive enabled"); @@ -655,7 +657,7 @@ public class HFileOutputFormat2 Configuration conf = job.getConfiguration(); job.setOutputKeyClass(ImmutableBytesWritable.class); - job.setOutputValueClass(KeyValue.class); + job.setOutputValueClass(MapReduceCell.class); job.setOutputFormatClass(HFileOutputFormat2.class); ArrayList singleTableDescriptor = new ArrayList<>(1); diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java index 28962bb51a6..7af77383126 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java @@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; 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.ZooKeeperWatcher; 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"; - public static class KeyValueWritableComparablePartitioner - extends Partitioner { - private static KeyValueWritableComparable[] START_KEYS = null; + public static class CellWritableComparablePartitioner + extends Partitioner { + private static CellWritableComparable[] START_KEYS = null; @Override - public int getPartition(KeyValueWritableComparable key, KeyValue value, + public int getPartition(CellWritableComparable key, Cell value, int numPartitions) { for (int i = 0; i < START_KEYS.length; ++i) { 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 { + private static KeyValueWritableComparable[] START_KEYS = null; + + @Override + public int getPartition(KeyValueWritableComparable key, KeyValue value, int numPartitions) { + for (int i = 0; i < START_KEYS.length; ++i) { + if (key.compareTo(START_KEYS[i]) <= 0) { + return i; + } + } + return START_KEYS.length; + } + } + public static class KeyValueWritableComparable implements WritableComparable { @@ -119,8 +140,7 @@ public class Import extends Configured implements Tool { static { // register this comparator - WritableComparator.define(KeyValueWritableComparable.class, - new KeyValueWritableComparator()); + WritableComparator.define(KeyValueWritableComparable.class, new KeyValueWritableComparator()); } public KeyValueWritableComparable() { @@ -141,10 +161,10 @@ public class Import extends Configured implements Tool { } @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") + @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(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 { @@ -166,18 +186,93 @@ public class Import extends Configured implements Tool { } + public static class CellWritableComparable + implements WritableComparable { + + 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 - extends - Reducer { - protected void reduce( - KeyValueWritableComparable row, - Iterable kvs, - Reducer.Context context) + extends Reducer { + protected void reduce(KeyValueWritableComparable row, Iterable kvs, + Reducer.Context context) throws java.io.IOException, InterruptedException { int index = 0; for (KeyValue kv : kvs) { context.write(new ImmutableBytesWritable(kv.getRowArray()), kv); + if (++index % 100 == 0) context.setStatus("Wrote " + index + " KeyValues, " + + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray())); + } + } + } + + public static class CellReducer + extends + Reducer { + protected void reduce( + CellWritableComparable row, + Iterable kvs, + Reducer.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) context.setStatus("Wrote " + index + " KeyValues, " + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray())); @@ -185,30 +280,31 @@ public class Import extends Configured implements Tool { } } + /** + * @deprecated Use {@link CellSortImporter}. Will be removed from 3.0 onwards + */ + @Deprecated public static class KeyValueSortImporter extends TableMapper { private Map cfRenameMap; 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 value The columns. - * @param context The current context. + * @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 { + 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())); + 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()))) { + 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 @@ -249,14 +345,57 @@ public class Import extends Configured implements Tool { /** * 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", - justification="Writables are going away and this has been this way forever") + @Deprecated + @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 { private Map cfRenameMap; private Filter filter; private static final Log LOG = LogFactory.getLog(KeyValueImporter.class); + /** + * @param row The current table row key. + * @param value The columns. + * @param context The current context. + * @throws IOException When something is broken with the data. + */ + @Override + public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException { + try { + if (LOG.isTraceEnabled()) { + LOG.trace( + "Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength())); + } + if (filter == null || !filter.filterRowKey( + 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 { + private Map cfRenameMap; + private Filter filter; + private static final Log LOG = LogFactory.getLog(CellImporter.class); + /** * @param row The current table row key. * @param value The columns. @@ -279,8 +418,72 @@ public class Import extends Configured implements Tool { 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))); + Cell ret = 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 { + private Map 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) { @@ -505,21 +708,21 @@ public class Import extends Configured implements Tool { if(cfRenameMap != null) { // If there's a rename mapping for this CF, create a new KeyValue byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv)); - if(newCfName != null) { - kv = new KeyValue(kv.getRowArray(), // row buffer - kv.getRowOffset(), // row offset - kv.getRowLength(), // row length - newCfName, // CF buffer - 0, // CF offset - newCfName.length, // CF length - kv.getQualifierArray(), // qualifier buffer - kv.getQualifierOffset(), // qualifier offset - kv.getQualifierLength(), // qualifier length - kv.getTimestamp(), // timestamp - KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type - kv.getValueArray(), // value buffer - kv.getValueOffset(), // value offset - kv.getValueLength()); // value length + if (newCfName != null) { + kv = new KeyValue(kv.getRowArray(), // row buffer + kv.getRowOffset(), // row offset + kv.getRowLength(), // row length + newCfName, // CF buffer + 0, // CF offset + newCfName.length, // CF length + kv.getQualifierArray(), // qualifier buffer + kv.getQualifierOffset(), // qualifier offset + kv.getQualifierLength(), // qualifier length + kv.getTimestamp(), // timestamp + KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type + kv.getValueArray(), // value buffer + kv.getValueOffset(), // value offset + kv.getValueLength()); // value length } } return kv; @@ -626,35 +829,35 @@ public class Import extends Configured implements Tool { Table table = conn.getTable(tableName); RegionLocator regionLocator = conn.getRegionLocator(tableName)) { HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); - job.setMapperClass(KeyValueSortImporter.class); - job.setReducerClass(KeyValueReducer.class); + job.setMapperClass(CellSortImporter.class); + job.setReducerClass(CellReducer.class); Path outputDir = new Path(hfileOutPath); FileOutputFormat.setOutputPath(job, outputDir); - job.setMapOutputKeyClass(KeyValueWritableComparable.class); - job.setMapOutputValueClass(KeyValue.class); + job.setMapOutputKeyClass(CellWritableComparable.class); + job.setMapOutputValueClass(MapReduceCell.class); job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class", - KeyValueWritableComparable.KeyValueWritableComparator.class, + CellWritableComparable.CellWritableComparator.class, RawComparator.class); Path partitionsPath = new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration())); FileSystem fs = FileSystem.get(job.getConfiguration()); fs.deleteOnExit(partitionsPath); - job.setPartitionerClass(KeyValueWritableComparablePartitioner.class); + job.setPartitionerClass(CellWritableComparablePartitioner.class); job.setNumReduceTasks(regionLocator.getStartKeys().length); TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class); } } else if (hfileOutPath != null) { LOG.info("writing to hfiles for bulk load."); - job.setMapperClass(KeyValueImporter.class); + job.setMapperClass(CellImporter.class); try (Connection conn = ConnectionFactory.createConnection(conf); Table table = conn.getTable(tableName); RegionLocator regionLocator = conn.getRegionLocator(tableName)){ - job.setReducerClass(KeyValueSortReducer.class); + job.setReducerClass(CellSortReducer.class); Path outputDir = new Path(hfileOutPath); FileOutputFormat.setOutputPath(job, outputDir); job.setMapOutputKeyClass(ImmutableBytesWritable.class); - job.setMapOutputValueClass(KeyValue.class); + job.setMapOutputValueClass(MapReduceCell.class); HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class); diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java index 4a1dea88fdd..d672803b4bc 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java @@ -584,7 +584,7 @@ public class ImportTsv extends Configured implements Tool { job.getConfiguration().setStrings("io.serializations", job.getConfiguration().get("io.serializations"), MutationSerialization.class.getName(), ResultSerialization.class.getName(), - KeyValueSerialization.class.getName()); + CellSerialization.class.getName()); } TableMapReduceUtil.addDependencyJars(job); TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java index d0cc00e0e5b..3207712f72b 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java @@ -29,7 +29,13 @@ 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 + * @deprecated Use {@link CellSerialization}. Will be + * removed from 3.0 onwards + */ +@Deprecated @InterfaceAudience.Public public class KeyValueSerialization implements Serialization { @Override diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java index 824f23dd6bc..3ba5198dd16 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java @@ -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 * memory sorting. * @see HFileOutputFormat2 + * @deprecated Use {@link CellSortReducer}. Will be removed from + * 3.0 onwards */ +@Deprecated @InterfaceAudience.Public public class KeyValueSortReducer extends Reducer { diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java index 49902f4ff69..bb935c3c1a4 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java @@ -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 * memory sorting. * @see HFileOutputFormat2 - * @see KeyValueSortReducer + * @see CellSortReducer */ @InterfaceAudience.Public public class PutSortReducer extends diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java index c99af4e5e99..cdecf145ec0 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java @@ -208,7 +208,7 @@ public class TableMapReduceUtil { conf.set(TableInputFormat.SCAN, convertScanToString(scan)); conf.setStrings("io.serializations", conf.get("io.serializations"), MutationSerialization.class.getName(), ResultSerialization.class.getName(), - KeyValueSerialization.class.getName()); + CellSerialization.class.getName()); if (addDependencyJars) { addDependencyJars(job); } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java index 14da314860f..2aaa4ebc6c4 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java @@ -45,7 +45,7 @@ import org.apache.hadoop.util.StringUtils; /** * Emits Sorted KeyValues. Parse the passed text and creates KeyValues. Sorts them before emit. * @see HFileOutputFormat2 - * @see KeyValueSortReducer + * @see CellSortReducer * @see PutSortReducer */ @InterfaceAudience.Public diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java index e1d8d28e884..02c46400cd0 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; 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.ConnectionFactory; 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.io.ImmutableBytesWritable; 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.MapReduceCell; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.mapreduce.Job; 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.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; /** * 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. * This one can be used together with {@link KeyValueSortReducer} + * @deprecated Use {@link WALCellMapper}. Will be removed from 3.0 onwards */ + @Deprecated static class WALKeyValueMapper extends Mapper { 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 { + 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"); } TableName tableName = TableName.valueOf(tables[0]); - job.setMapperClass(WALKeyValueMapper.class); - job.setReducerClass(KeyValueSortReducer.class); + job.setMapperClass(WALCellMapper.class); + job.setReducerClass(CellSortReducer.class); Path outputDir = new Path(hfileOutPath); FileOutputFormat.setOutputPath(job, outputDir); - job.setMapOutputValueClass(KeyValue.class); + job.setMapOutputValueClass(MapReduceCell.class); try (Connection conn = ConnectionFactory.createConnection(conf); Table table = conn.getTable(tableName); RegionLocator regionLocator = conn.getRegionLocator(tableName)) { diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java new file mode 100644 index 00000000000..c0f74a5c673 --- /dev/null +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java @@ -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); + } + } +} diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java new file mode 100644 index 00000000000..28a7be1c877 --- /dev/null +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java @@ -0,0 +1,1496 @@ +/** + * + * 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.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.ArrayBackedTag; +import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HDFSBlocksDistribution; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.HadoopShims; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PerformanceEvaluation; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.TagUtil; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; +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.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFile.Reader; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem; +import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestRule; +import org.mockito.Mockito; + +/** + * Simple test for {@link HFileOutputFormat2}. + * Sets up and runs a mapreduce job that writes hfile output. + * Creates a few inner classes to implement splits and an inputformat that + * emits keys and values like those of {@link PerformanceEvaluation}. + */ +@Category({VerySlowMapReduceTests.class, LargeTests.class}) +public class TestCellBasedHFileOutputFormat2 { + @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). + withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + private final static int ROWSPERSPLIT = 1024; + + public static final byte[] FAMILY_NAME = TestHRegionFileSystem.FAMILY_NAME; + private static final byte[][] FAMILIES = { + Bytes.add(FAMILY_NAME, Bytes.toBytes("-A")), Bytes.add(FAMILY_NAME, Bytes.toBytes("-B"))}; + private static final TableName[] TABLE_NAMES = Stream.of("TestTable", "TestTable2", + "TestTable3").map(TableName::valueOf).toArray(TableName[]::new); + + private HBaseTestingUtility util = new HBaseTestingUtility(); + + private static final Log LOG = LogFactory.getLog(TestCellBasedHFileOutputFormat2.class); + + /** + * Simple mapper that makes KeyValue output. + */ + static class RandomKVGeneratingMapper + extends Mapper { + + private int keyLength; + private static final int KEYLEN_DEFAULT=10; + private static final String KEYLEN_CONF="randomkv.key.length"; + + private int valLength; + private static final int VALLEN_DEFAULT=10; + private static final String VALLEN_CONF="randomkv.val.length"; + private static final byte [] QUALIFIER = Bytes.toBytes("data"); + private boolean multiTableMapper = false; + private TableName[] tables = null; + + + @Override + protected void setup(Context context) throws IOException, + InterruptedException { + super.setup(context); + + Configuration conf = context.getConfiguration(); + keyLength = conf.getInt(KEYLEN_CONF, KEYLEN_DEFAULT); + valLength = conf.getInt(VALLEN_CONF, VALLEN_DEFAULT); + multiTableMapper = conf.getBoolean(HFileOutputFormat2.MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, + false); + if (multiTableMapper) { + tables = TABLE_NAMES; + } else { + tables = new TableName[]{TABLE_NAMES[0]}; + } + } + + @Override + protected void map( + NullWritable n1, NullWritable n2, + Mapper.Context context) + throws java.io.IOException ,InterruptedException + { + + byte keyBytes[] = new byte[keyLength]; + byte valBytes[] = new byte[valLength]; + + int taskId = context.getTaskAttemptID().getTaskID().getId(); + assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!"; + Random random = new Random(); + byte[] key; + for (int j = 0; j < tables.length; ++j) { + for (int i = 0; i < ROWSPERSPLIT; i++) { + random.nextBytes(keyBytes); + // Ensure that unique tasks generate unique keys + keyBytes[keyLength - 1] = (byte) (taskId & 0xFF); + random.nextBytes(valBytes); + key = keyBytes; + if (multiTableMapper) { + key = MultiTableHFileOutputFormat.createCompositeKey(tables[j].getName(), keyBytes); + } + + for (byte[] family : TestCellBasedHFileOutputFormat2.FAMILIES) { + Cell kv = new KeyValue(keyBytes, family, QUALIFIER, valBytes); + context.write(new ImmutableBytesWritable(key), kv); + } + } + } + } + } + + /** + * Simple mapper that makes Put output. + */ + static class RandomPutGeneratingMapper + extends Mapper { + + private int keyLength; + private static final int KEYLEN_DEFAULT = 10; + private static final String KEYLEN_CONF = "randomkv.key.length"; + + private int valLength; + private static final int VALLEN_DEFAULT = 10; + private static final String VALLEN_CONF = "randomkv.val.length"; + private static final byte[] QUALIFIER = Bytes.toBytes("data"); + private boolean multiTableMapper = false; + private TableName[] tables = null; + + @Override + protected void setup(Context context) throws IOException, + InterruptedException { + super.setup(context); + + Configuration conf = context.getConfiguration(); + keyLength = conf.getInt(KEYLEN_CONF, KEYLEN_DEFAULT); + valLength = conf.getInt(VALLEN_CONF, VALLEN_DEFAULT); + multiTableMapper = conf.getBoolean(HFileOutputFormat2.MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, + false); + if (multiTableMapper) { + tables = TABLE_NAMES; + } else { + tables = new TableName[]{TABLE_NAMES[0]}; + } + } + + @Override + protected void map( + NullWritable n1, NullWritable n2, + Mapper.Context context) + throws java.io.IOException, InterruptedException { + + byte keyBytes[] = new byte[keyLength]; + byte valBytes[] = new byte[valLength]; + + int taskId = context.getTaskAttemptID().getTaskID().getId(); + assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!"; + + Random random = new Random(); + byte[] key; + for (int j = 0; j < tables.length; ++j) { + for (int i = 0; i < ROWSPERSPLIT; i++) { + random.nextBytes(keyBytes); + // Ensure that unique tasks generate unique keys + keyBytes[keyLength - 1] = (byte) (taskId & 0xFF); + random.nextBytes(valBytes); + key = keyBytes; + if (multiTableMapper) { + key = MultiTableHFileOutputFormat.createCompositeKey(tables[j].getName(), keyBytes); + } + + for (byte[] family : TestCellBasedHFileOutputFormat2.FAMILIES) { + Put p = new Put(keyBytes); + p.addColumn(family, QUALIFIER, valBytes); + // set TTL to very low so that the scan does not return any value + p.setTTL(1l); + context.write(new ImmutableBytesWritable(key), p); + } + } + } + } + } + + private void setupRandomGeneratorMapper(Job job, boolean putSortReducer) { + if (putSortReducer) { + job.setInputFormatClass(NMapInputFormat.class); + job.setMapperClass(RandomPutGeneratingMapper.class); + job.setMapOutputKeyClass(ImmutableBytesWritable.class); + job.setMapOutputValueClass(Put.class); + } else { + job.setInputFormatClass(NMapInputFormat.class); + job.setMapperClass(RandomKVGeneratingMapper.class); + job.setMapOutputKeyClass(ImmutableBytesWritable.class); + job.setMapOutputValueClass(KeyValue.class); + } + } + + /** + * Test that {@link HFileOutputFormat2} RecordWriter amends timestamps if + * passed a keyvalue whose timestamp is {@link HConstants#LATEST_TIMESTAMP}. + * @see HBASE-2615 + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void test_LATEST_TIMESTAMP_isReplaced() + throws Exception { + Configuration conf = new Configuration(this.util.getConfiguration()); + RecordWriter writer = null; + TaskAttemptContext context = null; + Path dir = + util.getDataTestDir("test_LATEST_TIMESTAMP_isReplaced"); + try { + Job job = new Job(conf); + FileOutputFormat.setOutputPath(job, dir); + context = createTestTaskAttemptContext(job); + HFileOutputFormat2 hof = new HFileOutputFormat2(); + writer = hof.getRecordWriter(context); + final byte [] b = Bytes.toBytes("b"); + + // Test 1. Pass a KV that has a ts of LATEST_TIMESTAMP. It should be + // changed by call to write. Check all in kv is same but ts. + KeyValue kv = new KeyValue(b, b, b); + KeyValue original = kv.clone(); + writer.write(new ImmutableBytesWritable(), kv); + assertFalse(original.equals(kv)); + assertTrue(Bytes.equals(CellUtil.cloneRow(original), CellUtil.cloneRow(kv))); + assertTrue(Bytes.equals(CellUtil.cloneFamily(original), CellUtil.cloneFamily(kv))); + assertTrue(Bytes.equals(CellUtil.cloneQualifier(original), CellUtil.cloneQualifier(kv))); + assertNotSame(original.getTimestamp(), kv.getTimestamp()); + assertNotSame(HConstants.LATEST_TIMESTAMP, kv.getTimestamp()); + + // Test 2. Now test passing a kv that has explicit ts. It should not be + // changed by call to record write. + kv = new KeyValue(b, b, b, kv.getTimestamp() - 1, b); + original = kv.clone(); + writer.write(new ImmutableBytesWritable(), kv); + assertTrue(original.equals(kv)); + } finally { + if (writer != null && context != null) writer.close(context); + dir.getFileSystem(conf).delete(dir, true); + } + } + + private TaskAttemptContext createTestTaskAttemptContext(final Job job) + throws Exception { + HadoopShims hadoop = CompatibilitySingletonFactory.getInstance(HadoopShims.class); + TaskAttemptContext context = hadoop.createTestTaskAttemptContext( + job, "attempt_201402131733_0001_m_000000_0"); + return context; + } + + /* + * Test that {@link HFileOutputFormat2} creates an HFile with TIMERANGE + * metadata used by time-restricted scans. + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void test_TIMERANGE() throws Exception { + Configuration conf = new Configuration(this.util.getConfiguration()); + RecordWriter writer = null; + TaskAttemptContext context = null; + Path dir = + util.getDataTestDir("test_TIMERANGE_present"); + LOG.info("Timerange dir writing to dir: "+ dir); + try { + // build a record writer using HFileOutputFormat2 + Job job = new Job(conf); + FileOutputFormat.setOutputPath(job, dir); + context = createTestTaskAttemptContext(job); + HFileOutputFormat2 hof = new HFileOutputFormat2(); + writer = hof.getRecordWriter(context); + + // Pass two key values with explicit times stamps + final byte [] b = Bytes.toBytes("b"); + + // value 1 with timestamp 2000 + KeyValue kv = new KeyValue(b, b, b, 2000, b); + KeyValue original = kv.clone(); + writer.write(new ImmutableBytesWritable(), kv); + assertEquals(original,kv); + + // value 2 with timestamp 1000 + kv = new KeyValue(b, b, b, 1000, b); + original = kv.clone(); + writer.write(new ImmutableBytesWritable(), kv); + assertEquals(original, kv); + + // verify that the file has the proper FileInfo. + writer.close(context); + + // the generated file lives 1 directory down from the attempt directory + // and is the only file, e.g. + // _attempt__0000_r_000000_0/b/1979617994050536795 + FileSystem fs = FileSystem.get(conf); + Path attemptDirectory = hof.getDefaultWorkFile(context, "").getParent(); + FileStatus[] sub1 = fs.listStatus(attemptDirectory); + FileStatus[] file = fs.listStatus(sub1[0].getPath()); + + // open as HFile Reader and pull out TIMERANGE FileInfo. + HFile.Reader rd = + HFile.createReader(fs, file[0].getPath(), new CacheConfig(conf), true, conf); + Map finfo = rd.loadFileInfo(); + byte[] range = finfo.get("TIMERANGE".getBytes("UTF-8")); + assertNotNull(range); + + // unmarshall and check values. + TimeRangeTracker timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.SYNC); + Writables.copyWritable(range, timeRangeTracker); + LOG.info(timeRangeTracker.getMin() + + "...." + timeRangeTracker.getMax()); + assertEquals(1000, timeRangeTracker.getMin()); + assertEquals(2000, timeRangeTracker.getMax()); + rd.close(); + } finally { + if (writer != null && context != null) writer.close(context); + dir.getFileSystem(conf).delete(dir, true); + } + } + + /** + * Run small MR job. + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testWritingPEData() throws Exception { + Configuration conf = util.getConfiguration(); + Path testDir = util.getDataTestDirOnTestFS("testWritingPEData"); + FileSystem fs = testDir.getFileSystem(conf); + + // Set down this value or we OOME in eclipse. + conf.setInt("mapreduce.task.io.sort.mb", 20); + // Write a few files. + conf.setLong(HConstants.HREGION_MAX_FILESIZE, 64 * 1024); + + Job job = new Job(conf, "testWritingPEData"); + setupRandomGeneratorMapper(job, false); + // This partitioner doesn't work well for number keys but using it anyways + // just to demonstrate how to configure it. + byte[] startKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT]; + byte[] endKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT]; + + Arrays.fill(startKey, (byte)0); + Arrays.fill(endKey, (byte)0xff); + + job.setPartitionerClass(SimpleTotalOrderPartitioner.class); + // Set start and end rows for partitioner. + SimpleTotalOrderPartitioner.setStartKey(job.getConfiguration(), startKey); + SimpleTotalOrderPartitioner.setEndKey(job.getConfiguration(), endKey); + job.setReducerClass(CellSortReducer.class); + job.setOutputFormatClass(HFileOutputFormat2.class); + job.setNumReduceTasks(4); + job.getConfiguration().setStrings("io.serializations", conf.get("io.serializations"), + MutationSerialization.class.getName(), ResultSerialization.class.getName(), + CellSerialization.class.getName()); + + FileOutputFormat.setOutputPath(job, testDir); + assertTrue(job.waitForCompletion(false)); + FileStatus [] files = fs.listStatus(testDir); + assertTrue(files.length > 0); + } + + /** + * Test that {@link HFileOutputFormat2} RecordWriter writes tags such as ttl into + * hfile. + */ + @Test + public void test_WritingTagData() + throws Exception { + Configuration conf = new Configuration(this.util.getConfiguration()); + final String HFILE_FORMAT_VERSION_CONF_KEY = "hfile.format.version"; + conf.setInt(HFILE_FORMAT_VERSION_CONF_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS); + RecordWriter writer = null; + TaskAttemptContext context = null; + Path dir = + util.getDataTestDir("WritingTagData"); + try { + conf.set(HFileOutputFormat2.OUTPUT_TABLE_NAME_CONF_KEY, TABLE_NAMES[0].getNameAsString()); + // turn locality off to eliminate getRegionLocation fail-and-retry time when writing kvs + conf.setBoolean(HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY, false); + Job job = new Job(conf); + FileOutputFormat.setOutputPath(job, dir); + context = createTestTaskAttemptContext(job); + HFileOutputFormat2 hof = new HFileOutputFormat2(); + writer = hof.getRecordWriter(context); + final byte [] b = Bytes.toBytes("b"); + + List< Tag > tags = new ArrayList<>(); + tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(978670))); + KeyValue kv = new KeyValue(b, b, b, HConstants.LATEST_TIMESTAMP, b, tags); + writer.write(new ImmutableBytesWritable(), kv); + writer.close(context); + writer = null; + FileSystem fs = dir.getFileSystem(conf); + RemoteIterator iterator = fs.listFiles(dir, true); + while(iterator.hasNext()) { + LocatedFileStatus keyFileStatus = iterator.next(); + HFile.Reader reader = + HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf), true, conf); + HFileScanner scanner = reader.getScanner(false, false, false); + scanner.seekTo(); + Cell cell = scanner.getCell(); + List tagsFromCell = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + assertTrue(tagsFromCell.size() > 0); + for (Tag tag : tagsFromCell) { + assertTrue(tag.getType() == TagType.TTL_TAG_TYPE); + } + } + } finally { + if (writer != null && context != null) writer.close(context); + dir.getFileSystem(conf).delete(dir, true); + } + } + + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testJobConfiguration() throws Exception { + Configuration conf = new Configuration(this.util.getConfiguration()); + conf.set(HConstants.TEMPORARY_FS_DIRECTORY_KEY, util.getDataTestDir("testJobConfiguration") + .toString()); + Job job = new Job(conf); + job.setWorkingDirectory(util.getDataTestDir("testJobConfiguration")); + Table table = Mockito.mock(Table.class); + RegionLocator regionLocator = Mockito.mock(RegionLocator.class); + setupMockStartKeys(regionLocator); + setupMockTableName(regionLocator); + HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator); + assertEquals(job.getNumReduceTasks(), 4); + } + + private byte [][] generateRandomStartKeys(int numKeys) { + Random random = new Random(); + byte[][] ret = new byte[numKeys][]; + // first region start key is always empty + ret[0] = HConstants.EMPTY_BYTE_ARRAY; + for (int i = 1; i < numKeys; i++) { + ret[i] = + PerformanceEvaluation.generateData(random, PerformanceEvaluation.DEFAULT_VALUE_LENGTH); + } + return ret; + } + + private byte[][] generateRandomSplitKeys(int numKeys) { + Random random = new Random(); + byte[][] ret = new byte[numKeys][]; + for (int i = 0; i < numKeys; i++) { + ret[i] = + PerformanceEvaluation.generateData(random, PerformanceEvaluation.DEFAULT_VALUE_LENGTH); + } + return ret; + } + + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testMRIncrementalLoad() throws Exception { + LOG.info("\nStarting test testMRIncrementalLoad\n"); + doIncrementalLoadTest(false, false, false, "testMRIncrementalLoad"); + } + + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testMRIncrementalLoadWithSplit() throws Exception { + LOG.info("\nStarting test testMRIncrementalLoadWithSplit\n"); + doIncrementalLoadTest(true, false, false, "testMRIncrementalLoadWithSplit"); + } + + /** + * Test for HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY = true + * This test could only check the correctness of original logic if LOCALITY_SENSITIVE_CONF_KEY + * is set to true. Because MiniHBaseCluster always run with single hostname (and different ports), + * it's not possible to check the region locality by comparing region locations and DN hostnames. + * When MiniHBaseCluster supports explicit hostnames parameter (just like MiniDFSCluster does), + * we could test region locality features more easily. + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testMRIncrementalLoadWithLocality() throws Exception { + LOG.info("\nStarting test testMRIncrementalLoadWithLocality\n"); + doIncrementalLoadTest(false, true, false, "testMRIncrementalLoadWithLocality1"); + doIncrementalLoadTest(true, true, false, "testMRIncrementalLoadWithLocality2"); + } + + //@Ignore("Wahtevs") + @Test + public void testMRIncrementalLoadWithPutSortReducer() throws Exception { + LOG.info("\nStarting test testMRIncrementalLoadWithPutSortReducer\n"); + doIncrementalLoadTest(false, false, true, "testMRIncrementalLoadWithPutSortReducer"); + } + + private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality, + boolean putSortReducer, String tableStr) throws Exception { + doIncrementalLoadTest(shouldChangeRegions, shouldKeepLocality, putSortReducer, + Arrays.asList(tableStr)); + } + + @Test + public void testMultiMRIncrementalLoadWithPutSortReducer() throws Exception { + LOG.info("\nStarting test testMultiMRIncrementalLoadWithPutSortReducer\n"); + doIncrementalLoadTest(false, false, true, + Arrays.stream(TABLE_NAMES).map(TableName::getNameAsString).collect(Collectors.toList + ())); + } + + private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality, + boolean putSortReducer, List tableStr) throws Exception { + util = new HBaseTestingUtility(); + Configuration conf = util.getConfiguration(); + conf.setBoolean(MultiTableHFileOutputFormat.LOCALITY_SENSITIVE_CONF_KEY, shouldKeepLocality); + int hostCount = 1; + int regionNum = 5; + if (shouldKeepLocality) { + // We should change host count higher than hdfs replica count when MiniHBaseCluster supports + // explicit hostnames parameter just like MiniDFSCluster does. + hostCount = 3; + regionNum = 20; + } + + String[] hostnames = new String[hostCount]; + for (int i = 0; i < hostCount; ++i) { + hostnames[i] = "datanode_" + i; + } + util.startMiniCluster(1, hostCount, hostnames); + + Map allTables = new HashMap<>(tableStr.size()); + List tableInfo = new ArrayList<>(tableStr.size()); + boolean writeMultipleTables = tableStr.size() > 1; + for (String tableStrSingle : tableStr) { + byte[][] splitKeys = generateRandomSplitKeys(regionNum - 1); + TableName tableName = TableName.valueOf(tableStrSingle); + Table table = util.createTable(tableName, FAMILIES, splitKeys); + + RegionLocator r = util.getConnection().getRegionLocator(tableName); + assertEquals("Should start with empty table", 0, util.countRows(table)); + int numRegions = r.getStartKeys().length; + assertEquals("Should make " + regionNum + " regions", numRegions, regionNum); + + allTables.put(tableStrSingle, table); + tableInfo.add(new HFileOutputFormat2.TableInfo(table.getTableDescriptor(), r)); + } + Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad"); + // Generate the bulk load files + runIncrementalPELoad(conf, tableInfo, testDir, putSortReducer); + + for (Table tableSingle : allTables.values()) { + // This doesn't write into the table, just makes files + assertEquals("HFOF should not touch actual table", 0, util.countRows(tableSingle)); + } + int numTableDirs = 0; + for (FileStatus tf : testDir.getFileSystem(conf).listStatus(testDir)) { + Path tablePath = testDir; + + if (writeMultipleTables) { + if (allTables.containsKey(tf.getPath().getName())) { + ++numTableDirs; + tablePath = tf.getPath(); + } + else { + continue; + } + } + + // Make sure that a directory was created for every CF + int dir = 0; + for (FileStatus f : tablePath.getFileSystem(conf).listStatus(tablePath)) { + for (byte[] family : FAMILIES) { + if (Bytes.toString(family).equals(f.getPath().getName())) { + ++dir; + } + } + } + assertEquals("Column family not found in FS.", FAMILIES.length, dir); + } + if (writeMultipleTables) { + assertEquals("Dir for all input tables not created", numTableDirs, allTables.size()); + } + + Admin admin = util.getConnection().getAdmin(); + try { + // handle the split case + if (shouldChangeRegions) { + Table chosenTable = allTables.values().iterator().next(); + // Choose a semi-random table if multiple tables are available + LOG.info("Changing regions in table " + chosenTable.getName().getNameAsString()); + admin.disableTable(chosenTable.getName()); + util.waitUntilNoRegionsInTransition(); + + util.deleteTable(chosenTable.getName()); + byte[][] newSplitKeys = generateRandomSplitKeys(14); + Table table = util.createTable(chosenTable.getName(), FAMILIES, newSplitKeys); + + while (util.getConnection().getRegionLocator(chosenTable.getName()) + .getAllRegionLocations().size() != 15 || + !admin.isTableAvailable(table.getName())) { + Thread.sleep(200); + LOG.info("Waiting for new region assignment to happen"); + } + } + + // Perform the actual load + for (HFileOutputFormat2.TableInfo singleTableInfo : tableInfo) { + Path tableDir = testDir; + String tableNameStr = singleTableInfo.getHTableDescriptor().getNameAsString(); + LOG.info("Running LoadIncrementalHFiles on table" + tableNameStr); + if (writeMultipleTables) { + tableDir = new Path(testDir, tableNameStr); + } + Table currentTable = allTables.get(tableNameStr); + TableName currentTableName = currentTable.getName(); + new LoadIncrementalHFiles(conf).doBulkLoad(tableDir, admin, currentTable, singleTableInfo + .getRegionLocator()); + + // Ensure data shows up + int expectedRows = 0; + if (putSortReducer) { + // no rows should be extracted + assertEquals("LoadIncrementalHFiles should put expected data in table", expectedRows, + util.countRows(currentTable)); + } else { + expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT; + assertEquals("LoadIncrementalHFiles should put expected data in table", expectedRows, + util.countRows(currentTable)); + Scan scan = new Scan(); + ResultScanner results = currentTable.getScanner(scan); + for (Result res : results) { + assertEquals(FAMILIES.length, res.rawCells().length); + Cell first = res.rawCells()[0]; + for (Cell kv : res.rawCells()) { + assertTrue(CellUtil.matchingRow(first, kv)); + assertTrue(Bytes.equals(CellUtil.cloneValue(first), CellUtil.cloneValue(kv))); + } + } + results.close(); + } + String tableDigestBefore = util.checksumRows(currentTable); + // Check region locality + HDFSBlocksDistribution hbd = new HDFSBlocksDistribution(); + for (HRegion region : util.getHBaseCluster().getRegions(currentTableName)) { + hbd.add(region.getHDFSBlocksDistribution()); + } + for (String hostname : hostnames) { + float locality = hbd.getBlockLocalityIndex(hostname); + LOG.info("locality of [" + hostname + "]: " + locality); + assertEquals(100, (int) (locality * 100)); + } + + // Cause regions to reopen + admin.disableTable(currentTableName); + while (!admin.isTableDisabled(currentTableName)) { + Thread.sleep(200); + LOG.info("Waiting for table to disable"); + } + admin.enableTable(currentTableName); + util.waitTableAvailable(currentTableName); + assertEquals("Data should remain after reopening of regions", + tableDigestBefore, util.checksumRows(currentTable)); + } + } finally { + for (HFileOutputFormat2.TableInfo tableInfoSingle : tableInfo) { + tableInfoSingle.getRegionLocator().close(); + } + for (Entry singleTable : allTables.entrySet() ) { + singleTable.getValue().close(); + util.deleteTable(singleTable.getValue().getName()); + } + testDir.getFileSystem(conf).delete(testDir, true); + util.shutdownMiniCluster(); + } + } + + private void runIncrementalPELoad(Configuration conf, List tableInfo, Path outDir, + boolean putSortReducer) throws IOException, + InterruptedException, ClassNotFoundException { + Job job = new Job(conf, "testLocalMRIncrementalLoad"); + job.setWorkingDirectory(util.getDataTestDirOnTestFS("runIncrementalPELoad")); + job.getConfiguration().setStrings("io.serializations", conf.get("io.serializations"), + MutationSerialization.class.getName(), ResultSerialization.class.getName(), + CellSerialization.class.getName()); + setupRandomGeneratorMapper(job, putSortReducer); + if (tableInfo.size() > 1) { + MultiTableHFileOutputFormat.configureIncrementalLoad(job, tableInfo); + int sum = 0; + for (HFileOutputFormat2.TableInfo tableInfoSingle : tableInfo) { + sum += tableInfoSingle.getRegionLocator().getAllRegionLocations().size(); + } + assertEquals(sum, job.getNumReduceTasks()); + } + else { + RegionLocator regionLocator = tableInfo.get(0).getRegionLocator(); + HFileOutputFormat2.configureIncrementalLoad(job, tableInfo.get(0).getHTableDescriptor(), + regionLocator); + assertEquals(regionLocator.getAllRegionLocations().size(), job.getNumReduceTasks()); + } + + FileOutputFormat.setOutputPath(job, outDir); + + assertFalse(util.getTestFileSystem().exists(outDir)) ; + + assertTrue(job.waitForCompletion(true)); + } + + /** + * Test for {@link HFileOutputFormat2#configureCompression(Configuration, HTableDescriptor)} and + * {@link HFileOutputFormat2#createFamilyCompressionMap(Configuration)}. + * Tests that the compression map is correctly serialized into + * and deserialized from configuration + * + * @throws IOException + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testSerializeDeserializeFamilyCompressionMap() throws IOException { + for (int numCfs = 0; numCfs <= 3; numCfs++) { + Configuration conf = new Configuration(this.util.getConfiguration()); + Map familyToCompression = + getMockColumnFamiliesForCompression(numCfs); + Table table = Mockito.mock(Table.class); + setupMockColumnFamiliesForCompression(table, familyToCompression); + conf.set(HFileOutputFormat2.COMPRESSION_FAMILIES_CONF_KEY, + HFileOutputFormat2.serializeColumnFamilyAttribute + (HFileOutputFormat2.compressionDetails, + Arrays.asList(table.getTableDescriptor()))); + + // read back family specific compression setting from the configuration + Map retrievedFamilyToCompressionMap = HFileOutputFormat2 + .createFamilyCompressionMap(conf); + + // test that we have a value for all column families that matches with the + // used mock values + for (Entry entry : familyToCompression.entrySet()) { + assertEquals("Compression configuration incorrect for column family:" + + entry.getKey(), entry.getValue(), + retrievedFamilyToCompressionMap.get(entry.getKey().getBytes("UTF-8"))); + } + } + } + + private void setupMockColumnFamiliesForCompression(Table table, + Map familyToCompression) throws IOException { + HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]); + for (Entry entry : familyToCompression.entrySet()) { + mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey()) + .setMaxVersions(1) + .setCompressionType(entry.getValue()) + .setBlockCacheEnabled(false) + .setTimeToLive(0)); + } + Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor(); + } + + /** + * @return a map from column family names to compression algorithms for + * testing column family compression. Column family names have special characters + */ + private Map + getMockColumnFamiliesForCompression (int numCfs) { + Map familyToCompression = new HashMap<>(); + // use column family names having special characters + if (numCfs-- > 0) { + familyToCompression.put("Family1!@#!@#&", Compression.Algorithm.LZO); + } + if (numCfs-- > 0) { + familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.SNAPPY); + } + if (numCfs-- > 0) { + familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.GZ); + } + if (numCfs-- > 0) { + familyToCompression.put("Family3", Compression.Algorithm.NONE); + } + return familyToCompression; + } + + + /** + * Test for {@link HFileOutputFormat2#configureBloomType(HTableDescriptor, Configuration)} and + * {@link HFileOutputFormat2#createFamilyBloomTypeMap(Configuration)}. + * Tests that the compression map is correctly serialized into + * and deserialized from configuration + * + * @throws IOException + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testSerializeDeserializeFamilyBloomTypeMap() throws IOException { + for (int numCfs = 0; numCfs <= 2; numCfs++) { + Configuration conf = new Configuration(this.util.getConfiguration()); + Map familyToBloomType = + getMockColumnFamiliesForBloomType(numCfs); + Table table = Mockito.mock(Table.class); + setupMockColumnFamiliesForBloomType(table, + familyToBloomType); + conf.set(HFileOutputFormat2.BLOOM_TYPE_FAMILIES_CONF_KEY, + HFileOutputFormat2.serializeColumnFamilyAttribute(HFileOutputFormat2.bloomTypeDetails, + Arrays.asList(table.getTableDescriptor()))); + + // read back family specific data block encoding settings from the + // configuration + Map retrievedFamilyToBloomTypeMap = + HFileOutputFormat2 + .createFamilyBloomTypeMap(conf); + + // test that we have a value for all column families that matches with the + // used mock values + for (Entry entry : familyToBloomType.entrySet()) { + assertEquals("BloomType configuration incorrect for column family:" + + entry.getKey(), entry.getValue(), + retrievedFamilyToBloomTypeMap.get(entry.getKey().getBytes("UTF-8"))); + } + } + } + + private void setupMockColumnFamiliesForBloomType(Table table, + Map familyToDataBlockEncoding) throws IOException { + HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]); + for (Entry entry : familyToDataBlockEncoding.entrySet()) { + mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey()) + .setMaxVersions(1) + .setBloomFilterType(entry.getValue()) + .setBlockCacheEnabled(false) + .setTimeToLive(0)); + } + Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor(); + } + + /** + * @return a map from column family names to compression algorithms for + * testing column family compression. Column family names have special characters + */ + private Map + getMockColumnFamiliesForBloomType (int numCfs) { + Map familyToBloomType = new HashMap<>(); + // use column family names having special characters + if (numCfs-- > 0) { + familyToBloomType.put("Family1!@#!@#&", BloomType.ROW); + } + if (numCfs-- > 0) { + familyToBloomType.put("Family2=asdads&!AASD", + BloomType.ROWCOL); + } + if (numCfs-- > 0) { + familyToBloomType.put("Family3", BloomType.NONE); + } + return familyToBloomType; + } + + /** + * Test for {@link HFileOutputFormat2#configureBlockSize(HTableDescriptor, Configuration)} and + * {@link HFileOutputFormat2#createFamilyBlockSizeMap(Configuration)}. + * Tests that the compression map is correctly serialized into + * and deserialized from configuration + * + * @throws IOException + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testSerializeDeserializeFamilyBlockSizeMap() throws IOException { + for (int numCfs = 0; numCfs <= 3; numCfs++) { + Configuration conf = new Configuration(this.util.getConfiguration()); + Map familyToBlockSize = + getMockColumnFamiliesForBlockSize(numCfs); + Table table = Mockito.mock(Table.class); + setupMockColumnFamiliesForBlockSize(table, + familyToBlockSize); + conf.set(HFileOutputFormat2.BLOCK_SIZE_FAMILIES_CONF_KEY, + HFileOutputFormat2.serializeColumnFamilyAttribute + (HFileOutputFormat2.blockSizeDetails, Arrays.asList(table + .getTableDescriptor()))); + + // read back family specific data block encoding settings from the + // configuration + Map retrievedFamilyToBlockSizeMap = + HFileOutputFormat2 + .createFamilyBlockSizeMap(conf); + + // test that we have a value for all column families that matches with the + // used mock values + for (Entry entry : familyToBlockSize.entrySet() + ) { + assertEquals("BlockSize configuration incorrect for column family:" + + entry.getKey(), entry.getValue(), + retrievedFamilyToBlockSizeMap.get(entry.getKey().getBytes("UTF-8"))); + } + } + } + + private void setupMockColumnFamiliesForBlockSize(Table table, + Map familyToDataBlockEncoding) throws IOException { + HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]); + for (Entry entry : familyToDataBlockEncoding.entrySet()) { + mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey()) + .setMaxVersions(1) + .setBlocksize(entry.getValue()) + .setBlockCacheEnabled(false) + .setTimeToLive(0)); + } + Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor(); + } + + /** + * @return a map from column family names to compression algorithms for + * testing column family compression. Column family names have special characters + */ + private Map + getMockColumnFamiliesForBlockSize (int numCfs) { + Map familyToBlockSize = new HashMap<>(); + // use column family names having special characters + if (numCfs-- > 0) { + familyToBlockSize.put("Family1!@#!@#&", 1234); + } + if (numCfs-- > 0) { + familyToBlockSize.put("Family2=asdads&!AASD", + Integer.MAX_VALUE); + } + if (numCfs-- > 0) { + familyToBlockSize.put("Family2=asdads&!AASD", + Integer.MAX_VALUE); + } + if (numCfs-- > 0) { + familyToBlockSize.put("Family3", 0); + } + return familyToBlockSize; + } + + /** + * Test for {@link HFileOutputFormat2#configureDataBlockEncoding(HTableDescriptor, Configuration)} + * and {@link HFileOutputFormat2#createFamilyDataBlockEncodingMap(Configuration)}. + * Tests that the compression map is correctly serialized into + * and deserialized from configuration + * + * @throws IOException + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testSerializeDeserializeFamilyDataBlockEncodingMap() throws IOException { + for (int numCfs = 0; numCfs <= 3; numCfs++) { + Configuration conf = new Configuration(this.util.getConfiguration()); + Map familyToDataBlockEncoding = + getMockColumnFamiliesForDataBlockEncoding(numCfs); + Table table = Mockito.mock(Table.class); + setupMockColumnFamiliesForDataBlockEncoding(table, + familyToDataBlockEncoding); + HTableDescriptor tableDescriptor = table.getTableDescriptor(); + conf.set(HFileOutputFormat2.DATABLOCK_ENCODING_FAMILIES_CONF_KEY, + HFileOutputFormat2.serializeColumnFamilyAttribute + (HFileOutputFormat2.dataBlockEncodingDetails, Arrays + .asList(tableDescriptor))); + + // read back family specific data block encoding settings from the + // configuration + Map retrievedFamilyToDataBlockEncodingMap = + HFileOutputFormat2 + .createFamilyDataBlockEncodingMap(conf); + + // test that we have a value for all column families that matches with the + // used mock values + for (Entry entry : familyToDataBlockEncoding.entrySet()) { + assertEquals("DataBlockEncoding configuration incorrect for column family:" + + entry.getKey(), entry.getValue(), + retrievedFamilyToDataBlockEncodingMap.get(entry.getKey().getBytes("UTF-8"))); + } + } + } + + private void setupMockColumnFamiliesForDataBlockEncoding(Table table, + Map familyToDataBlockEncoding) throws IOException { + HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAMES[0]); + for (Entry entry : familyToDataBlockEncoding.entrySet()) { + mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey()) + .setMaxVersions(1) + .setDataBlockEncoding(entry.getValue()) + .setBlockCacheEnabled(false) + .setTimeToLive(0)); + } + Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor(); + } + + /** + * @return a map from column family names to compression algorithms for + * testing column family compression. Column family names have special characters + */ + private Map + getMockColumnFamiliesForDataBlockEncoding (int numCfs) { + Map familyToDataBlockEncoding = new HashMap<>(); + // use column family names having special characters + if (numCfs-- > 0) { + familyToDataBlockEncoding.put("Family1!@#!@#&", DataBlockEncoding.DIFF); + } + if (numCfs-- > 0) { + familyToDataBlockEncoding.put("Family2=asdads&!AASD", + DataBlockEncoding.FAST_DIFF); + } + if (numCfs-- > 0) { + familyToDataBlockEncoding.put("Family2=asdads&!AASD", + DataBlockEncoding.PREFIX); + } + if (numCfs-- > 0) { + familyToDataBlockEncoding.put("Family3", DataBlockEncoding.NONE); + } + return familyToDataBlockEncoding; + } + + private void setupMockStartKeys(RegionLocator table) throws IOException { + byte[][] mockKeys = new byte[][] { + HConstants.EMPTY_BYTE_ARRAY, + Bytes.toBytes("aaa"), + Bytes.toBytes("ggg"), + Bytes.toBytes("zzz") + }; + Mockito.doReturn(mockKeys).when(table).getStartKeys(); + } + + private void setupMockTableName(RegionLocator table) throws IOException { + TableName mockTableName = TableName.valueOf("mock_table"); + Mockito.doReturn(mockTableName).when(table).getName(); + } + + /** + * Test that {@link HFileOutputFormat2} RecordWriter uses compression and + * bloom filter settings from the column family descriptor + */ + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testColumnFamilySettings() throws Exception { + Configuration conf = new Configuration(this.util.getConfiguration()); + RecordWriter writer = null; + TaskAttemptContext context = null; + Path dir = util.getDataTestDir("testColumnFamilySettings"); + + // Setup table descriptor + Table table = Mockito.mock(Table.class); + RegionLocator regionLocator = Mockito.mock(RegionLocator.class); + HTableDescriptor htd = new HTableDescriptor(TABLE_NAMES[0]); + Mockito.doReturn(htd).when(table).getTableDescriptor(); + for (HColumnDescriptor hcd: HBaseTestingUtility.generateColumnDescriptors()) { + htd.addFamily(hcd); + } + + // set up the table to return some mock keys + setupMockStartKeys(regionLocator); + + try { + // partial map red setup to get an operational writer for testing + // We turn off the sequence file compression, because DefaultCodec + // pollutes the GZip codec pool with an incompatible compressor. + conf.set("io.seqfile.compression.type", "NONE"); + conf.set("hbase.fs.tmp.dir", dir.toString()); + // turn locality off to eliminate getRegionLocation fail-and-retry time when writing kvs + conf.setBoolean(HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY, false); + + Job job = new Job(conf, "testLocalMRIncrementalLoad"); + job.setWorkingDirectory(util.getDataTestDirOnTestFS("testColumnFamilySettings")); + setupRandomGeneratorMapper(job, false); + HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator); + FileOutputFormat.setOutputPath(job, dir); + context = createTestTaskAttemptContext(job); + HFileOutputFormat2 hof = new HFileOutputFormat2(); + writer = hof.getRecordWriter(context); + + // write out random rows + writeRandomKeyValues(writer, context, htd.getFamiliesKeys(), ROWSPERSPLIT); + writer.close(context); + + // Make sure that a directory was created for every CF + FileSystem fs = dir.getFileSystem(conf); + + // commit so that the filesystem has one directory per column family + hof.getOutputCommitter(context).commitTask(context); + hof.getOutputCommitter(context).commitJob(context); + FileStatus[] families = FSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs)); + assertEquals(htd.getFamilies().size(), families.length); + for (FileStatus f : families) { + String familyStr = f.getPath().getName(); + HColumnDescriptor hcd = htd.getFamily(Bytes.toBytes(familyStr)); + // verify that the compression on this file matches the configured + // compression + Path dataFilePath = fs.listStatus(f.getPath())[0].getPath(); + Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), true, conf); + Map fileInfo = reader.loadFileInfo(); + + byte[] bloomFilter = fileInfo.get(BLOOM_FILTER_TYPE_KEY); + if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE"); + assertEquals("Incorrect bloom filter used for column family " + familyStr + + "(reader: " + reader + ")", + hcd.getBloomFilterType(), BloomType.valueOf(Bytes.toString(bloomFilter))); + assertEquals("Incorrect compression used for column family " + familyStr + + "(reader: " + reader + ")", hcd.getCompressionType(), reader.getFileContext().getCompression()); + } + } finally { + dir.getFileSystem(conf).delete(dir, true); + } + } + + /** + * Write random values to the writer assuming a table created using + * {@link #FAMILIES} as column family descriptors + */ + private void writeRandomKeyValues(RecordWriter writer, + TaskAttemptContext context, Set families, int numRows) + throws IOException, InterruptedException { + byte keyBytes[] = new byte[Bytes.SIZEOF_INT]; + int valLength = 10; + byte valBytes[] = new byte[valLength]; + + int taskId = context.getTaskAttemptID().getTaskID().getId(); + assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!"; + final byte [] qualifier = Bytes.toBytes("data"); + Random random = new Random(); + for (int i = 0; i < numRows; i++) { + + Bytes.putInt(keyBytes, 0, i); + random.nextBytes(valBytes); + ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes); + + for (byte[] family : families) { + Cell kv = new KeyValue(keyBytes, family, qualifier, valBytes); + writer.write(key, kv); + } + } + } + + /** + * This test is to test the scenario happened in HBASE-6901. + * All files are bulk loaded and excluded from minor compaction. + * Without the fix of HBASE-6901, an ArrayIndexOutOfBoundsException + * will be thrown. + */ + @Ignore ("Flakey: See HBASE-9051") @Test + public void testExcludeAllFromMinorCompaction() throws Exception { + Configuration conf = util.getConfiguration(); + conf.setInt("hbase.hstore.compaction.min", 2); + generateRandomStartKeys(5); + + util.startMiniCluster(); + try (Connection conn = ConnectionFactory.createConnection(); + Admin admin = conn.getAdmin(); + Table table = util.createTable(TABLE_NAMES[0], FAMILIES); + RegionLocator locator = conn.getRegionLocator(TABLE_NAMES[0])) { + final FileSystem fs = util.getDFSCluster().getFileSystem(); + assertEquals("Should start with empty table", 0, util.countRows(table)); + + // deep inspection: get the StoreFile dir + final Path storePath = new Path( + FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]), + new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(), + Bytes.toString(FAMILIES[0]))); + assertEquals(0, fs.listStatus(storePath).length); + + // Generate two bulk load files + conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude", + true); + + for (int i = 0; i < 2; i++) { + Path testDir = util.getDataTestDirOnTestFS("testExcludeAllFromMinorCompaction_" + i); + runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(table + .getTableDescriptor(), conn.getRegionLocator(TABLE_NAMES[0]))), testDir, false); + // Perform the actual load + new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, locator); + } + + // Ensure data shows up + int expectedRows = 2 * NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT; + assertEquals("LoadIncrementalHFiles should put expected data in table", + expectedRows, util.countRows(table)); + + // should have a second StoreFile now + assertEquals(2, fs.listStatus(storePath).length); + + // minor compactions shouldn't get rid of the file + admin.compact(TABLE_NAMES[0]); + try { + quickPoll(new Callable() { + @Override + public Boolean call() throws Exception { + List regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]); + for (HRegion region : regions) { + for (HStore store : region.getStores()) { + store.closeAndArchiveCompactedFiles(); + } + } + return fs.listStatus(storePath).length == 1; + } + }, 5000); + throw new IOException("SF# = " + fs.listStatus(storePath).length); + } catch (AssertionError ae) { + // this is expected behavior + } + + // a major compaction should work though + admin.majorCompact(TABLE_NAMES[0]); + quickPoll(new Callable() { + @Override + public Boolean call() throws Exception { + List regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]); + for (HRegion region : regions) { + for (HStore store : region.getStores()) { + store.closeAndArchiveCompactedFiles(); + } + } + return fs.listStatus(storePath).length == 1; + } + }, 5000); + + } finally { + util.shutdownMiniCluster(); + } + } + + @Ignore("Goes zombie too frequently; needs work. See HBASE-14563") @Test + public void testExcludeMinorCompaction() throws Exception { + Configuration conf = util.getConfiguration(); + conf.setInt("hbase.hstore.compaction.min", 2); + generateRandomStartKeys(5); + + util.startMiniCluster(); + try (Connection conn = ConnectionFactory.createConnection(conf); + Admin admin = conn.getAdmin()){ + Path testDir = util.getDataTestDirOnTestFS("testExcludeMinorCompaction"); + final FileSystem fs = util.getDFSCluster().getFileSystem(); + Table table = util.createTable(TABLE_NAMES[0], FAMILIES); + assertEquals("Should start with empty table", 0, util.countRows(table)); + + // deep inspection: get the StoreFile dir + final Path storePath = new Path( + FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]), + new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(), + Bytes.toString(FAMILIES[0]))); + assertEquals(0, fs.listStatus(storePath).length); + + // put some data in it and flush to create a storefile + Put p = new Put(Bytes.toBytes("test")); + p.addColumn(FAMILIES[0], Bytes.toBytes("1"), Bytes.toBytes("1")); + table.put(p); + admin.flush(TABLE_NAMES[0]); + assertEquals(1, util.countRows(table)); + quickPoll(new Callable() { + @Override + public Boolean call() throws Exception { + return fs.listStatus(storePath).length == 1; + } + }, 5000); + + // Generate a bulk load file with more rows + conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude", + true); + + RegionLocator regionLocator = conn.getRegionLocator(TABLE_NAMES[0]); + runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(table + .getTableDescriptor(), regionLocator)), testDir, false); + + // Perform the actual load + new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, regionLocator); + + // Ensure data shows up + int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT; + assertEquals("LoadIncrementalHFiles should put expected data in table", + expectedRows + 1, util.countRows(table)); + + // should have a second StoreFile now + assertEquals(2, fs.listStatus(storePath).length); + + // minor compactions shouldn't get rid of the file + admin.compact(TABLE_NAMES[0]); + try { + quickPoll(new Callable() { + @Override + public Boolean call() throws Exception { + return fs.listStatus(storePath).length == 1; + } + }, 5000); + throw new IOException("SF# = " + fs.listStatus(storePath).length); + } catch (AssertionError ae) { + // this is expected behavior + } + + // a major compaction should work though + admin.majorCompact(TABLE_NAMES[0]); + quickPoll(new Callable() { + @Override + public Boolean call() throws Exception { + return fs.listStatus(storePath).length == 1; + } + }, 5000); + + } finally { + util.shutdownMiniCluster(); + } + } + + private void quickPoll(Callable c, int waitMs) throws Exception { + int sleepMs = 10; + int retries = (int) Math.ceil(((double) waitMs) / sleepMs); + while (retries-- > 0) { + if (c.call().booleanValue()) { + return; + } + Thread.sleep(sleepMs); + } + fail(); + } + + public static void main(String args[]) throws Exception { + new TestCellBasedHFileOutputFormat2().manualTest(args); + } + + public void manualTest(String args[]) throws Exception { + Configuration conf = HBaseConfiguration.create(); + util = new HBaseTestingUtility(conf); + if ("newtable".equals(args[0])) { + TableName tname = TableName.valueOf(args[1]); + byte[][] splitKeys = generateRandomSplitKeys(4); + Table table = util.createTable(tname, FAMILIES, splitKeys); + } else if ("incremental".equals(args[0])) { + TableName tname = TableName.valueOf(args[1]); + try(Connection c = ConnectionFactory.createConnection(conf); + Admin admin = c.getAdmin(); + RegionLocator regionLocator = c.getRegionLocator(tname)) { + Path outDir = new Path("incremental-out"); + runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(admin + .getTableDescriptor(tname), regionLocator)), outDir, false); + } + } else { + throw new RuntimeException( + "usage: TestHFileOutputFormat2 newtable | incremental"); + } + } + + @Test + public void testBlockStoragePolicy() throws Exception { + util = new HBaseTestingUtility(); + Configuration conf = util.getConfiguration(); + conf.set(HFileOutputFormat2.STORAGE_POLICY_PROPERTY, "ALL_SSD"); + + conf.set(HFileOutputFormat2.STORAGE_POLICY_PROPERTY_CF_PREFIX + + Bytes.toString(HFileOutputFormat2.combineTableNameSuffix( + TABLE_NAMES[0].getName(), FAMILIES[0])), "ONE_SSD"); + Path cf1Dir = new Path(util.getDataTestDir(), Bytes.toString(FAMILIES[0])); + Path cf2Dir = new Path(util.getDataTestDir(), Bytes.toString(FAMILIES[1])); + util.startMiniDFSCluster(3); + FileSystem fs = util.getDFSCluster().getFileSystem(); + try { + fs.mkdirs(cf1Dir); + fs.mkdirs(cf2Dir); + + // the original block storage policy would be HOT + String spA = getStoragePolicyName(fs, cf1Dir); + String spB = getStoragePolicyName(fs, cf2Dir); + LOG.debug("Storage policy of cf 0: [" + spA + "]."); + LOG.debug("Storage policy of cf 1: [" + spB + "]."); + assertEquals("HOT", spA); + assertEquals("HOT", spB); + + // alter table cf schema to change storage policies + HFileOutputFormat2.configureStoragePolicy(conf, fs, + HFileOutputFormat2.combineTableNameSuffix(TABLE_NAMES[0].getName(), FAMILIES[0]), cf1Dir); + HFileOutputFormat2.configureStoragePolicy(conf, fs, + HFileOutputFormat2.combineTableNameSuffix(TABLE_NAMES[0].getName(), FAMILIES[1]), cf2Dir); + spA = getStoragePolicyName(fs, cf1Dir); + spB = getStoragePolicyName(fs, cf2Dir); + LOG.debug("Storage policy of cf 0: [" + spA + "]."); + LOG.debug("Storage policy of cf 1: [" + spB + "]."); + assertNotNull(spA); + assertEquals("ONE_SSD", spA); + assertNotNull(spB); + assertEquals("ALL_SSD", spB); + } finally { + fs.delete(cf1Dir, true); + fs.delete(cf2Dir, true); + util.shutdownMiniDFSCluster(); + } + } + + private String getStoragePolicyName(FileSystem fs, Path path) { + try { + Object blockStoragePolicySpi = ReflectionUtils.invokeMethod(fs, "getStoragePolicy", path); + return (String) ReflectionUtils.invokeMethod(blockStoragePolicySpi, "getName"); + } catch (Exception e) { + // Maybe fail because of using old HDFS version, try the old way + if (LOG.isTraceEnabled()) { + LOG.trace("Failed to get policy directly", e); + } + String policy = getStoragePolicyNameForOldHDFSVersion(fs, path); + return policy == null ? "HOT" : policy;// HOT by default + } + } + + private String getStoragePolicyNameForOldHDFSVersion(FileSystem fs, Path path) { + try { + if (fs instanceof DistributedFileSystem) { + DistributedFileSystem dfs = (DistributedFileSystem) fs; + HdfsFileStatus status = dfs.getClient().getFileInfo(path.toUri().getPath()); + if (null != status) { + byte storagePolicyId = status.getStoragePolicy(); + Field idUnspecified = BlockStoragePolicySuite.class.getField("ID_UNSPECIFIED"); + if (storagePolicyId != idUnspecified.getByte(BlockStoragePolicySuite.class)) { + BlockStoragePolicy[] policies = dfs.getStoragePolicies(); + for (BlockStoragePolicy policy : policies) { + if (policy.getId() == storagePolicyId) { + return policy.getName(); + } + } + } + } + } + } catch (Throwable e) { + LOG.warn("failed to get block storage policy of [" + path + "]", e); + } + + return null; + } +} + diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java new file mode 100644 index 00000000000..59fb55322af --- /dev/null +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java @@ -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=")); + 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 ]* [ " + + "[ []] [^[regex pattern] or [Prefix] to filter]]")); + assertTrue( + errMsg.contains("-D hbase.mapreduce.scan.column.family=,, ...")); + 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() { + + @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 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; + } + } +} diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java new file mode 100644 index 00000000000..99bd5c1147a --- /dev/null +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java @@ -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.Context context = mock(Context.class); + when(context.getConfiguration()).thenReturn(configuration); + + WALEdit value = mock(WALEdit.class); + ArrayList 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() { + + @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] " + + " []")); + assertTrue(data.toString().contains("-Dwal.bulk.output=/path/for/output")); + } + + } finally { + System.setErr(oldPrintStream); + System.setSecurityManager(SECURITY_MANAGER); + } + + } + +} diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 42df264c632..ec19b2e49c9 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -119,6 +119,7 @@ import org.mockito.Mockito; * emits keys and values like those of {@link PerformanceEvaluation}. */ @Category({VerySlowMapReduceTests.class, LargeTests.class}) +//TODO : Remove this in 3.0 public class TestHFileOutputFormat2 { @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). withTimeout(this.getClass()).withLookingForStuckThread(true).build(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index b2b9c4dd979..1928e81efb9 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -91,6 +91,7 @@ import org.mockito.stubbing.Answer; * Tests the table import and table export MR job functionality */ @Category({VerySlowMapReduceTests.class, MediumTests.class}) +//TODO : Remove this in 3.0 public class TestImportExport { private static final Log LOG = LogFactory.getLog(TestImportExport.class); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java index eef9cffc94b..13b1b1e5cb7 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java @@ -70,6 +70,7 @@ import org.mockito.stubbing.Answer; * Basic test for the WALPlayer M/R tool */ @Category({MapReduceTests.class, LargeTests.class}) +//TODO : Remove this in 3.0 public class TestWALPlayer { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static MiniHBaseCluster cluster;