Address IntelliJ inspection findings (#10583)

diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java
index d957e82..417785c 100644
--- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java
+++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java
@@ -29,7 +29,7 @@
 abstract class BaseOSSFile {
   private final OSS client;
   private final OSSURI uri;
-  private AliyunProperties aliyunProperties;
+  private final AliyunProperties aliyunProperties;
   private SimplifiedObjectMeta metadata;
   private final MetricsContext metrics;
 
diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java
index 5a9b060..9aae5b7 100644
--- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java
+++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java
@@ -101,7 +101,7 @@
   }
 
   public static class Builder {
-    private Map<String, Object> props = Maps.newHashMap();
+    private final Map<String, Object> props = Maps.newHashMap();
 
     public Builder silent() {
       props.put(AliyunOSSMockApp.PROP_SILENT, true);
diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java
index e98f228..7f7546e 100644
--- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java
+++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java
@@ -420,11 +420,11 @@
      * A caller has caused a request that would cross the {@code maxLength} boundary.
      *
      * @param maxLength The max count of bytes to read.
-     * @param count The count of bytes read.
+     * @param bytesRead The count of bytes read.
      * @throws IOException Subclasses may throw.
      * @since 2.12.0
      */
-    protected void onMaxLength(final long maxLength, final long pCount) throws IOException {
+    protected void onMaxLength(final long maxLength, final long bytesRead) throws IOException {
       // for subclasses
     }
 
diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java
index 8f1df79..08a1c4f 100644
--- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java
+++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java
@@ -365,7 +365,7 @@
     private final Schema schema;
     private final List<PartitionField> fields = Lists.newArrayList();
     private final Set<String> partitionNames = Sets.newHashSet();
-    private Map<Map.Entry<Integer, String>, PartitionField> dedupFields = Maps.newHashMap();
+    private final Map<Map.Entry<Integer, String>, PartitionField> dedupFields = Maps.newHashMap();
     private int specId = 0;
     private final AtomicInteger lastAssignedFieldId =
         new AtomicInteger(unpartitionedLastAssignedId());
@@ -409,8 +409,7 @@
               name);
         }
       }
-      Preconditions.checkArgument(
-          name != null && !name.isEmpty(), "Cannot use empty or null partition name: %s", name);
+      Preconditions.checkArgument(!name.isEmpty(), "Cannot use empty partition name: %s", name);
       Preconditions.checkArgument(
           !partitionNames.contains(name), "Cannot use partition name more than once: %s", name);
       partitionNames.add(name);
diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java
index d5ec3f2..7ff712b 100644
--- a/api/src/main/java/org/apache/iceberg/Schema.java
+++ b/api/src/main/java/org/apache/iceberg/Schema.java
@@ -67,8 +67,8 @@
   private transient Map<Integer, Accessor<StructLike>> idToAccessor = null;
   private transient Map<Integer, String> idToName = null;
   private transient Set<Integer> identifierFieldIdSet = null;
-  private transient Map<Integer, Integer> idsToReassigned;
-  private transient Map<Integer, Integer> idsToOriginal;
+  private final transient Map<Integer, Integer> idsToReassigned;
+  private final transient Map<Integer, Integer> idsToOriginal;
 
   public Schema(List<NestedField> columns, Map<String, Integer> aliases) {
     this(columns, aliases, ImmutableSet.of());
diff --git a/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java b/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java
index 4a112e2..48d3f7b 100644
--- a/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java
+++ b/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java
@@ -113,5 +113,5 @@
   default R caseSensitive(boolean caseSensitive) {
     throw new UnsupportedOperationException(
         this.getClass().getName() + " doesn't implement caseSensitive");
-  };
+  }
 }
diff --git a/api/src/main/java/org/apache/iceberg/io/FileIO.java b/api/src/main/java/org/apache/iceberg/io/FileIO.java
index a521cbf..de4bc2e 100644
--- a/api/src/main/java/org/apache/iceberg/io/FileIO.java
+++ b/api/src/main/java/org/apache/iceberg/io/FileIO.java
@@ -94,7 +94,7 @@
    */
   default Map<String, String> properties() {
     throw new UnsupportedOperationException(
-        String.format("%s does not expose configuration properties", this.getClass().toString()));
+        String.format("%s does not expose configuration properties", this.getClass()));
   }
 
   /**
diff --git a/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java b/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java
index 6cf0b38..4ad0c8f 100644
--- a/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java
+++ b/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java
@@ -66,31 +66,22 @@
 
   @Override
   public <T> T time(Supplier<T> supplier) {
-    Timed timed = start();
-    try {
+    try (Timed ignore = start()) {
       return supplier.get();
-    } finally {
-      timed.stop();
     }
   }
 
   @Override
   public <T> T timeCallable(Callable<T> callable) throws Exception {
-    Timed timed = start();
-    try {
+    try (Timed ignore = start()) {
       return callable.call();
-    } finally {
-      timed.stop();
     }
   }
 
   @Override
   public void time(Runnable runnable) {
-    Timed timed = start();
-    try {
+    try (Timed ignore = start()) {
       runnable.run();
-    } finally {
-      timed.stop();
     }
   }
 
diff --git a/api/src/main/java/org/apache/iceberg/util/BucketUtil.java b/api/src/main/java/org/apache/iceberg/util/BucketUtil.java
index 1f3a68a..113850c 100644
--- a/api/src/main/java/org/apache/iceberg/util/BucketUtil.java
+++ b/api/src/main/java/org/apache/iceberg/util/BucketUtil.java
@@ -36,7 +36,7 @@
   private BucketUtil() {}
 
   public static int hash(int value) {
-    return MURMUR3.hashLong((long) value).asInt();
+    return MURMUR3.hashLong(value).asInt();
   }
 
   public static int hash(long value) {
@@ -56,7 +56,7 @@
   }
 
   public static int hash(float value) {
-    return MURMUR3.hashLong(doubleToLongBits((double) value)).asInt();
+    return MURMUR3.hashLong(doubleToLongBits(value)).asInt();
   }
 
   public static int hash(double value) {
diff --git a/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java b/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java
index abd088c..adb6a60 100644
--- a/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java
+++ b/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java
@@ -22,7 +22,7 @@
 
 public class TestableCloseableIterable implements CloseableIterable<Integer> {
   private Boolean closed = false;
-  private TestableCloseableIterator iterator = new TestableCloseableIterator();
+  private final TestableCloseableIterator iterator = new TestableCloseableIterator();
 
   @Override
   public CloseableIterator<Integer> iterator() {
diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java
index f397f94..28d01ef 100644
--- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java
+++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java
@@ -383,7 +383,7 @@
    */
   private static UUID newUUID(byte[] bytes) {
     try {
-      return uuidBytesConstructor.newInstance((Object) bytes);
+      return uuidBytesConstructor.newInstance(bytes);
     } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) {
       throw new RuntimeException(e);
     }
diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java b/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java
index 52a6d2c..9ff11ef 100644
--- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java
+++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java
@@ -32,6 +32,7 @@
 
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.UUID;
 import java.util.stream.Collectors;
 import org.apache.iceberg.PartitionSpec;
@@ -309,7 +310,7 @@
 
   @Test
   public void testBucketByteBufferStrict() throws Exception {
-    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8"));
+    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8));
     Schema schema = new Schema(optional(1, "value", Types.BinaryType.get()));
     PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("value", 10).build();
 
@@ -322,7 +323,7 @@
     assertProjectionStrictValue(
         spec, greaterThanOrEqual("value", value), Expression.Operation.FALSE);
 
-    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8"));
+    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8));
     assertProjectionStrict(
         spec, notIn("value", value, anotherValue), Expression.Operation.NOT_IN, "[4, 6]");
     assertProjectionStrictValue(spec, in("value", value, anotherValue), Expression.Operation.FALSE);
@@ -330,7 +331,7 @@
 
   @Test
   public void testBucketByteBufferInclusive() throws Exception {
-    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8"));
+    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8));
     Schema schema = new Schema(optional(1, "value", Types.BinaryType.get()));
     PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("value", 10).build();
 
@@ -344,7 +345,7 @@
     assertProjectionInclusiveValue(
         spec, greaterThanOrEqual("value", value), Expression.Operation.TRUE);
 
-    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8"));
+    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8));
     assertProjectionInclusive(
         spec, in("value", value, anotherValue), Expression.Operation.IN, "[4, 6]");
     assertProjectionInclusiveValue(
diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java b/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java
index 588f6fc..67fc27a 100644
--- a/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java
+++ b/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java
@@ -32,6 +32,7 @@
 
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.stream.Collectors;
 import org.apache.iceberg.PartitionSpec;
 import org.apache.iceberg.Schema;
@@ -447,10 +448,11 @@
 
   @Test
   public void testBinaryStrict() throws Exception {
-    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8"));
+    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8));
     Schema schema = new Schema(optional(1, "value", Types.BinaryType.get()));
     PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("value", 5).build();
-    String expectedValue = TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes("UTF-8")));
+    String expectedValue =
+        TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes(StandardCharsets.UTF_8)));
 
     assertProjectionStrict(spec, lessThan("value", value), Expression.Operation.LT, expectedValue);
     assertProjectionStrict(
@@ -463,7 +465,7 @@
         spec, notEqual("value", value), Expression.Operation.NOT_EQ, expectedValue);
     assertProjectionStrictValue(spec, equal("value", value), Expression.Operation.FALSE);
 
-    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8"));
+    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8));
     assertProjectionStrict(
         spec,
         notIn("value", value, anotherValue),
@@ -474,10 +476,11 @@
 
   @Test
   public void testBinaryInclusive() throws Exception {
-    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8"));
+    ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8));
     Schema schema = new Schema(optional(1, "value", Types.BinaryType.get()));
     PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("value", 5).build();
-    String expectedValue = TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes("UTF-8")));
+    String expectedValue =
+        TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes(StandardCharsets.UTF_8)));
 
     assertProjectionInclusive(
         spec, lessThan("value", value), Expression.Operation.LT_EQ, expectedValue);
@@ -490,7 +493,7 @@
     assertProjectionInclusive(spec, equal("value", value), Expression.Operation.EQ, expectedValue);
     assertProjectionInclusiveValue(spec, notEqual("value", value), Expression.Operation.TRUE);
 
-    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8"));
+    ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8));
     assertProjectionInclusive(
         spec,
         in("value", value, anotherValue),
diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java
index 2154cf3..0d74aa5 100644
--- a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java
+++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java
@@ -31,8 +31,8 @@
 
   @Test
   public void nullString() {
-    assertThat(CharSequenceMap.create()).doesNotContainKey((String) null);
-    assertThat(CharSequenceMap.create()).doesNotContainValue((String) null);
+    assertThat(CharSequenceMap.create()).doesNotContainKey(null);
+    assertThat(CharSequenceMap.create()).doesNotContainValue(null);
   }
 
   @Test
diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
index 247f4d4..b0a75cc 100644
--- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
+++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
@@ -145,7 +145,7 @@
     int ch3 = inputStream.read();
     int ch2 = inputStream.read();
     int ch1 = inputStream.read();
-    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4;
   }
 
   /** Reads the next byteWidth little endian int. */
@@ -166,7 +166,7 @@
           int ch3 = inputStream.read();
           int ch2 = inputStream.read();
           int ch1 = inputStream.read();
