diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml
index 47353e1a5d695..c0f672f24d099 100644
--- a/plugin/trino-iceberg/pom.xml
+++ b/plugin/trino-iceberg/pom.xml
@@ -25,7 +25,7 @@
-->
instances
- 0.59.0
+ 0.71.0
@@ -249,11 +249,6 @@
parquet-hadoop
-
- org.gaul
- modernizer-maven-annotations
-
-
org.jdbi
jdbi3-core
diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java
index d3b25589f2d12..4530ed950dc0c 100644
--- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java
+++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java
@@ -37,7 +37,7 @@
import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
-import static org.apache.iceberg.parquet.TrinoParquetUtil.footerMetrics;
+import static org.apache.iceberg.parquet.ParquetUtil.footerMetrics;
public final class IcebergParquetFileWriter
implements IcebergFileWriter
diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieCatalogModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieCatalogModule.java
index 809c39589bd5f..f0e5de3ec30a5 100644
--- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieCatalogModule.java
+++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieCatalogModule.java
@@ -48,7 +48,6 @@ public static NessieIcebergClient createNessieIcebergClient(IcebergNessieCatalog
return new NessieIcebergClient(
HttpClientBuilder.builder()
.withUri(icebergNessieCatalogConfig.getServerUri())
- .withEnableApiCompatibilityCheck(false)
.build(NessieApiV1.class),
icebergNessieCatalogConfig.getDefaultReferenceName(),
null,
diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java
index 9ab4470b836c2..6c57853c9e4b8 100644
--- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java
+++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java
@@ -67,7 +67,7 @@
import org.apache.iceberg.mapping.MappingUtil;
import org.apache.iceberg.mapping.NameMapping;
import org.apache.iceberg.orc.OrcMetrics;
-import org.apache.iceberg.parquet.TrinoParquetUtil;
+import org.apache.iceberg.parquet.ParquetUtil;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
@@ -382,7 +382,7 @@ private static Metrics loadMetrics(TrinoInputFile file, HiveStorageFormat storag
InputFile inputFile = new ForwardingInputFile(file);
return switch (storageFormat) {
case ORC -> OrcMetrics.fromInputFile(inputFile, METRICS_CONFIG, nameMapping);
- case PARQUET -> TrinoParquetUtil.fileMetrics(inputFile, METRICS_CONFIG, nameMapping);
+ case PARQUET -> ParquetUtil.fileMetrics(inputFile, METRICS_CONFIG, nameMapping);
case AVRO -> new Metrics(Avro.rowCount(inputFile), null, null, null, null);
default -> throw new TrinoException(NOT_SUPPORTED, "Unsupported storage format: " + storageFormat);
};
diff --git a/plugin/trino-iceberg/src/main/java/org/apache/iceberg/parquet/TrinoParquetUtil.java b/plugin/trino-iceberg/src/main/java/org/apache/iceberg/parquet/TrinoParquetUtil.java
deleted file mode 100644
index 19a6f5f29614e..0000000000000
--- a/plugin/trino-iceberg/src/main/java/org/apache/iceberg/parquet/TrinoParquetUtil.java
+++ /dev/null
@@ -1,450 +0,0 @@
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.iceberg.parquet;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.iceberg.FieldMetrics;
-import org.apache.iceberg.Metrics;
-import org.apache.iceberg.MetricsConfig;
-import org.apache.iceberg.MetricsModes;
-import org.apache.iceberg.MetricsModes.MetricsMode;
-import org.apache.iceberg.MetricsUtil;
-import org.apache.iceberg.Schema;
-import org.apache.iceberg.exceptions.RuntimeIOException;
-import org.apache.iceberg.expressions.Literal;
-import org.apache.iceberg.io.InputFile;
-import org.apache.iceberg.mapping.NameMapping;
-import org.apache.iceberg.types.Conversions;
-import org.apache.iceberg.types.Type;
-import org.apache.iceberg.types.Types;
-import org.apache.iceberg.util.BinaryUtil;
-import org.apache.iceberg.util.UnicodeUtil;
-import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.Dictionary;
-import org.apache.parquet.column.Encoding;
-import org.apache.parquet.column.EncodingStats;
-import org.apache.parquet.column.page.DictionaryPage;
-import org.apache.parquet.column.page.PageReader;
-import org.apache.parquet.column.statistics.Statistics;
-import org.apache.parquet.hadoop.ParquetFileReader;
-import org.apache.parquet.hadoop.metadata.BlockMetaData;
-import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
-import org.apache.parquet.hadoop.metadata.ColumnPath;
-import org.apache.parquet.hadoop.metadata.ParquetMetadata;
-import org.apache.parquet.io.ParquetDecodingException;
-import org.apache.parquet.schema.MessageType;
-import org.apache.parquet.schema.PrimitiveType;
-import org.gaul.modernizer_maven_annotations.SuppressModernizer;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-// Copied from https://github.com/apache/iceberg/blob/master/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java
-@Deprecated // This class will be removed after new Iceberg library including https://github.com/apache/iceberg/pull/8559 is released
-@SuppressModernizer
-public class TrinoParquetUtil // Renamed from ParquetUtil for avoiding duplicate resources
-{
- // not meant to be instantiated
- private TrinoParquetUtil() {}
-
- private static final long UNIX_EPOCH_JULIAN = 2_440_588L;
-
- public static Metrics fileMetrics(InputFile file, MetricsConfig metricsConfig)
- {
- return fileMetrics(file, metricsConfig, null);
- }
-
- public static Metrics fileMetrics(
- InputFile file, MetricsConfig metricsConfig, NameMapping nameMapping)
- {
- try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(file))) {
- return footerMetrics(reader.getFooter(), Stream.empty(), metricsConfig, nameMapping);
- }
- catch (IOException e) {
- throw new RuntimeIOException(e, "Failed to read footer of file: %s", file);
- }
- }
-
- public static Metrics footerMetrics(
- ParquetMetadata metadata, Stream> fieldMetrics, MetricsConfig metricsConfig)
- {
- return footerMetrics(metadata, fieldMetrics, metricsConfig, null);
- }
-
- @SuppressWarnings("checkstyle:CyclomaticComplexity")
- public static Metrics footerMetrics(
- ParquetMetadata metadata,
- Stream> fieldMetrics,
- MetricsConfig metricsConfig,
- NameMapping nameMapping)
- {
- checkNotNull(fieldMetrics, "fieldMetrics should not be null");
-
- long rowCount = 0;
- Map columnSizes = Maps.newHashMap();
- Map valueCounts = Maps.newHashMap();
- Map nullValueCounts = Maps.newHashMap();
- Map> lowerBounds = Maps.newHashMap();
- Map> upperBounds = Maps.newHashMap();
- Set missingStats = Sets.newHashSet();
-
- // ignore metrics for fields we failed to determine reliable IDs
- MessageType parquetTypeWithIds = getParquetTypeWithIds(metadata, nameMapping);
- Schema fileSchema = ParquetSchemaUtil.convertAndPrune(parquetTypeWithIds);
-
- Map> fieldMetricsMap =
- fieldMetrics.collect(Collectors.toMap(FieldMetrics::id, Function.identity()));
-
- List blocks = metadata.getBlocks();
- for (BlockMetaData block : blocks) {
- rowCount += block.getRowCount();
- for (ColumnChunkMetaData column : block.getColumns()) {
- Integer fieldId = fileSchema.aliasToId(column.getPath().toDotString());
- if (fieldId == null) {
- // fileSchema may contain a subset of columns present in the file
- // as we prune columns we could not assign ids
- continue;
- }
-
- increment(columnSizes, fieldId, column.getTotalSize());
-
- MetricsMode metricsMode = MetricsUtil.metricsMode(fileSchema, metricsConfig, fieldId);
- if (metricsMode == MetricsModes.None.get()) {
- continue;
- }
- increment(valueCounts, fieldId, column.getValueCount());
-
- Statistics stats = column.getStatistics();
- if (stats != null && !stats.isEmpty()) {
- increment(nullValueCounts, fieldId, stats.getNumNulls());
-
- // when there are metrics gathered by Iceberg for a column, we should use those instead
- // of the ones from Parquet
- if (metricsMode != MetricsModes.Counts.get() && !fieldMetricsMap.containsKey(fieldId)) {
- Types.NestedField field = fileSchema.findField(fieldId);
- if (field != null && stats.hasNonNullValue() && shouldStoreBounds(column, fileSchema)) {
- Literal> min =
- ParquetConversions.fromParquetPrimitive(
- field.type(), column.getPrimitiveType(), stats.genericGetMin());
- updateMin(lowerBounds, fieldId, field.type(), min, metricsMode);
- Literal> max =
- ParquetConversions.fromParquetPrimitive(
- field.type(), column.getPrimitiveType(), stats.genericGetMax());
- updateMax(upperBounds, fieldId, field.type(), max, metricsMode);
- }
- }
- }
- else if (!stats.isEmpty()) {
- missingStats.add(fieldId);
- }
- }
- }
-
- // discard accumulated values if any stats were missing
- for (Integer fieldId : missingStats) {
- nullValueCounts.remove(fieldId);
- lowerBounds.remove(fieldId);
- upperBounds.remove(fieldId);
- }
-
- updateFromFieldMetrics(fieldMetricsMap, metricsConfig, fileSchema, lowerBounds, upperBounds);
-
- return new Metrics(
- rowCount,
- columnSizes,
- valueCounts,
- nullValueCounts,
- MetricsUtil.createNanValueCounts(
- fieldMetricsMap.values().stream(), metricsConfig, fileSchema),
- toBufferMap(fileSchema, lowerBounds),
- toBufferMap(fileSchema, upperBounds));
- }
-
- private static void updateFromFieldMetrics(
- Map> idToFieldMetricsMap,
- MetricsConfig metricsConfig,
- Schema schema,
- Map> lowerBounds,
- Map> upperBounds)
- {
- idToFieldMetricsMap
- .entrySet()
- .forEach(
- entry -> {
- int fieldId = entry.getKey();
- FieldMetrics> metrics = entry.getValue();
- MetricsMode metricsMode = MetricsUtil.metricsMode(schema, metricsConfig, fieldId);
-
- // only check for MetricsModes.None, since we don't truncate float/double values.
- if (metricsMode != MetricsModes.None.get()) {
- if (!metrics.hasBounds()) {
- lowerBounds.remove(fieldId);
- upperBounds.remove(fieldId);
- }
- else if (metrics.upperBound() instanceof Float) {
- lowerBounds.put(fieldId, Literal.of((Float) metrics.lowerBound()));
- upperBounds.put(fieldId, Literal.of((Float) metrics.upperBound()));
- }
- else if (metrics.upperBound() instanceof Double) {
- lowerBounds.put(fieldId, Literal.of((Double) metrics.lowerBound()));
- upperBounds.put(fieldId, Literal.of((Double) metrics.upperBound()));
- }
- else {
- throw new UnsupportedOperationException(
- "Expected only float or double column metrics");
- }
- }
- });
- }
-
- private static MessageType getParquetTypeWithIds(
- ParquetMetadata metadata, NameMapping nameMapping)
- {
- MessageType type = metadata.getFileMetaData().getSchema();
-
- if (ParquetSchemaUtil.hasIds(type)) {
- return type;
- }
-
- if (nameMapping != null) {
- return ParquetSchemaUtil.applyNameMapping(type, nameMapping);
- }
-
- return ParquetSchemaUtil.addFallbackIds(type);
- }
-
- /**
- * Returns a list of offsets in ascending order determined by the starting position of the row
- * groups.
- */
- public static List getSplitOffsets(ParquetMetadata md)
- {
- List splitOffsets = Lists.newArrayListWithExpectedSize(md.getBlocks().size());
- for (BlockMetaData blockMetaData : md.getBlocks()) {
- splitOffsets.add(blockMetaData.getStartingPos());
- }
- Collections.sort(splitOffsets);
- return splitOffsets;
- }
-
- // we allow struct nesting, but not maps or arrays
- private static boolean shouldStoreBounds(ColumnChunkMetaData column, Schema schema)
- {
- if (column.getPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT96) {
- // stats for INT96 are not reliable
- return false;
- }
-
- ColumnPath columnPath = column.getPath();
- Iterator pathIterator = columnPath.iterator();
- Type currentType = schema.asStruct();
-
- while (pathIterator.hasNext()) {
- if (currentType == null || !currentType.isStructType()) {
- return false;
- }
- String fieldName = pathIterator.next();
- currentType = currentType.asStructType().fieldType(fieldName);
- }
-
- return currentType != null && currentType.isPrimitiveType();
- }
-
- private static void increment(Map columns, int fieldId, long amount)
- {
- if (columns != null) {
- if (columns.containsKey(fieldId)) {
- columns.put(fieldId, columns.get(fieldId) + amount);
- }
- else {
- columns.put(fieldId, amount);
- }
- }
- }
-
- @SuppressWarnings("unchecked")
- private static void updateMin(
- Map> lowerBounds,
- int id,
- Type type,
- Literal min,
- MetricsMode metricsMode)
- {
- Literal currentMin = (Literal) lowerBounds.get(id);
- if (currentMin == null || min.comparator().compare(min.value(), currentMin.value()) < 0) {
- if (metricsMode == MetricsModes.Full.get()) {
- lowerBounds.put(id, min);
- }
- else {
- MetricsModes.Truncate truncateMode = (MetricsModes.Truncate) metricsMode;
- int truncateLength = truncateMode.length();
- switch (type.typeId()) {
- case STRING:
- lowerBounds.put(
- id, UnicodeUtil.truncateStringMin((Literal) min, truncateLength));
- break;
- case FIXED:
- case BINARY:
- lowerBounds.put(
- id, BinaryUtil.truncateBinaryMin((Literal) min, truncateLength));
- break;
- default:
- lowerBounds.put(id, min);
- }
- }
- }
- }
-
- @SuppressWarnings("unchecked")
- private static void updateMax(
- Map> upperBounds,
- int id,
- Type type,
- Literal max,
- MetricsMode metricsMode)
- {
- Literal currentMax = (Literal) upperBounds.get(id);
- if (currentMax == null || max.comparator().compare(max.value(), currentMax.value()) > 0) {
- if (metricsMode == MetricsModes.Full.get()) {
- upperBounds.put(id, max);
- }
- else {
- MetricsModes.Truncate truncateMode = (MetricsModes.Truncate) metricsMode;
- int truncateLength = truncateMode.length();
- switch (type.typeId()) {
- case STRING:
- Literal truncatedMaxString =
- UnicodeUtil.truncateStringMax((Literal) max, truncateLength);
- if (truncatedMaxString != null) {
- upperBounds.put(id, truncatedMaxString);
- }
- break;
- case FIXED:
- case BINARY:
- Literal truncatedMaxBinary =
- BinaryUtil.truncateBinaryMax((Literal) max, truncateLength);
- if (truncatedMaxBinary != null) {
- upperBounds.put(id, truncatedMaxBinary);
- }
- break;
- default:
- upperBounds.put(id, max);
- }
- }
- }
- }
-
- private static Map toBufferMap(Schema schema, Map> map)
- {
- Map bufferMap = Maps.newHashMap();
- for (Map.Entry> entry : map.entrySet()) {
- bufferMap.put(
- entry.getKey(),
- Conversions.toByteBuffer(schema.findType(entry.getKey()), entry.getValue().value()));
- }
- return bufferMap;
- }
-
- @SuppressWarnings("deprecation")
- public static boolean hasNonDictionaryPages(ColumnChunkMetaData meta)
- {
- EncodingStats stats = meta.getEncodingStats();
- if (stats != null) {
- return stats.hasNonDictionaryEncodedPages();
- }
-
- // without EncodingStats, fall back to testing the encoding list
- Set encodings = Sets.newHashSet(meta.getEncodings());
- if (encodings.remove(Encoding.PLAIN_DICTIONARY)) {
- // if remove returned true, PLAIN_DICTIONARY was present, which means at
- // least one page was dictionary encoded and 1.0 encodings are used
-
- // RLE and BIT_PACKED are only used for repetition or definition levels
- encodings.remove(Encoding.RLE);
- encodings.remove(Encoding.BIT_PACKED);
-
- // when empty, no encodings other than dictionary or rep/def levels
- return !encodings.isEmpty();
- }
- else {
- // if PLAIN_DICTIONARY wasn't present, then either the column is not
- // dictionary-encoded, or the 2.0 encoding, RLE_DICTIONARY, was used.
- // for 2.0, this cannot determine whether a page fell back without
- // page encoding stats
- return true;
- }
- }
-
- public static boolean hasNoBloomFilterPages(ColumnChunkMetaData meta)
- {
- return meta.getBloomFilterOffset() <= 0;
- }
-
- public static Dictionary readDictionary(ColumnDescriptor desc, PageReader pageSource)
- {
- DictionaryPage dictionaryPage = pageSource.readDictionaryPage();
- if (dictionaryPage != null) {
- try {
- return dictionaryPage.getEncoding().initDictionary(desc, dictionaryPage);
- }
- catch (IOException e) {
- throw new ParquetDecodingException("could not decode the dictionary for " + desc, e);
- }
- }
- return null;
- }
-
- public static boolean isIntType(PrimitiveType primitiveType)
- {
- if (primitiveType.getOriginalType() != null) {
- switch (primitiveType.getOriginalType()) {
- case INT_8:
- case INT_16:
- case INT_32:
- case DATE:
- return true;
- default:
- return false;
- }
- }
- return primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT32;
- }
-
- /**
- * Method to read timestamp (parquet Int96) from bytebuffer. Read 12 bytes in byteBuffer: 8 bytes
- * (time of day nanos) + 4 bytes(julianDay)
- */
- public static long extractTimestampInt96(ByteBuffer buffer)
- {
- // 8 bytes (time of day nanos)
- long timeOfDayNanos = buffer.getLong();
- // 4 bytes(julianDay)
- int julianDay = buffer.getInt();
- return TimeUnit.DAYS.toMicros(julianDay - UNIX_EPOCH_JULIAN)
- + TimeUnit.NANOSECONDS.toMicros(timeOfDayNanos);
- }
-}
diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java
index 1a48e32799952..cf63e56125072 100644
--- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java
+++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java
@@ -4704,7 +4704,7 @@ protected void verifyIcebergTableProperties(MaterializedResult actual)
assertThat(actual).isNotNull();
MaterializedResult expected = resultBuilder(getSession())
.row("write.format.default", format.name())
- .build();
+ .row("write.parquet.compression-codec", "zstd").build();
assertEqualsIgnoreOrder(actual.getMaterializedRows(), expected.getMaterializedRows());
}
diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestPartitionFields.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestPartitionFields.java
index 527fe1bd36eee..3cd07dd337dac 100644
--- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestPartitionFields.java
+++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestPartitionFields.java
@@ -71,10 +71,10 @@ public void testParse()
assertInvalid("bucket()", "Invalid partition field declaration: bucket()");
assertInvalid("abc", "Cannot find source column: abc");
assertInvalid("notes", "Cannot partition by non-primitive source field: list");
- assertInvalid("bucket(price, 42)", "Cannot bucket by type: double");
- assertInvalid("bucket(notes, 88)", "Cannot bucket by type: list");
- assertInvalid("truncate(ts, 13)", "Cannot truncate type: timestamp");
- assertInvalid("year(order_key)", "Cannot partition type long by year");
+ assertInvalid("bucket(price, 42)", "Invalid source type double for transform: bucket[42]");
+ assertInvalid("bucket(notes, 88)", "Cannot partition by non-primitive source field: list");
+ assertInvalid("truncate(ts, 13)", "Invalid source type timestamp for transform: truncate[13]");
+ assertInvalid("year(order_key)", "Invalid source type long for transform: year");
assertInvalid("\"test\"", "Cannot find source column: test");
assertInvalid("\"test with space\"", "Cannot find source column: test with space");
assertInvalid("\"test \"with space\"", "Invalid partition field declaration: \"test \"with space\"");
diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java
index aa7cf7e5b22a6..47761caea762f 100644
--- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java
+++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java
@@ -51,6 +51,8 @@ public class TestIcebergFileMetastoreCreateTableFailure
private static final String ICEBERG_CATALOG = "iceberg";
private static final String SCHEMA_NAME = "test_schema";
+ private static final String METADATA_GLOB = "glob:**.metadata.json";
+
private Path dataDirectory;
private HiveMetastore metastore;
private final AtomicReference testException = new AtomicReference<>();
@@ -125,10 +127,11 @@ protected void testCreateTableFailure(String expectedExceptionMessage, boolean s
Path metadataDirectory = Path.of(tableLocation, "metadata");
if (shouldMetadataFileExist) {
- assertThat(metadataDirectory).as("Metadata file should exist").isDirectoryContaining("glob:**.metadata.json");
+ assertThat(metadataDirectory).as("Metadata file should exist").isDirectoryContaining(METADATA_GLOB);
}
else {
- assertThat(metadataDirectory).as("Metadata file should not exist").isEmptyDirectory();
+ // file cleanup is more conservative since https://github.com/apache/iceberg/pull/8599
+ assertThat(metadataDirectory).as("Metadata file should not exist").isDirectoryNotContaining(METADATA_GLOB);
}
}
}
diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/NessieContainer.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/NessieContainer.java
index f41213c894733..d5f27d8f0ff92 100644
--- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/NessieContainer.java
+++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/NessieContainer.java
@@ -28,7 +28,7 @@ public class NessieContainer
{
private static final Logger log = Logger.get(NessieContainer.class);
- public static final String DEFAULT_IMAGE = "projectnessie/nessie:0.59.0";
+ public static final String DEFAULT_IMAGE = "projectnessie/nessie:0.71.0";
public static final String DEFAULT_HOST_NAME = "nessie";
public static final String VERSION_STORE_TYPE = "INMEMORY";
diff --git a/pom.xml b/pom.xml
index 55e50d7382b1e..37dea5e40b4af 100644
--- a/pom.xml
+++ b/pom.xml
@@ -176,7 +176,7 @@
3.6.0
4.17.0
8.5.6
- 1.3.1
+ 1.4.1
3.24.4
4.5.0
4.1.100.Final
diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIcebergNessie.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIcebergNessie.java
index 8634536fe25ac..4c3a9857083ca 100644
--- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIcebergNessie.java
+++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIcebergNessie.java
@@ -37,7 +37,7 @@ public class EnvSinglenodeSparkIcebergNessie
{
private static final int SPARK_THRIFT_PORT = 10213;
private static final int NESSIE_PORT = 19120;
- private static final String NESSIE_VERSION = "0.59.0";
+ private static final String NESSIE_VERSION = "0.71.0";
private static final String SPARK = "spark";
private final DockerFiles dockerFiles;