Skip to content

Commit be93248

Browse files
chenxu14openinx
authored andcommitted
HBASE-22965 RS Crash due to DBE reference to an reused ByteBuff (#603)
Signed-off-by: huzheng <[email protected]>
1 parent 9f703fc commit be93248

File tree

6 files changed

+79
-7
lines changed

6 files changed

+79
-7
lines changed

hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodingState.java

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,8 @@
1919
package org.apache.hadoop.hbase.io.encoding;
2020

2121
import org.apache.hadoop.hbase.Cell;
22+
import org.apache.hadoop.hbase.KeyValueUtil;
2223
import org.apache.yetus.audience.InterfaceAudience;
23-
2424
/**
2525
* Keeps track of the encoding state.
2626
*/
@@ -31,4 +31,12 @@ public class EncodingState {
3131
* The previous Cell the encoder encoded.
3232
*/
3333
protected Cell prevCell = null;
34+
35+
public void beforeShipped() {
36+
if (this.prevCell != null) {
37+
// can't use KeyValueUtil#toNewKeyCell, because we need both key and value
38+
// from the prevCell in FastDiffDeltaEncoder
39+
this.prevCell = KeyValueUtil.copyToNewKeyValue(this.prevCell);
40+
}
41+
}
3442
}

hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,13 @@ public class RowIndexCodecV1 extends AbstractDataBlockEncoder {
5353

5454
private static class RowIndexEncodingState extends EncodingState {
5555
RowIndexEncoderV1 encoder = null;
56+
57+
@Override
58+
public void beforeShipped() {
59+
if (encoder != null) {
60+
encoder.beforeShipped();
61+
}
62+
}
5663
}
5764

5865
@Override

hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515

1616
import org.apache.hadoop.hbase.Cell;
1717
import org.apache.hadoop.hbase.CellComparatorImpl;
18+
import org.apache.hadoop.hbase.KeyValueUtil;
1819
import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
1920
import org.apache.yetus.audience.InterfaceAudience;
2021
import org.slf4j.Logger;
@@ -30,11 +31,9 @@ public class RowIndexEncoderV1 {
3031
private DataOutputStream out;
3132
private NoneEncoder encoder;
3233
private int startOffset = -1;
33-
private ByteArrayOutputStream rowsOffsetBAOS = new ByteArrayOutputStream(
34-
64 * 4);
34+
private ByteArrayOutputStream rowsOffsetBAOS = new ByteArrayOutputStream(64 * 4);
3535

36-
public RowIndexEncoderV1(DataOutputStream out,
37-
HFileBlockDefaultEncodingContext encodingCtx) {
36+
public RowIndexEncoderV1(DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) {
3837
this.out = out;
3938
this.encoder = new NoneEncoder(out, encodingCtx);
4039
}
@@ -85,4 +84,9 @@ public void flush() throws IOException {
8584
}
8685
}
8786

87+
void beforeShipped() {
88+
if (this.lastCell != null) {
89+
this.lastCell = KeyValueUtil.toNewKeyCell(this.lastCell);
90+
}
91+
}
8892
}

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -46,13 +46,15 @@
4646
import org.apache.hadoop.hbase.io.ByteBufferWriterDataOutputStream;
4747
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
4848
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
49+
import org.apache.hadoop.hbase.io.encoding.EncodingState;
4950
import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
5051
import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
5152
import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
5253
import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
5354
import org.apache.hadoop.hbase.nio.ByteBuff;
5455
import org.apache.hadoop.hbase.nio.MultiByteBuff;
5556
import org.apache.hadoop.hbase.nio.SingleByteBuff;
57+
import org.apache.hadoop.hbase.regionserver.ShipperListener;
5658
import org.apache.hadoop.hbase.util.Bytes;
5759
import org.apache.hadoop.hbase.util.ChecksumType;
5860
import org.apache.hadoop.hbase.util.ClassSize;
@@ -762,7 +764,7 @@ public boolean isSharedMem() {
762764
* </ol>
763765
* <p>
764766
*/
765-
static class Writer {
767+
static class Writer implements ShipperListener {
766768
private enum State {
767769
INIT,
768770
WRITING,
@@ -841,6 +843,17 @@ private enum State {
841843
/** Meta data that holds information about the hfileblock**/
842844
private HFileContext fileContext;
843845

846+
@Override
847+
public void beforeShipped() {
848+
if (getEncodingState() != null) {
849+
getEncodingState().beforeShipped();
850+
}
851+
}
852+
853+
EncodingState getEncodingState() {
854+
return dataBlockEncodingCtx.getEncodingState();
855+
}
856+
844857
/**
845858
* @param dataBlockEncoder data block encoding algorithm to use
846859
*/

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -764,6 +764,7 @@ public void append(final Cell cell) throws IOException {
764764

765765
@Override
766766
public void beforeShipped() throws IOException {
767+
this.blockWriter.beforeShipped();
767768
// Add clone methods for every cell
768769
if (this.lastCell != null) {
769770
this.lastCell = KeyValueUtil.toNewKeyCell(this.lastCell);

hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java

Lines changed: 40 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@
4545
import org.apache.hadoop.fs.FileSystem;
4646
import org.apache.hadoop.fs.Path;
4747
import org.apache.hadoop.hbase.ArrayBackedTag;
48+
import org.apache.hadoop.hbase.ByteBufferKeyValue;
4849
import org.apache.hadoop.hbase.Cell;
4950
import org.apache.hadoop.hbase.CellComparatorImpl;
5051
import org.apache.hadoop.hbase.CellUtil;
@@ -60,12 +61,15 @@
6061
import org.apache.hadoop.hbase.Tag;
6162
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
6263
import org.apache.hadoop.hbase.io.compress.Compression;
64+
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
65+
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
6366
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
6467
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
6568
import org.apache.hadoop.hbase.nio.ByteBuff;
6669
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
6770
import org.apache.hadoop.hbase.testclassification.IOTests;
6871
import org.apache.hadoop.hbase.testclassification.SmallTests;
72+
import org.apache.hadoop.hbase.util.ByteBufferUtils;
6973
import org.apache.hadoop.hbase.util.Bytes;
7074
import org.apache.hadoop.io.Writable;
7175
import org.junit.Assert;
@@ -759,5 +763,40 @@ public void testGetShortMidpoint() {
759763
0, expectedArray.length);
760764
}
761765

766+
@Test
767+
public void testDBEShipped() throws IOException {
768+
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
769+
DataBlockEncoder encoder = encoding.getEncoder();
770+
if (encoder == null) {
771+
continue;
772+
}
773+
Path f = new Path(ROOT_DIR, testName.getMethodName() + "_" + encoding);
774+
HFileContext context = new HFileContextBuilder()
775+
.withIncludesTags(false)
776+
.withDataBlockEncoding(encoding).build();
777+
HFileWriterImpl writer = (HFileWriterImpl) HFile.getWriterFactory(conf, cacheConf)
778+
.withPath(fs, f).withFileContext(context).create();
779+
780+
KeyValue kv = new KeyValue(Bytes.toBytes("testkey1"), Bytes.toBytes("family"),
781+
Bytes.toBytes("qual"), Bytes.toBytes("testvalue"));
782+
KeyValue kv2 = new KeyValue(Bytes.toBytes("testkey2"), Bytes.toBytes("family"),
783+
Bytes.toBytes("qual"), Bytes.toBytes("testvalue"));
784+
KeyValue kv3 = new KeyValue(Bytes.toBytes("testkey3"), Bytes.toBytes("family"),
785+
Bytes.toBytes("qual"), Bytes.toBytes("testvalue"));
786+
787+
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
788+
ByteBuffer buffer2 = ByteBuffer.wrap(kv2.getBuffer());
789+
ByteBuffer buffer3 = ByteBuffer.wrap(kv3.getBuffer());
790+
791+
writer.append(new ByteBufferKeyValue(buffer, 0, buffer.remaining()));
792+
writer.beforeShipped();
793+
794+
// pollute first cell's backing ByteBuffer
795+
ByteBufferUtils.copyFromBufferToBuffer(buffer3, buffer);
796+
797+
// write another cell, if DBE not Shipped, test will fail
798+
writer.append(new ByteBufferKeyValue(buffer2, 0, buffer2.remaining()));
799+
writer.close();
800+
}
801+
}
762802
}
763-

0 commit comments

Comments
 (0)