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