1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * Test cases that ensure that file system level errors are bubbled up
56   * appropriately to clients, rather than swallowed.
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     * Injects errors into the pread calls of an on-disk file, and makes
66     * sure those bubble up to the HFile scanner
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      // Do at least one successful read
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); // end of test so evictOnClose
111   }
112 
113   /**
114    * Injects errors into the pread calls of an on-disk file, and makes
115    * sure those bubble up to the StoreFileScanner
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     // Do at least one successful read
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    * Cluster test which starts a region server with a region, then
163    * removes the data from HDFS underneath it, and ensures that
164    * errors are bubbled to the client.
165    */
166   @Test
167   public void testFullSystemBubblesFSErrors() throws Exception {
168     try {
169       // We set it not to run or it will trigger server shutdown while sync'ing
170       // because all the datanodes are bad
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       // Make it fail faster.
185       util.getConfiguration().setInt("hbase.client.retries.number", 1);
186       // Make a new Configuration so it makes a new connection that has the
187       // above configuration on it; else we use the old one w/ 10 as default.
188       HTable table = new HTable(new Configuration(util.getConfiguration()), tableName);
189 
190       // Load some data
191       util.loadTable(table, fam);
192       table.flushCommits();
193       util.flush();
194       util.countRows(table);
195 
196       // Kill the DFS cluster
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      * Starts to simulate faults on all streams opened so far
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