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