You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by uw...@apache.org on 2018/04/23 20:51:38 UTC

[parquet-cpp] branch master updated: PARQUET-1279: [C++] Adding use of ASSERT_NO_FATAL_FAILURE in unit tests when calling helper functions that call ASSERT_ macros

This is an automated email from the ASF dual-hosted git repository.

uwe pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-cpp.git


The following commit(s) were added to refs/heads/master by this push:
     new f973727  PARQUET-1279: [C++] Adding use of ASSERT_NO_FATAL_FAILURE in unit tests when calling helper functions that call ASSERT_ macros
f973727 is described below

commit f973727a0bec75a6e352c0f51a7af2630ac3ec42
Author: Joshua Storck <jo...@twosigma.com>
AuthorDate: Mon Apr 23 22:51:32 2018 +0200

    PARQUET-1279: [C++] Adding use of ASSERT_NO_FATAL_FAILURE in unit tests when calling helper functions that call ASSERT_ macros
    
    There were a number of helper functions in the unit tests that has ASSERT_ macros. However, the callers of these functions were not asserting if the helper functions had encountered an assertion. In those circumstances, the unit test will continue because the ASSERT_ macros only return in the block in which they are used. This commit adds the use of ASSERT_NO_FATAL_FAILURE to the calls of those helper functions at the scope of the unit test, which will cause the unit test to exit. Thi [...]
    
    Author: Joshua Storck <jo...@twosigma.com>
    
    Closes #458 from joshuastorck/use_assert_no_fatal_failure and squashes the following commits:
    
    fa89f5d [Joshua Storck] There were a number of helper functions in the unit tests that has ASSERT_ macros. However, the callers of these functions were not asserting if the helper functions had encountered an assertion. In those circumstances, the unit test will continue because the ASSERT_ macros only return in the block in which they are used. This commit adds the use of ASSERT_NO_FATAL_FAILURE to the calls of those helper functions at the scope of the unit test, which will cause th [...]
---
 src/parquet/arrow/arrow-reader-writer-test.cc | 196 ++++++++++++++------------
 src/parquet/arrow/arrow-schema-test.cc        |  42 +++---
 src/parquet/column_reader-test.cc             |  12 +-
 src/parquet/column_scanner-test.cc            |  23 +--
 src/parquet/column_writer-test.cc             |  20 +--
 src/parquet/encoding-test.cc                  |  14 +-
 src/parquet/file-deserialize-test.cc          |  23 +--
 src/parquet/file-serialize-test.cc            |  18 ++-
 src/parquet/schema-test.cc                    |  14 +-
 src/parquet/statistics-test.cc                |  19 ++-
 10 files changed, 205 insertions(+), 176 deletions(-)

diff --git a/src/parquet/arrow/arrow-reader-writer-test.cc b/src/parquet/arrow/arrow-reader-writer-test.cc
index cb38b8f..47226a3 100644
--- a/src/parquet/arrow/arrow-reader-writer-test.cc
+++ b/src/parquet/arrow/arrow-reader-writer-test.cc
@@ -405,7 +405,8 @@ void DoSimpleRoundtrip(const std::shared_ptr<Table>& table, int num_threads,
                        const std::shared_ptr<ArrowWriterProperties>& arrow_properties =
                            default_arrow_writer_properties()) {
   std::shared_ptr<Buffer> buffer;
-  WriteTableToBuffer(table, num_threads, row_group_size, arrow_properties, &buffer);
+  ASSERT_NO_FATAL_FAILURE(
+      WriteTableToBuffer(table, num_threads, row_group_size, arrow_properties, &buffer));
 
   std::unique_ptr<FileReader> reader;
   ASSERT_OK_NO_THROW(OpenFile(std::make_shared<BufferReader>(buffer),
@@ -427,7 +428,7 @@ void CheckSimpleRoundtrip(const std::shared_ptr<Table>& table, int64_t row_group
                               default_arrow_writer_properties()) {
   std::shared_ptr<Table> result;
   DoSimpleRoundtrip(table, 1, row_group_size, {}, &result, arrow_properties);
-  AssertTablesEqual(*table, *result, false);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*table, *result, false));
 }
 
 static std::shared_ptr<GroupNode> MakeSimpleSchema(const ::DataType& type,
@@ -611,9 +612,9 @@ TYPED_TEST(TestParquetIO, SingleColumnRequiredWrite) {
 
   std::shared_ptr<GroupNode> schema =
       MakeSimpleSchema(*values->type(), Repetition::REQUIRED);
-  this->WriteColumn(schema, values);
+  ASSERT_NO_FATAL_FAILURE(this->WriteColumn(schema, values));
 
-  this->ReadAndCheckSingleColumnFile(*values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnTableRequiredWrite) {
@@ -626,8 +627,8 @@ TYPED_TEST(TestParquetIO, SingleColumnTableRequiredWrite) {
 
   std::shared_ptr<Table> out;
   std::unique_ptr<FileReader> reader;
-  this->ReaderFromSink(&reader);
-  this->ReadTableFromFile(std::move(reader), &out);
+  ASSERT_NO_FATAL_FAILURE(this->ReaderFromSink(&reader));
+  ASSERT_NO_FATAL_FAILURE(this->ReadTableFromFile(std::move(reader), &out));
   ASSERT_EQ(1, out->num_columns());
   ASSERT_EQ(100, out->num_rows());
 
@@ -645,9 +646,9 @@ TYPED_TEST(TestParquetIO, SingleColumnOptionalReadWrite) {
 
   std::shared_ptr<GroupNode> schema =
       MakeSimpleSchema(*values->type(), Repetition::OPTIONAL);
-  this->WriteColumn(schema, values);
+  ASSERT_NO_FATAL_FAILURE(this->WriteColumn(schema, values));
 
-  this->ReadAndCheckSingleColumnFile(*values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnOptionalDictionaryWrite) {
@@ -666,9 +667,9 @@ TYPED_TEST(TestParquetIO, SingleColumnOptionalDictionaryWrite) {
   std::shared_ptr<Array> dict_values = MakeArray(out.array());
   std::shared_ptr<GroupNode> schema =
       MakeSimpleSchema(*dict_values->type(), Repetition::OPTIONAL);
-  this->WriteColumn(schema, dict_values);
+  ASSERT_NO_FATAL_FAILURE(this->WriteColumn(schema, dict_values));
 
-  this->ReadAndCheckSingleColumnFile(*values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnRequiredSliceWrite) {
@@ -678,13 +679,13 @@ TYPED_TEST(TestParquetIO, SingleColumnRequiredSliceWrite) {
       MakeSimpleSchema(*values->type(), Repetition::REQUIRED);
 
   std::shared_ptr<Array> sliced_values = values->Slice(SMALL_SIZE / 2, SMALL_SIZE);
-  this->WriteColumn(schema, sliced_values);
-  this->ReadAndCheckSingleColumnFile(*sliced_values);
+  ASSERT_NO_FATAL_FAILURE(this->WriteColumn(schema, sliced_values));
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*sliced_values));
 
   // Slice offset 1 higher
   sliced_values = values->Slice(SMALL_SIZE / 2 + 1, SMALL_SIZE);
-  this->WriteColumn(schema, sliced_values);
-  this->ReadAndCheckSingleColumnFile(*sliced_values);
+  ASSERT_NO_FATAL_FAILURE(this->WriteColumn(schema, sliced_values));
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*sliced_values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnOptionalSliceWrite) {
@@ -694,13 +695,13 @@ TYPED_TEST(TestParquetIO, SingleColumnOptionalSliceWrite) {
       MakeSimpleSchema(*values->type(), Repetition::OPTIONAL);
 
   std::shared_ptr<Array> sliced_values = values->Slice(SMALL_SIZE / 2, SMALL_SIZE);
-  this->WriteColumn(schema, sliced_values);
-  this->ReadAndCheckSingleColumnFile(*sliced_values);
+  ASSERT_NO_FATAL_FAILURE(this->WriteColumn(schema, sliced_values));
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*sliced_values));
 
   // Slice offset 1 higher, thus different null bitmap.
   sliced_values = values->Slice(SMALL_SIZE / 2 + 1, SMALL_SIZE);
-  this->WriteColumn(schema, sliced_values);
-  this->ReadAndCheckSingleColumnFile(*sliced_values);
+  ASSERT_NO_FATAL_FAILURE(this->WriteColumn(schema, sliced_values));
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*sliced_values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnTableOptionalReadWrite) {
@@ -709,37 +710,38 @@ TYPED_TEST(TestParquetIO, SingleColumnTableOptionalReadWrite) {
 
   ASSERT_OK(NullableArray<TypeParam>(SMALL_SIZE, 10, kDefaultSeed, &values));
   std::shared_ptr<Table> table = MakeSimpleTable(values, true);
-  this->CheckRoundTrip(table);
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundTrip(table));
 }
 
 TYPED_TEST(TestParquetIO, SingleNullableListNullableColumnReadWrite) {
   std::shared_ptr<Table> table;
-  this->PrepareListTable(SMALL_SIZE, true, true, 10, &table);
-  this->CheckRoundTrip(table);
+  ASSERT_NO_FATAL_FAILURE(this->PrepareListTable(SMALL_SIZE, true, true, 10, &table));
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundTrip(table));
 }
 
 TYPED_TEST(TestParquetIO, SingleRequiredListNullableColumnReadWrite) {
   std::shared_ptr<Table> table;
-  this->PrepareListTable(SMALL_SIZE, false, true, 10, &table);
-  this->CheckRoundTrip(table);
+  ASSERT_NO_FATAL_FAILURE(this->PrepareListTable(SMALL_SIZE, false, true, 10, &table));
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundTrip(table));
 }
 
 TYPED_TEST(TestParquetIO, SingleNullableListRequiredColumnReadWrite) {
   std::shared_ptr<Table> table;
-  this->PrepareListTable(SMALL_SIZE, true, false, 10, &table);
-  this->CheckRoundTrip(table);
+  ASSERT_NO_FATAL_FAILURE(this->PrepareListTable(SMALL_SIZE, true, false, 10, &table));
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundTrip(table));
 }
 
 TYPED_TEST(TestParquetIO, SingleRequiredListRequiredColumnReadWrite) {
   std::shared_ptr<Table> table;
-  this->PrepareListTable(SMALL_SIZE, false, false, 0, &table);
-  this->CheckRoundTrip(table);
+  ASSERT_NO_FATAL_FAILURE(this->PrepareListTable(SMALL_SIZE, false, false, 0, &table));
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundTrip(table));
 }
 
 TYPED_TEST(TestParquetIO, SingleNullableListRequiredListRequiredColumnReadWrite) {
   std::shared_ptr<Table> table;
-  this->PrepareListOfListTable(SMALL_SIZE, true, false, false, 0, &table);
-  this->CheckRoundTrip(table);
+  ASSERT_NO_FATAL_FAILURE(
+      this->PrepareListOfListTable(SMALL_SIZE, true, false, false, 0, &table));
+  ASSERT_NO_FATAL_FAILURE(this->CheckRoundTrip(table));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnRequiredChunkedWrite) {
@@ -757,7 +759,7 @@ TYPED_TEST(TestParquetIO, SingleColumnRequiredChunkedWrite) {
   }
   ASSERT_OK_NO_THROW(writer.Close());
 
-  this->ReadAndCheckSingleColumnFile(*values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnTableRequiredChunkedWrite) {
@@ -768,7 +770,7 @@ TYPED_TEST(TestParquetIO, SingleColumnTableRequiredChunkedWrite) {
   ASSERT_OK_NO_THROW(WriteTable(*table, default_memory_pool(), this->sink_, 512,
                                 default_writer_properties()));
 
-  this->ReadAndCheckSingleColumnTable(values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnTable(values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnTableRequiredChunkedWriteArrowIO) {
@@ -794,7 +796,7 @@ TYPED_TEST(TestParquetIO, SingleColumnTableRequiredChunkedWriteArrowIO) {
   std::shared_ptr<::arrow::Table> out;
   std::unique_ptr<FileReader> reader;
   ASSERT_OK_NO_THROW(OpenFile(source, ::arrow::default_memory_pool(), &reader));
-  this->ReadTableFromFile(std::move(reader), &out);
+  ASSERT_NO_FATAL_FAILURE(this->ReadTableFromFile(std::move(reader), &out));
   ASSERT_EQ(1, out->num_columns());
   ASSERT_EQ(values->length(), out->num_rows());
 
@@ -820,7 +822,7 @@ TYPED_TEST(TestParquetIO, SingleColumnOptionalChunkedWrite) {
   }
   ASSERT_OK_NO_THROW(writer.Close());
 
-  this->ReadAndCheckSingleColumnFile(*values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*values));
 }
 
 TYPED_TEST(TestParquetIO, SingleColumnTableOptionalChunkedWrite) {
@@ -833,7 +835,7 @@ TYPED_TEST(TestParquetIO, SingleColumnTableOptionalChunkedWrite) {
   ASSERT_OK_NO_THROW(WriteTable(*table, ::arrow::default_memory_pool(), this->sink_, 512,
                                 default_writer_properties()));
 
-  this->ReadAndCheckSingleColumnTable(values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnTable(values));
 }
 
 using TestInt96ParquetIO = TestParquetIO<::arrow::TimestampType>;
@@ -888,7 +890,7 @@ TEST_F(TestInt96ParquetIO, ReadIntoTimestamp) {
   ASSERT_OK(builder.Append(val));
   std::shared_ptr<Array> values;
   ASSERT_OK(builder.Finish(&values));
-  this->ReadAndCheckSingleColumnFile(*values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnFile(*values));
 }
 
 using TestUInt32ParquetIO = TestParquetIO<::arrow::UInt32Type>;
@@ -908,7 +910,7 @@ TEST_F(TestUInt32ParquetIO, Parquet_2_0_Compability) {
           ->build();
   ASSERT_OK_NO_THROW(
       WriteTable(*table, default_memory_pool(), this->sink_, 512, properties));
-  this->ReadAndCheckSingleColumnTable(values);
+  ASSERT_NO_FATAL_FAILURE(this->ReadAndCheckSingleColumnTable(values));
 }
 
 TEST_F(TestUInt32ParquetIO, Parquet_1_0_Compability) {
@@ -986,8 +988,8 @@ TEST_F(TestStringParquetIO, EmptyStringColumnRequiredWrite) {
 
   std::shared_ptr<Table> out;
   std::unique_ptr<FileReader> reader;
-  this->ReaderFromSink(&reader);
-  this->ReadTableFromFile(std::move(reader), &out);
+  ASSERT_NO_FATAL_FAILURE(this->ReaderFromSink(&reader));
+  ASSERT_NO_FATAL_FAILURE(this->ReadTableFromFile(std::move(reader), &out));
   ASSERT_EQ(1, out->num_columns());
   ASSERT_EQ(100, out->num_rows());
 
@@ -1011,8 +1013,8 @@ TEST_F(TestNullParquetIO, NullColumn) {
 
     std::shared_ptr<Table> out;
     std::unique_ptr<FileReader> reader;
-    this->ReaderFromSink(&reader);
-    this->ReadTableFromFile(std::move(reader), &out);
+    ASSERT_NO_FATAL_FAILURE(this->ReaderFromSink(&reader));
+    ASSERT_NO_FATAL_FAILURE(this->ReadTableFromFile(std::move(reader), &out));
     ASSERT_EQ(1, out->num_columns());
     ASSERT_EQ(num_rows, out->num_rows());
 
@@ -1067,8 +1069,8 @@ TEST_F(TestNullParquetIO, NullDictionaryColumn) {
 
   std::shared_ptr<Table> out;
   std::unique_ptr<FileReader> reader;
-  this->ReaderFromSink(&reader);
-  this->ReadTableFromFile(std::move(reader), &out);
+  ASSERT_NO_FATAL_FAILURE(this->ReaderFromSink(&reader));
+  ASSERT_NO_FATAL_FAILURE(this->ReadTableFromFile(std::move(reader), &out));
   ASSERT_EQ(1, out->num_columns());
   ASSERT_EQ(100, out->num_rows());
 
@@ -1127,7 +1129,7 @@ class TestPrimitiveParquetIO : public TestParquetIO<TestType> {
   void CheckSingleColumnRequiredTableRead(int num_chunks) {
     std::vector<T> values(SMALL_SIZE, test_traits<TestType>::value);
     std::unique_ptr<FileReader> file_reader;
-    ASSERT_NO_THROW(MakeTestFile(values, num_chunks, &file_reader));
+    ASSERT_NO_FATAL_FAILURE(MakeTestFile(values, num_chunks, &file_reader));
 
     std::shared_ptr<Table> out;
     this->ReadTableFromFile(std::move(file_reader), &out);
@@ -1142,7 +1144,7 @@ class TestPrimitiveParquetIO : public TestParquetIO<TestType> {
   void CheckSingleColumnRequiredRead(int num_chunks) {
     std::vector<T> values(SMALL_SIZE, test_traits<TestType>::value);
     std::unique_ptr<FileReader> file_reader;
-    ASSERT_NO_THROW(MakeTestFile(values, num_chunks, &file_reader));
+    ASSERT_NO_FATAL_FAILURE(MakeTestFile(values, num_chunks, &file_reader));
 
     std::shared_ptr<Array> out;
     this->ReadSingleColumnFile(std::move(file_reader), &out);
@@ -1160,19 +1162,19 @@ typedef ::testing::Types<::arrow::BooleanType, ::arrow::UInt8Type, ::arrow::Int8
 TYPED_TEST_CASE(TestPrimitiveParquetIO, PrimitiveTestTypes);
 
 TYPED_TEST(TestPrimitiveParquetIO, SingleColumnRequiredRead) {
-  this->CheckSingleColumnRequiredRead(1);
+  ASSERT_NO_FATAL_FAILURE(this->CheckSingleColumnRequiredRead(1));
 }
 
 TYPED_TEST(TestPrimitiveParquetIO, SingleColumnRequiredTableRead) {
-  this->CheckSingleColumnRequiredTableRead(1);
+  ASSERT_NO_FATAL_FAILURE(this->CheckSingleColumnRequiredTableRead(1));
 }
 
 TYPED_TEST(TestPrimitiveParquetIO, SingleColumnRequiredChunkedRead) {
-  this->CheckSingleColumnRequiredRead(4);
+  ASSERT_NO_FATAL_FAILURE(this->CheckSingleColumnRequiredRead(4));
 }
 
 TYPED_TEST(TestPrimitiveParquetIO, SingleColumnRequiredChunkedTableRead) {
-  this->CheckSingleColumnRequiredTableRead(4);
+  ASSERT_NO_FATAL_FAILURE(this->CheckSingleColumnRequiredTableRead(4));
 }
 
 void MakeDateTimeTypesTable(std::shared_ptr<Table>* out, bool nanos_as_micros = false) {
@@ -1228,18 +1230,18 @@ TEST(TestArrowReadWrite, DateTimeTypes) {
 
   // Use deprecated INT96 type
   std::shared_ptr<Table> result;
-  DoSimpleRoundtrip(
+  ASSERT_NO_FATAL_FAILURE(DoSimpleRoundtrip(
       table, 1, table->num_rows(), {}, &result,
-      ArrowWriterProperties::Builder().enable_deprecated_int96_timestamps()->build());
+      ArrowWriterProperties::Builder().enable_deprecated_int96_timestamps()->build()));
 
-  AssertTablesEqual(*table, *result);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*table, *result));
 
   // Cast nanaoseconds to microseconds and use INT64 physical type
-  DoSimpleRoundtrip(table, 1, table->num_rows(), {}, &result);
+  ASSERT_NO_FATAL_FAILURE(DoSimpleRoundtrip(table, 1, table->num_rows(), {}, &result));
   std::shared_ptr<Table> expected;
   MakeDateTimeTypesTable(&table, true);
 
-  AssertTablesEqual(*table, *result);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*table, *result));
 }
 
 TEST(TestArrowReadWrite, CoerceTimestamps) {
@@ -1297,17 +1299,17 @@ TEST(TestArrowReadWrite, CoerceTimestamps) {
        std::make_shared<Column>("f_us", a_us), std::make_shared<Column>("f_ns", a_us)});
 
   std::shared_ptr<Table> milli_result;
-  DoSimpleRoundtrip(
+  ASSERT_NO_FATAL_FAILURE(DoSimpleRoundtrip(
       input, 1, input->num_rows(), {}, &milli_result,
-      ArrowWriterProperties::Builder().coerce_timestamps(TimeUnit::MILLI)->build());
+      ArrowWriterProperties::Builder().coerce_timestamps(TimeUnit::MILLI)->build()));
 
-  AssertTablesEqual(*ex_milli_result, *milli_result);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*ex_milli_result, *milli_result));
 
   std::shared_ptr<Table> micro_result;
-  DoSimpleRoundtrip(
+  ASSERT_NO_FATAL_FAILURE(DoSimpleRoundtrip(
       input, 1, input->num_rows(), {}, &micro_result,
-      ArrowWriterProperties::Builder().coerce_timestamps(TimeUnit::MICRO)->build());
-  AssertTablesEqual(*ex_micro_result, *micro_result);
+      ArrowWriterProperties::Builder().coerce_timestamps(TimeUnit::MICRO)->build()));
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*ex_micro_result, *micro_result));
 }
 
 TEST(TestArrowReadWrite, CoerceTimestampsLosePrecision) {
@@ -1431,9 +1433,9 @@ TEST(TestArrowReadWrite, ConvertedDateTimeTypes) {
   auto ex_table = Table::Make(ex_schema, ex_columns);
 
   std::shared_ptr<Table> result;
-  DoSimpleRoundtrip(table, 1, table->num_rows(), {}, &result);
+  ASSERT_NO_FATAL_FAILURE(DoSimpleRoundtrip(table, 1, table->num_rows(), {}, &result));
 
-  AssertTablesEqual(*ex_table, *result);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*ex_table, *result));
 }
 
 // Regression for ARROW-2802
@@ -1563,12 +1565,13 @@ TEST(TestArrowReadWrite, MultithreadedRead) {
   const int num_threads = 4;
 
   std::shared_ptr<Table> table;
-  MakeDoubleTable(num_columns, num_rows, 1, &table);
+  ASSERT_NO_FATAL_FAILURE(MakeDoubleTable(num_columns, num_rows, 1, &table));
 
   std::shared_ptr<Table> result;
-  DoSimpleRoundtrip(table, num_threads, table->num_rows(), {}, &result);
+  ASSERT_NO_FATAL_FAILURE(
+      DoSimpleRoundtrip(table, num_threads, table->num_rows(), {}, &result));
 
-  AssertTablesEqual(*table, *result);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*table, *result));
 }
 
 TEST(TestArrowReadWrite, ReadSingleRowGroup) {
@@ -1576,10 +1579,11 @@ TEST(TestArrowReadWrite, ReadSingleRowGroup) {
   const int num_rows = 1000;
 
   std::shared_ptr<Table> table;
-  MakeDoubleTable(num_columns, num_rows, 1, &table);
+  ASSERT_NO_FATAL_FAILURE(MakeDoubleTable(num_columns, num_rows, 1, &table));
 
   std::shared_ptr<Buffer> buffer;
-  WriteTableToBuffer(table, 1, num_rows / 2, default_arrow_writer_properties(), &buffer);
+  ASSERT_NO_FATAL_FAILURE(WriteTableToBuffer(table, 1, num_rows / 2,
+                                             default_arrow_writer_properties(), &buffer));
 
   std::unique_ptr<FileReader> reader;
   ASSERT_OK_NO_THROW(OpenFile(std::make_shared<BufferReader>(buffer),
@@ -1604,10 +1608,11 @@ TEST(TestArrowReadWrite, GetRecordBatchReader) {
   const int num_rows = 1000;
 
   std::shared_ptr<Table> table;
-  MakeDoubleTable(num_columns, num_rows, 1, &table);
+  ASSERT_NO_FATAL_FAILURE(MakeDoubleTable(num_columns, num_rows, 1, &table));
 
   std::shared_ptr<Buffer> buffer;
-  WriteTableToBuffer(table, 1, num_rows / 2, default_arrow_writer_properties(), &buffer);
+  ASSERT_NO_FATAL_FAILURE(WriteTableToBuffer(table, 1, num_rows / 2,
+                                             default_arrow_writer_properties(), &buffer));
 
   std::unique_ptr<FileReader> reader;
   ASSERT_OK_NO_THROW(OpenFile(std::make_shared<BufferReader>(buffer),
@@ -1636,10 +1641,11 @@ TEST(TestArrowReadWrite, ScanContents) {
   const int num_rows = 1000;
 
   std::shared_ptr<Table> table;
-  MakeDoubleTable(num_columns, num_rows, 1, &table);
+  ASSERT_NO_FATAL_FAILURE(MakeDoubleTable(num_columns, num_rows, 1, &table));
 
   std::shared_ptr<Buffer> buffer;
-  WriteTableToBuffer(table, 1, num_rows / 2, default_arrow_writer_properties(), &buffer);
+  ASSERT_NO_FATAL_FAILURE(WriteTableToBuffer(table, 1, num_rows / 2,
+                                             default_arrow_writer_properties(), &buffer));
 
   std::unique_ptr<FileReader> reader;
   ASSERT_OK_NO_THROW(OpenFile(std::make_shared<BufferReader>(buffer),
@@ -1660,11 +1666,12 @@ TEST(TestArrowReadWrite, ReadColumnSubset) {
   const int num_threads = 4;
 
   std::shared_ptr<Table> table;
-  MakeDoubleTable(num_columns, num_rows, 1, &table);
+  ASSERT_NO_FATAL_FAILURE(MakeDoubleTable(num_columns, num_rows, 1, &table));
 
   std::shared_ptr<Table> result;
   std::vector<int> column_subset = {0, 4, 8, 10};
-  DoSimpleRoundtrip(table, num_threads, table->num_rows(), column_subset, &result);
+  ASSERT_NO_FATAL_FAILURE(
+      DoSimpleRoundtrip(table, num_threads, table->num_rows(), column_subset, &result));
 
   std::vector<std::shared_ptr<::arrow::Column>> ex_columns;
   std::vector<std::shared_ptr<::arrow::Field>> ex_fields;
@@ -1675,7 +1682,7 @@ TEST(TestArrowReadWrite, ReadColumnSubset) {
 
   auto ex_schema = ::arrow::schema(ex_fields);
   auto expected = Table::Make(ex_schema, ex_columns);
-  AssertTablesEqual(*expected, *result);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*expected, *result));
 }
 
 TEST(TestArrowReadWrite, ListLargeRecords) {
@@ -1690,7 +1697,8 @@ TEST(TestArrowReadWrite, ListLargeRecords) {
   std::shared_ptr<Table> table = Table::Make(schema, {list_array});
 
   std::shared_ptr<Buffer> buffer;
-  WriteTableToBuffer(table, 1, 100, default_arrow_writer_properties(), &buffer);
+  ASSERT_NO_FATAL_FAILURE(
+      WriteTableToBuffer(table, 1, 100, default_arrow_writer_properties(), &buffer));
 
   std::unique_ptr<FileReader> reader;
   ASSERT_OK_NO_THROW(OpenFile(std::make_shared<BufferReader>(buffer),
@@ -1700,7 +1708,7 @@ TEST(TestArrowReadWrite, ListLargeRecords) {
   // Read everything
   std::shared_ptr<Table> result;
   ASSERT_OK_NO_THROW(reader->ReadTable(&result));
-  AssertTablesEqual(*table, *result);
+  ASSERT_NO_FATAL_FAILURE(AssertTablesEqual(*table, *result));
 
   ASSERT_OK_NO_THROW(OpenFile(std::make_shared<BufferReader>(buffer),
                               ::arrow::default_memory_pool(),
@@ -1790,9 +1798,9 @@ TEST(TestArrowReadWrite, TableWithChunkedColumns) {
     auto col = std::make_shared<::arrow::Column>(field, arrays);
     auto table = Table::Make(schema, {col});
 
-    CheckSimpleRoundtrip(table, 2);
-    CheckSimpleRoundtrip(table, 3);
-    CheckSimpleRoundtrip(table, 10);
+    ASSERT_NO_FATAL_FAILURE(CheckSimpleRoundtrip(table, 2));
+    ASSERT_NO_FATAL_FAILURE(CheckSimpleRoundtrip(table, 3));
+    ASSERT_NO_FATAL_FAILURE(CheckSimpleRoundtrip(table, 10));
   }
 }
 
@@ -1814,7 +1822,7 @@ TEST(TestArrowReadWrite, TableWithDuplicateColumns) {
 
   auto table = Table::Make(schema, {std::make_shared<Column>(f0->name(), a0),
                                     std::make_shared<Column>(f1->name(), a1)});
-  CheckSimpleRoundtrip(table, table->num_rows());
+  ASSERT_NO_FATAL_FAILURE(CheckSimpleRoundtrip(table, table->num_rows()));
 }
 
 TEST(TestArrowReadWrite, DictionaryColumnChunkedWrite) {
@@ -1885,7 +1893,7 @@ TEST(TestArrowWrite, CheckChunkSize) {
   const int num_rows = 128;
   const int64_t chunk_size = 0;  // note the chunk_size is 0
   std::shared_ptr<Table> table;
-  MakeDoubleTable(num_columns, num_rows, 1, &table);
+  ASSERT_NO_FATAL_FAILURE(MakeDoubleTable(num_columns, num_rows, 1, &table));
 
   auto sink = std::make_shared<InMemoryOutputStream>();
 
@@ -2149,14 +2157,14 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
 };
 
 TEST_F(TestNestedSchemaRead, ReadIntoTableFull) {
-  CreateSimpleNestedParquet(Repetition::OPTIONAL);
+  ASSERT_NO_FATAL_FAILURE(CreateSimpleNestedParquet(Repetition::OPTIONAL));
 
   std::shared_ptr<Table> table;
   ASSERT_OK_NO_THROW(reader_->ReadTable(&table));
   ASSERT_EQ(table->num_rows(), NUM_SIMPLE_TEST_ROWS);
   ASSERT_EQ(table->num_columns(), 2);
   ASSERT_EQ(table->schema()->field(0)->type()->num_children(), 2);
-  ValidateTableArrayTypes(*table);
+  ASSERT_NO_FATAL_FAILURE(ValidateTableArrayTypes(*table));
 
   auto struct_field_array =
       std::static_pointer_cast<::arrow::StructArray>(table->column(0)->data()->chunk(0));
@@ -2170,17 +2178,18 @@ TEST_F(TestNestedSchemaRead, ReadIntoTableFull) {
   // validate struct and leaf arrays
 
   // validate struct array
-  ValidateArray(*struct_field_array, NUM_SIMPLE_TEST_ROWS / 3);
+  ASSERT_NO_FATAL_FAILURE(ValidateArray(*struct_field_array, NUM_SIMPLE_TEST_ROWS / 3));
   // validate leaf1
-  ValidateColumnArray(*leaf1_array, NUM_SIMPLE_TEST_ROWS / 3);
+  ASSERT_NO_FATAL_FAILURE(ValidateColumnArray(*leaf1_array, NUM_SIMPLE_TEST_ROWS / 3));
   // validate leaf2
-  ValidateColumnArray(*leaf2_array, NUM_SIMPLE_TEST_ROWS * 2 / 3);
+  ASSERT_NO_FATAL_FAILURE(
+      ValidateColumnArray(*leaf2_array, NUM_SIMPLE_TEST_ROWS * 2 / 3));
   // validate leaf3
-  ValidateColumnArray(*leaf3_array, 0);
+  ASSERT_NO_FATAL_FAILURE(ValidateColumnArray(*leaf3_array, 0));
 }
 
 TEST_F(TestNestedSchemaRead, ReadTablePartial) {
-  CreateSimpleNestedParquet(Repetition::OPTIONAL);
+  ASSERT_NO_FATAL_FAILURE(CreateSimpleNestedParquet(Repetition::OPTIONAL));
   std::shared_ptr<Table> table;
 
   // columns: {group1.leaf1, leaf3}
@@ -2190,7 +2199,7 @@ TEST_F(TestNestedSchemaRead, ReadTablePartial) {
   ASSERT_EQ(table->schema()->field(0)->name(), "group1");
   ASSERT_EQ(table->schema()->field(1)->name(), "leaf3");
   ASSERT_EQ(table->schema()->field(0)->type()->num_children(), 1);
-  ValidateTableArrayTypes(*table);
+  ASSERT_NO_FATAL_FAILURE(ValidateTableArrayTypes(*table));
 
   // columns: {group1.leaf1, group1.leaf2}
   ASSERT_OK_NO_THROW(reader_->ReadTable({0, 1}, &table));
@@ -2198,7 +2207,7 @@ TEST_F(TestNestedSchemaRead, ReadTablePartial) {
   ASSERT_EQ(table->num_columns(), 1);
   ASSERT_EQ(table->schema()->field(0)->name(), "group1");
   ASSERT_EQ(table->schema()->field(0)->type()->num_children(), 2);
-  ValidateTableArrayTypes(*table);
+  ASSERT_NO_FATAL_FAILURE(ValidateTableArrayTypes(*table));
 
   // columns: {leaf3}
   ASSERT_OK_NO_THROW(reader_->ReadTable({2}, &table));
@@ -2206,7 +2215,7 @@ TEST_F(TestNestedSchemaRead, ReadTablePartial) {
   ASSERT_EQ(table->num_columns(), 1);
   ASSERT_EQ(table->schema()->field(0)->name(), "leaf3");
   ASSERT_EQ(table->schema()->field(0)->type()->num_children(), 0);
-  ValidateTableArrayTypes(*table);
+  ASSERT_NO_FATAL_FAILURE(ValidateTableArrayTypes(*table));
 
   // Test with different ordering
   ASSERT_OK_NO_THROW(reader_->ReadTable({2, 0}, &table));
@@ -2215,11 +2224,11 @@ TEST_F(TestNestedSchemaRead, ReadTablePartial) {
   ASSERT_EQ(table->schema()->field(0)->name(), "leaf3");
   ASSERT_EQ(table->schema()->field(1)->name(), "group1");
   ASSERT_EQ(table->schema()->field(1)->type()->num_children(), 1);
-  ValidateTableArrayTypes(*table);
+  ASSERT_NO_FATAL_FAILURE(ValidateTableArrayTypes(*table));
 }
 
 TEST_F(TestNestedSchemaRead, StructAndListTogetherUnsupported) {
-  CreateSimpleNestedParquet(Repetition::REPEATED);
+  ASSERT_NO_FATAL_FAILURE(CreateSimpleNestedParquet(Repetition::REPEATED));
   std::shared_ptr<Table> table;
   ASSERT_RAISES(NotImplemented, reader_->ReadTable(&table));
 }
@@ -2229,7 +2238,8 @@ TEST_P(TestNestedSchemaRead, DeepNestedSchemaRead) {
   const int depth = 5;
   const int num_children = 3;
   int num_rows = SMALL_SIZE * (depth + 2);
-  CreateMultiLevelNestedParquet(num_trees, depth, num_children, num_rows, GetParam());
+  ASSERT_NO_FATAL_FAILURE(CreateMultiLevelNestedParquet(num_trees, depth, num_children,
+                                                        num_rows, GetParam()));
   std::shared_ptr<Table> table;
   ASSERT_OK_NO_THROW(reader_->ReadTable(&table));
   ASSERT_EQ(table->num_columns(), num_trees);
diff --git a/src/parquet/arrow/arrow-schema-test.cc b/src/parquet/arrow/arrow-schema-test.cc
index da6af52..5c16c04 100644
--- a/src/parquet/arrow/arrow-schema-test.cc
+++ b/src/parquet/arrow/arrow-schema-test.cc
@@ -62,8 +62,8 @@ class TestConvertParquetSchema : public ::testing::Test {
     for (int i = 0; i < expected_schema->num_fields(); ++i) {
       auto lhs = result_schema_->field(i);
       auto rhs = expected_schema->field(i);
-      EXPECT_TRUE(lhs->Equals(rhs))
-          << i << " " << lhs->ToString() << " != " << rhs->ToString();
+      EXPECT_TRUE(lhs->Equals(rhs)) << i << " " << lhs->ToString()
+                                    << " != " << rhs->ToString();
     }
   }
 
@@ -162,7 +162,7 @@ TEST_F(TestConvertParquetSchema, ParquetFlatPrimitives) {
   auto arrow_schema = std::make_shared<::arrow::Schema>(arrow_fields);
   ASSERT_OK(ConvertSchema(parquet_fields));
 
-  CheckFlatSchema(arrow_schema);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema));
 }
 
 TEST_F(TestConvertParquetSchema, DuplicateFieldNames) {
@@ -179,15 +179,18 @@ TEST_F(TestConvertParquetSchema, DuplicateFieldNames) {
 
   ASSERT_OK(ConvertSchema(parquet_fields));
   arrow_fields = {arrow_field1, arrow_field2};
-  CheckFlatSchema(std::make_shared<::arrow::Schema>(arrow_fields));
+  ASSERT_NO_FATAL_FAILURE(
+      CheckFlatSchema(std::make_shared<::arrow::Schema>(arrow_fields)));
 
   ASSERT_OK(ConvertSchema(parquet_fields, std::vector<int>({0, 1})));
   arrow_fields = {arrow_field1, arrow_field2};
-  CheckFlatSchema(std::make_shared<::arrow::Schema>(arrow_fields));
+  ASSERT_NO_FATAL_FAILURE(
+      CheckFlatSchema(std::make_shared<::arrow::Schema>(arrow_fields)));
 
   ASSERT_OK(ConvertSchema(parquet_fields, std::vector<int>({1, 0})));
   arrow_fields = {arrow_field2, arrow_field1};
-  CheckFlatSchema(std::make_shared<::arrow::Schema>(arrow_fields));
+  ASSERT_NO_FATAL_FAILURE(
+      CheckFlatSchema(std::make_shared<::arrow::Schema>(arrow_fields)));
 }
 
 TEST_F(TestConvertParquetSchema, ParquetKeyValueMetadata) {
@@ -256,7 +259,7 @@ TEST_F(TestConvertParquetSchema, ParquetFlatDecimals) {
   auto arrow_schema = std::make_shared<::arrow::Schema>(arrow_fields);
   ASSERT_OK(ConvertSchema(parquet_fields));
 
-  CheckFlatSchema(arrow_schema);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema));
 }
 
 TEST_F(TestConvertParquetSchema, ParquetLists) {
@@ -438,7 +441,7 @@ TEST_F(TestConvertParquetSchema, ParquetLists) {
   auto arrow_schema = std::make_shared<::arrow::Schema>(arrow_fields);
   ASSERT_OK(ConvertSchema(parquet_fields));
 
-  CheckFlatSchema(arrow_schema);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema));
 }
 
 TEST_F(TestConvertParquetSchema, UnsupportedThings) {
@@ -476,7 +479,7 @@ TEST_F(TestConvertParquetSchema, ParquetNestedSchema) {
   auto arrow_schema = std::make_shared<::arrow::Schema>(arrow_fields);
   ASSERT_OK(ConvertSchema(parquet_fields));
 
-  CheckFlatSchema(arrow_schema);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema));
 }
 
 TEST_F(TestConvertParquetSchema, ParquetNestedSchemaPartial) {
@@ -527,7 +530,7 @@ TEST_F(TestConvertParquetSchema, ParquetNestedSchemaPartial) {
   auto arrow_schema = std::make_shared<::arrow::Schema>(arrow_fields);
   ASSERT_OK(ConvertSchema(parquet_fields, std::vector<int>{0, 3, 4}));
 
-  CheckFlatSchema(arrow_schema);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema));
 }
 
 TEST_F(TestConvertParquetSchema, ParquetNestedSchemaPartialOrdering) {
@@ -578,7 +581,7 @@ TEST_F(TestConvertParquetSchema, ParquetNestedSchemaPartialOrdering) {
   auto arrow_schema = std::make_shared<::arrow::Schema>(arrow_fields);
   ASSERT_OK(ConvertSchema(parquet_fields, std::vector<int>{3, 4, 0}));
 
-  CheckFlatSchema(arrow_schema);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema));
 }
 TEST_F(TestConvertParquetSchema, ParquetRepeatedNestedSchema) {
   std::vector<NodePtr> parquet_fields;
@@ -604,10 +607,9 @@ TEST_F(TestConvertParquetSchema, ParquetRepeatedNestedSchema) {
     auto inner_group_type = std::make_shared<::arrow::StructType>(inner_group_fields);
     auto outer_group_fields = {
         std::make_shared<Field>("leaf2", INT32, true),
-        std::make_shared<Field>(
-            "innerGroup",
-            ::arrow::list(std::make_shared<Field>("innerGroup", inner_group_type, false)),
-            false)};
+        std::make_shared<Field>("innerGroup", ::arrow::list(std::make_shared<Field>(
+                                                  "innerGroup", inner_group_type, false)),
+                                false)};
     auto outer_group_type = std::make_shared<::arrow::StructType>(outer_group_fields);
 
     arrow_fields.push_back(std::make_shared<Field>("leaf1", INT32, true));
@@ -619,7 +621,7 @@ TEST_F(TestConvertParquetSchema, ParquetRepeatedNestedSchema) {
   auto arrow_schema = std::make_shared<::arrow::Schema>(arrow_fields);
   ASSERT_OK(ConvertSchema(parquet_fields));
 
-  CheckFlatSchema(arrow_schema);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema));
 }
 
 class TestConvertArrowSchema : public ::testing::Test {
@@ -705,7 +707,7 @@ TEST_F(TestConvertArrowSchema, ParquetFlatPrimitives) {
 
   ASSERT_OK(ConvertSchema(arrow_fields));
 
-  CheckFlatSchema(parquet_fields);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
 TEST_F(TestConvertArrowSchema, ParquetFlatPrimitivesAsDictionaries) {
@@ -765,7 +767,7 @@ TEST_F(TestConvertArrowSchema, ParquetFlatPrimitivesAsDictionaries) {
 
   ASSERT_OK(ConvertSchema(arrow_fields));
 
-  CheckFlatSchema(parquet_fields);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
 TEST_F(TestConvertArrowSchema, ParquetLists) {
@@ -810,7 +812,7 @@ TEST_F(TestConvertArrowSchema, ParquetLists) {
 
   ASSERT_OK(ConvertSchema(arrow_fields));
 
-  CheckFlatSchema(parquet_fields);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
 TEST_F(TestConvertArrowSchema, UnsupportedTypes) {
@@ -830,7 +832,7 @@ TEST_F(TestConvertArrowSchema, ParquetFlatDecimals) {
 
   ASSERT_OK(ConvertSchema(arrow_fields));
 
-  CheckFlatSchema(parquet_fields);
+  ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(parquet_fields));
 }
 
 TEST(InvalidSchema, ParquetNegativeDecimalScale) {
diff --git a/src/parquet/column_reader-test.cc b/src/parquet/column_reader-test.cc
index ebb70f1..15ddc8b 100644
--- a/src/parquet/column_reader-test.cc
+++ b/src/parquet/column_reader-test.cc
@@ -234,8 +234,8 @@ TEST_F(TestPrimitiveReader, TestInt32FlatRequired) {
   max_rep_level_ = 0;
   NodePtr type = schema::Int32("a", Repetition::REQUIRED);
   const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  ExecutePlain(num_pages, levels_per_page, &descr);
-  ExecuteDict(num_pages, levels_per_page, &descr);
+  ASSERT_NO_FATAL_FAILURE(ExecutePlain(num_pages, levels_per_page, &descr));
+  ASSERT_NO_FATAL_FAILURE(ExecuteDict(num_pages, levels_per_page, &descr));
 }
 
 TEST_F(TestPrimitiveReader, TestInt32FlatOptional) {
@@ -245,8 +245,8 @@ TEST_F(TestPrimitiveReader, TestInt32FlatOptional) {
   max_rep_level_ = 0;
   NodePtr type = schema::Int32("b", Repetition::OPTIONAL);
   const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  ExecutePlain(num_pages, levels_per_page, &descr);
-  ExecuteDict(num_pages, levels_per_page, &descr);
+  ASSERT_NO_FATAL_FAILURE(ExecutePlain(num_pages, levels_per_page, &descr));
+  ASSERT_NO_FATAL_FAILURE(ExecuteDict(num_pages, levels_per_page, &descr));
 }
 
 TEST_F(TestPrimitiveReader, TestInt32FlatRepeated) {
@@ -256,8 +256,8 @@ TEST_F(TestPrimitiveReader, TestInt32FlatRepeated) {
   max_rep_level_ = 2;
   NodePtr type = schema::Int32("c", Repetition::REPEATED);
   const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  ExecutePlain(num_pages, levels_per_page, &descr);
-  ExecuteDict(num_pages, levels_per_page, &descr);
+  ASSERT_NO_FATAL_FAILURE(ExecutePlain(num_pages, levels_per_page, &descr));
+  ASSERT_NO_FATAL_FAILURE(ExecuteDict(num_pages, levels_per_page, &descr));
 }
 
 TEST_F(TestPrimitiveReader, TestInt32FlatRequiredSkip) {
diff --git a/src/parquet/column_scanner-test.cc b/src/parquet/column_scanner-test.cc
index 1ebc719..d39b910 100644
--- a/src/parquet/column_scanner-test.cc
+++ b/src/parquet/column_scanner-test.cc
@@ -154,30 +154,33 @@ using TestFLBAFlatScanner = TestFlatScanner<FLBAType>;
 TYPED_TEST_CASE(TestFlatScanner, TestTypes);
 
 TYPED_TEST(TestFlatScanner, TestPlainScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0, Encoding::PLAIN);
+  ASSERT_NO_FATAL_FAILURE(
+      this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0, Encoding::PLAIN));
 }
 
 TYPED_TEST(TestFlatScanner, TestDictScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0,
-                   Encoding::RLE_DICTIONARY);
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0,
+                                           Encoding::RLE_DICTIONARY));
 }
 
 TEST_F(TestBooleanFlatScanner, TestPlainScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0);
+  ASSERT_NO_FATAL_FAILURE(
+      this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0));
 }
 
 TEST_F(TestFLBAFlatScanner, TestPlainScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH);
+  ASSERT_NO_FATAL_FAILURE(
+      this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH));
 }
 
 TEST_F(TestFLBAFlatScanner, TestDictScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH,
-                   Encoding::RLE_DICTIONARY);
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteAll(num_pages, num_levels_per_page, batch_size,
+                                           FLBA_LENGTH, Encoding::RLE_DICTIONARY));
 }
 
 TEST_F(TestFLBAFlatScanner, TestPlainDictScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH,
-                   Encoding::PLAIN_DICTIONARY);
+  ASSERT_NO_FATAL_FAILURE(this->ExecuteAll(num_pages, num_levels_per_page, batch_size,
+                                           FLBA_LENGTH, Encoding::PLAIN_DICTIONARY));
 }
 
 // PARQUET 502
@@ -190,7 +193,7 @@ TEST_F(TestFLBAFlatScanner, TestSmallBatch) {
                                     data_buffer_, pages_);
   num_levels_ = 1 * 100;
   InitScanner(&d);
-  CheckResults(1, &d);
+  ASSERT_NO_FATAL_FAILURE(CheckResults(1, &d));
 }
 
 TEST_F(TestFLBAFlatScanner, TestDescriptorAPI) {
diff --git a/src/parquet/column_writer-test.cc b/src/parquet/column_writer-test.cc
index 224a23d..7a5f379 100644
--- a/src/parquet/column_writer-test.cc
+++ b/src/parquet/column_writer-test.cc
@@ -110,11 +110,11 @@ class TestPrimitiveWriter : public PrimitiveTypedTest<TestType> {
 
     this->WriteRequiredWithSettings(encoding, compression, enable_dictionary,
                                     enable_statistics, num_rows);
-    this->ReadAndCompare(compression, num_rows);
+    ASSERT_NO_FATAL_FAILURE(this->ReadAndCompare(compression, num_rows));
 
     this->WriteRequiredWithSettingsSpaced(encoding, compression, enable_dictionary,
                                           enable_statistics, num_rows);
-    this->ReadAndCompare(compression, num_rows);
+    ASSERT_NO_FATAL_FAILURE(this->ReadAndCompare(compression, num_rows));
   }
 
   void WriteRequiredWithSettings(Encoding::type encoding, Compression::type compression,
@@ -644,9 +644,11 @@ TEST(TestLevels, TestLevelsDecodeMultipleBitWidth) {
       int16_t max_level = static_cast<int16_t>((1 << bit_width) - 1);
       // Generate levels
       GenerateLevels(min_repeat_factor, max_repeat_factor, max_level, input_levels);
-      EncodeLevels(encoding, max_level, static_cast<int>(input_levels.size()),
-                   input_levels.data(), bytes);
-      VerifyDecodingLevels(encoding, max_level, input_levels, bytes);
+      ASSERT_NO_FATAL_FAILURE(EncodeLevels(encoding, max_level,
+                                           static_cast<int>(input_levels.size()),
+                                           input_levels.data(), bytes));
+      ASSERT_NO_FATAL_FAILURE(
+          VerifyDecodingLevels(encoding, max_level, input_levels, bytes));
       input_levels.clear();
     }
   }
@@ -672,10 +674,12 @@ TEST(TestLevels, TestLevelsDecodeMultipleSetData) {
     Encoding::type encoding = encodings[encode];
     for (int rf = 0; rf < setdata_factor; rf++) {
       int offset = rf * split_level_size;
-      EncodeLevels(encoding, max_level, split_level_size,
-                   reinterpret_cast<int16_t*>(input_levels.data()) + offset, bytes[rf]);
+      ASSERT_NO_FATAL_FAILURE(EncodeLevels(
+          encoding, max_level, split_level_size,
+          reinterpret_cast<int16_t*>(input_levels.data()) + offset, bytes[rf]));
     }
-    VerifyDecodingMultipleSetData(encoding, max_level, input_levels, bytes);
+    ASSERT_NO_FATAL_FAILURE(
+        VerifyDecodingMultipleSetData(encoding, max_level, input_levels, bytes));
   }
 }
 
diff --git a/src/parquet/encoding-test.cc b/src/parquet/encoding-test.cc
index 0da32cf..8d97bff 100644
--- a/src/parquet/encoding-test.cc
+++ b/src/parquet/encoding-test.cc
@@ -225,7 +225,7 @@ class TestPlainEncoding : public TestEncodingBase<Type> {
                     static_cast<int>(encode_buffer_->size()));
     int values_decoded = decoder.Decode(decode_buf_, num_values_);
     ASSERT_EQ(num_values_, values_decoded);
-    VerifyResults<T>(decode_buf_, draws_, num_values_);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<T>(decode_buf_, draws_, num_values_));
   }
 
  protected:
@@ -234,7 +234,9 @@ class TestPlainEncoding : public TestEncodingBase<Type> {
 
 TYPED_TEST_CASE(TestPlainEncoding, ParquetTypes);
 
-TYPED_TEST(TestPlainEncoding, BasicRoundTrip) { this->Execute(10000, 1); }
+TYPED_TEST(TestPlainEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(10000, 1));
+}
 
 // ----------------------------------------------------------------------
 // Dictionary encoding tests
@@ -278,14 +280,14 @@ class TestDictionaryEncoding : public TestEncodingBase<Type> {
     // TODO(wesm): The DictionaryDecoder must stay alive because the decoded
     // values' data is owned by a buffer inside the DictionaryEncoder. We
     // should revisit when data lifetime is reviewed more generally.
-    VerifyResults<T>(decode_buf_, draws_, num_values_);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<T>(decode_buf_, draws_, num_values_));
 
     // Also test spaced decoding
     decoder.SetData(num_values_, indices->data(), static_cast<int>(indices->size()));
     values_decoded =
         decoder.DecodeSpaced(decode_buf_, num_values_, 0, valid_bits.data(), 0);
     ASSERT_EQ(num_values_, values_decoded);
-    VerifyResults<T>(decode_buf_, draws_, num_values_);
+    ASSERT_NO_FATAL_FAILURE(VerifyResults<T>(decode_buf_, draws_, num_values_));
   }
 
  protected:
@@ -295,7 +297,9 @@ class TestDictionaryEncoding : public TestEncodingBase<Type> {
 
 TYPED_TEST_CASE(TestDictionaryEncoding, DictEncodedTypes);
 
-TYPED_TEST(TestDictionaryEncoding, BasicRoundTrip) { this->Execute(2500, 2); }
+TYPED_TEST(TestDictionaryEncoding, BasicRoundTrip) {
+  ASSERT_NO_FATAL_FAILURE(this->Execute(2500, 2));
+}
 
 TEST(TestDictionaryEncoding, CannotDictDecodeBoolean) {
   PlainDecoder<BooleanType> dict_decoder(nullptr);
diff --git a/src/parquet/file-deserialize-test.cc b/src/parquet/file-deserialize-test.cc
index 5e17375..6b01ac2 100644
--- a/src/parquet/file-deserialize-test.cc
+++ b/src/parquet/file-deserialize-test.cc
@@ -129,10 +129,10 @@ TEST_F(TestPageSerde, DataPage) {
   AddDummyStats(stats_size, data_page_header_);
   data_page_header_.num_values = num_rows;
 
-  WriteDataPageHeader();
+  ASSERT_NO_FATAL_FAILURE(WriteDataPageHeader());
   InitSerializedPageReader(num_rows);
   std::shared_ptr<Page> current_page = page_reader_->NextPage();
-  CheckDataPageHeader(data_page_header_, current_page.get());
+  ASSERT_NO_FATAL_FAILURE(CheckDataPageHeader(data_page_header_, current_page.get()));
 }
 
 TEST_F(TestPageSerde, TestLargePageHeaders) {
@@ -144,7 +144,7 @@ TEST_F(TestPageSerde, TestLargePageHeaders) {
   data_page_header_.num_values = num_rows;
 
   int max_header_size = 512 * 1024;  // 512 KB
-  WriteDataPageHeader(max_header_size);
+  ASSERT_NO_FATAL_FAILURE(WriteDataPageHeader(max_header_size));
   ASSERT_GE(max_header_size, out_stream_->Tell());
 
   // check header size is between 256 KB to 16 MB
@@ -153,7 +153,7 @@ TEST_F(TestPageSerde, TestLargePageHeaders) {
 
   InitSerializedPageReader(num_rows);
   std::shared_ptr<Page> current_page = page_reader_->NextPage();
-  CheckDataPageHeader(data_page_header_, current_page.get());
+  ASSERT_NO_FATAL_FAILURE(CheckDataPageHeader(data_page_header_, current_page.get()));
 }
 
 TEST_F(TestPageSerde, TestFailLargePageHeaders) {
@@ -164,7 +164,7 @@ TEST_F(TestPageSerde, TestFailLargePageHeaders) {
 
   // Serialize the Page header
   int max_header_size = 512 * 1024;  // 512 KB
-  WriteDataPageHeader(max_header_size);
+  ASSERT_NO_FATAL_FAILURE(WriteDataPageHeader(max_header_size));
   ASSERT_GE(max_header_size, out_stream_->Tell());
 
   int smaller_max_size = 128 * 1024;
@@ -209,7 +209,8 @@ TEST_F(TestPageSerde, Compression) {
       ASSERT_OK(codec->Compress(data_size, data, max_compressed_size, &buffer[0],
                                 &actual_size));
 
-      WriteDataPageHeader(1024, data_size, static_cast<int32_t>(actual_size));
+      ASSERT_NO_FATAL_FAILURE(
+          WriteDataPageHeader(1024, data_size, static_cast<int32_t>(actual_size)));
       out_stream_->Write(buffer.data(), actual_size);
     }
 
@@ -233,7 +234,7 @@ TEST_F(TestPageSerde, LZONotSupported) {
   // Must await PARQUET-530
   int data_size = 1024;
   std::vector<uint8_t> faux_data(data_size);
-  WriteDataPageHeader(1024, data_size, data_size);
+  ASSERT_NO_FATAL_FAILURE(WriteDataPageHeader(1024, data_size, data_size));
   out_stream_->Write(faux_data.data(), data_size);
   ASSERT_THROW(InitSerializedPageReader(data_size, Compression::LZO), ParquetException);
 }
@@ -262,7 +263,7 @@ TEST_F(TestParquetFileReader, InvalidHeader) {
 
   auto buffer = std::make_shared<Buffer>(reinterpret_cast<const uint8_t*>(bad_header),
                                          strlen(bad_header));
-  AssertInvalidFileThrows(buffer);
+  ASSERT_NO_FATAL_FAILURE(AssertInvalidFileThrows(buffer));
 }
 
 TEST_F(TestParquetFileReader, InvalidFooter) {
@@ -270,13 +271,13 @@ TEST_F(TestParquetFileReader, InvalidFooter) {
   const char* bad_file = "PAR1PAR";
   auto buffer = std::make_shared<Buffer>(reinterpret_cast<const uint8_t*>(bad_file),
                                          strlen(bad_file));
-  AssertInvalidFileThrows(buffer);
+  ASSERT_NO_FATAL_FAILURE(AssertInvalidFileThrows(buffer));
 
   // Magic number incorrect
   const char* bad_file2 = "PAR1PAR2";
   buffer = std::make_shared<Buffer>(reinterpret_cast<const uint8_t*>(bad_file2),
                                     strlen(bad_file2));
-  AssertInvalidFileThrows(buffer);
+  ASSERT_NO_FATAL_FAILURE(AssertInvalidFileThrows(buffer));
 }
 
 TEST_F(TestParquetFileReader, IncompleteMetadata) {
@@ -292,7 +293,7 @@ TEST_F(TestParquetFileReader, IncompleteMetadata) {
   stream.Write(reinterpret_cast<const uint8_t*>(magic), strlen(magic));
 
   auto buffer = stream.GetBuffer();
-  AssertInvalidFileThrows(buffer);
+  ASSERT_NO_FATAL_FAILURE(AssertInvalidFileThrows(buffer));
 }
 
 }  // namespace parquet
diff --git a/src/parquet/file-serialize-test.cc b/src/parquet/file-serialize-test.cc
index b4df77e..16a7c4f 100644
--- a/src/parquet/file-serialize-test.cc
+++ b/src/parquet/file-serialize-test.cc
@@ -184,7 +184,7 @@ typedef ::testing::Types<Int32Type, Int64Type, Int96Type, FloatType, DoubleType,
 TYPED_TEST_CASE(TestSerialize, TestTypes);
 
 TYPED_TEST(TestSerialize, SmallFileUncompressed) {
-  this->FileSerializeTest(Compression::UNCOMPRESSED);
+  ASSERT_NO_FATAL_FAILURE(this->FileSerializeTest(Compression::UNCOMPRESSED));
 }
 
 TYPED_TEST(TestSerialize, TooFewRows) {
@@ -202,18 +202,24 @@ TYPED_TEST(TestSerialize, RepeatedTooFewRows) {
 }
 
 TYPED_TEST(TestSerialize, SmallFileSnappy) {
-  this->FileSerializeTest(Compression::SNAPPY);
+  ASSERT_NO_FATAL_FAILURE(this->FileSerializeTest(Compression::SNAPPY));
 }
 
 TYPED_TEST(TestSerialize, SmallFileBrotli) {
-  this->FileSerializeTest(Compression::BROTLI);
+  ASSERT_NO_FATAL_FAILURE(this->FileSerializeTest(Compression::BROTLI));
 }
 
-TYPED_TEST(TestSerialize, SmallFileGzip) { this->FileSerializeTest(Compression::GZIP); }
+TYPED_TEST(TestSerialize, SmallFileGzip) {
+  ASSERT_NO_FATAL_FAILURE(this->FileSerializeTest(Compression::GZIP));
+}
 
-TYPED_TEST(TestSerialize, SmallFileLz4) { this->FileSerializeTest(Compression::LZ4); }
+TYPED_TEST(TestSerialize, SmallFileLz4) {
+  ASSERT_NO_FATAL_FAILURE(this->FileSerializeTest(Compression::LZ4));
+}
 
-TYPED_TEST(TestSerialize, SmallFileZstd) { this->FileSerializeTest(Compression::ZSTD); }
+TYPED_TEST(TestSerialize, SmallFileZstd) {
+  ASSERT_NO_FATAL_FAILURE(this->FileSerializeTest(Compression::ZSTD));
+}
 
 }  // namespace test
 
diff --git a/src/parquet/schema-test.cc b/src/parquet/schema-test.cc
index ec9aff4..a734a92 100644
--- a/src/parquet/schema-test.cc
+++ b/src/parquet/schema-test.cc
@@ -140,7 +140,7 @@ TEST_F(TestPrimitiveNode, Attrs) {
 TEST_F(TestPrimitiveNode, FromParquet) {
   SchemaElement elt =
       NewPrimitive(name_, FieldRepetitionType::OPTIONAL, format::Type::INT32, 0);
-  Convert(&elt);
+  ASSERT_NO_FATAL_FAILURE(Convert(&elt));
   ASSERT_EQ(name_, prim_node_->name());
   ASSERT_EQ(id_, prim_node_->id());
   ASSERT_EQ(Repetition::OPTIONAL, prim_node_->repetition());
@@ -151,7 +151,7 @@ TEST_F(TestPrimitiveNode, FromParquet) {
   elt = NewPrimitive(name_, FieldRepetitionType::REQUIRED, format::Type::BYTE_ARRAY, 0);
   elt.__set_converted_type(ConvertedType::UTF8);
 
-  Convert(&elt);
+  ASSERT_NO_FATAL_FAILURE(Convert(&elt));
   ASSERT_EQ(Repetition::REQUIRED, prim_node_->repetition());
   ASSERT_EQ(Type::BYTE_ARRAY, prim_node_->physical_type());
   ASSERT_EQ(LogicalType::UTF8, prim_node_->logical_type());
@@ -161,7 +161,7 @@ TEST_F(TestPrimitiveNode, FromParquet) {
                      format::Type::FIXED_LEN_BYTE_ARRAY, 0);
   elt.__set_type_length(16);
 
-  Convert(&elt);
+  ASSERT_NO_FATAL_FAILURE(Convert(&elt));
   ASSERT_EQ(name_, prim_node_->name());
   ASSERT_EQ(id_, prim_node_->id());
   ASSERT_EQ(Repetition::OPTIONAL, prim_node_->repetition());
@@ -176,7 +176,7 @@ TEST_F(TestPrimitiveNode, FromParquet) {
   elt.__set_scale(2);
   elt.__set_precision(12);
 
-  Convert(&elt);
+  ASSERT_NO_FATAL_FAILURE(Convert(&elt));
   ASSERT_EQ(Type::FIXED_LEN_BYTE_ARRAY, prim_node_->physical_type());
   ASSERT_EQ(LogicalType::DECIMAL, prim_node_->logical_type());
   ASSERT_EQ(6, prim_node_->type_length());
@@ -432,7 +432,7 @@ TEST_F(TestSchemaConverter, NestedExample) {
   elements.push_back(
       NewPrimitive("item", FieldRepetitionType::OPTIONAL, format::Type::INT64, 4));
 
-  Convert(&elements[0], static_cast<int>(elements.size()));
+  ASSERT_NO_FATAL_FAILURE(Convert(&elements[0], static_cast<int>(elements.size())));
 
   // Construct the expected schema
   NodeVector fields;
@@ -470,10 +470,10 @@ TEST_F(TestSchemaConverter, InvalidRoot) {
   // practicality matter.
   elements[0] = NewGroup("not-repeated", FieldRepetitionType::REQUIRED, 1, 0);
   elements[1] = NewPrimitive("a", FieldRepetitionType::REQUIRED, format::Type::INT32, 1);
-  Convert(elements, 2);
+  ASSERT_NO_FATAL_FAILURE(Convert(elements, 2));
 
   elements[0] = NewGroup("not-repeated", FieldRepetitionType::OPTIONAL, 1, 0);
-  Convert(elements, 2);
+  ASSERT_NO_FATAL_FAILURE(Convert(elements, 2));
 }
 
 TEST_F(TestSchemaConverter, NotEnoughChildren) {
diff --git a/src/parquet/statistics-test.cc b/src/parquet/statistics-test.cc
index 5474016..bf3d196 100644
--- a/src/parquet/statistics-test.cc
+++ b/src/parquet/statistics-test.cc
@@ -194,9 +194,8 @@ bool* TestRowGroupStatistics<BooleanType>::GetValuesPointer(std::vector<bool>& v
 }
 
 template <typename TestType>
-typename std::vector<typename TestType::c_type>
-TestRowGroupStatistics<TestType>::GetDeepCopy(
-    const std::vector<typename TestType::c_type>& values) {
+typename std::vector<typename TestType::c_type> TestRowGroupStatistics<
+    TestType>::GetDeepCopy(const std::vector<typename TestType::c_type>& values) {
   return values;
 }
 
@@ -284,19 +283,19 @@ TYPED_TEST_CASE(TestRowGroupStatistics, TestTypes);
 
 TYPED_TEST(TestRowGroupStatistics, MinMaxEncode) {
   this->SetUpSchema(Repetition::REQUIRED);
-  this->TestMinMaxEncode();
+  ASSERT_NO_FATAL_FAILURE(this->TestMinMaxEncode());
 }
 
 TYPED_TEST(TestRowGroupStatistics, Reset) {
   this->SetUpSchema(Repetition::OPTIONAL);
-  this->TestReset();
+  ASSERT_NO_FATAL_FAILURE(this->TestReset());
 }
 
 TYPED_TEST(TestRowGroupStatistics, FullRoundtrip) {
   this->SetUpSchema(Repetition::OPTIONAL);
-  this->TestFullRoundtrip(100, 31);
-  this->TestFullRoundtrip(1000, 415);
-  this->TestFullRoundtrip(10000, 926);
+  ASSERT_NO_FATAL_FAILURE(this->TestFullRoundtrip(100, 31));
+  ASSERT_NO_FATAL_FAILURE(this->TestFullRoundtrip(1000, 415));
+  ASSERT_NO_FATAL_FAILURE(this->TestFullRoundtrip(10000, 926));
 }
 
 template <typename TestType>
@@ -308,7 +307,7 @@ TYPED_TEST_CASE(TestNumericRowGroupStatistics, NumericTypes);
 
 TYPED_TEST(TestNumericRowGroupStatistics, Merge) {
   this->SetUpSchema(Repetition::OPTIONAL);
-  this->TestMerge();
+  ASSERT_NO_FATAL_FAILURE(this->TestMerge());
 }
 
 // Statistics are restricted for few types in older parquet version
@@ -632,7 +631,7 @@ TYPED_TEST(TestStatistics, MinMax) {
   this->AddNodes("Column ");
   this->SetUpSchema();
   this->WriteParquet();
-  this->VerifyParquetStats();
+  ASSERT_NO_FATAL_FAILURE(this->VerifyParquetStats());
 }
 
 // Ensure UNKNOWN sort order is handled properly

-- 
To stop receiving notification emails like this one, please contact
uwe@apache.org.