1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import static org.junit.Assert.assertNotNull;
22 import static org.junit.Assert.assertTrue;
23 import static org.junit.Assert.fail;
24
25 import java.io.IOException;
26 import java.io.InputStream;
27 import java.lang.ref.SoftReference;
28 import java.util.ArrayList;
29 import java.util.Collections;
30 import java.util.List;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.fs.FSDataInputStream;
35 import org.apache.hadoop.fs.FileSystem;
36 import org.apache.hadoop.fs.FilterFileSystem;
37 import org.apache.hadoop.fs.Path;
38 import org.apache.hadoop.fs.PositionedReadable;
39 import org.apache.hadoop.hbase.HBaseTestingUtility;
40 import org.apache.hadoop.hbase.HColumnDescriptor;
41 import org.apache.hadoop.hbase.HConstants;
42 import org.apache.hadoop.hbase.HTableDescriptor;
43 import org.apache.hadoop.hbase.KeyValue;
44 import org.apache.hadoop.hbase.client.Table;
45 import org.apache.hadoop.hbase.testclassification.MediumTests;
46 import org.apache.hadoop.hbase.TableName;
47 import org.apache.hadoop.hbase.client.Admin;
48 import org.apache.hadoop.hbase.client.HBaseAdmin;
49 import org.apache.hadoop.hbase.fs.HFileSystem;
50 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
51 import org.apache.hadoop.hbase.io.hfile.HFileContext;
52 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
53 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
54 import org.apache.hadoop.hbase.util.Bytes;
55 import org.junit.Assume;
56 import org.junit.Test;
57 import org.junit.experimental.categories.Category;
58
59
60
61
62
63 @Category(MediumTests.class)
64 public class TestFSErrorsExposed {
65 private static final Log LOG = LogFactory.getLog(TestFSErrorsExposed.class);
66
67 HBaseTestingUtility util = new HBaseTestingUtility();
68
69
70
71
72
73 @Test
74 public void testHFileScannerThrowsErrors() throws IOException {
75 Path hfilePath = new Path(new Path(
76 util.getDataTestDir("internalScannerExposesErrors"),
77 "regionname"), "familyname");
78 HFileSystem hfs = (HFileSystem)util.getTestFileSystem();
79 FaultyFileSystem faultyfs = new FaultyFileSystem(hfs.getBackingFs());
80 FileSystem fs = new HFileSystem(faultyfs);
81 CacheConfig cacheConf = new CacheConfig(util.getConfiguration());
82 HFileContext meta = new HFileContextBuilder().withBlockSize(2 * 1024).build();
83 StoreFile.Writer writer = new StoreFile.WriterBuilder(
84 util.getConfiguration(), cacheConf, hfs)
85 .withOutputDir(hfilePath)
86 .withFileContext(meta)
87 .build();
88 TestStoreFile.writeStoreFile(
89 writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
90
91 StoreFile sf = new StoreFile(fs, writer.getPath(),
92 util.getConfiguration(), cacheConf, BloomType.NONE);
93
94 StoreFile.Reader reader = sf.createReader();
95 HFileScanner scanner = reader.getScanner(false, true);
96
97 FaultyInputStream inStream = faultyfs.inStreams.get(0).get();
98 assertNotNull(inStream);
99
100 scanner.seekTo();
101
102 assertTrue(scanner.next());
103
104 faultyfs.startFaults();
105
106 try {
107 int scanned=0;
108 while (scanner.next()) {
109 scanned++;
110 }
111 fail("Scanner didn't throw after faults injected");
112 } catch (IOException ioe) {
113 LOG.info("Got expected exception", ioe);
114 assertTrue(ioe.getMessage().contains("Fault"));
115 }
116 reader.close(true);
117 }
118
119
120
121
122
123 @Test
124 public void testStoreFileScannerThrowsErrors() throws IOException {
125 Path hfilePath = new Path(new Path(
126 util.getDataTestDir("internalScannerExposesErrors"),
127 "regionname"), "familyname");
128 HFileSystem hfs = (HFileSystem)util.getTestFileSystem();
129 FaultyFileSystem faultyfs = new FaultyFileSystem(hfs.getBackingFs());
130 HFileSystem fs = new HFileSystem(faultyfs);
131 CacheConfig cacheConf = new CacheConfig(util.getConfiguration());
132 HFileContext meta = new HFileContextBuilder().withBlockSize(2 * 1024).build();
133 StoreFile.Writer writer = new StoreFile.WriterBuilder(
134 util.getConfiguration(), cacheConf, hfs)
135 .withOutputDir(hfilePath)
136 .withFileContext(meta)
137 .build();
138 TestStoreFile.writeStoreFile(
139 writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
140
141 StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(),
142 cacheConf, BloomType.NONE);
143
144 List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
145 Collections.singletonList(sf), false, true, false,
146
147 0);
148 KeyValueScanner scanner = scanners.get(0);
149
150 FaultyInputStream inStream = faultyfs.inStreams.get(0).get();
151 assertNotNull(inStream);
152
153 scanner.seek(KeyValue.LOWESTKEY);
154
155 assertNotNull(scanner.next());
156 faultyfs.startFaults();
157
158 try {
159 int scanned=0;
160 while (scanner.next() != null) {
161 scanned++;
162 }
163 fail("Scanner didn't throw after faults injected");
164 } catch (IOException ioe) {
165 LOG.info("Got expected exception", ioe);
166 assertTrue(ioe.getMessage().contains("Could not iterate"));
167 }
168 scanner.close();
169 }
170
171
172
173
174
175
176 @Test(timeout=5 * 60 * 1000)
177 public void testFullSystemBubblesFSErrors() throws Exception {
178
179
180 Assume.assumeTrue(!util.isReadShortCircuitOn());
181
182 try {
183
184 util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
185
186 util.startMiniCluster(1);
187 TableName tableName = TableName.valueOf("table");
188 byte[] fam = Bytes.toBytes("fam");
189
190 Admin admin = new HBaseAdmin(util.getConfiguration());
191 HTableDescriptor desc = new HTableDescriptor(tableName);
192 desc.addFamily(new HColumnDescriptor(fam)
193 .setMaxVersions(1)
194 .setBlockCacheEnabled(false)
195 );
196 admin.createTable(desc);
197
198
199
200 try (Table table = util.getConnection().getTable(tableName)) {
201
202 util.loadTable(table, fam, false);
203 util.flush();
204 util.countRows(table);
205
206
207 util.getDFSCluster().shutdownDataNodes();
208
209 try {
210 util.countRows(table);
211 fail("Did not fail to count after removing data");
212 } catch (Exception e) {
213 LOG.info("Got expected error", e);
214 assertTrue(e.getMessage().contains("Could not seek"));
215 }
216 }
217
218
219 util.getDFSCluster().restartDataNodes();
220
221 } finally {
222 util.getMiniHBaseCluster().killAll();
223 util.shutdownMiniCluster();
224 }
225 }
226
227 static class FaultyFileSystem extends FilterFileSystem {
228 List<SoftReference<FaultyInputStream>> inStreams =
229 new ArrayList<SoftReference<FaultyInputStream>>();
230
231 public FaultyFileSystem(FileSystem testFileSystem) {
232 super(testFileSystem);
233 }
234
235 @Override
236 public FSDataInputStream open(Path p, int bufferSize) throws IOException {
237 FSDataInputStream orig = fs.open(p, bufferSize);
238 FaultyInputStream faulty = new FaultyInputStream(orig);
239 inStreams.add(new SoftReference<FaultyInputStream>(faulty));
240 return faulty;
241 }
242
243
244
245
246 public void startFaults() {
247 for (SoftReference<FaultyInputStream> is: inStreams) {
248 is.get().startFaults();
249 }
250 }
251 }
252
253 static class FaultyInputStream extends FSDataInputStream {
254 boolean faultsStarted = false;
255
256 public FaultyInputStream(InputStream in) throws IOException {
257 super(in);
258 }
259
260 public void startFaults() {
261 faultsStarted = true;
262 }
263
264 @Override
265 public int read(long position, byte[] buffer, int offset, int length)
266 throws IOException {
267 injectFault();
268 return ((PositionedReadable)in).read(position, buffer, offset, length);
269 }
270
271 private void injectFault() throws IOException {
272 if (faultsStarted) {
273 throw new IOException("Fault injected");
274 }
275 }
276 }
277
278
279
280 }