diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index f4019d688cb8..526bb42ea687 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -262,7 +262,7 @@ private boolean fileContent(BoundReference ref) { return ref.fieldId() == DataFile.CONTENT.fieldId(); } - private boolean contentMatch(Integer fileContentId) { + private boolean contentMatch(Integer fileContentId) { if (FileContent.DATA.id() == fileContentId) { return ManifestContent.DATA.id() == manifestContentId; } else if (FileContent.EQUALITY_DELETES.id() == fileContentId diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java index fb3c27220cb2..cea7003c1a38 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java @@ -229,7 +229,8 @@ protected long numOutputFiles(long inputSize) { // the remainder file is of a valid size for this rewrite so keep it return numFilesWithRemainder; - } else if (avgFileSizeWithoutRemainder < Math.min(1.1 * targetFileSize, writeMaxFileSize())) { + } else if (avgFileSizeWithoutRemainder + < Math.min(1.1 * targetFileSize, (double) writeMaxFileSize())) { // if the reminder is distributed amongst other files, // the average file size will be no more than 10% bigger than the target file size // so round down and distribute remainder amongst other files diff --git a/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java b/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java index aadb97bc7112..263b3c305af0 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java +++ b/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java @@ -149,7 +149,7 @@ public TimeValue getRetryInterval(HttpResponse response, int execCount, HttpCont } } - int delayMillis = 1000 * (int) Math.min(Math.pow(2.0, (long) execCount - 1), 64.0); + int delayMillis = 1000 * (int) Math.min(Math.pow(2.0, (long) execCount - 1.0), 64.0); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMillis * 0.1))); return TimeValue.ofMilliseconds(delayMillis + jitter); 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 bd3a934f6f04..e36321c8e2c9 100644 --- a/core/src/main/java/org/apache/iceberg/util/Pair.java +++ b/core/src/main/java/org/apache/iceberg/util/Pair.java @@ -58,11 +58,6 @@ public Schema load(Pair, Class> key) { private X first; private Y second; - /** Constructor used by Avro */ - private Pair(Schema schema) { - this.schema = schema; - } - private Pair(X first, Y second) { this.first = first; this.second = second; diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 16fa6f3d8537..27cd96a39733 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -101,6 +101,7 @@ private ParallelIterator( } @Override + @SuppressWarnings("FutureReturnValueIgnored") public void close() { // close first, avoid new task submit this.closed.set(true); 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 02d2b834311f..14804e040755 100644 --- a/core/src/main/java/org/apache/iceberg/util/Tasks.java +++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java @@ -450,7 +450,9 @@ private void runTaskWithRetry(Task task, I item) thr } int delayMs = - (int) Math.min(minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), maxSleepTimeMs); + (int) + Math.min( + minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), (double) maxSleepTimeMs); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMs * 0.1))); LOG.warn("Retrying task after failure: {}", e.getMessage(), e); 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 18b452f98367..df96b90eb728 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -102,6 +102,7 @@ public ViewMetadata refresh() { } @Override + @SuppressWarnings("ImmutablesReferenceEquality") public void commit(ViewMetadata base, ViewMetadata metadata) { // if the metadata is already out of date, reject it if (base != current()) { diff --git a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index 007b423e592a..592e7ff16241 100644 --- a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.nio.charset.StandardCharsets; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -139,7 +140,7 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - return prefix + new String(buffer); + return prefix + new String(buffer, StandardCharsets.UTF_8); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */