|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.hadoop.hbase.wal; |
| 19 | + |
| 20 | +import static org.junit.Assert.assertEquals; |
| 21 | +import static org.junit.Assert.assertThrows; |
| 22 | + |
| 23 | +import java.io.IOException; |
| 24 | +import java.util.ArrayList; |
| 25 | +import java.util.List; |
| 26 | +import org.apache.hadoop.fs.FSDataInputStream; |
| 27 | +import org.apache.hadoop.fs.FSDataOutputStream; |
| 28 | +import org.apache.hadoop.fs.FileSystem; |
| 29 | +import org.apache.hadoop.fs.Path; |
| 30 | +import org.apache.hadoop.hbase.Cell.Type; |
| 31 | +import org.apache.hadoop.hbase.CellBuilderFactory; |
| 32 | +import org.apache.hadoop.hbase.CellBuilderType; |
| 33 | +import org.apache.hadoop.hbase.HBaseClassTestRule; |
| 34 | +import org.apache.hadoop.hbase.HBaseCommonTestingUtil; |
| 35 | +import org.apache.hadoop.hbase.HConstants; |
| 36 | +import org.apache.hadoop.hbase.TableName; |
| 37 | +import org.apache.hadoop.hbase.client.RegionInfo; |
| 38 | +import org.apache.hadoop.hbase.client.RegionInfoBuilder; |
| 39 | +import org.apache.hadoop.hbase.regionserver.wal.WALHeaderEOFException; |
| 40 | +import org.apache.hadoop.hbase.testclassification.MediumTests; |
| 41 | +import org.apache.hadoop.hbase.testclassification.RegionServerTests; |
| 42 | +import org.apache.hadoop.hbase.util.Bytes; |
| 43 | +import org.apache.hadoop.hbase.util.CommonFSUtils; |
| 44 | +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; |
| 45 | +import org.junit.AfterClass; |
| 46 | +import org.junit.BeforeClass; |
| 47 | +import org.junit.ClassRule; |
| 48 | +import org.junit.Ignore; |
| 49 | +import org.junit.Test; |
| 50 | +import org.junit.experimental.categories.Category; |
| 51 | + |
| 52 | +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer; |
| 53 | + |
| 54 | +@Category({ RegionServerTests.class, MediumTests.class }) |
| 55 | +public class TestParsePartialWALFile { |
| 56 | + |
| 57 | + @ClassRule |
| 58 | + public static final HBaseClassTestRule CLASS_RULE = |
| 59 | + HBaseClassTestRule.forClass(TestParsePartialWALFile.class); |
| 60 | + |
| 61 | + private static final HBaseCommonTestingUtil UTIL = new HBaseCommonTestingUtil(); |
| 62 | + |
| 63 | + @BeforeClass |
| 64 | + public static void setUp() { |
| 65 | + UTIL.getConfiguration().setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, false); |
| 66 | + } |
| 67 | + |
| 68 | + @AfterClass |
| 69 | + public static void tearDown() { |
| 70 | + UTIL.cleanupTestDir(); |
| 71 | + } |
| 72 | + |
| 73 | + @Test |
| 74 | + public void testTrailerSize() { |
| 75 | + // make sure that the trailer size is currently still zero, as when parsing broken WAL file |
| 76 | + // without a valid trailer, we need this assumption to determine whether there is a broken |
| 77 | + // trailer |
| 78 | + WALTrailer trailer = WALTrailer.newBuilder().build(); |
| 79 | + assertEquals(0, trailer.getSerializedSize()); |
| 80 | + } |
| 81 | + |
| 82 | + private Path generateBrokenWALFile(FileSystem fs, byte[] content, int length) throws IOException { |
| 83 | + Path walFile = UTIL.getDataTestDir("wal-" + length); |
| 84 | + try (FSDataOutputStream out = fs.create(walFile)) { |
| 85 | + out.write(content, 0, length); |
| 86 | + } |
| 87 | + return walFile; |
| 88 | + } |
| 89 | + |
| 90 | + // TODO: can not pass yet, need modify the way on how we parse PB message |
| 91 | + @Ignore |
| 92 | + @Test |
| 93 | + public void testPartialParse() throws Exception { |
| 94 | + FileSystem fs = FileSystem.getLocal(UTIL.getConfiguration()); |
| 95 | + if (!fs.mkdirs(UTIL.getDataTestDir())) { |
| 96 | + throw new IOException("can not create " + UTIL.getDataTestDir()); |
| 97 | + } |
| 98 | + TableName tn = TableName.valueOf("test"); |
| 99 | + RegionInfo ri = RegionInfoBuilder.newBuilder(tn).build(); |
| 100 | + byte[] row = Bytes.toBytes("row"); |
| 101 | + byte[] family = Bytes.toBytes("family"); |
| 102 | + byte[] qual = Bytes.toBytes("qualifier"); |
| 103 | + byte[] value = Bytes.toBytes("value"); |
| 104 | + Path walFile = UTIL.getDataTestDir("wal"); |
| 105 | + |
| 106 | + long headerLength; |
| 107 | + List<Long> endOffsets = new ArrayList<>(); |
| 108 | + try (WALProvider.Writer writer = |
| 109 | + WALFactory.createWALWriter(fs, walFile, UTIL.getConfiguration())) { |
| 110 | + headerLength = writer.getLength(); |
| 111 | + for (int i = 0; i < 2; i++) { |
| 112 | + WALKeyImpl key = new WALKeyImpl(ri.getEncodedNameAsBytes(), tn, i, |
| 113 | + EnvironmentEdgeManager.currentTime(), HConstants.DEFAULT_CLUSTER_ID); |
| 114 | + WALEdit edit = new WALEdit(); |
| 115 | + edit.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setType(Type.Put) |
| 116 | + .setRow(row).setFamily(family).setQualifier(qual).setValue(value).build()); |
| 117 | + writer.append(new WAL.Entry(key, edit)); |
| 118 | + writer.sync(true); |
| 119 | + endOffsets.add(writer.getLength()); |
| 120 | + } |
| 121 | + } |
| 122 | + long fileLength = fs.getFileStatus(walFile).getLen(); |
| 123 | + byte[] content = new byte[(int) fileLength]; |
| 124 | + try (FSDataInputStream in = fs.open(walFile)) { |
| 125 | + in.readFully(content); |
| 126 | + } |
| 127 | + // partial header, should throw WALHeaderEOFException |
| 128 | + for (int i = 0; i < headerLength; i++) { |
| 129 | + Path brokenFile = generateBrokenWALFile(fs, content, i); |
| 130 | + assertThrows(WALHeaderEOFException.class, |
| 131 | + () -> WALFactory.createStreamReader(fs, brokenFile, UTIL.getConfiguration())); |
| 132 | + assertThrows(WALHeaderEOFException.class, |
| 133 | + () -> WALFactory.createTailingReader(fs, brokenFile, UTIL.getConfiguration(), -1)); |
| 134 | + fs.delete(brokenFile, false); |
| 135 | + } |
| 136 | + } |
| 137 | +} |
0 commit comments