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;