-          return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+          return (ch1 << 16) + (ch2 << 8) + ch3;
         }
       case 4:
         {
diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java
index 5e0f666..630db7a 100644
--- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java
+++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java
@@ -128,15 +128,12 @@
 
   @BeforeAll
   public static void beforeClass() throws Exception {
-    lfRegisterPathRoleName = LF_REGISTER_PATH_ROLE_PREFIX + UUID.randomUUID().toString();
-    lfPrivilegedRoleName = LF_PRIVILEGED_ROLE_PREFIX + UUID.randomUUID().toString();
-    lfRegisterPathRoleS3PolicyName =
-        LF_REGISTER_PATH_ROLE_S3_POLICY_PREFIX + UUID.randomUUID().toString();
-    lfRegisterPathRoleLfPolicyName =
-        LF_REGISTER_PATH_ROLE_LF_POLICY_PREFIX + UUID.randomUUID().toString();
-    lfRegisterPathRoleIamPolicyName =
-        LF_REGISTER_PATH_ROLE_IAM_POLICY_PREFIX + UUID.randomUUID().toString();
-    lfPrivilegedRolePolicyName = LF_PRIVILEGED_ROLE_POLICY_PREFIX + UUID.randomUUID().toString();
+    lfRegisterPathRoleName = LF_REGISTER_PATH_ROLE_PREFIX + UUID.randomUUID();
+    lfPrivilegedRoleName = LF_PRIVILEGED_ROLE_PREFIX + UUID.randomUUID();
+    lfRegisterPathRoleS3PolicyName = LF_REGISTER_PATH_ROLE_S3_POLICY_PREFIX + UUID.randomUUID();
+    lfRegisterPathRoleLfPolicyName = LF_REGISTER_PATH_ROLE_LF_POLICY_PREFIX + UUID.randomUUID();
+    lfRegisterPathRoleIamPolicyName = LF_REGISTER_PATH_ROLE_IAM_POLICY_PREFIX + UUID.randomUUID();
+    lfPrivilegedRolePolicyName = LF_PRIVILEGED_ROLE_POLICY_PREFIX + UUID.randomUUID();
 
     iam =
         IamClient.builder()
diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java
index 9a52ae5..18abb82 100644
--- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java
+++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java
@@ -119,7 +119,7 @@
 
   @BeforeEach
   public void beforeEach() {
-    objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    objectKey = String.format("%s/%s", prefix, UUID.randomUUID());
     objectUri = String.format("s3://%s/%s", bucketName, objectKey);
     clientFactory.initialize(Maps.newHashMap());
   }
diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java
index ac34807..29d4c48 100644
--- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java
+++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java
@@ -64,7 +64,7 @@
 
   @BeforeEach
   public void before() {
-    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString());
+    String objectKey = String.format("%s/%s", prefix, UUID.randomUUID());
     objectUri = String.format("s3://%s/%s", bucketName, objectKey);
   }
 
diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java
index 1d03a0a..0c91f86 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java
@@ -67,7 +67,7 @@
   public static final String CLIENT_REGION = "client.region";
 
   private String clientRegion;
-  private String clientCredentialsProvider;
+  private final String clientCredentialsProvider;
   private final Map<String, String> clientCredentialsProviderProperties;
 
   public AwsClientProperties() {
diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
index 0d75e24..5c3afc2 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
@@ -213,25 +213,25 @@
   private static final String HTTP_CLIENT_PREFIX = "http-client.";
   private final Set<software.amazon.awssdk.services.sts.model.Tag> stsClientAssumeRoleTags;
 
-  private String clientAssumeRoleArn;
-  private String clientAssumeRoleExternalId;
-  private int clientAssumeRoleTimeoutSec;
-  private String clientAssumeRoleRegion;
-  private String clientAssumeRoleSessionName;
-  private String clientCredentialsProvider;
+  private final String clientAssumeRoleArn;
+  private final String clientAssumeRoleExternalId;
+  private final int clientAssumeRoleTimeoutSec;
+  private final String clientAssumeRoleRegion;
+  private final String clientAssumeRoleSessionName;
+  private final String clientCredentialsProvider;
   private final Map<String, String> clientCredentialsProviderProperties;
 
-  private String glueEndpoint;
+  private final String glueEndpoint;
   private String glueCatalogId;
   private boolean glueCatalogSkipArchive;
   private boolean glueCatalogSkipNameValidation;
   private boolean glueLakeFormationEnabled;
 
   private String dynamoDbTableName;
-  private String dynamoDbEndpoint;
+  private final String dynamoDbEndpoint;
 
   private String restSigningRegion;
-  private String restSigningName;
+  private final String restSigningName;
   private String restAccessKeyId;
   private String restSecretAccessKey;
   private String restSessionToken;
diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
index cdbcc79..c6b157b 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
@@ -387,7 +387,7 @@
           "Cannot rename %s to %s because namespace %s does not exist", from, to, to.namespace());
     }
     // keep metadata
-    Table fromTable = null;
+    Table fromTable;
     String fromTableDbName =
         IcebergToGlueConverter.getDatabaseName(from, awsProperties.glueCatalogSkipNameValidation());
     String fromTableName =
