1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.io.hfile;
21
22 import java.io.ByteArrayInputStream;
23 import java.io.ByteArrayOutputStream;
24 import java.io.DataInputStream;
25 import java.io.DataOutputStream;
26 import java.io.IOException;
27 import java.util.ArrayList;
28 import java.util.Collection;
29 import java.util.List;
30
31 import org.junit.Before;
32 import org.junit.Test;
33 import org.junit.runner.RunWith;
34 import org.junit.runners.Parameterized;
35 import org.junit.runners.Parameterized.Parameters;
36
37 import static org.junit.Assert.*;
38
39 import org.apache.commons.logging.Log;
40 import org.apache.commons.logging.LogFactory;
41 import org.apache.hadoop.fs.FSDataInputStream;
42 import org.apache.hadoop.fs.FSDataOutputStream;
43 import org.apache.hadoop.fs.FileSystem;
44 import org.apache.hadoop.fs.Path;
45 import org.apache.hadoop.hbase.HBaseTestingUtility;
46 import org.apache.hadoop.hbase.KeyValue;
47
48 @RunWith(Parameterized.class)
49 public class TestFixedFileTrailer {
50
51 private static final Log LOG = LogFactory.getLog(TestFixedFileTrailer.class);
52
53
54 private static final int[] NUM_FIELDS_BY_VERSION = new int[] { 8, 13 };
55
56 private HBaseTestingUtility util = new HBaseTestingUtility();
57 private FileSystem fs;
58 private ByteArrayOutputStream baos = new ByteArrayOutputStream();
59 private int version;
60
61 static {
62 assert NUM_FIELDS_BY_VERSION.length == HFile.MAX_FORMAT_VERSION
63 - HFile.MIN_FORMAT_VERSION + 1;
64 }
65
66 public TestFixedFileTrailer(int version) {
67 this.version = version;
68 }
69
70 @Parameters
71 public static Collection<Object[]> getParameters() {
72 List<Object[]> versionsToTest = new ArrayList<Object[]>();
73 for (int v = HFile.MIN_FORMAT_VERSION; v <= HFile.MAX_FORMAT_VERSION; ++v)
74 versionsToTest.add(new Integer[] { v } );
75 return versionsToTest;
76 }
77
78 @Before
79 public void setUp() throws IOException {
80 fs = FileSystem.get(util.getConfiguration());
81 }
82
83 @Test
84 public void testTrailer() throws IOException {
85 FixedFileTrailer t = new FixedFileTrailer(version);
86 t.setDataIndexCount(3);
87 t.setEntryCount(((long) Integer.MAX_VALUE) + 1);
88
89 if (version == 1) {
90 t.setFileInfoOffset(876);
91 }
92
93 if (version == 2) {
94 t.setLastDataBlockOffset(291);
95 t.setNumDataIndexLevels(3);
96 t.setComparatorClass(KeyValue.KEY_COMPARATOR.getClass());
97 t.setFirstDataBlockOffset(9081723123L);
98 t.setUncompressedDataIndexSize(827398717L);
99 }
100
101 t.setLoadOnOpenOffset(128);
102 t.setMetaIndexCount(7);
103
104 t.setTotalUncompressedBytes(129731987);
105
106 {
107 DataOutputStream dos = new DataOutputStream(baos);
108 t.serialize(dos);
109 dos.flush();
110 assertEquals(dos.size(), FixedFileTrailer.getTrailerSize(version));
111 }
112
113 byte[] bytes = baos.toByteArray();
114 baos.reset();
115
116 assertEquals(bytes.length, FixedFileTrailer.getTrailerSize(version));
117
118 ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
119
120
121 {
122 DataInputStream dis = new DataInputStream(bais);
123 FixedFileTrailer t2 = new FixedFileTrailer(version);
124 t2.deserialize(dis);
125 assertEquals(-1, bais.read());
126 checkLoadedTrailer(version, t, t2);
127 }
128
129
130 Path trailerPath = new Path(util.getDataTestDir(), "trailer_"
131 + version);
132
133 {
134 for (byte invalidVersion : new byte[] { HFile.MIN_FORMAT_VERSION - 1,
135 HFile.MAX_FORMAT_VERSION + 1}) {
136 bytes[bytes.length - 1] = invalidVersion;
137 writeTrailer(trailerPath, null, bytes);
138 try {
139 readTrailer(trailerPath);
140 fail("Exception expected");
141 } catch (IOException ex) {
142
143 String msg = ex.getMessage();
144 String cleanMsg = msg.replaceAll(
145 "^(java(\\.[a-zA-Z]+)+:\\s+)?|\\s+\\(.*\\)\\s*$", "");
146 assertEquals("Actual exception message is \"" + msg + "\".\n" +
147 "Cleaned-up message",
148 "Invalid HFile version: " + invalidVersion, cleanMsg);
149 LOG.info("Got an expected exception: " + msg);
150 }
151 }
152
153 }
154
155
156 writeTrailer(trailerPath, t, null);
157
158 FixedFileTrailer t4 = readTrailer(trailerPath);
159
160 checkLoadedTrailer(version, t, t4);
161
162 String trailerStr = t.toString();
163 assertEquals("Invalid number of fields in the string representation "
164 + "of the trailer: " + trailerStr, NUM_FIELDS_BY_VERSION[version - 1],
165 trailerStr.split(", ").length);
166 assertEquals(trailerStr, t4.toString());
167 }
168
169 private FixedFileTrailer readTrailer(Path trailerPath) throws IOException {
170 FSDataInputStream fsdis = fs.open(trailerPath);
171 FixedFileTrailer trailerRead = FixedFileTrailer.readFromStream(fsdis,
172 fs.getFileStatus(trailerPath).getLen());
173 fsdis.close();
174 return trailerRead;
175 }
176
177 private void writeTrailer(Path trailerPath, FixedFileTrailer t,
178 byte[] useBytesInstead) throws IOException {
179 assert (t == null) != (useBytesInstead == null);
180
181 FSDataOutputStream fsdos = fs.create(trailerPath);
182 fsdos.write(135);
183 if (useBytesInstead != null) {
184 fsdos.write(useBytesInstead);
185 } else {
186 t.serialize(fsdos);
187 }
188 fsdos.close();
189 }
190
191 private void checkLoadedTrailer(int version, FixedFileTrailer expected,
192 FixedFileTrailer loaded) throws IOException {
193 assertEquals(version, loaded.getVersion());
194 assertEquals(expected.getDataIndexCount(), loaded.getDataIndexCount());
195
196 assertEquals(Math.min(expected.getEntryCount(),
197 version == 1 ? Integer.MAX_VALUE : Long.MAX_VALUE),
198 loaded.getEntryCount());
199
200 if (version == 1) {
201 assertEquals(expected.getFileInfoOffset(), loaded.getFileInfoOffset());
202 }
203
204 if (version == 2) {
205 assertEquals(expected.getLastDataBlockOffset(),
206 loaded.getLastDataBlockOffset());
207 assertEquals(expected.getNumDataIndexLevels(),
208 loaded.getNumDataIndexLevels());
209 assertEquals(expected.createComparator().getClass().getName(),
210 loaded.createComparator().getClass().getName());
211 assertEquals(expected.getFirstDataBlockOffset(),
212 loaded.getFirstDataBlockOffset());
213 assertTrue(
214 expected.createComparator() instanceof KeyValue.KeyComparator);
215 assertEquals(expected.getUncompressedDataIndexSize(),
216 loaded.getUncompressedDataIndexSize());
217 }
218
219 assertEquals(expected.getLoadOnOpenDataOffset(),
220 loaded.getLoadOnOpenDataOffset());
221 assertEquals(expected.getMetaIndexCount(), loaded.getMetaIndexCount());
222
223 assertEquals(expected.getTotalUncompressedBytes(),
224 loaded.getTotalUncompressedBytes());
225 }
226
227 }