001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver;
019
020import static org.junit.Assert.assertNotNull;
021import static org.junit.Assert.assertTrue;
022import static org.junit.Assert.fail;
023
024import java.io.IOException;
025import java.io.InputStream;
026import java.lang.ref.SoftReference;
027import java.util.ArrayList;
028import java.util.Collections;
029import java.util.List;
030import org.apache.hadoop.fs.FSDataInputStream;
031import org.apache.hadoop.fs.FileSystem;
032import org.apache.hadoop.fs.FilterFileSystem;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.fs.PositionedReadable;
035import org.apache.hadoop.hbase.HBaseClassTestRule;
036import org.apache.hadoop.hbase.HBaseTestingUtil;
037import org.apache.hadoop.hbase.HConstants;
038import org.apache.hadoop.hbase.KeyValue;
039import org.apache.hadoop.hbase.SingleProcessHBaseCluster;
040import org.apache.hadoop.hbase.TableName;
041import org.apache.hadoop.hbase.client.Admin;
042import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
043import org.apache.hadoop.hbase.client.Table;
044import org.apache.hadoop.hbase.client.TableDescriptor;
045import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
046import org.apache.hadoop.hbase.fs.HFileSystem;
047import org.apache.hadoop.hbase.io.hfile.CacheConfig;
048import org.apache.hadoop.hbase.io.hfile.HFileContext;
049import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
050import org.apache.hadoop.hbase.io.hfile.HFileScanner;
051import org.apache.hadoop.hbase.testclassification.LargeTests;
052import org.apache.hadoop.hbase.testclassification.RegionServerTests;
053import org.apache.hadoop.hbase.util.Bytes;
054import org.junit.Assume;
055import org.junit.ClassRule;
056import org.junit.Rule;
057import org.junit.Test;
058import org.junit.experimental.categories.Category;
059import org.junit.rules.TestName;
060import org.slf4j.Logger;
061import org.slf4j.LoggerFactory;
062
063/**
064 * Test cases that ensure that file system level errors are bubbled up
065 * appropriately to clients, rather than swallowed.
066 */
067@Category({RegionServerTests.class, LargeTests.class})
068public class TestFSErrorsExposed {
069
070  @ClassRule
071  public static final HBaseClassTestRule CLASS_RULE =
072      HBaseClassTestRule.forClass(TestFSErrorsExposed.class);
073
074  private static final Logger LOG = LoggerFactory.getLogger(TestFSErrorsExposed.class);
075
076  HBaseTestingUtil util = new HBaseTestingUtil();
077
078  @Rule
079  public TestName name = new TestName();
080
081  /**
082   * Injects errors into the pread calls of an on-disk file, and makes
083   * sure those bubble up to the HFile scanner
084   */
085  @Test
086  public void testHFileScannerThrowsErrors() throws IOException {
087    Path hfilePath = new Path(new Path(
088        util.getDataTestDir("internalScannerExposesErrors"),
089        "regionname"), "familyname");
090    HFileSystem hfs = (HFileSystem)util.getTestFileSystem();
091    FaultyFileSystem faultyfs = new FaultyFileSystem(hfs.getBackingFs());
092    FileSystem fs = new HFileSystem(faultyfs);
093    CacheConfig cacheConf = new CacheConfig(util.getConfiguration());
094    HFileContext meta = new HFileContextBuilder().withBlockSize(2 * 1024).build();
095    StoreFileWriter writer = new StoreFileWriter.Builder(
096        util.getConfiguration(), cacheConf, hfs)
097            .withOutputDir(hfilePath)
098            .withFileContext(meta)
099            .build();
100    TestHStoreFile.writeStoreFile(
101        writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
102
103    HStoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
104        BloomType.NONE, true);
105    sf.initReader();
106    StoreFileReader reader = sf.getReader();
107    HFileScanner scanner = reader.getScanner(false, true);
108
109    FaultyInputStream inStream = faultyfs.inStreams.get(0).get();
110    assertNotNull(inStream);
111
112    scanner.seekTo();
113    // Do at least one successful read
114    assertTrue(scanner.next());
115
116    faultyfs.startFaults();
117
118    try {
119      int scanned=0;
120      while (scanner.next()) {
121        scanned++;
122      }
123      fail("Scanner didn't throw after faults injected");
124    } catch (IOException ioe) {
125      LOG.info("Got expected exception", ioe);
126      assertTrue(ioe.getMessage().contains("Fault"));
127    }
128    reader.close(true); // end of test so evictOnClose
129  }
130
131  /**
132   * Injects errors into the pread calls of an on-disk file, and makes
133   * sure those bubble up to the StoreFileScanner
134   */
135  @Test
136  public void testStoreFileScannerThrowsErrors() throws IOException {
137    Path hfilePath = new Path(new Path(
138        util.getDataTestDir("internalScannerExposesErrors"),
139        "regionname"), "familyname");
140    HFileSystem hfs = (HFileSystem)util.getTestFileSystem();
141    FaultyFileSystem faultyfs = new FaultyFileSystem(hfs.getBackingFs());
142    HFileSystem fs = new HFileSystem(faultyfs);
143    CacheConfig cacheConf = new CacheConfig(util.getConfiguration());
144    HFileContext meta = new HFileContextBuilder().withBlockSize(2 * 1024).build();
145    StoreFileWriter writer = new StoreFileWriter.Builder(
146        util.getConfiguration(), cacheConf, hfs)
147            .withOutputDir(hfilePath)
148            .withFileContext(meta)
149            .build();
150    TestHStoreFile.writeStoreFile(
151        writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
152
153    HStoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
154        BloomType.NONE, true);
155
156    List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
157        Collections.singletonList(sf), false, true, false, false,
158        // 0 is passed as readpoint because this test operates on HStoreFile directly
159        0);
160    KeyValueScanner scanner = scanners.get(0);
161
162    FaultyInputStream inStream = faultyfs.inStreams.get(0).get();
163    assertNotNull(inStream);
164
165    scanner.seek(KeyValue.LOWESTKEY);
166    // Do at least one successful read
167    assertNotNull(scanner.next());
168    faultyfs.startFaults();
169
170    try {
171      int scanned=0;
172      while (scanner.next() != null) {
173        scanned++;
174      }
175      fail("Scanner didn't throw after faults injected");
176    } catch (IOException ioe) {
177      LOG.info("Got expected exception", ioe);
178      assertTrue(ioe.getMessage().contains("Could not iterate"));
179    }
180    scanner.close();
181  }
182
183  /**
184   * Cluster test which starts a region server with a region, then
185   * removes the data from HDFS underneath it, and ensures that
186   * errors are bubbled to the client.
187   */
188  @Test
189  public void testFullSystemBubblesFSErrors() throws Exception {
190    // We won't have an error if the datanode is not there if we use short circuit
191    //  it's a known 'feature'.
192    Assume.assumeTrue(!util.isReadShortCircuitOn());
193
194    try {
195      // Make it fail faster.
196      util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
197      util.getConfiguration().setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 90000);
198      util.getConfiguration().setInt("hbase.lease.recovery.timeout", 10000);
199      util.getConfiguration().setInt("hbase.lease.recovery.dfs.timeout", 1000);
200      util.startMiniCluster(1);
201      final TableName tableName = TableName.valueOf(name.getMethodName());
202      byte[] fam = Bytes.toBytes("fam");
203
204      Admin admin = util.getAdmin();
205      TableDescriptor tableDescriptor =
206        TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
207          .newBuilder(fam).setMaxVersions(1).setBlockCacheEnabled(false).build()).build();
208      admin.createTable(tableDescriptor);
209
210      // Make a new Configuration so it makes a new connection that has the
211      // above configuration on it; else we use the old one w/ 10 as default.
212      try (Table table = util.getConnection().getTable(tableName)) {
213        // Load some data
214        util.loadTable(table, fam, false);
215        util.flush();
216        util.countRows(table);
217
218        // Kill the DFS cluster
219        util.getDFSCluster().shutdownDataNodes();
220
221        try {
222          util.countRows(table);
223          fail("Did not fail to count after removing data");
224        } catch (Exception e) {
225          LOG.info("Got expected error", e);
226          assertTrue(e.getMessage().contains("Could not seek"));
227        }
228      }
229
230      // Restart data nodes so that HBase can shut down cleanly.
231      util.getDFSCluster().restartDataNodes();
232
233    } finally {
234      SingleProcessHBaseCluster cluster = util.getMiniHBaseCluster();
235      if (cluster != null) cluster.killAll();
236      util.shutdownMiniCluster();
237    }
238  }
239
240  static class FaultyFileSystem extends FilterFileSystem {
241    List<SoftReference<FaultyInputStream>> inStreams = new ArrayList<>();
242
243    public FaultyFileSystem(FileSystem testFileSystem) {
244      super(testFileSystem);
245    }
246
247    @Override
248    public FSDataInputStream open(Path p, int bufferSize) throws IOException  {
249      FSDataInputStream orig = fs.open(p, bufferSize);
250      FaultyInputStream faulty = new FaultyInputStream(orig);
251      inStreams.add(new SoftReference<>(faulty));
252      return faulty;
253    }
254
255    /**
256     * Starts to simulate faults on all streams opened so far
257     */
258    public void startFaults() {
259      for (SoftReference<FaultyInputStream> is: inStreams) {
260        is.get().startFaults();
261      }
262    }
263  }
264
265  static class FaultyInputStream extends FSDataInputStream {
266    boolean faultsStarted = false;
267
268    public FaultyInputStream(InputStream in) throws IOException {
269      super(in);
270    }
271
272    public void startFaults() {
273      faultsStarted = true;
274    }
275
276    @Override
277    public int read(long position, byte[] buffer, int offset, int length)
278      throws IOException {
279      injectFault();
280      return ((PositionedReadable)in).read(position, buffer, offset, length);
281    }
282
283    private void injectFault() throws IOException {
284      if (faultsStarted) {
285        throw new IOException("Fault injected");
286      }
287    }
288  }
289}