diff --git a/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java
index fb0b1c2..552da4b 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java
@@ -139,8 +139,8 @@
   }
 
   static class LakeFormationCredentialsProvider implements AwsCredentialsProvider {
-    private LakeFormationClient client;
-    private String tableArn;
+    private final LakeFormationClient client;
+    private final String tableArn;
 
     LakeFormationCredentialsProvider(LakeFormationClient lakeFormationClient, String tableArn) {
       this.client = lakeFormationClient;
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java
index 1bcbdf7..3414c9d 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java
@@ -396,9 +396,9 @@
   private String sseType;
   private String sseKey;
   private String sseMd5;
-  private String accessKeyId;
-  private String secretAccessKey;
-  private String sessionToken;
+  private final String accessKeyId;
+  private final String secretAccessKey;
+  private final String sessionToken;
   private boolean isS3AccessGrantsEnabled;
   private boolean isS3AccessGrantsFallbackToIamEnabled;
   private int multipartUploadThreads;
@@ -416,11 +416,11 @@
   private boolean isDeleteEnabled;
   private final Map<String, String> bucketToAccessPointMapping;
   private boolean isPreloadClientEnabled;
-  private boolean isDualStackEnabled;
-  private boolean isPathStyleAccess;
-  private boolean isUseArnRegionEnabled;
-  private boolean isAccelerationEnabled;
-  private String endpoint;
+  private final boolean isDualStackEnabled;
+  private final boolean isPathStyleAccess;
+  private final boolean isUseArnRegionEnabled;
+  private final boolean isAccelerationEnabled;
+  private final String endpoint;
   private final boolean isRemoteSigningEnabled;
   private String writeStorageClass;
   private final Map<String, String> allProperties;
diff --git a/common/src/main/java/org/apache/iceberg/common/DynClasses.java b/common/src/main/java/org/apache/iceberg/common/DynClasses.java
index af33ade..3d42171 100644
--- a/common/src/main/java/org/apache/iceberg/common/DynClasses.java
+++ b/common/src/main/java/org/apache/iceberg/common/DynClasses.java
@@ -34,7 +34,7 @@
     private ClassLoader loader = Thread.currentThread().getContextClassLoader();
     private Class<?> foundClass = null;
     private boolean nullOk = false;
-    private Set<String> classNames = Sets.newLinkedHashSet();
+    private final Set<String> classNames = Sets.newLinkedHashSet();
 
     private Builder() {}
 
diff --git a/common/src/main/java/org/apache/iceberg/common/DynFields.java b/common/src/main/java/org/apache/iceberg/common/DynFields.java
index 01afbad..a96da85 100644
--- a/common/src/main/java/org/apache/iceberg/common/DynFields.java
+++ b/common/src/main/java/org/apache/iceberg/common/DynFields.java
@@ -414,7 +414,7 @@
   }
 
   private static class MakeFieldAccessible implements PrivilegedAction<Void> {
-    private Field hidden;
+    private final Field hidden;
 
     MakeFieldAccessible(Field hidden) {
       this.hidden = hidden;
diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java
index ae331da..98de1e7 100644
--- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java
+++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java
@@ -508,7 +508,7 @@
   }
 
   private static class MakeAccessible implements PrivilegedAction<Void> {
-    private Method hidden;
+    private final Method hidden;
 
     MakeAccessible(Method hidden) {
       this.hidden = hidden;
diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java
index 4e485d5..5f35704 100644
--- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java
+++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java
@@ -54,7 +54,7 @@
   public Schema schema() {
     StructType partitionType = Partitioning.partitionType(table());
     Schema schema = ManifestEntry.getSchema(partitionType);
-    if (partitionType.fields().size() < 1) {
+    if (partitionType.fields().isEmpty()) {
       // avoid returning an empty struct, which is not always supported.
       // instead, drop the partition field (id 102)
       schema = TypeUtil.selectNot(schema, Sets.newHashSet(DataFile.PARTITION_ID));
diff --git a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java
index 62e6f8a..011abff 100644
--- a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java
+++ b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java
@@ -48,7 +48,7 @@
   public Schema schema() {
     StructType partitionType = Partitioning.partitionType(table());
     Schema schema = new Schema(DataFile.getType(partitionType).fields());
-    if (partitionType.fields().size() < 1) {
+    if (partitionType.fields().isEmpty()) {
       // avoid returning an empty struct, which is not always supported.
       // instead, drop the partition field
       schema = TypeUtil.selectNot(schema, Sets.newHashSet(DataFile.PARTITION_ID));
diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
index 6443cf6..50266b2 100644
--- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
+++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
@@ -355,7 +355,7 @@
     }
 
     try {
-      return Integer.valueOf(metadataLocation.substring(versionStart, versionEnd));
+      return Integer.parseInt(metadataLocation.substring(versionStart, versionEnd));
     } catch (NumberFormatException e) {
       LOG.warn("Unable to parse version from metadata location: {}", metadataLocation, e);
       return -1;
diff --git a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
index d3a8edb..04611f5 100644
--- a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
+++ b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
@@ -111,7 +111,7 @@
 
   @Override
   public List<ManifestFile> apply(TableMetadata base, Snapshot snapshot) {
-    if (dataSpec().fields().size() <= 0) {
+    if (dataSpec().fields().isEmpty()) {
       // replace all data in an unpartitioned table
       deleteByRowFilter(Expressions.alwaysTrue());
     }
diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
index f3f8e5f..dce6d4a 100644
--- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
+++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
@@ -200,7 +200,7 @@
       return false;
     }
 
-    if (rewrittenManifests.size() == 0) {
+    if (rewrittenManifests.isEmpty()) {
       // nothing yet processed so perform a full rewrite
       return true;
     }
diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java
index 2f05146..eb8dbd2 100644
--- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java
+++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java
@@ -71,7 +71,7 @@
   private final Set<String> deletedFiles =
       Sets.newHashSet(); // keep track of files deleted in the most recent commit
   private final Consumer<String> enqueueDelete = deletedFiles::add;
-  private TransactionType type;
+  private final TransactionType type;
   private TableMetadata base;
   private TableMetadata current;
   private boolean hasLastOpCommitted;
diff --git a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java
index c3534fa..f454c4a 100644
--- a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java
+++ b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java
@@ -40,7 +40,7 @@
   private volatile int currentSize;
   private boolean closed;
 
-  private int connectionRetryWaitPeriodMs = 1000;
+  private static final int CONNECTION_RETRY_WAIT_PERIOD_MS = 1000;
 
   public ClientPoolImpl(int poolSize, Class<? extends E> reconnectExc, boolean retryByDefault) {
     this(poolSize, reconnectExc, retryByDefault, 1);
@@ -80,7 +80,7 @@
           } catch (Exception e) {
             if (isConnectionException(e)) {
               retryAttempts++;
-              Thread.sleep(connectionRetryWaitPeriodMs);
+              Thread.sleep(CONNECTION_RETRY_WAIT_PERIOD_MS);
             } else {
               throw reconnectExc.cast(exc);
             }
diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java
index d133b76..49fb1fe 100644
--- a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java
+++ b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java
@@ -336,9 +336,9 @@
     private final String refName;
     private final Long snapshotId;
     private final SnapshotRefType type;
-    private Integer minSnapshotsToKeep;
-    private Long maxSnapshotAgeMs;
-    private Long maxRefAgeMs;
+    private final Integer minSnapshotsToKeep;
+    private final Long maxSnapshotAgeMs;
+    private final Long maxRefAgeMs;
 
     public SetSnapshotRef(
         String refName,
diff --git a/core/src/main/java/org/apache/iceberg/MetricsModes.java b/core/src/main/java/org/apache/iceberg/MetricsModes.java
index fd6b9c5..3dc2434 100644
--- a/core/src/main/java/org/apache/iceberg/MetricsModes.java
+++ b/core/src/main/java/org/apache/iceberg/MetricsModes.java
@@ -160,7 +160,7 @@
   }
 
   private static class MetricsModeProxy implements Serializable {
-    private String modeAsString;
+    private final String modeAsString;
 
     MetricsModeProxy(String modeAsString) {
       this.modeAsString = modeAsString;
diff --git a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java
index 9168b84..35338a6 100644
--- a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java
+++ b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java
@@ -62,7 +62,7 @@
   public UpdateProperties remove(String key) {
     Preconditions.checkNotNull(key, "Key cannot be null");
     Preconditions.checkArgument(
-        !updates.keySet().contains(key), "Cannot remove and update the same key: %s", key);
+        !updates.containsKey(key), "Cannot remove and update the same key: %s", key);
 
     removals.add(key);
 
diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java
index 218daca..9f4bcbc 100644
--- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java
+++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java
@@ -370,82 +370,84 @@
   public void commit() {
     // this is always set to the latest commit attempt's snapshot id.
     AtomicLong newSnapshotId = new AtomicLong(-1L);
-    Timed totalDuration = commitMetrics().totalDuration().start();
-    try {
-      Tasks.foreach(ops)
-          .retry(base.propertyAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT))
-          .exponentialBackoff(
-              base.propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
-              base.propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
-              base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
-              2.0 /* exponential */)
-          .onlyRetryOn(CommitFailedException.class)
-          .countAttempts(commitMetrics().attempts())
-          .run(
-              taskOps -> {
-                Snapshot newSnapshot = apply();
-                newSnapshotId.set(newSnapshot.snapshotId());
-                TableMetadata.Builder update = TableMetadata.buildFrom(base);
-                if (base.snapshot(newSnapshot.snapshotId()) != null) {
-                  // this is a rollback operation
-                  update.setBranchSnapshot(newSnapshot.snapshotId(), targetBranch);
-                } else if (stageOnly) {
-                  update.addSnapshot(newSnapshot);
-                } else {
-                  update.setBranchSnapshot(newSnapshot, targetBranch);
-                }
+    try (Timed ignore = commitMetrics().totalDuration().start()) {
+      try {
+        Tasks.foreach(ops)
+            .retry(base.propertyAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT))
+            .exponentialBackoff(
+                base.propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
+                base.propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
+                base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
+                2.0 /* exponential */)
+            .onlyRetryOn(CommitFailedException.class)
+            .countAttempts(commitMetrics().attempts())
+            .run(
+                taskOps -> {
+                  Snapshot newSnapshot = apply();
+                  newSnapshotId.set(newSnapshot.snapshotId());
+                  TableMetadata.Builder update = TableMetadata.buildFrom(base);
+                  if (base.snapshot(newSnapshot.snapshotId()) != null) {
+                    // this is a rollback operation
+                    update.setBranchSnapshot(newSnapshot.snapshotId(), targetBranch);
+                  } else if (stageOnly) {
+                    update.addSnapshot(newSnapshot);
+                  } else {
+                    update.setBranchSnapshot(newSnapshot, targetBranch);
+                  }
 
-                TableMetadata updated = update.build();
-                if (updated.changes().isEmpty()) {
-                  // do not commit if the metadata has not changed. for example, this may happen
-                  // when setting the current
-                  // snapshot to an ID that is already current. note that this check uses identity.
-                  return;
-                }
+                  TableMetadata updated = update.build();
+                  if (updated.changes().isEmpty()) {
+                    // do not commit if the metadata has not changed. for example, this may happen
+                    // when setting the current
+                    // snapshot to an ID that is already current. note that this check uses
+                    // identity.
+                    return;
+                  }
 
-                // if the table UUID is missing, add it here. the UUID will be re-created each time
-                // this operation retries
-                // to ensure that if a concurrent operation assigns the UUID, this operation will
-                // not fail.
-                taskOps.commit(base, updated.withUUID());
-              });
+                  // if the table UUID is missing, add it here. the UUID will be re-created each
+                  // time
+                  // this operation retries
+                  // to ensure that if a concurrent operation assigns the UUID, this operation will
+                  // not fail.
+                  taskOps.commit(base, updated.withUUID());
+                });
 
-    } catch (CommitStateUnknownException commitStateUnknownException) {
-      throw commitStateUnknownException;
-    } catch (RuntimeException e) {
-      if (!strictCleanup || e instanceof CleanableFailure) {
-        Exceptions.suppressAndThrow(e, this::cleanAll);
-      }
-
-      throw e;
-    }
-
-    try {
-      LOG.info("Committed snapshot {} ({})", newSnapshotId.get(), getClass().getSimpleName());
-
-      // at this point, the commit must have succeeded. after a refresh, the snapshot is loaded by
-      // id in case another commit was added between this commit and the refresh.
-      Snapshot saved = ops.refresh().snapshot(newSnapshotId.get());
-      if (saved != null) {
-        cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io())));
-        // also clean up unused manifest lists created by multiple attempts
-        for (String manifestList : manifestLists) {
-          if (!saved.manifestListLocation().equals(manifestList)) {
-            deleteFile(manifestList);
-          }
+      } catch (CommitStateUnknownException commitStateUnknownException) {
+        throw commitStateUnknownException;
+      } catch (RuntimeException e) {
+        if (!strictCleanup || e instanceof CleanableFailure) {
+          Exceptions.suppressAndThrow(e, this::cleanAll);
         }
-      } else {
-        // saved may not be present if the latest metadata couldn't be loaded due to eventual
-        // consistency problems in refresh. in that case, don't clean up.
-        LOG.warn("Failed to load committed snapshot, skipping manifest clean-up");
+
+        throw e;
       }
 
-    } catch (Throwable e) {
-      LOG.warn(
-          "Failed to load committed table metadata or during cleanup, skipping further cleanup", e);
-    }
+      try {
+        LOG.info("Committed snapshot {} ({})", newSnapshotId.get(), getClass().getSimpleName());
 
-    totalDuration.stop();
+        // at this point, the commit must have succeeded. after a refresh, the snapshot is loaded by
+        // id in case another commit was added between this commit and the refresh.
+        Snapshot saved = ops.refresh().snapshot(newSnapshotId.get());
+        if (saved != null) {
+          cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io())));
+          // also clean up unused manifest lists created by multiple attempts
+          for (String manifestList : manifestLists) {
+            if (!saved.manifestListLocation().equals(manifestList)) {
+              deleteFile(manifestList);
+            }
+          }
+        } else {
+          // saved may not be present if the latest metadata couldn't be loaded due to eventual
+          // consistency problems in refresh. in that case, don't clean up.
+          LOG.warn("Failed to load committed snapshot, skipping manifest clean-up");
+        }
+
+      } catch (Throwable e) {
+        LOG.warn(
+            "Failed to load committed table metadata or during cleanup, skipping further cleanup",
+            e);
+      }
+    }
 
     try {
       notifyListeners();
diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java
index a7eddde..6b23c33 100644
--- a/core/src/main/java/org/apache/iceberg/TableMetadata.java
+++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java
@@ -1434,7 +1434,7 @@
       // what is in the metadata file, which does not store changes. metadata location with changes
       // is inconsistent.
       Preconditions.checkArgument(
-          changes.size() == 0 || discardChanges || metadataLocation == null,
+          changes.isEmpty() || discardChanges || metadataLocation == null,
           "Cannot set metadata location with changes to table metadata: %s changes",
           changes.size());
 
diff --git a/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java b/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java
index 9d15bf0..f7ccea7 100644
--- a/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java
+++ b/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java
@@ -31,7 +31,7 @@
 
   private final TableOperations ops;
   private final Map<String, SnapshotRef> updatedRefs;
-  private TableMetadata base;
+  private final TableMetadata base;
 
   UpdateSnapshotReferencesOperation(TableOperations ops) {
     this.ops = ops;
diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java
index a3b0292..df2fdfd 100644
--- a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java
+++ b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java
@@ -27,8 +27,8 @@
   private static final RewriteDataFilesActionResult EMPTY =
       new RewriteDataFilesActionResult(ImmutableList.of(), ImmutableList.of());
 
-  private List<DataFile> deletedDataFiles;
-  private List<DataFile> addedDataFiles;
+  private final List<DataFile> deletedDataFiles;
+  private final List<DataFile> addedDataFiles;
 
   public RewriteDataFilesActionResult(
       List<DataFile> deletedDataFiles, List<DataFile> addedDataFiles) {
diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java
index 575e44a..69159b6 100644
--- a/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java
+++ b/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java
@@ -68,7 +68,7 @@
     }
   }
 
-  private Deque<String> recordLevels = Lists.newLinkedList();
+  private final Deque<String> recordLevels = Lists.newLinkedList();
 
   public T record(Schema record, List<String> names, Iterable<F> fields) {
     return null;
diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
index c69a862..a583f9a 100644
--- a/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
+++ b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java
@@ -34,11 +34,11 @@
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 
 class AvroFileAppender<D> implements FileAppender<D> {
-  private PositionOutputStream stream;
+  private final PositionOutputStream stream;
   private DataFileWriter<D> writer;
-  private DatumWriter<?> datumWriter;
-  private org.apache.iceberg.Schema icebergSchema;
-  private MetricsConfig metricsConfig;
+  private final DatumWriter<?> datumWriter;
+  private final org.apache.iceberg.Schema icebergSchema;
+  private final MetricsConfig metricsConfig;
   private long numRecords = 0L;
   private boolean isClosed = false;
 
diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java
index 57c99fa..032d631 100644
--- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java
+++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java
@@ -222,7 +222,7 @@
   static Schema createMap(int keyId, Schema keySchema, int valueId, Schema valueSchema) {
     String keyValueName = "k" + keyId + "_v" + valueId;
 
-    Schema.Field keyField = new Schema.Field("key", keySchema, null, (Object) null);
+    Schema.Field keyField = new Schema.Field("key", keySchema, null, null);
     keyField.addProp(FIELD_ID_PROP, keyId);
 
     Schema.Field valueField =
@@ -250,7 +250,7 @@
       Schema valueSchema) {
     String keyValueName = "k" + keyId + "_v" + valueId;
 
-    Schema.Field keyField = new Schema.Field("key", keySchema, null, (Object) null);
+    Schema.Field keyField = new Schema.Field("key", keySchema, null, null);
     if (!"key".equals(keyName)) {
       keyField.addAlias(keyName);
     }
diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java
index f22a359..9819924 100644
--- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java
+++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java
@@ -71,8 +71,8 @@
     }
   }
 
-  private Deque<String> recordLevels = Lists.newLinkedList();
-  private Deque<String> fieldNames = Lists.newLinkedList();
+  private final Deque<String> recordLevels = Lists.newLinkedList();
+  private final Deque<String> fieldNames = Lists.newLinkedList();
 
   protected Deque<String> fieldNames() {
     return fieldNames;
diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java
index 85a8718..45892d3 100644
--- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java
+++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java
@@ -115,7 +115,7 @@
     }
   }
 
-  private Deque<String> recordLevels = Lists.newLinkedList();
+  private final Deque<String> recordLevels = Lists.newLinkedList();
 
   public T record(Types.StructType iStruct, Schema record, List<String> names, List<T> fields) {
     return null;
diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java
index 2c08000..126a637 100644
--- a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java
+++ b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java
@@ -146,7 +146,7 @@
   }
 
   /** Just for checking state. */
-  private Deque<String> recordLevels = Lists.newLinkedList();
+  private final Deque<String> recordLevels = Lists.newLinkedList();
 
   // ---------------------------------- Partner type methods
   // ---------------------------------------------
diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java
index 0147dbf..b23b195 100644
--- a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java
+++ b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java
@@ -67,7 +67,7 @@
   }
 
   /** Used to fail on recursive types. */
-  private Deque<String> recordLevels = Lists.newLinkedList();
+  private final Deque<String> recordLevels = Lists.newLinkedList();
 
   public R record(P partner, Schema record, List<R> fieldResults) {
     return null;
diff --git a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java
index d89489d..93bfa23 100644
--- a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java
+++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java
@@ -43,7 +43,7 @@
   private final Types.StructType expectedType;
   private ClassLoader loader = Thread.currentThread().getContextClassLoader();
   private Map<String, String> renames = ImmutableMap.of();
-  private Map<Integer, ?> idToConstant = ImmutableMap.of();
+  private final Map<Integer, ?> idToConstant = ImmutableMap.of();
   private Schema fileSchema = null;
   private ValueReader<T> reader = null;
 
diff --git a/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java
index 1658db9..3844ede 100644
--- a/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java
+++ b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java
@@ -232,7 +232,7 @@
         throw new IllegalArgumentException("Cannot write null to required string column");
       } else {
         throw new IllegalArgumentException(
-            "Cannot write unknown string type: " + s.getClass().getName() + ": " + s.toString());
+            "Cannot write unknown string type: " + s.getClass().getName() + ": " + s);
       }
     }
   }
diff --git a/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java b/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
index f11506a..2cade89 100644
--- a/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
+++ b/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
@@ -27,8 +27,8 @@
  * encryption support (Parquet and ORC).
  */
 public class NativeFileCryptoParameters {
-  private ByteBuffer fileKey;
-  private EncryptionAlgorithm fileEncryptionAlgorithm;
+  private final ByteBuffer fileKey;
+  private final EncryptionAlgorithm fileEncryptionAlgorithm;
 
   private NativeFileCryptoParameters(
       ByteBuffer fileKey, EncryptionAlgorithm fileEncryptionAlgorithm) {
@@ -48,7 +48,7 @@
   }
 
   public static class Builder {
-    private ByteBuffer fileKey;
+    private final ByteBuffer fileKey;
     private EncryptionAlgorithm fileEncryptionAlgorithm;
 
     private Builder(ByteBuffer fileKey) {
diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java
index 4e815ce..1e0cf44 100644
--- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java
+++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java
@@ -153,7 +153,7 @@
             TableProperties.METADATA_COMPRESSION, TableProperties.METADATA_COMPRESSION_DEFAULT);
     TableMetadataParser.Codec codec = TableMetadataParser.Codec.fromName(codecName);
     String fileExtension = TableMetadataParser.getFileExtension(codec);
-    Path tempMetadataFile = metadataPath(UUID.randomUUID().toString() + fileExtension);
+    Path tempMetadataFile = metadataPath(UUID.randomUUID() + fileExtension);
     TableMetadataParser.write(metadata, io().newOutputFile(tempMetadataFile.toString()));
 
     int nextVersion = (current.first() != null ? current.first() : 0) + 1;
@@ -296,7 +296,7 @@
     FileSystem fs = getFileSystem(versionHintFile, conf);
 
     try {
-      Path tempVersionHintFile = metadataPath(UUID.randomUUID().toString() + "-version-hint.temp");
+      Path tempVersionHintFile = metadataPath(UUID.randomUUID() + "-version-hint.temp");
       writeVersionToPath(fs, tempVersionHintFile, versionToWrite);
       fs.delete(versionHintFile, false /* recursive delete */);
       fs.rename(tempVersionHintFile, versionHintFile);
diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
index 487b840..9b6fca9 100644
--- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
+++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
@@ -53,7 +53,7 @@
   private final String dbUrl;
   private final Map<String, String> properties;
 
-  private Set<String> retryableStatusCodes;
+  private final Set<String> retryableStatusCodes;
 
   public JdbcClientPool(String dbUrl, Map<String, String> props) {
     this(
diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java
index 5543259..da93b82 100644
--- a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java
+++ b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java
@@ -28,10 +28,10 @@
 /** Standard response body for all API errors */
 public class ErrorResponse implements RESTResponse {
 
-  private String message;
-  private String type;
-  private int code;
-  private List<String> stack;
+  private final String message;
+  private final String type;
+  private final int code;
+  private final List<String> stack;
 
   private ErrorResponse(String message, String type, int code, List<String> stack) {
     this.message = message;
diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java
index a389479..5e5353c 100644
--- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java
+++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java
@@ -84,7 +84,7 @@
   public static class Builder {
     private String metadataLocation;
     private TableMetadata metadata;
-    private Map<String, String> config = Maps.newHashMap();
+    private final Map<String, String> config = Maps.newHashMap();
 
     private Builder() {}
 
diff --git a/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java b/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java
index 2916777..1497ba5 100644
--- a/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java
+++ b/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java
@@ -34,7 +34,7 @@
 
   private final UpdateSchema api;
   private final Schema partnerSchema;
-  private boolean caseSensitive;
+  private final boolean caseSensitive;
 
   private UnionByNameVisitor(UpdateSchema api, Schema partnerSchema, boolean caseSensitive) {
     this.api = api;
diff --git a/core/src/main/java/org/apache/iceberg/util/ArrayUtil.java b/core/src/main/java/org/apache/iceberg/util/ArrayUtil.java
index 4c9941b..3a2ae0b 100644
--- a/core/src/main/java/org/apache/iceberg/util/ArrayUtil.java
+++ b/core/src/main/java/org/apache/iceberg/util/ArrayUtil.java
@@ -96,7 +96,7 @@
     }
     final boolean[] result = new boolean[array.length];
     for (int i = 0; i < array.length; i++) {
-      result[i] = array[i].booleanValue();
+      result[i] = array[i];
     }
     return result;
   }
@@ -120,7 +120,7 @@
     }
     final byte[] result = new byte[array.length];
     for (int i = 0; i < array.length; i++) {
-      result[i] = array[i].byteValue();
+      result[i] = array[i];
     }
     return result;
   }
@@ -144,7 +144,7 @@
     }
     final short[] result = new short[array.length];
     for (int i = 0; i < array.length; i++) {
-      result[i] = array[i].shortValue();
+      result[i] = array[i];
     }
     return result;
   }
@@ -168,7 +168,7 @@
     }
     final int[] result = new int[array.length];
     for (int i = 0; i < array.length; i++) {
-      result[i] = array[i].intValue();
+      result[i] = array[i];
     }
     return result;
   }
@@ -192,7 +192,7 @@
     }
     final long[] result = new long[array.length];
     for (int i = 0; i < array.length; i++) {
-      result[i] = array[i].longValue();
+      result[i] = array[i];
     }
     return result;
   }
@@ -216,7 +216,7 @@
     }
     final float[] result = new float[array.length];
     for (int i = 0; i < array.length; i++) {
-      result[i] = array[i].floatValue();
+      result[i] = array[i];
     }
     return result;
   }
@@ -240,7 +240,7 @@
     }
     final double[] result = new double[array.length];
     for (int i = 0; i < array.length; i++) {
-      result[i] = array[i].doubleValue();
+      result[i] = array[i];
     }
     return result;
   }
diff --git a/core/src/main/java/org/apache/iceberg/util/Pair.java b/core/src/main/java/org/apache/iceberg/util/Pair.java
index 89b697a..bd3a934 100644
--- a/core/src/main/java/org/apache/iceberg/util/Pair.java
+++ b/core/src/main/java/org/apache/iceberg/util/Pair.java
@@ -49,8 +49,8 @@
                       null,
                       false,
                       Lists.newArrayList(
-                          new Schema.Field("x", xSchema, null, (Object) null),
-                          new Schema.Field("y", ySchema, null, (Object) null)));
+                          new Schema.Field("x", xSchema, null, null),
+                          new Schema.Field("y", ySchema, null, null)));
                 }
               });
 
@@ -109,7 +109,7 @@
 
   @Override
   public String toString() {
-    return "(" + String.valueOf(first) + ", " + String.valueOf(second) + ")";
+    return "(" + first + ", " + second + ")";
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/iceberg/util/Tasks.java b/core/src/main/java/org/apache/iceberg/util/Tasks.java
index e420145..02d2b83 100644
--- a/core/src/main/java/org/apache/iceberg/util/Tasks.java
+++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java
@@ -78,7 +78,7 @@
     private boolean stopAbortsOnFailure = false;
 
     // retry settings
-    private List<Class<? extends Exception>> stopRetryExceptions =
+    private final List<Class<? extends Exception>> stopRetryExceptions =
         Lists.newArrayList(UnrecoverableException.class);
     private List<Class<? extends Exception>> onlyRetryExceptions = null;
     private Predicate<Exception> shouldRetryPredicate = null;
@@ -496,7 +496,7 @@
 
           } catch (ExecutionException e) {
             Throwable cause = e.getCause();
-            if (Error.class.isInstance(cause)) {
+            if (cause instanceof Error) {
               for (Throwable t : uncaught) {
                 cause.addSuppressed(t);
               }
diff --git a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
index c687fc4..4a58057 100644
--- a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
+++ b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
@@ -76,9 +76,7 @@
    * ByteBuffer)}
    */
   public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) {
-    ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE);
-    bytes.putLong(((long) val) ^ 0x8000000000000000L);
-    return bytes;
+    return intToOrderedBytes(val, reuse);
   }
 
   /**
@@ -86,9 +84,7 @@
    * ByteBuffer)}
    */
   public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
-    ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE);
-    bytes.putLong(((long) val) ^ 0x8000000000000000L);
-    return bytes;
+    return intToOrderedBytes(val, reuse);
   }
 
   /**
@@ -100,11 +96,7 @@
    * into lexicographically comparable bytes
    */
   public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) {
-    ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE);
-    long lval = Double.doubleToLongBits(val);
-    lval ^= ((lval >> (Integer.SIZE - 1)) | Long.MIN_VALUE);
-    bytes.putLong(lval);
-    return bytes;
+    return doubleToOrderedBytes(val, reuse);
   }
 
   /** Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)} */
diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java
index 7a4f546..18b452f 100644
--- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java
+++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java
@@ -219,7 +219,7 @@
     }
 
     try {
-      return Integer.valueOf(metadataLocation.substring(versionStart, versionEnd));
+      return Integer.parseInt(metadataLocation.substring(versionStart, versionEnd));
     } catch (NumberFormatException e) {
       LOG.warn("Unable to parse version from metadata location: {}", metadataLocation, e);
       return -1;
diff --git a/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java
index 5581818..56e3f7b 100644
--- a/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java
+++ b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java
@@ -36,7 +36,6 @@
 import java.util.zip.GZIPInputStream;
 import java.util.zip.ZipException;
 import org.apache.iceberg.TableMetadataParser.Codec;
-import org.apache.iceberg.io.FileIO;
 import org.apache.iceberg.io.OutputFile;
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types.BooleanType;
@@ -69,7 +68,7 @@
     TableMetadataParser.write(metadata, outputFile);
     assertThat(isCompressed(fileName)).isEqualTo(codec == Codec.GZIP);
     TableMetadata actualMetadata =
-        TableMetadataParser.read((FileIO) null, Files.localInput(new File(fileName)));
+        TableMetadataParser.read(null, Files.localInput(new File(fileName)));
     verifyMetadata(metadata, actualMetadata);
   }
 
@@ -88,7 +87,7 @@
   }
 
   private boolean isCompressed(String path) throws IOException {
-    try (InputStream ignored = new GZIPInputStream(new FileInputStream(new File(path)))) {
+    try (InputStream ignored = new GZIPInputStream(new FileInputStream(path))) {
       return true;
     } catch (ZipException e) {
       if (e.getMessage().equals("Not in GZIP format")) {
diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java
index dc7bf0e..2322062 100644
--- a/core/src/test/java/org/apache/iceberg/TestBase.java
+++ b/core/src/test/java/org/apache/iceberg/TestBase.java
@@ -390,7 +390,7 @@
 
   Snapshot apply(SnapshotUpdate<?> snapshotUpdate, String branch) {
     if (branch.equals(SnapshotRef.MAIN_BRANCH)) {
-      return ((SnapshotProducer<?>) snapshotUpdate).apply();
+      return snapshotUpdate.apply();
     } else {
       return ((SnapshotProducer<?>) snapshotUpdate.toBranch(branch)).apply();
     }
diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java
index 8075372..e11cc50 100644
--- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java
+++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java
@@ -537,7 +537,7 @@
     List<MetadataLogEntry> previousMetadataLog = Lists.newArrayList();
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp, "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json"));
 
     TableMetadata base =
         new TableMetadata(
@@ -606,17 +606,14 @@
     List<MetadataLogEntry> previousMetadataLog = Lists.newArrayList();
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 100,
-            "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 100, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 90,
-            "/tmp/000002-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 90, "/tmp/000002-" + UUID.randomUUID() + ".metadata.json"));
 
     MetadataLogEntry latestPreviousMetadata =
         new MetadataLogEntry(
-            currentTimestamp - 80,
-            "/tmp/000003-" + UUID.randomUUID().toString() + ".metadata.json");
+            currentTimestamp - 80, "/tmp/000003-" + UUID.randomUUID() + ".metadata.json");
 
     TableMetadata base =
         new TableMetadata(
@@ -691,29 +688,23 @@
     List<MetadataLogEntry> previousMetadataLog = Lists.newArrayList();
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 100,
-            "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 100, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 90,
-            "/tmp/000002-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 90, "/tmp/000002-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 80,
-            "/tmp/000003-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 80, "/tmp/000003-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 70,
-            "/tmp/000004-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 70, "/tmp/000004-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 60,
-            "/tmp/000005-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 60, "/tmp/000005-" + UUID.randomUUID() + ".metadata.json"));
 
     MetadataLogEntry latestPreviousMetadata =
         new MetadataLogEntry(
-            currentTimestamp - 50,
-            "/tmp/000006-" + UUID.randomUUID().toString() + ".metadata.json");
+            currentTimestamp - 50, "/tmp/000006-" + UUID.randomUUID() + ".metadata.json");
 
     TableMetadata base =
         new TableMetadata(
@@ -792,29 +783,23 @@
     List<MetadataLogEntry> previousMetadataLog = Lists.newArrayList();
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 100,
-            "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 100, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 90,
-            "/tmp/000002-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 90, "/tmp/000002-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 80,
-            "/tmp/000003-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 80, "/tmp/000003-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 70,
-            "/tmp/000004-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 70, "/tmp/000004-" + UUID.randomUUID() + ".metadata.json"));
     previousMetadataLog.add(
         new MetadataLogEntry(
-            currentTimestamp - 60,
-            "/tmp/000005-" + UUID.randomUUID().toString() + ".metadata.json"));
+            currentTimestamp - 60, "/tmp/000005-" + UUID.randomUUID() + ".metadata.json"));
 
     MetadataLogEntry latestPreviousMetadata =
         new MetadataLogEntry(
-            currentTimestamp - 50,
-            "/tmp/000006-" + UUID.randomUUID().toString() + ".metadata.json");
+            currentTimestamp - 50, "/tmp/000006-" + UUID.randomUUID() + ".metadata.json");
 
     TableMetadata base =
         new TableMetadata(
diff --git a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java
index 28695e2..80f4d05 100644
--- a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java
+++ b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java
@@ -70,7 +70,7 @@
     verifyManifestSequenceNumber(dataManifest, 0, 0);
 
     // update table version to 2
-    TableOperations ops = ((BaseTable) table).operations();
+    TableOperations ops = table.operations();
     TableMetadata base = ops.current();
     ops.commit(base, base.upgradeToFormatVersion(2));
 
@@ -127,7 +127,7 @@
     table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit();
 
     // update table version to 2
-    TableOperations ops = ((BaseTable) table).operations();
+    TableOperations ops = table.operations();
     TableMetadata base = ops.current();
     ops.commit(base, base.upgradeToFormatVersion(2));
 
@@ -162,7 +162,7 @@
     table.newAppend().appendFile(FILE_A).commit();
 
     // update table version to 2
-    TableOperations ops = ((BaseTable) table).operations();
+    TableOperations ops = table.operations();
     TableMetadata base = ops.current();
     ops.commit(base, base.upgradeToFormatVersion(2));
 
diff --git a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java
index 686456b..e3b2596 100644
--- a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java
+++ b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java
@@ -59,41 +59,39 @@
                 new Schema.Field(
                     "complexUnion",
                     Schema.createUnion(
-                        new Schema[] {
-                          Schema.create(Schema.Type.NULL),
-                          Schema.create(Schema.Type.STRING),
-                          Schema.createRecord(
-                              "innerRecord1",
-                              null,
-                              "namespace1",
-                              false,
-                              Lists.newArrayList(
-                                  new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)),
-                                  new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))),
-                          Schema.createRecord(
-                              "innerRecord2",
-                              null,
-                              "namespace2",
-                              false,
-                              Lists.newArrayList(
-                                  new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)),
-                                  new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))),
-                          Schema.createRecord(
-                              "innerRecord3",
-                              null,
-                              "namespace3",
-                              false,
-                              Lists.newArrayList(
-                                  new Schema.Field(
-                                      "innerUnion",
-                                      Schema.createUnion(
-                                          Lists.newArrayList(
-                                              Schema.create(Schema.Type.STRING),
-                                              Schema.create(Schema.Type.INT)))))),
-                          Schema.createEnum(
-                              "timezone", null, null, Lists.newArrayList("UTC", "PST", "EST")),
-                          Schema.createFixed("bitmap", null, null, 1)
-                        }))));
+                        Schema.create(Schema.Type.NULL),
+                        Schema.create(Schema.Type.STRING),
+                        Schema.createRecord(
+                            "innerRecord1",
+                            null,
+                            "namespace1",
+                            false,
+                            Lists.newArrayList(
+                                new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)),
+                                new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))),
+                        Schema.createRecord(
+                            "innerRecord2",
+                            null,
+                            "namespace2",
+                            false,
+                            Lists.newArrayList(
+                                new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)),
+                                new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))),
+                        Schema.createRecord(
+                            "innerRecord3",
+                            null,
+                            "namespace3",
+                            false,
+                            Lists.newArrayList(
+                                new Schema.Field(
+                                    "innerUnion",
+                                    Schema.createUnion(
+                                        Lists.newArrayList(
+                                            Schema.create(Schema.Type.STRING),
+                                            Schema.create(Schema.Type.INT)))))),
+                        Schema.createEnum(
+                            "timezone", null, null, Lists.newArrayList("UTC", "PST", "EST")),
+                        Schema.createFixed("bitmap", null, null, 1)))));
 
     NameMappingWithAvroSchema nameMappingWithAvroSchema = new NameMappingWithAvroSchema();
 
