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