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

etudenhoefner pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/main by this push:
     new 134345dd20 Parquet: Remove TestHelpers in parquet module (#10428)
134345dd20 is described below

commit 134345dd20d99b4800aaefdab1397fd22819d1ff
Author: advancedxy <[email protected]>
AuthorDate: Mon Jun 3 22:49:40 2024 +0800

    Parquet: Remove TestHelpers in parquet module (#10428)
---
 .../test/java/org/apache/iceberg/TestHelpers.java  | 85 ----------------------
 .../iceberg/parquet/TestBloomRowGroupFilter.java   | 36 ++++-----
 .../iceberg/parquet/TestParquetEncryption.java     | 37 +++++-----
 3 files changed, 37 insertions(+), 121 deletions(-)

diff --git a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java 
b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java
deleted file mode 100644
index 0e7627cab1..0000000000
--- a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iceberg;
-
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-import java.util.concurrent.Callable;
-import org.apache.avro.AvroRuntimeException;
-import org.apache.avro.generic.GenericRecord;
-import org.assertj.core.api.AbstractThrowableAssert;
-
-public class TestHelpers {
-
-  private TestHelpers() {}
-
-  /**
-   * A convenience method to avoid a large number of @Test(expected=...) tests
-   *
-   * @param message A String message to describe this assertion
-   * @param expected An Exception class that the Runnable should throw
-   * @param containedInMessage A String that should be contained by the thrown 
exception's message
-   * @param callable A Callable that is expected to throw the exception
-   */
-  public static void assertThrows(
-      String message,
-      Class<? extends Exception> expected,
-      String containedInMessage,
-      Callable callable) {
-    AbstractThrowableAssert<?, ? extends Throwable> check =
-        assertThatThrownBy(callable::call).as(message).isInstanceOf(expected);
-    if (null != containedInMessage) {
-      check.hasMessageContaining(containedInMessage);
-    }
-  }
-
-  /**
-   * A convenience method to avoid a large number of @Test(expected=...) tests
-   *
-   * @param message A String message to describe this assertion
-   * @param expected An Exception class that the Runnable should throw
-   * @param containedInMessage A String that should be contained by the thrown 
exception's message
-   * @param runnable A Runnable that is expected to throw the runtime exception
-   */
-  public static void assertThrows(
-      String message,
-      Class<? extends Exception> expected,
-      String containedInMessage,
-      Runnable runnable) {
-    AbstractThrowableAssert<?, ? extends Throwable> check =
-        assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected);
-    if (null != containedInMessage) {
-      check.hasMessageContaining(containedInMessage);
-    }
-  }
-
-  /**
-   * A convenience method to assert if an Avro field is empty
-   *
-   * @param record The record to read from
-   * @param field The name of the field
-   */
-  public static void assertEmptyAvroField(GenericRecord record, String field) {
-    TestHelpers.assertThrows(
-        "Not a valid schema field: " + field,
-        AvroRuntimeException.class,
-        "Not a valid schema field: " + field,
-        () -> record.get(field));
-  }
-}
diff --git 
a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java 
b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
index 56736ae42f..5f64c72306 100644
--- 
a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
+++ 
b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
@@ -40,6 +40,7 @@ import static org.apache.iceberg.expressions.Expressions.year;
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.apache.iceberg.types.Types.NestedField.required;
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import java.io.File;
 import java.io.IOException;
@@ -54,7 +55,6 @@ import org.apache.avro.generic.GenericData.Record;
 import org.apache.avro.generic.GenericRecordBuilder;
 import org.apache.iceberg.Files;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.TestHelpers;
 import org.apache.iceberg.avro.AvroSchemaUtil;
 import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.expressions.Expression;
@@ -413,13 +413,13 @@ public class TestBloomRowGroupFilter {
 
   @Test
   public void testMissingColumn() {
-    TestHelpers.assertThrows(
-        "Should complain about missing column in expression",
-        ValidationException.class,
-        "Cannot find field 'missing'",
-        () ->
-            new ParquetBloomRowGroupFilter(SCHEMA, lessThan("missing", 5))
-                .shouldRead(parquetSchema, rowGroupMetadata, bloomStore));
+    assertThatThrownBy(
+            () ->
+                new ParquetBloomRowGroupFilter(SCHEMA, equal("missing", 5))
+                    .shouldRead(parquetSchema, rowGroupMetadata, bloomStore))
+        .as("Should complain about missing column in expression")
+        .isInstanceOf(ValidationException.class)
+        .hasMessageContaining("Cannot find field 'missing'");
   }
 
   @Test
@@ -973,16 +973,16 @@ public class TestBloomRowGroupFilter {
 
   @Test
   public void testMissingBloomFilterForColumn() {
-    TestHelpers.assertThrows(
-        "Should complain about missing bloom filter",
-        IllegalStateException.class,
-        "Failed to read required bloom filter for id: 10",
-        () ->
-            new ParquetBloomRowGroupFilter(SCHEMA, equal("some_nulls", "some"))
-                .shouldRead(
-                    parquetSchema,
-                    rowGroupMetadata,
-                    new DummyBloomFilterReader(null, rowGroupMetadata)));
+    assertThatThrownBy(
+            () ->
+                new ParquetBloomRowGroupFilter(SCHEMA, equal("some_nulls", 
"some"))
+                    .shouldRead(
+                        parquetSchema,
+                        rowGroupMetadata,
+                        new DummyBloomFilterReader(null, rowGroupMetadata)))
+        .as("Should complain about missing bloom filter")
+        .isInstanceOf(IllegalStateException.class)
+        .hasMessageContaining("Failed to read required bloom filter for id: 
10");
   }
 
   private static class DummyBloomFilterReader extends BloomFilterReader {
diff --git 
a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java 
b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java
index c68a79c689..ea47ecb1c2 100644
--- 
a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java
+++ 
b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java
@@ -23,6 +23,7 @@ import static org.apache.iceberg.Files.localOutput;
 import static 
org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile;
 import static org.apache.iceberg.types.Types.NestedField.optional;
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import java.io.Closeable;
 import java.io.File;
@@ -33,7 +34,6 @@ import java.security.SecureRandom;
 import java.util.List;
 import org.apache.avro.generic.GenericData;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.TestHelpers;
 import org.apache.iceberg.avro.AvroSchemaUtil;
 import org.apache.iceberg.io.CloseableIterator;
 import org.apache.iceberg.io.FileAppender;
@@ -85,27 +85,28 @@ public class TestParquetEncryption {
 
   @Test
   public void testReadEncryptedFileWithoutKeys() throws IOException {
-    TestHelpers.assertThrows(
-        "Decrypted without keys",
-        ParquetCryptoRuntimeException.class,
-        "Trying to read file with encrypted footer. No keys available",
-        () -> 
Parquet.read(localInput(file)).project(schema).callInit().build().iterator());
+    assertThatThrownBy(
+            () -> 
Parquet.read(localInput(file)).project(schema).callInit().build().iterator())
+        .as("Decrypted without keys")
+        .isInstanceOf(ParquetCryptoRuntimeException.class)
+        .hasMessage("Trying to read file with encrypted footer. No keys 
available");
   }
 
   @Test
   public void testReadEncryptedFileWithoutAADPrefix() throws IOException {
-    TestHelpers.assertThrows(
-        "Decrypted without AAD prefix",
-        ParquetCryptoRuntimeException.class,
-        "AAD prefix used for file encryption, "
-            + "but not stored in file and not supplied in decryption 
properties",
-        () ->
-            Parquet.read(localInput(file))
-                .project(schema)
-                .withFileEncryptionKey(fileDek)
-                .callInit()
-                .build()
-                .iterator());
+    assertThatThrownBy(
+            () ->
+                Parquet.read(localInput(file))
+                    .project(schema)
+                    .withFileEncryptionKey(fileDek)
+                    .callInit()
+                    .build()
+                    .iterator())
+        .as("Decrypted without AAD prefix")
+        .isInstanceOf(ParquetCryptoRuntimeException.class)
+        .hasMessage(
+            "AAD prefix used for file encryption, "
+                + "but not stored in file and not supplied in decryption 
properties");
   }
 
   @Test

Reply via email to