diff --git a/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java b/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java
index 9b35546..0024363 100644
--- a/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java
+++ b/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java
@@ -40,6 +40,7 @@
     DecoderResolver.DECODER_CACHES.get().clear();
   }
 
+  @SuppressWarnings("UnusedAssignment") // the unused assignments are necessary for this test
   @Test
   public void testDecoderCachingReadSchemaSameAsFileSchema() throws Exception {
     Decoder dummyDecoder = DecoderFactory.get().binaryDecoder(new byte[] {}, null);
@@ -84,6 +85,7 @@
     checkCachedSize(0);
   }
 
+  @SuppressWarnings("UnusedAssignment") // the unused assignments are necessary for this test
   @Test
   public void testDecoderCachingReadSchemaNotSameAsFileSchema() throws Exception {
     Decoder dummyDecoder = DecoderFactory.get().binaryDecoder(new byte[] {}, null);
diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java
index 95d0b6a..109c88d 100644
--- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java
+++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java
@@ -86,7 +86,7 @@
   @Test
   public void testFileExists() throws IOException {
     Path parent = new Path(tempDir.toURI());
-    Path randomFilePath = new Path(parent, "random-file-" + UUID.randomUUID().toString());
+    Path randomFilePath = new Path(parent, "random-file-" + UUID.randomUUID());
     fs.createNewFile(randomFilePath);
 
     // check existence of the created file
diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java
index fbd6f83..6512b24 100644
--- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java
+++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java
@@ -58,7 +58,7 @@
 import org.junit.jupiter.params.provider.ValueSource;
 
 public class TestHadoopCatalog extends HadoopTableTestBase {
-  private static ImmutableMap<String, String> meta = ImmutableMap.of();
+  private static final ImmutableMap<String, String> meta = ImmutableMap.of();
 
   @ParameterizedTest
   @ValueSource(ints = {1, 2})
@@ -632,7 +632,7 @@
     catalog.createTable(identifier, SCHEMA);
     Table registeringTable = catalog.loadTable(identifier);
     TableOperations ops = ((HasTableOperations) registeringTable).operations();
-    String metadataLocation = ((HadoopTableOperations) ops).current().metadataFileLocation();
+    String metadataLocation = ops.current().metadataFileLocation();
     assertThat(catalog.registerTable(identifier2, metadataLocation)).isNotNull();
     assertThat(catalog.loadTable(identifier2)).isNotNull();
     assertThat(catalog.dropTable(identifier)).isTrue();
@@ -646,7 +646,7 @@
     catalog.createTable(identifier, SCHEMA);
     Table registeringTable = catalog.loadTable(identifier);
     TableOperations ops = ((HasTableOperations) registeringTable).operations();
-    String metadataLocation = ((HadoopTableOperations) ops).current().metadataFileLocation();
+    String metadataLocation = ops.current().metadataFileLocation();
     assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation))
         .isInstanceOf(AlreadyExistsException.class)
         .hasMessage("Table already exists: a.t1");
diff --git a/core/src/test/java/org/apache/iceberg/io/MockInputStream.java b/core/src/test/java/org/apache/iceberg/io/MockInputStream.java
index 17a6d53..00ce7fb 100644
--- a/core/src/test/java/org/apache/iceberg/io/MockInputStream.java
+++ b/core/src/test/java/org/apache/iceberg/io/MockInputStream.java
@@ -24,7 +24,7 @@
 
   static final byte[] TEST_ARRAY = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
 
-  private int[] lengths;
+  private final int[] lengths;
   private int current = 0;
 
   MockInputStream(int... actualReadLengths) {
diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java
index c0c9afa..148a483 100644
--- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java
+++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java
@@ -1130,7 +1130,7 @@
   private String createMetadataLocationViaJdbcCatalog(TableIdentifier identifier)
       throws SQLException {
     // temporary connection just to actually create a concrete metadata location
-    String jdbcUrl = null;
+    String jdbcUrl;
     try {
       java.nio.file.Path dbFile = Files.createTempFile("temp", "metadata");
       jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath();
diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java
index 9525001..4e542b8 100644
--- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java
+++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java
@@ -149,7 +149,7 @@
   static {
     StringBuilder sb = new StringBuilder();
     for (int i = 0; i < 200; i += 1) {
-      sb.append(UUID.randomUUID().toString());
+      sb.append(UUID.randomUUID());
     }
     TOO_LONG_FOR_STATS_PARQUET = sb.toString();
   }
diff --git a/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java b/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java
index 85ddeb7..fb93a79 100644
--- a/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java
+++ b/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java
@@ -217,8 +217,8 @@
 
   private static class TestTaskWriter extends BaseTaskWriter<Record> {
 
-    private RollingFileWriter dataWriter;
-    private RollingEqDeleteWriter deleteWriter;
+    private final RollingFileWriter dataWriter;
+    private final RollingEqDeleteWriter deleteWriter;
 
     private TestTaskWriter(
         PartitionSpec spec,
diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java
index f9db7f0..c51241b 100644
--- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java
+++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java
@@ -110,7 +110,7 @@
   private void cleanUp() {
     if (mock) {
       // clean up
-      TEST_RULE_FOR_MOCK_CLIENT.set(null);
+      TEST_RULE_FOR_MOCK_CLIENT.remove();
     } else {
       if (bucketCreated) {
         deleteBucket();
diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java
index 08ffacb..5f50dcc 100644
--- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java
+++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java
@@ -72,7 +72,7 @@
 
   public S3ObjectMetadata createFullMetadata() {
     S3ObjectMetadata metadata = new S3ObjectMetadata();
-    MessageDigest md = null;
+    MessageDigest md;
     try {
       md = MessageDigest.getInstance("MD5");
     } catch (NoSuchAlgorithmException e) {
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
index 06ac546..b96b47c 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
@@ -30,8 +30,8 @@
           // disable classloader check as Avro may cache class/object in the serializers.
           .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false);
 
-  private StreamExecutionEnvironment env;
-  private Table table;
+  private final StreamExecutionEnvironment env;
+  private final Table table;
 
   private Actions(StreamExecutionEnvironment env, Table table) {
     this.env = env;
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java
index 9876bb3..670abeb 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java
@@ -31,7 +31,7 @@
 
 public class RewriteDataFilesAction extends BaseRewriteDataFilesAction<RewriteDataFilesAction> {
 
-  private StreamExecutionEnvironment env;
+  private final StreamExecutionEnvironment env;
   private int maxParallelism;
 
   public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) {
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java
index 04e1683..f7e8e0c 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java
@@ -55,7 +55,7 @@
   public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) {
     DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString());
     LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType);
-    RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new));
+    RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0]));
     return new AvroGenericRecordToRowDataMapper(rowType);
   }
 }
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java
index 22dd238..05b943f 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java
@@ -43,7 +43,7 @@
 
   @Override
   public boolean isEmpty() {
-    return keyFrequency.size() == 0;
+    return keyFrequency.isEmpty();
   }
 
   @Override
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java
index 8f95e3e..8ef1f1f 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java
@@ -64,7 +64,7 @@
   public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) {
     DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString());
     LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType);
-    RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new));
+    RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0]));
     return new RowDataToAvroGenericRecordConverter(rowType, avroSchema);
   }
 }
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java
index e9e3c15..fef4ec4 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java
@@ -121,7 +121,7 @@
       LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId());
       return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition);
     } else {
-      Long lastConsumedSnapshotId = lastPosition != null ? lastPosition.snapshotId() : null;
+      Long lastConsumedSnapshotId = lastPosition.snapshotId();
       Snapshot toSnapshotInclusive =
           toSnapshotInclusive(
               lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount());
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java
index 024d0b1..26fbad4 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java
@@ -29,7 +29,7 @@
 public class IcebergEnumeratorState implements Serializable {
   @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition;
   private final Collection<IcebergSourceSplitState> pendingSplits;
-  private int[] enumerationSplitCountHistory;
+  private final int[] enumerationSplitCountHistory;
 
   public IcebergEnumeratorState(Collection<IcebergSourceSplitState> pendingSplits) {
     this(null, pendingSplits);
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java
index 95d6db2..f76f8a6 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java
@@ -174,10 +174,8 @@
   private static void serializeEnumerationSplitCountHistory(
       DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException {
     out.writeInt(enumerationSplitCountHistory.length);
-    if (enumerationSplitCountHistory.length > 0) {
-      for (int enumerationSplitCount : enumerationSplitCountHistory) {
-        out.writeInt(enumerationSplitCount);
-      }
+    for (int enumerationSplitCount : enumerationSplitCountHistory) {
+      out.writeInt(enumerationSplitCount);
     }
   }
 
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java
index a0395f2..8419696 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java
@@ -129,7 +129,7 @@
           if (count > utflen) {
             throw new UTFDataFormatException("malformed input: partial character at end");
           }
-          char2 = (int) bytearr[count - 1];
+          char2 = bytearr[count - 1];
           if ((char2 & 0xC0) != 0x80) {
             throw new UTFDataFormatException("malformed input around byte " + count);
           }
@@ -141,8 +141,8 @@
           if (count > utflen) {
             throw new UTFDataFormatException("malformed input: partial character at end");
           }
-          char2 = (int) bytearr[count - 2];
-          char3 = (int) bytearr[count - 1];
+          char2 = bytearr[count - 2];
+          char3 = bytearr[count - 1];
           if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
             throw new UTFDataFormatException("malformed input around byte " + (count - 1));
           }
diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
index 353cee5..20b33e6 100644
--- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
+++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
@@ -32,7 +32,7 @@
   /** Returns Flink version string like x.y.z */
   public static String version() {
     if (null == VERSION.get()) {
-      String detectedVersion = null;
+      String detectedVersion;
       try {
         detectedVersion = versionFromJar();
         // use unknown version in case exact implementation version can't be found from the jar
diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java
index 97ab787..e69e1ac 100644
--- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java
+++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java
@@ -39,7 +39,7 @@
 
 public class TestFlinkCatalogTablePartitions extends CatalogTestBase {
 
-  private String tableName = "test_table";
+  private final String tableName = "test_table";
 
   @Parameter(index = 2)
   private FileFormat format;
diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java
index 4ad302d..a7c58e5 100644
--- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java
+++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java
@@ -23,7 +23,7 @@
 import org.apache.iceberg.TestTables;
 
 public class TestTableLoader implements TableLoader {
-  private File dir;
+  private final File dir;
 
   public static TableLoader of(String dir) {
     return new TestTableLoader(dir);
diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java
index 90790b3..91d36aa 100644
--- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java
+++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java
@@ -26,6 +26,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.TableEnvironment;
 import org.apache.flink.table.api.TableResult;
@@ -89,7 +90,9 @@
     tables.forEach(t -> exec("DROP TABLE IF EXISTS %s", t.getField(0)));
     exec("USE CATALOG default_catalog");
     exec("DROP CATALOG IF EXISTS %s", catalogName);
-    Files.walk(warehouse).sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete);
+    try (Stream<Path> files = Files.walk(warehouse)) {
+      files.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete);
+    }
   }
 
   /**
diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java
index 7033fd3..187f2b0 100644
--- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java
+++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java
@@ -122,7 +122,7 @@
         new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS);
 
     assertThat(extractor.extractWatermark(split(0)))
-        .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue() / 1000L);
+        .isEqualTo(MIN_VALUES.get(0).get(columnName) / 1000L);
   }
 
   @TestTemplate
diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java
index b0ecb0c..c5328d8 100644
--- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java
+++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java
@@ -100,8 +100,7 @@
   @Override
   protected boolean isConnectionException(Exception e) {
     return super.isConnectionException(e)
-        || (e != null
-            && e instanceof MetaException
+        || (e instanceof MetaException
             && e.getMessage()
                 .contains("Got exception: org.apache.thrift.transport.TTransportException"));
   }
diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java
index dfad78d..da9ca21 100644
--- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java
+++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java
@@ -42,7 +42,7 @@
   private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaConverter.class);
 
   private int id;
-  private boolean autoConvert;
+  private final boolean autoConvert;
 
   private HiveSchemaConverter(boolean autoConvert) {
     this.autoConvert = autoConvert;
diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java
index a0b9e8f..5b318fc 100644
--- a/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java
+++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java
@@ -33,10 +33,10 @@
 
   private static final String DEFAULT_DELIMITER = ";";
 
-  private Connection connection;
+  private final Connection connection;
 
-  private boolean stopOnError;
-  private boolean autoCommit;
+  private final boolean stopOnError;
+  private final boolean autoCommit;
 
   private PrintWriter logWriter = new PrintWriter(System.out);
   private PrintWriter errorLogWriter = new PrintWriter(System.err);
@@ -110,7 +110,7 @@
     StringBuilder command = null;
     try {
       LineNumberReader lineReader = new LineNumberReader(reader);
-      String line = null;
+      String line;
       while ((line = lineReader.readLine()) != null) {
         if (command == null) {
           command = new StringBuilder();
@@ -118,13 +118,11 @@
         String trimmedLine = line.trim();
         if (trimmedLine.startsWith("--")) {
           println(trimmedLine);
-        } else if (trimmedLine.length() < 1 || trimmedLine.startsWith("//")) {
-          // Do nothing
-        } else if (trimmedLine.length() < 1 || trimmedLine.startsWith("--")) {
+        } else if (trimmedLine.isEmpty() || trimmedLine.startsWith("//")) {
           // Do nothing
         } else if (!fullLineDelimiter && trimmedLine.endsWith(getDelimiter())
             || fullLineDelimiter && trimmedLine.equals(getDelimiter())) {
-          command.append(line.substring(0, line.lastIndexOf(getDelimiter())));
+          command.append(line, 0, line.lastIndexOf(getDelimiter()));
           command.append(" ");
           Statement statement = conn.createStatement();
 
diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
index 95bf6c6..ccb8fc3 100644
--- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
+++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
@@ -93,7 +93,7 @@
  * Run all the tests from abstract of {@link CatalogTests} with few specific tests related to HIVE.
  */
 public class TestHiveCatalog extends CatalogTests<HiveCatalog> {
-  private static ImmutableMap meta =
+  private static final ImmutableMap meta =
       ImmutableMap.of(
           "owner", "apache",
           "group", "iceberg",
diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java
index 44707f2..61af2c7 100644
--- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java
+++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java
@@ -86,7 +86,7 @@
   private static HiveClientPool spyClientPool = null;
   private static CachedClientPool spyCachedClientPool = null;
   private static Configuration overriddenHiveConf;
-  private static AtomicReference<IMetaStoreClient> spyClientRef = new AtomicReference<>();
+  private static final AtomicReference<IMetaStoreClient> spyClientRef = new AtomicReference<>();
   private static IMetaStoreClient spyClient = null;
   HiveTableOperations ops = null;
   TableMetadata metadataV1 = null;
@@ -387,7 +387,7 @@
 
   /** Wraps an ArgumentCaptor to provide data based on the request */
   private class ShowLocksResponseElementWrapper extends ShowLocksResponseElement {
-    private ArgumentCaptor<LockRequest> wrapped;
+    private final ArgumentCaptor<LockRequest> wrapped;
 
     private ShowLocksResponseElementWrapper(ArgumentCaptor<LockRequest> wrapped) {
       this.wrapped = wrapped;
diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java b/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java
index 62ce493..585242c 100644
--- a/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java
+++ b/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java
@@ -41,7 +41,7 @@
 import org.apache.iceberg.types.Types.StructType;
 
 class Deserializer {
-  private FieldDeserializer fieldDeserializer;
+  private final FieldDeserializer fieldDeserializer;
 
   /**
    * Builder to create a Deserializer instance. Requires an Iceberg Schema and the Hive
@@ -279,8 +279,8 @@
    * the provided writerInspector.
    */
   private static class ObjectInspectorPair {
-    private ObjectInspector writerInspector;
-    private ObjectInspector sourceInspector;
+    private final ObjectInspector writerInspector;
+    private final ObjectInspector sourceInspector;
 
     ObjectInspectorPair(ObjectInspector writerInspector, ObjectInspector sourceInspector) {
       this.writerInspector = writerInspector;
diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
index f206bba..59bb38e 100644
--- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
+++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
@@ -54,8 +54,9 @@
 
   private ObjectInspector inspector;
   private Schema tableSchema;
-  private Map<ObjectInspector, Deserializer> deserializers = Maps.newHashMapWithExpectedSize(1);
-  private Container<Record> row = new Container<>();
+  private final Map<ObjectInspector, Deserializer> deserializers =
+      Maps.newHashMapWithExpectedSize(1);
+  private final Container<Record> row = new Container<>();
 
   @Override
   public void initialize(@Nullable Configuration configuration, Properties serDeProperties)
diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java b/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java
index 72475f7..f9f2334 100644
--- a/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java
+++ b/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java
@@ -20,7 +20,6 @@
 
 import java.io.IOException;
 import java.nio.file.Path;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -40,6 +39,7 @@
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 import org.junit.rules.TemporaryFolder;
 
 public class TestHelper {
@@ -152,7 +152,7 @@
 
   public static class RecordsBuilder {
 
-    private final List<Record> records = new ArrayList<Record>();
+    private final List<Record> records = Lists.newArrayList();
     private final Schema schema;
 
     private RecordsBuilder(Schema schema) {
diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
index 43a61d8..4e17794 100644
--- a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
+++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
@@ -39,6 +39,7 @@
 import java.util.List;
 import java.util.UUID;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
@@ -279,11 +280,14 @@
    */
   public static void validateFiles(Table table, Configuration conf, JobID jobId, int dataFileNum)
       throws IOException {
-    List<Path> dataFiles =
-        Files.walk(Paths.get(table.location() + "/data"))
-            .filter(Files::isRegularFile)
-            .filter(path -> !path.getFileName().toString().startsWith("."))
-            .collect(Collectors.toList());
+    List<Path> dataFiles;
+    try (Stream<Path> files = Files.walk(Paths.get(table.location() + "/data"))) {
+      dataFiles =
+          files
+              .filter(Files::isRegularFile)
+              .filter(path -> !path.getFileName().toString().startsWith("."))
+              .collect(Collectors.toList());
+    }
 
     assertThat(dataFiles).hasSize(dataFileNum);
     assertThat(
diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
index fd15a9a..9018d45 100644
--- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
+++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
@@ -781,7 +781,7 @@
       expected.addAll(data.get(partition));
     }
 
-    List<Object[]> descRows = shell.executeStatement("SELECT * FROM " + identifier.toString());
+    List<Object[]> descRows = shell.executeStatement("SELECT * FROM " + identifier);
     List<Record> records = HiveIcebergTestUtils.valueForRow(icebergTable.schema(), descRows);
 
     HiveIcebergTestUtils.validateData(expected, records, 0);
diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java
index 18fc4f3..ce3a6fd 100644
--- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java
+++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java
@@ -1341,14 +1341,15 @@
     } else if (type instanceof Types.StructType) {
       query.append("named_struct(");
       ((GenericRecord) field)
-          .struct().fields().stream()
-              .forEach(
-                  f ->
-                      query
-                          .append(buildComplexTypeInnerQuery(f.name(), Types.StringType.get()))
-                          .append(
-                              buildComplexTypeInnerQuery(
-                                  ((GenericRecord) field).getField(f.name()), f.type())));
+          .struct()
+          .fields()
+          .forEach(
+              f ->
+                  query
+                      .append(buildComplexTypeInnerQuery(f.name(), Types.StringType.get()))
+                      .append(
+                          buildComplexTypeInnerQuery(
+                              ((GenericRecord) field).getField(f.name()), f.type())));
       query.setLength(query.length() - 1);
       query.append("),");
     } else if (type instanceof Types.StringType) {
diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java
index 8c8cf89..f271029 100644
--- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java
+++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java
@@ -552,10 +552,9 @@
   private String getStringValueForInsert(Object value, Type type) {
     String template = "\'%s\'";
     if (type.equals(Types.TimestampType.withoutZone())) {
-      return String.format(template, Timestamp.valueOf((LocalDateTime) value).toString());
+      return String.format(template, Timestamp.valueOf((LocalDateTime) value));
     } else if (type.equals(Types.TimestampType.withZone())) {
-      return String.format(
-          template, Timestamp.from(((OffsetDateTime) value).toInstant()).toString());
+      return String.format(template, Timestamp.from(((OffsetDateTime) value).toInstant()));
     } else if (type.equals(Types.BooleanType.get())) {
       // in hive2 boolean type values must not be surrounded in apostrophes. Otherwise the value is
       // translated to true.
diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java
index 3fc38e4..8bf6bc1 100644
--- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java
+++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java
@@ -192,7 +192,7 @@
 
   private TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
     List<String> elements = entry.getName().getElements();
-    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+    return TableIdentifier.of(elements.toArray(new String[0]));
   }
 
   public IcebergTable table(TableIdentifier tableIdentifier) {
diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
index d56d07f..40e3375 100644
--- a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
+++ b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
@@ -172,7 +172,7 @@
       createMissingNamespaces(tableIdentifier);
       return catalog.createTable(tableIdentifier, schema(count));
     } catch (Throwable t) {
-      LOG.error("unable to do create " + tableIdentifier.toString(), t);
+      LOG.error("unable to do create {}", tableIdentifier, t);
       throw t;
     }
   }
diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java
index 05ee755..b1f844c 100644
--- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java
+++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java
@@ -168,7 +168,7 @@
         .hasMessageContaining("Namespace already exists: a");
 
     client.commitTable(
-        null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl"));
+        null, newTableMetadata(), "file:///tmp/iceberg", null, ContentKey.of("a", "tbl"));
 
     assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of()))
         .isInstanceOf(AlreadyExistsException.class)
@@ -284,7 +284,7 @@
     client.createNamespace(Namespace.of("a"), Map.of());
 
     client.commitTable(
-        null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl"));
+        null, newTableMetadata(), "file:///tmp/iceberg", null, ContentKey.of("a", "tbl"));
 
     assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a", "tbl")))
         .isInstanceOf(NoSuchNamespaceException.class)
diff --git a/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java b/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java
index 18ce07a..2dde66c 100644
--- a/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java
+++ b/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java
@@ -209,7 +209,7 @@
         Types.StructType structType,
         Map<Integer, ?> idToConstant) {
       super(readers, structType, idToConstant);
-      this.template = structType != null ? GenericRecord.create(structType) : null;
+      this.template = GenericRecord.create(structType);
     }
 
     @Override
diff --git a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java
index 609c8ae..2be6404 100644
--- a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java
+++ b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java
@@ -60,8 +60,8 @@
       ImmutableSet.of(
           TypeID.BINARY, TypeID.FIXED, TypeID.UUID, TypeID.STRUCT, TypeID.MAP, TypeID.LIST);
 
-  private SearchArgument.Builder builder;
-  private Map<Integer, String> idToColumnName;
+  private final SearchArgument.Builder builder;
+  private final Map<Integer, String> idToColumnName;
 
   private ExpressionToSearchArgument(
       SearchArgument.Builder builder, Map<Integer, String> idToColumnName) {
diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java
index 7d1405b..18186fe 100644
--- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java
+++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java
@@ -115,7 +115,7 @@
     private final Configuration conf;
     private Schema schema = null;
     private BiFunction<Schema, TypeDescription, OrcRowWriter<?>> createWriterFunc;
-    private Map<String, byte[]> metadata = Maps.newHashMap();
+    private final Map<String, byte[]> metadata = Maps.newHashMap();
     private MetricsConfig metricsConfig;
     private Function<Map<String, String>, Context> createContextFunc = Context::dataContext;
     private final Map<String, String> config = Maps.newLinkedHashMap();
diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
index fae1a76..ba6d6ee 100644
--- a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
+++ b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
@@ -232,7 +232,7 @@
             .map(Optional::get)
             .collect(Collectors.toList());
 
-    if (fields.size() == 0) {
+    if (fields.isEmpty()) {
       throw new IllegalArgumentException("ORC schema does not contain Iceberg IDs");
     }
 
@@ -326,9 +326,7 @@
         } else {
           if (isRequired) {
             throw new IllegalArgumentException(
-                String.format(
-                    "Field %d of type %s is required and was not found.",
-                    fieldId, type.toString()));
+                String.format("Field %d of type %s is required and was not found.", fieldId, type));
           }
 
           orcType = convert(fieldId, type, false);
diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java
index cfc2962..2941fb8 100644
--- a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java
+++ b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java
@@ -107,7 +107,7 @@
 
     try (Reader reader =
         OrcFile.createReader(
-            new Path(outFile.location()), new OrcFile.ReaderOptions(new Configuration())); ) {
+            new Path(outFile.location()), new OrcFile.ReaderOptions(new Configuration()))) {
       boolean[] readCols = new boolean[] {false, true, true, false};
       RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
       OrcIndex indices = rows.readRowIndex(0, null, readCols);
diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java
index 79585b0..38fd693 100644
--- a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java
+++ b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java
@@ -362,21 +362,19 @@
         case FIXED_LEN_BYTE_ARRAY:
           return new FixedReader(desc);
         case BINARY:
-          if (expected != null
-              && expected.typeId() == org.apache.iceberg.types.Type.TypeID.STRING) {
+          if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.STRING) {
             return new ParquetValueReaders.StringReader(desc);
           } else {
             return new ParquetValueReaders.BytesReader(desc);
           }
         case INT32:
-          if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+          if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
             return new ParquetValueReaders.IntAsLongReader(desc);
           } else {
             return new ParquetValueReaders.UnboxedReader<>(desc);
           }
         case FLOAT:
-          if (expected != null
-              && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+          if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
             return new ParquetValueReaders.FloatAsDoubleReader(desc);
           } else {
             return new ParquetValueReaders.UnboxedReader<>(desc);
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
index c97512a..3421c0b 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
@@ -346,7 +346,7 @@
         for (Map.Entry<String, String> entry : columnBloomFilterEnabled.entrySet()) {
           String colPath = entry.getKey();
           String bloomEnabled = entry.getValue();
-          propsBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled));
+          propsBuilder.withBloomFilterEnabled(colPath, Boolean.parseBoolean(bloomEnabled));
         }
 
         for (Map.Entry<String, String> entry : columnBloomFilterFpp.entrySet()) {
@@ -389,7 +389,7 @@
         for (Map.Entry<String, String> entry : columnBloomFilterEnabled.entrySet()) {
           String colPath = entry.getKey();
           String bloomEnabled = entry.getValue();
-          parquetWriteBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled));
+          parquetWriteBuilder.withBloomFilterEnabled(colPath, Boolean.parseBoolean(bloomEnabled));
         }
 
         for (Map.Entry<String, String> entry : columnBloomFilterFpp.entrySet()) {
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java
index 680eb14..65c4721 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java
@@ -301,7 +301,7 @@
     @Override
     public Schema primitive(Schema primitive) {
       LogicalType logicalType = primitive.getLogicalType();
-      if (logicalType != null && logicalType instanceof LogicalTypes.Decimal) {
+      if (logicalType instanceof LogicalTypes.Decimal) {
         LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType;
         if (decimal.getPrecision() <= 9) {
           return new ParquetDecimal(decimal.getPrecision(), decimal.getScale())
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
index 11201bd..553b8a0 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
@@ -277,7 +277,7 @@
 
     private <T> boolean shouldRead(
         PrimitiveType primitiveType, T value, BloomFilter bloom, Type type) {
-      long hashValue = 0;
+      long hashValue;
       switch (primitiveType.getPrimitiveTypeName()) {
         case INT32:
           switch (type.typeId()) {
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
index 1f6e855..17518a1 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
@@ -90,7 +90,7 @@
   }
 
   private static <T> T visitListElement(Type listElement, ParquetTypeVisitor<T> visitor) {
-    T elementResult = null;
+    T elementResult;
 
     visitor.beforeElementField(listElement);
     try {
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java
index a335fe1..62a49da 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java
@@ -343,7 +343,7 @@
   }
 
   public static class BinaryAsDecimalReader extends PrimitiveReader<BigDecimal> {
-    private int scale;
+    private final int scale;
 
     public BinaryAsDecimalReader(ColumnDescriptor desc, int scale) {
       super(desc);
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java
index 4eddf91..9076698 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java
@@ -371,7 +371,7 @@
         List<FieldMetrics<?>> fieldMetricsFromWriter =
             writer.metrics().collect(Collectors.toList());
 
-        if (fieldMetricsFromWriter.size() == 0) {
+        if (fieldMetricsFromWriter.isEmpty()) {
           // we are not tracking field metrics for this type ourselves
           return Stream.empty();
         } else if (fieldMetricsFromWriter.size() == 1) {
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java
index 049fed0..a4a3255 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java
@@ -38,7 +38,7 @@
 @Deprecated
 public class ParquetWriteAdapter<D> implements FileAppender<D> {
   private ParquetWriter<D> writer;
-  private MetricsConfig metricsConfig;
+  private final MetricsConfig metricsConfig;
   private ParquetMetadata footer;
 
   public ParquetWriteAdapter(ParquetWriter<D> writer, MetricsConfig metricsConfig) {
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java
index 773e0f7..35d94f3 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java
@@ -45,7 +45,7 @@
   private final ParquetReadOptions options;
   private final Function<MessageType, VectorizedReader<?>> batchReaderFunc;
   private final Expression filter;
-  private boolean reuseContainers;
+  private final boolean reuseContainers;
   private final boolean caseSensitive;
   private final int batchSize;
   private final NameMapping nameMapping;
diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java
index 09673e6..bced36a 100644
--- a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java
+++ b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java
@@ -71,7 +71,7 @@
       GenericData.Record... records)
       throws IOException {
 
-    long len = 0;
+    long len;
 
     FileAppender<GenericData.Record> writer =
         Parquet.write(localOutput(file))
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 5f64c72..2da9b2b 100644
--- a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
+++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
@@ -149,7 +149,7 @@
   static {
     StringBuilder sb = new StringBuilder();
     for (int i = 0; i < 200; i += 1) {
-      sb.append(UUID.randomUUID().toString());
+      sb.append(UUID.randomUUID());
     }
     TOO_LONG_FOR_STATS = sb.toString();
   }
diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java
index cf2c118..d690d3c 100644
--- a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java
+++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java
@@ -140,7 +140,7 @@
   static {
     StringBuilder sb = new StringBuilder();
     for (int i = 0; i < 200; i += 1) {
-      sb.append(UUID.randomUUID().toString());
+      sb.append(UUID.randomUUID());
     }
     TOO_LONG_FOR_STATS = sb.toString();
   }
diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java
index 5122166..5f3d3ac 100644
--- a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java
+++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java
@@ -70,8 +70,8 @@
 
   public static final String PIG_ICEBERG_TABLES_IMPL = "pig.iceberg.tables.impl";
   private static Tables iceberg;
-  private static Map<String, Table> tables = Maps.newConcurrentMap();
-  private static Map<String, String> locations = Maps.newConcurrentMap();
+  private static final Map<String, Table> tables = Maps.newConcurrentMap();
+  private static final Map<String, String> locations = Maps.newConcurrentMap();
 
   private String signature;
 
diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java
index d283fa9..e947db3 100644
--- a/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java
+++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java
@@ -191,13 +191,11 @@
 
     final String finalQuery = "SHOW TABLES IN SCHEMA IDENTIFIER(?) LIMIT 1";
 
-    List<SnowflakeIdentifier> tables;
     try {
-      tables =
-          connectionPool.run(
-              conn ->
-                  queryHarness.query(
-                      conn, finalQuery, TABLE_RESULT_SET_HANDLER, schema.toIdentifierString()));
+      connectionPool.run(
+          conn ->
+              queryHarness.query(
+                  conn, finalQuery, TABLE_RESULT_SET_HANDLER, schema.toIdentifierString()));
     } catch (SQLException e) {
       if (SCHEMA_NOT_FOUND_ERROR_CODES.contains(e.getErrorCode())) {
         return false;
diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java
index 06dacad..7f64b4c 100644
--- a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java
+++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java
@@ -176,7 +176,7 @@
   @Override
   public List<Namespace> listNamespaces(Namespace namespace) {
     SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
-    List<SnowflakeIdentifier> results = null;
+    List<SnowflakeIdentifier> results;
     switch (scope.type()) {
       case ROOT:
         results = snowflakeClient.listDatabases();
diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java
index c550b3e..512c818 100644
--- a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java
+++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java
@@ -93,7 +93,7 @@
   }
 
   public String toDebugString() {
-    return String.format("%s, rawJsonVal: %s", toString(), rawJsonVal);
+    return String.format("%s, rawJsonVal: %s", this, rawJsonVal);
   }
 
   /**
diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java
index b3d5cef..ecad072 100644
--- a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java
+++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java
@@ -89,7 +89,7 @@
         "s3://tab1/metadata/v3.metadata.json",
         TableMetadataParser.toJson(
                 TableMetadata.newTableMetadata(
-                    schema, partitionSpec, "s3://tab1", ImmutableMap.<String, String>of()))
+                    schema, partitionSpec, "s3://tab1", ImmutableMap.of()))
             .getBytes());
     fakeFileIO.addFile(
         "wasbs://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
@@ -98,13 +98,13 @@
                     schema,
                     partitionSpec,
                     "wasbs://mycontainer@myaccount.blob.core.windows.net/tab1/",
-                    ImmutableMap.<String, String>of()))
+                    ImmutableMap.of()))
             .getBytes());
     fakeFileIO.addFile(
         "gs://tab5/metadata/v793.metadata.json",
         TableMetadataParser.toJson(
                 TableMetadata.newTableMetadata(
-                    schema, partitionSpec, "gs://tab5/", ImmutableMap.<String, String>of()))
+                    schema, partitionSpec, "gs://tab5/", ImmutableMap.of()))
             .getBytes());
 
     fakeFileIOFactory =
diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java
index f00b942..7af9dfc 100644
--- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java
+++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java
@@ -255,7 +255,7 @@
   private Dataset<Row> toDS(String schema, String jsonData) {
     List<String> jsonRows =
         Arrays.stream(jsonData.split("\n"))
-            .filter(str -> str.trim().length() > 0)
+            .filter(str -> !str.trim().isEmpty())
             .collect(Collectors.toList());
     Dataset<String> jsonDS = spark.createDataset(jsonRows, Encoders.STRING());
 
diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java
index 63d24f7..d8f9301 100644
--- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java
+++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java
@@ -29,7 +29,7 @@
 
 class RandomGeneratingUDF implements Serializable {
   private final long uniqueValues;
-  private Random rand = new Random();
+  private final Random rand = new Random();
 
   RandomGeneratingUDF(long uniqueValues) {
     this.uniqueValues = uniqueValues;
@@ -43,8 +43,7 @@
 
   UserDefinedFunction randomString() {
     return udf(
-            () -> (String) RandomUtil.generatePrimitive(Types.StringType.get(), rand),
-            DataTypes.StringType)
+            () -> RandomUtil.generatePrimitive(Types.StringType.get(), rand), DataTypes.StringType)
         .asNondeterministic()
         .asNonNullable();
   }
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java
index bbbb46e..f76f123 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java
@@ -203,7 +203,7 @@
         "Cannot project a map of optional values as required values: %s",
         map);
     Preconditions.checkArgument(
-        StringType.class.isInstance(requestedMap.keyType()),
+        requestedMap.keyType() instanceof StringType,
         "Invalid map key type (not string): %s",
         requestedMap.keyType());
 
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java
index 39518ff..af0fa84 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java
@@ -888,7 +888,7 @@
             JavaConverters.collectionAsScalaIterableConverter(ImmutableList.of(rootPath))
                 .asScala()
                 .toSeq(),
-            scala.collection.immutable.Map$.MODULE$.<String, String>empty(),
+            scala.collection.immutable.Map$.MODULE$.empty(),
             userSpecifiedSchema,
             fileStatusCache,
             Option.empty(),
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
index 8447dbd..7a96e97 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java
@@ -183,7 +183,7 @@
       Option<scala.collection.immutable.Map<String, String>> scalaPartitionFilter;
       if (partitionFilter != null && !partitionFilter.isEmpty()) {
         Builder<Tuple2<String, String>, scala.collection.immutable.Map<String, String>> builder =
-            Map$.MODULE$.<String, String>newBuilder();
+            Map$.MODULE$.newBuilder();
         partitionFilter.forEach((key, value) -> builder.$plus$eq(Tuple2.apply(key, value)));
         scalaPartitionFilter = Option.apply(builder.result());
       } else {
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java
index b00ed42..5fbb411 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java
@@ -116,7 +116,7 @@
   private Map<String, String> equalSchemes = flattenMap(EQUAL_SCHEMES_DEFAULT);
   private Map<String, String> equalAuthorities = Collections.emptyMap();
   private PrefixMismatchMode prefixMismatchMode = PrefixMismatchMode.ERROR;
-  private String location = null;
+  private String location;
   private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3);
   private Dataset<Row> compareToFileList;
   private Consumer<String> deleteFunc = null;
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java
index e301ad5..d33e5e5 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java
@@ -393,7 +393,7 @@
   Stream<RewriteFileGroup> toGroupStream(
       RewriteExecutionContext ctx, Map<StructLike, List<List<FileScanTask>>> groupsByPartition) {
     return groupsByPartition.entrySet().stream()
-        .filter(e -> e.getValue().size() != 0)
+        .filter(e -> !e.getValue().isEmpty())
         .flatMap(
             e -> {
               StructLike partition = e.getKey();
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java
index e9edfeb..60e2b11 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java
@@ -103,9 +103,9 @@
   private final long targetManifestSizeBytes;
   private final boolean shouldStageManifests;
 
-  private PartitionSpec spec = null;
+  private PartitionSpec spec;
   private Predicate<ManifestFile> predicate = manifest -> true;
-  private String outputLocation = null;
+  private String outputLocation;
 
   RewriteManifestsSparkAction(SparkSession spark, Table table) {
     super(spark);
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java
index 1166740..282222a 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java
@@ -309,7 +309,7 @@
     // stop commit service
     commitService.close();
     List<RewritePositionDeletesGroup> commitResults = commitService.results();
-    if (commitResults.size() == 0) {
+    if (commitResults.isEmpty()) {
       LOG.error(
           "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
               + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
@@ -331,7 +331,7 @@
       RewriteExecutionContext ctx,
       Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
     return groupsByPartition.entrySet().stream()
-        .filter(e -> e.getValue().size() != 0)
+        .filter(e -> !e.getValue().isEmpty())
         .flatMap(
             e -> {
               StructLike partition = e.getKey();
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java
index 209c06b..678ebd2 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java
@@ -560,7 +560,7 @@
 
     private InternalRowWriter(List<ParquetValueWriter<?>> writers, List<DataType> types) {
       super(writers);
-      this.types = types.toArray(new DataType[types.size()]);
+      this.types = types.toArray(new DataType[0]);
     }
 
     @Override
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java
index 9b075b6..50a1259 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java
@@ -48,7 +48,7 @@
   private StructType dataSparkType;
   private StructType equalityDeleteSparkType;
   private StructType positionDeleteSparkType;
-  private Map<String, String> writeProperties;
+  private final Map<String, String> writeProperties;
 
   SparkFileWriterFactory(
       Table table,
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java
index 320d2e1..49180e0 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java
@@ -395,7 +395,6 @@
         Snapshot nextValid = nextValidSnapshot(curSnapshot);
         if (nextValid == null) {
           // nextValide implies all the remaining snapshots should be skipped.
-          shouldContinueReading = false;
           break;
         }
         // we found the next available snapshot, continue from there.
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
index b430e6f..6b97e48 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
@@ -95,7 +95,7 @@
   private final List<String> metaColumns = Lists.newArrayList();
   private final InMemoryMetricsReporter metricsReporter;
 
-  private Schema schema = null;
+  private Schema schema;
   private boolean caseSensitive;
   private List<Expression> filterExpressions = null;
   private Predicate[] pushedPredicates = NO_PREDICATES;
diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java
index 2539388..c5c86c3 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java
@@ -44,7 +44,7 @@
   private final SparkReadConf readConf;
   private final List<String> metaColumns = Lists.newArrayList();
 
-  private Schema schema = null;
+  private Schema schema;
 
   SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) {
     this.spark = spark;
diff --git a/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java b/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java
index 075733a..9ed7167 100644
--- a/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java
+++ b/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java
@@ -32,7 +32,7 @@
         Option.empty(),
         Option.empty(),
         Option.empty(),
-        Map$.MODULE$.<String, String>empty(),
+        Map$.MODULE$.empty(),
         new QueryContext[0]);
   }
 }
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
index e7d5a0f..40bc4bb 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java
@@ -118,7 +118,7 @@
 
   protected List<Object[]> sql(String query, Object... args) {
     List<Row> rows = spark.sql(String.format(query, args)).collectAsList();
-    if (rows.size() < 1) {
+    if (rows.isEmpty()) {
       return ImmutableList.of();
     }
 
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java
index d0860ff..8f4c093 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java
@@ -38,7 +38,7 @@
 import org.junit.jupiter.api.TestTemplate;
 
 public class TestSparkCatalogOperations extends CatalogTestBase {
-  private static boolean useNullableQuerySchema = ThreadLocalRandom.current().nextBoolean();
+  private static final boolean useNullableQuerySchema = ThreadLocalRandom.current().nextBoolean();
 
   @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
   protected static Object[][] parameters() {
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java
index db77da4..a0f1fba 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java
@@ -141,7 +141,7 @@
     };
   }
 
-  private String baseTableName = "baseTable";
+  private final String baseTableName = "baseTable";
   private File tableDir;
   private String tableLocation;
 
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java
index 478afcf..360b9ff 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java
@@ -67,7 +67,7 @@
   public static Iterable<InternalRow> generateSpark(Schema schema, int numRecords, long seed) {
     return () ->
         new Iterator<InternalRow>() {
-          private SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed);
+          private final SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed);
           private int count = 0;
 
           @Override
@@ -114,7 +114,7 @@
     return () ->
         new Iterator<Record>() {
           private int count = 0;
-          private RandomDataGenerator generator = newGenerator.get();
+          private final RandomDataGenerator generator = newGenerator.get();
 
           @Override
           public boolean hasNext() {
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java
index 53a35ee..2627ec0 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java
@@ -22,7 +22,7 @@
 import java.util.concurrent.atomic.AtomicInteger;
 
 public class LogMessage {
-  private static AtomicInteger idCounter = new AtomicInteger(0);
+  private static final AtomicInteger idCounter = new AtomicInteger(0);
 
   static LogMessage debug(String date, String message) {
     return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message);
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java
index 27e7d7c..7258344 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java
@@ -87,7 +87,7 @@
   // Main reader class to test base class iteration logic.
   // Keeps track of iterator closure.
   private static class ClosureTrackingReader extends BaseReader<Integer, FileScanTask> {
-    private Map<String, CloseableIntegerRange> tracker = Maps.newHashMap();
+    private final Map<String, CloseableIntegerRange> tracker = Maps.newHashMap();
 
     ClosureTrackingReader(Table table, List<FileScanTask> tasks) {
       super(table, new BaseCombinedScanTask(tasks), null, null, false);
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java
index 34315bc..44885eb 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java
@@ -245,7 +245,7 @@
         return footer.getBlocks().get(0).getColumns().get(0).getCodec().name();
       default:
         FileContext fc = FileContext.getFileContext(CONF);
-        GenericDatumReader<Object> reader = new GenericDatumReader<Object>();
+        GenericDatumReader<Object> reader = new GenericDatumReader<>();
         DataFileReader fileReader =
             (DataFileReader)
                 DataFileReader.openReader(
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java
index e702f1f..336ee5a 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java
@@ -91,7 +91,7 @@
 
   private Map<String, String> tableProperties;
 
-  private org.apache.spark.sql.types.StructType sparkSchema =
+  private final org.apache.spark.sql.types.StructType sparkSchema =
       new org.apache.spark.sql.types.StructType(
           new org.apache.spark.sql.types.StructField[] {
             new org.apache.spark.sql.types.StructField(
@@ -106,16 +106,16 @@
                 org.apache.spark.sql.types.Metadata.empty())
           });
 
-  private Schema icebergSchema =
+  private final Schema icebergSchema =
       new Schema(
           Types.NestedField.optional(1, "optionalField", Types.StringType.get()),
           Types.NestedField.required(2, "requiredField", Types.StringType.get()));
 
-  private List<String> data0 =
+  private final List<String> data0 =
       Arrays.asList(
           "{\"optionalField\": \"a1\", \"requiredField\": \"bid_001\"}",
           "{\"optionalField\": \"a2\", \"requiredField\": \"bid_002\"}");
-  private List<String> data1 =
+  private final List<String> data1 =
       Arrays.asList(
           "{\"optionalField\": \"d1\", \"requiredField\": \"bid_101\"}",
           "{\"optionalField\": \"d2\", \"requiredField\": \"bid_102\"}",
@@ -220,16 +220,15 @@
     final int numPartitions = 10;
     final int partitionToFail = new Random().nextInt(numPartitions);
     MapPartitionsFunction<Row, Row> failOnFirstPartitionFunc =
-        (MapPartitionsFunction<Row, Row>)
-            input -> {
-              int partitionId = TaskContext.getPartitionId();
+        input -> {
+          int partitionId = TaskContext.getPartitionId();
 
-              if (partitionId == partitionToFail) {
-                throw new SparkException(
-                    String.format("Intended exception in partition %d !", partitionId));
-              }
-              return input;
-            };
+          if (partitionId == partitionToFail) {
+            throw new SparkException(
+                String.format("Intended exception in partition %d !", partitionId));
+          }
+          return input;
+        };
 
     Dataset<Row> df =
         createDataset(records, schema)
@@ -287,8 +286,8 @@
         .startsWith("2");
 
     File location = temp.resolve("parquet").resolve("test").toFile();
-    String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString());
-    String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString());
+    String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location);
+    String targetPath = String.format("%s/nullable_poc/targetFolder/", location);
 
     tableProperties = ImmutableMap.of(TableProperties.WRITE_DATA_LOCATION, targetPath);
 
@@ -341,8 +340,8 @@
         .startsWith("2");
 
     File location = temp.resolve("parquet").resolve("test").toFile();
-    String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString());
-    String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString());
+    String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location);
+    String targetPath = String.format("%s/nullable_poc/targetFolder/", location);
 
     tableProperties = ImmutableMap.of(TableProperties.WRITE_DATA_LOCATION, targetPath);
 
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java
index 0febfd6..35a6750 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java
@@ -134,7 +134,7 @@
 
   @TempDir private Path temp;
 
-  private PartitionSpec spec =
+  private final PartitionSpec spec =
       PartitionSpec.builderFor(LOG_SCHEMA).identity("date").identity("level").build();
   private Table table = null;
   private Dataset<Row> logs = null;
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java
index 2bd6dad..9464f68 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java
@@ -170,7 +170,7 @@
 
   @TempDir private java.nio.file.Path temp;
 
-  private PartitionSpec spec =
+  private final PartitionSpec spec =
       PartitionSpec.builderFor(LOG_SCHEMA)
           .identity("date")
           .identity("level")
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java
index 3d9d6eb..17db46b 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java
@@ -115,7 +115,7 @@
       query.stop();
 
       // remove the last commit to force Spark to reprocess batch #1
-      File lastCommitFile = new File(checkpoint.toString() + "/commits/1");
+      File lastCommitFile = new File(checkpoint + "/commits/1");
       assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue();
 
       // restart the query from the checkpoint
@@ -176,7 +176,7 @@
       query.stop();
 
       // remove the last commit to force Spark to reprocess batch #1
-      File lastCommitFile = new File(checkpoint.toString() + "/commits/1");
+      File lastCommitFile = new File(checkpoint + "/commits/1");
       assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue();
 
       // restart the query from the checkpoint
@@ -237,7 +237,7 @@
       query.stop();
 
       // remove the last commit to force Spark to reprocess batch #1
-      File lastCommitFile = new File(checkpoint.toString() + "/commits/1");
+      File lastCommitFile = new File(checkpoint + "/commits/1");
       assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue();
 
       // restart the query from the checkpoint
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java
index a2a315f..ae0aa2c 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java
@@ -295,7 +295,7 @@
     File tableLocation = Files.createTempDirectory(temp, "junit").toFile();
     assertThat(tableLocation.delete()).isTrue();
 
-    String location = "file:" + tableLocation.toString();
+    String location = "file:" + tableLocation;
 
     sql(
         "CREATE TABLE %s "
diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java
index ae9aee8..0ba4806 100644
--- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java
+++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java
@@ -64,7 +64,7 @@
         SparkCatalogConfig.HADOOP.catalogName(),
         SparkCatalogConfig.HADOOP.implementation(),
         SparkCatalogConfig.HADOOP.properties(),
-        SparkCatalogConfig.HADOOP.catalogName() + "." + NS.toString(),
+        SparkCatalogConfig.HADOOP.catalogName() + "." + NS,
         true
       },
       {