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.io.hfile;
019
020import static org.hamcrest.MatcherAssert.assertThat;
021import static org.hamcrest.Matchers.allOf;
022import static org.hamcrest.Matchers.hasProperty;
023import static org.hamcrest.Matchers.instanceOf;
024import static org.hamcrest.Matchers.startsWith;
025import static org.junit.Assert.assertEquals;
026import static org.junit.Assert.assertNotEquals;
027import static org.junit.Assert.assertTrue;
028import static org.junit.Assert.fail;
029
030import java.io.ByteArrayOutputStream;
031import java.io.Closeable;
032import java.io.IOException;
033import java.io.PrintStream;
034import java.nio.ByteBuffer;
035import java.nio.channels.SeekableByteChannel;
036import java.nio.charset.StandardCharsets;
037import java.nio.file.FileSystems;
038import java.nio.file.Files;
039import java.nio.file.StandardOpenOption;
040import java.time.Instant;
041import java.util.LinkedList;
042import java.util.List;
043import java.util.NoSuchElementException;
044import java.util.Random;
045import org.apache.hadoop.conf.Configuration;
046import org.apache.hadoop.fs.Path;
047import org.apache.hadoop.hbase.Cell;
048import org.apache.hadoop.hbase.CellBuilderType;
049import org.apache.hadoop.hbase.ExtendedCell;
050import org.apache.hadoop.hbase.ExtendedCellBuilder;
051import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
052import org.apache.hadoop.hbase.HBaseClassTestRule;
053import org.apache.hadoop.hbase.HBaseTestingUtil;
054import org.apache.hadoop.hbase.fs.HFileSystem;
055import org.apache.hadoop.hbase.nio.ByteBuff;
056import org.apache.hadoop.hbase.testclassification.IOTests;
057import org.apache.hadoop.hbase.testclassification.SmallTests;
058import org.apache.hadoop.hbase.util.Bytes;
059import org.apache.hadoop.hbase.util.ChecksumType;
060import org.hamcrest.Description;
061import org.hamcrest.Matcher;
062import org.hamcrest.TypeSafeMatcher;
063import org.junit.ClassRule;
064import org.junit.Rule;
065import org.junit.Test;
066import org.junit.experimental.categories.Category;
067import org.junit.rules.ExternalResource;
068import org.junit.rules.RuleChain;
069import org.junit.rules.TestName;
070import org.slf4j.Logger;
071import org.slf4j.LoggerFactory;
072
073/**
074 * This test provides coverage for HFileHeader block fields that are read and interpreted before
075 * HBase checksum validation can be applied. As of now, this is just
076 * {@code onDiskSizeWithoutHeader}.
077 */
078@Category({ IOTests.class, SmallTests.class })
079public class TestHFileBlockHeaderCorruption {
080
081  private static final Logger LOG = LoggerFactory.getLogger(TestHFileBlockHeaderCorruption.class);
082
083  @ClassRule
084  public static final HBaseClassTestRule CLASS_RULE =
085    HBaseClassTestRule.forClass(TestHFileBlockHeaderCorruption.class);
086
087  private final HFileTestRule hFileTestRule;
088
089  @Rule
090  public final RuleChain ruleChain;
091
092  public TestHFileBlockHeaderCorruption() throws IOException {
093    TestName testName = new TestName();
094    hFileTestRule = new HFileTestRule(new HBaseTestingUtil(), testName);
095    ruleChain = RuleChain.outerRule(testName).around(hFileTestRule);
096  }
097
098  @Test
099  public void testChecksumTypeCorruptionFirstBlock() throws Exception {
100    HFileBlockChannelPosition firstBlock = null;
101    try {
102      try (HFileBlockChannelPositionIterator it =
103        new HFileBlockChannelPositionIterator(hFileTestRule)) {
104        assertTrue(it.hasNext());
105        firstBlock = it.next();
106      }
107
108      Corrupter c = new Corrupter(firstBlock);
109
110      logHeader(firstBlock);
111
112      // test corrupted HFileBlock with unknown checksumType code -1
113      c.write(HFileBlock.Header.CHECKSUM_TYPE_INDEX, ByteBuffer.wrap(new byte[] { -1 }));
114      logHeader(firstBlock);
115      try (HFileBlockChannelPositionIterator it =
116        new HFileBlockChannelPositionIterator(hFileTestRule)) {
117        CountingConsumer consumer = new CountingConsumer(it);
118        try {
119          consumer.readFully();
120          fail();
121        } catch (Exception e) {
122          assertThat(e, new IsThrowableMatching().withInstanceOf(IOException.class)
123            .withMessage(startsWith("Unknown checksum type code")));
124        }
125        assertEquals(0, consumer.getItemsRead());
126      }
127
128      // valid checksumType code test
129      for (ChecksumType t : ChecksumType.values()) {
130        testValidChecksumTypeReadBlock(t.getCode(), c, firstBlock);
131      }
132
133      c.restore();
134      // test corrupted HFileBlock with unknown checksumType code 3
135      c.write(HFileBlock.Header.CHECKSUM_TYPE_INDEX, ByteBuffer.wrap(new byte[] { 3 }));
136      logHeader(firstBlock);
137      try (HFileBlockChannelPositionIterator it =
138        new HFileBlockChannelPositionIterator(hFileTestRule)) {
139        CountingConsumer consumer = new CountingConsumer(it);
140        try {
141          consumer.readFully();
142          fail();
143        } catch (Exception e) {
144          assertThat(e, new IsThrowableMatching().withInstanceOf(IOException.class)
145            .withMessage(startsWith("Unknown checksum type code")));
146        }
147        assertEquals(0, consumer.getItemsRead());
148      }
149    } finally {
150      if (firstBlock != null) {
151        firstBlock.close();
152      }
153    }
154  }
155
156  @Test
157  public void testChecksumTypeCorruptionSecondBlock() throws Exception {
158    HFileBlockChannelPosition secondBlock = null;
159    try {
160      try (HFileBlockChannelPositionIterator it =
161        new HFileBlockChannelPositionIterator(hFileTestRule)) {
162        assertTrue(it.hasNext());
163        it.next();
164        assertTrue(it.hasNext());
165        secondBlock = it.next();
166      }
167
168      Corrupter c = new Corrupter(secondBlock);
169
170      logHeader(secondBlock);
171      // test corrupted HFileBlock with unknown checksumType code -1
172      c.write(HFileBlock.Header.CHECKSUM_TYPE_INDEX, ByteBuffer.wrap(new byte[] { -1 }));
173      logHeader(secondBlock);
174      try (HFileBlockChannelPositionIterator it =
175        new HFileBlockChannelPositionIterator(hFileTestRule)) {
176        CountingConsumer consumer = new CountingConsumer(it);
177        try {
178          consumer.readFully();
179          fail();
180        } catch (Exception e) {
181          assertThat(e, new IsThrowableMatching().withInstanceOf(RuntimeException.class)
182            .withMessage(startsWith("Unknown checksum type code")));
183        }
184        assertEquals(1, consumer.getItemsRead());
185      }
186
187      // valid checksumType code test
188      for (ChecksumType t : ChecksumType.values()) {
189        testValidChecksumTypeReadBlock(t.getCode(), c, secondBlock);
190      }
191
192      c.restore();
193      // test corrupted HFileBlock with unknown checksumType code 3
194      c.write(HFileBlock.Header.CHECKSUM_TYPE_INDEX, ByteBuffer.wrap(new byte[] { 3 }));
195      logHeader(secondBlock);
196      try (HFileBlockChannelPositionIterator it =
197        new HFileBlockChannelPositionIterator(hFileTestRule)) {
198        CountingConsumer consumer = new CountingConsumer(it);
199        try {
200          consumer.readFully();
201          fail();
202        } catch (Exception e) {
203          assertThat(e, new IsThrowableMatching().withInstanceOf(RuntimeException.class)
204            .withMessage(startsWith("Unknown checksum type code")));
205        }
206        assertEquals(1, consumer.getItemsRead());
207      }
208    } finally {
209      if (secondBlock != null) {
210        secondBlock.close();
211      }
212    }
213  }
214
215  public void testValidChecksumTypeReadBlock(byte checksumTypeCode, Corrupter c,
216    HFileBlockChannelPosition testBlock) throws IOException {
217    c.restore();
218    c.write(HFileBlock.Header.CHECKSUM_TYPE_INDEX,
219      ByteBuffer.wrap(new byte[] { checksumTypeCode }));
220    logHeader(testBlock);
221    try (
222      HFileBlockChannelPositionIterator it = new HFileBlockChannelPositionIterator(hFileTestRule)) {
223      CountingConsumer consumer = new CountingConsumer(it);
224      try {
225        consumer.readFully();
226      } catch (Exception e) {
227        fail("test fail: valid checksumType are not executing properly");
228      }
229      assertNotEquals(0, consumer.getItemsRead());
230    }
231  }
232
233  @Test
234  public void testOnDiskSizeWithoutHeaderCorruptionFirstBlock() throws Exception {
235    HFileBlockChannelPosition firstBlock = null;
236    try {
237      try (HFileBlockChannelPositionIterator it =
238        new HFileBlockChannelPositionIterator(hFileTestRule)) {
239        assertTrue(it.hasNext());
240        firstBlock = it.next();
241      }
242
243      Corrupter c = new Corrupter(firstBlock);
244
245      logHeader(firstBlock);
246      c.write(HFileBlock.Header.ON_DISK_SIZE_WITHOUT_HEADER_INDEX,
247        ByteBuffer.wrap(Bytes.toBytes(Integer.MIN_VALUE)));
248      logHeader(firstBlock);
249      try (HFileBlockChannelPositionIterator it =
250        new HFileBlockChannelPositionIterator(hFileTestRule)) {
251        CountingConsumer consumer = new CountingConsumer(it);
252        try {
253          consumer.readFully();
254          fail();
255        } catch (Exception e) {
256          assertThat(e, new IsThrowableMatching().withInstanceOf(IOException.class)
257            .withMessage(startsWith("Invalid onDiskSizeWithHeader=")));
258        }
259        assertEquals(0, consumer.getItemsRead());
260      }
261
262      c.restore();
263      c.write(HFileBlock.Header.ON_DISK_SIZE_WITHOUT_HEADER_INDEX,
264        ByteBuffer.wrap(Bytes.toBytes(0)));
265      logHeader(firstBlock);
266      try (HFileBlockChannelPositionIterator it =
267        new HFileBlockChannelPositionIterator(hFileTestRule)) {
268        CountingConsumer consumer = new CountingConsumer(it);
269        try {
270          consumer.readFully();
271          fail();
272        } catch (Exception e) {
273          assertThat(e, new IsThrowableMatching().withInstanceOf(IllegalArgumentException.class));
274        }
275        assertEquals(0, consumer.getItemsRead());
276      }
277
278      c.restore();
279      c.write(HFileBlock.Header.ON_DISK_SIZE_WITHOUT_HEADER_INDEX,
280        ByteBuffer.wrap(Bytes.toBytes(Integer.MAX_VALUE)));
281      logHeader(firstBlock);
282      try (HFileBlockChannelPositionIterator it =
283        new HFileBlockChannelPositionIterator(hFileTestRule)) {
284        CountingConsumer consumer = new CountingConsumer(it);
285        try {
286          consumer.readFully();
287          fail();
288        } catch (Exception e) {
289          assertThat(e, new IsThrowableMatching().withInstanceOf(IOException.class)
290            .withMessage(startsWith("Invalid onDiskSizeWithHeader=")));
291        }
292        assertEquals(0, consumer.getItemsRead());
293      }
294    } finally {
295      if (firstBlock != null) {
296        firstBlock.close();
297      }
298    }
299  }
300
301  @Test
302  public void testOnDiskSizeWithoutHeaderCorruptionSecondBlock() throws Exception {
303    HFileBlockChannelPosition secondBlock = null;
304    try {
305      try (HFileBlockChannelPositionIterator it =
306        new HFileBlockChannelPositionIterator(hFileTestRule)) {
307        assertTrue(it.hasNext());
308        it.next();
309        assertTrue(it.hasNext());
310        secondBlock = it.next();
311      }
312
313      Corrupter c = new Corrupter(secondBlock);
314
315      logHeader(secondBlock);
316      c.write(HFileBlock.Header.ON_DISK_SIZE_WITHOUT_HEADER_INDEX,
317        ByteBuffer.wrap(Bytes.toBytes(Integer.MIN_VALUE)));
318      logHeader(secondBlock);
319      try (HFileBlockChannelPositionIterator it =
320        new HFileBlockChannelPositionIterator(hFileTestRule)) {
321        CountingConsumer consumer = new CountingConsumer(it);
322        try {
323          consumer.readFully();
324          fail();
325        } catch (Exception e) {
326          assertThat(e, new IsThrowableMatching().withInstanceOf(IOException.class)
327            .withMessage(startsWith("Invalid onDiskSizeWithHeader=")));
328        }
329        assertEquals(1, consumer.getItemsRead());
330      }
331
332      c.restore();
333      c.write(HFileBlock.Header.ON_DISK_SIZE_WITHOUT_HEADER_INDEX,
334        ByteBuffer.wrap(Bytes.toBytes(0)));
335      logHeader(secondBlock);
336      try (HFileBlockChannelPositionIterator it =
337        new HFileBlockChannelPositionIterator(hFileTestRule)) {
338        CountingConsumer consumer = new CountingConsumer(it);
339        try {
340          consumer.readFully();
341          fail();
342        } catch (Exception e) {
343          assertThat(e, new IsThrowableMatching().withInstanceOf(IllegalArgumentException.class));
344        }
345        assertEquals(1, consumer.getItemsRead());
346      }
347
348      c.restore();
349      c.write(HFileBlock.Header.ON_DISK_SIZE_WITHOUT_HEADER_INDEX,
350        ByteBuffer.wrap(Bytes.toBytes(Integer.MAX_VALUE)));
351      logHeader(secondBlock);
352      try (HFileBlockChannelPositionIterator it =
353        new HFileBlockChannelPositionIterator(hFileTestRule)) {
354        CountingConsumer consumer = new CountingConsumer(it);
355        try {
356          consumer.readFully();
357          fail();
358        } catch (Exception e) {
359          assertThat(e, new IsThrowableMatching().withInstanceOf(IOException.class)
360            .withMessage(startsWith("Invalid onDiskSizeWithHeader=")));
361        }
362        assertEquals(1, consumer.getItemsRead());
363      }
364    } finally {
365      if (secondBlock != null) {
366        secondBlock.close();
367      }
368    }
369  }
370
371  private static void logHeader(HFileBlockChannelPosition hbcp) throws IOException {
372    ByteBuff buf = ByteBuff.wrap(ByteBuffer.allocate(HFileBlock.headerSize(true)));
373    hbcp.rewind();
374    assertEquals(buf.capacity(), buf.read(hbcp.getChannel()));
375    buf.rewind();
376    hbcp.rewind();
377    logHeader(buf);
378  }
379
380  private static void logHeader(ByteBuff buf) {
381    byte[] blockMagic = new byte[8];
382    buf.get(blockMagic);
383    int onDiskSizeWithoutHeader = buf.getInt();
384    int uncompressedSizeWithoutHeader = buf.getInt();
385    long prevBlockOffset = buf.getLong();
386    byte checksumType = buf.get();
387    int bytesPerChecksum = buf.getInt();
388    int onDiskDataSizeWithHeader = buf.getInt();
389    LOG.debug(
390      "blockMagic={}, onDiskSizeWithoutHeader={}, uncompressedSizeWithoutHeader={}, "
391        + "prevBlockOffset={}, checksumType={}, bytesPerChecksum={}, onDiskDataSizeWithHeader={}",
392      Bytes.toStringBinary(blockMagic), onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
393      prevBlockOffset, checksumType, bytesPerChecksum, onDiskDataSizeWithHeader);
394  }
395
396  /**
397   * Data class to enabled messing with the bytes behind an {@link HFileBlock}.
398   */
399  public static class HFileBlockChannelPosition implements Closeable {
400    private final SeekableByteChannel channel;
401    private final long position;
402
403    public HFileBlockChannelPosition(SeekableByteChannel channel, long position) {
404      this.channel = channel;
405      this.position = position;
406    }
407
408    public SeekableByteChannel getChannel() {
409      return channel;
410    }
411
412    public long getPosition() {
413      return position;
414    }
415
416    public void rewind() throws IOException {
417      channel.position(position);
418    }
419
420    @Override
421    public void close() throws IOException {
422      channel.close();
423    }
424  }
425
426  /**
427   * Reads blocks off of an {@link HFileBlockChannelPositionIterator}, counting them as it does.
428   */
429  public static class CountingConsumer {
430    private final HFileBlockChannelPositionIterator iterator;
431    private int itemsRead = 0;
432
433    public CountingConsumer(HFileBlockChannelPositionIterator iterator) {
434      this.iterator = iterator;
435    }
436
437    public int getItemsRead() {
438      return itemsRead;
439    }
440
441    public Object readFully() throws IOException {
442      Object val = null;
443      for (itemsRead = 0; iterator.hasNext(); itemsRead++) {
444        val = iterator.next();
445      }
446      return val;
447    }
448  }
449
450  /**
451   * A simplified wrapper over an {@link HFileBlock.BlockIterator} that looks a lot like an
452   * {@link java.util.Iterator}.
453   */
454  public static class HFileBlockChannelPositionIterator implements Closeable {
455
456    private final HFileTestRule hFileTestRule;
457    private final HFile.Reader reader;
458    private final HFileBlock.BlockIterator iter;
459    private HFileBlockChannelPosition current = null;
460
461    public HFileBlockChannelPositionIterator(HFileTestRule hFileTestRule) throws IOException {
462      Configuration conf = hFileTestRule.getConfiguration();
463      HFileSystem hfs = hFileTestRule.getHFileSystem();
464      Path hfsPath = hFileTestRule.getPath();
465
466      HFile.Reader reader = null;
467      HFileBlock.BlockIterator iter = null;
468      try {
469        reader = HFile.createReader(hfs, hfsPath, CacheConfig.DISABLED, true, conf);
470        HFileBlock.FSReader fsreader = reader.getUncachedBlockReader();
471        // The read block offset cannot out of the range:0,loadOnOpenDataOffset
472        iter = fsreader.blockRange(0, reader.getTrailer().getLoadOnOpenDataOffset());
473      } catch (IOException e) {
474        if (reader != null) {
475          closeQuietly(reader::close);
476        }
477        throw e;
478      }
479
480      this.hFileTestRule = hFileTestRule;
481      this.reader = reader;
482      this.iter = iter;
483    }
484
485    public boolean hasNext() throws IOException {
486      HFileBlock next = iter.nextBlock();
487      SeekableByteChannel channel = hFileTestRule.getRWChannel();
488      if (next != null) {
489        current = new HFileBlockChannelPosition(channel, next.getOffset());
490      }
491      return next != null;
492    }
493
494    public HFileBlockChannelPosition next() {
495      if (current == null) {
496        throw new NoSuchElementException();
497      }
498      HFileBlockChannelPosition ret = current;
499      current = null;
500      return ret;
501    }
502
503    @Override
504    public void close() throws IOException {
505      if (current != null) {
506        closeQuietly(current::close);
507      }
508      closeQuietly(reader::close);
509    }
510
511    @FunctionalInterface
512    private interface CloseMethod {
513      void run() throws IOException;
514    }
515
516    private static void closeQuietly(CloseMethod closeMethod) {
517      try {
518        closeMethod.run();
519      } catch (Throwable e) {
520        LOG.debug("Ignoring thrown exception.", e);
521      }
522    }
523  }
524
525  /**
526   * Enables writing and rewriting portions of the file backing an {@link HFileBlock}.
527   */
528  public static class Corrupter {
529
530    private final HFileBlockChannelPosition channelAndPosition;
531    private final ByteBuffer originalHeader;
532
533    public Corrupter(HFileBlockChannelPosition channelAndPosition) throws IOException {
534      this.channelAndPosition = channelAndPosition;
535      this.originalHeader = readHeaderData(channelAndPosition);
536    }
537
538    private static ByteBuffer readHeaderData(HFileBlockChannelPosition channelAndPosition)
539      throws IOException {
540      SeekableByteChannel channel = channelAndPosition.getChannel();
541      ByteBuffer originalHeader = ByteBuffer.allocate(HFileBlock.headerSize(true));
542      channelAndPosition.rewind();
543      channel.read(originalHeader);
544      return originalHeader;
545    }
546
547    public void write(int offset, ByteBuffer src) throws IOException {
548      SeekableByteChannel channel = channelAndPosition.getChannel();
549      long position = channelAndPosition.getPosition();
550      channel.position(position + offset);
551      channel.write(src);
552    }
553
554    public void restore() throws IOException {
555      SeekableByteChannel channel = channelAndPosition.getChannel();
556      originalHeader.rewind();
557      channelAndPosition.rewind();
558      assertEquals(originalHeader.capacity(), channel.write(originalHeader));
559    }
560  }
561
562  public static class HFileTestRule extends ExternalResource {
563
564    private final HBaseTestingUtil testingUtility;
565    private final HFileSystem hfs;
566    private final HFileContext context;
567    private final TestName testName;
568    private Path path;
569
570    public HFileTestRule(HBaseTestingUtil testingUtility, TestName testName) throws IOException {
571      this.testingUtility = testingUtility;
572      this.testName = testName;
573      this.hfs = (HFileSystem) HFileSystem.get(testingUtility.getConfiguration());
574      this.context =
575        new HFileContextBuilder().withBlockSize(4 * 1024).withHBaseCheckSum(true).build();
576    }
577
578    public Configuration getConfiguration() {
579      return testingUtility.getConfiguration();
580    }
581
582    public HFileSystem getHFileSystem() {
583      return hfs;
584    }
585
586    public HFileContext getHFileContext() {
587      return context;
588    }
589
590    public Path getPath() {
591      return path;
592    }
593
594    public SeekableByteChannel getRWChannel() throws IOException {
595      java.nio.file.Path p = FileSystems.getDefault().getPath(path.toString());
596      return Files.newByteChannel(p, StandardOpenOption.READ, StandardOpenOption.WRITE,
597        StandardOpenOption.DSYNC);
598    }
599
600    @Override
601    protected void before() throws Throwable {
602      this.path = new Path(testingUtility.getDataTestDirOnTestFS(), testName.getMethodName());
603      HFile.WriterFactory factory =
604        HFile.getWriterFactory(testingUtility.getConfiguration(), CacheConfig.DISABLED)
605          .withPath(hfs, path).withFileContext(context);
606
607      ExtendedCellBuilder cellBuilder =
608        ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY);
609      Random rand = new Random(Instant.now().toEpochMilli());
610      byte[] family = Bytes.toBytes("f");
611      try (HFile.Writer writer = factory.create()) {
612        for (int i = 0; i < 40; i++) {
613          byte[] row = RandomKeyValueUtil.randomOrderedFixedLengthKey(rand, i, 100);
614          byte[] qualifier = RandomKeyValueUtil.randomRowOrQualifier(rand);
615          byte[] value = RandomKeyValueUtil.randomValue(rand);
616          ExtendedCell cell = cellBuilder.setType(Cell.Type.Put).setRow(row).setFamily(family)
617            .setQualifier(qualifier).setValue(value).build();
618          writer.append(cell);
619          cellBuilder.clear();
620        }
621      }
622    }
623  }
624
625  /**
626   * A Matcher implementation that can make basic assertions over a provided {@link Throwable}.
627   * Assertion failures include the full stacktrace in their description.
628   */
629  private static final class IsThrowableMatching extends TypeSafeMatcher<Throwable> {
630
631    private final List<Matcher<? super Throwable>> requirements = new LinkedList<>();
632
633    public IsThrowableMatching withInstanceOf(Class<?> type) {
634      requirements.add(instanceOf(type));
635      return this;
636    }
637
638    public IsThrowableMatching withMessage(Matcher<String> matcher) {
639      requirements.add(hasProperty("message", matcher));
640      return this;
641    }
642
643    @Override
644    protected boolean matchesSafely(Throwable throwable) {
645      return allOf(requirements).matches(throwable);
646    }
647
648    @Override
649    protected void describeMismatchSafely(Throwable item, Description mismatchDescription) {
650      allOf(requirements).describeMismatch(item, mismatchDescription);
651      // would be nice if `item` could be provided as the cause of the AssertionError instead.
652      mismatchDescription.appendText(String.format("%nProvided: "));
653      try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
654        try (PrintStream ps = new PrintStream(baos, false, StandardCharsets.UTF_8.name())) {
655          item.printStackTrace(ps);
656          ps.flush();
657        }
658        mismatchDescription.appendText(baos.toString(StandardCharsets.UTF_8.name()));
659      } catch (Exception e) {
660        throw new RuntimeException(e);
661      }
662    }
663
664    @Override
665    public void describeTo(Description description) {
666      description.appendDescriptionOf(allOf(requirements));
667    }
668  }
669}