From 91879249d1990fac4481c2f478d0f0c40e1a3d61 Mon Sep 17 00:00:00 2001 From: Kurtis Wright Date: Thu, 9 Apr 2026 18:14:50 +0000 Subject: [PATCH 01/16] Add unknown type support to Spark 3.4 and 3.5 Map Iceberg's UnknownType to Spark's NullType in both directions: - TypeToSparkType: UNKNOWN -> NullType (Iceberg to Spark) - SparkTypeToType: NullType -> UnknownType (Spark to Iceberg) This aligns Spark 3.x with the existing Spark 4.x behavior and allows reading v3 tables with unknown-typed columns without throwing UnsupportedOperationException. Spark has supported NullType since 2.x. --- .../main/java/org/apache/iceberg/spark/SparkTypeToType.java | 4 ++++ .../main/java/org/apache/iceberg/spark/TypeToSparkType.java | 5 ++++- .../main/java/org/apache/iceberg/spark/SparkTypeToType.java | 4 ++++ .../main/java/org/apache/iceberg/spark/TypeToSparkType.java | 5 ++++- 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index 8beaefc5cc8f..488e2c290ea0 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -35,6 +35,7 @@ import org.apache.spark.sql.types.IntegerType; import org.apache.spark.sql.types.LongType; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType; import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StringType; import org.apache.spark.sql.types.StructField; @@ -155,6 +156,9 @@ public Type atomic(DataType atomic) { ((DecimalType) atomic).precision(), ((DecimalType) atomic).scale()); } else if (atomic instanceof BinaryType) { return Types.BinaryType.get(); + + } else if (atomic instanceof NullType) { + return Types.UnknownType.get(); } throw new UnsupportedOperationException("Not a supported type: " + atomic.catalogString()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index dfb9b30be603..692a89f8a530 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -36,6 +36,7 @@ import org.apache.spark.sql.types.IntegerType$; import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.MapType$; +import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.StringType$; @@ -124,9 +125,11 @@ public DataType primitive(Type.PrimitiveType primitive) { case DECIMAL: Types.DecimalType decimal = (Types.DecimalType) primitive; return DecimalType$.MODULE$.apply(decimal.precision(), decimal.scale()); + case UNKNOWN: + return NullType$.MODULE$; default: throw new UnsupportedOperationException( - "Cannot convert unknown type to Spark: " + primitive); + "Cannot convert unsupported type to Spark: " + primitive); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index 8beaefc5cc8f..488e2c290ea0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -35,6 +35,7 @@ import org.apache.spark.sql.types.IntegerType; import org.apache.spark.sql.types.LongType; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType; import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StringType; import org.apache.spark.sql.types.StructField; @@ -155,6 +156,9 @@ public Type atomic(DataType atomic) { ((DecimalType) atomic).precision(), ((DecimalType) atomic).scale()); } else if (atomic instanceof BinaryType) { return Types.BinaryType.get(); + + } else if (atomic instanceof NullType) { + return Types.UnknownType.get(); } throw new UnsupportedOperationException("Not a supported type: " + atomic.catalogString()); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index dfb9b30be603..692a89f8a530 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -36,6 +36,7 @@ import org.apache.spark.sql.types.IntegerType$; import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.MapType$; +import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.StringType$; @@ -124,9 +125,11 @@ public DataType primitive(Type.PrimitiveType primitive) { case DECIMAL: Types.DecimalType decimal = (Types.DecimalType) primitive; return DecimalType$.MODULE$.apply(decimal.precision(), decimal.scale()); + case UNKNOWN: + return NullType$.MODULE$; default: throw new UnsupportedOperationException( - "Cannot convert unknown type to Spark: " + primitive); + "Cannot convert unsupported type to Spark: " + primitive); } } From d9d5a0342d0ac76e2c09e86a618c1ae7e4f240d0 Mon Sep 17 00:00:00 2001 From: Kurtis Wright Date: Thu, 9 Apr 2026 18:14:50 +0000 Subject: [PATCH 02/16] Spark: Add unknown type support to Spark 3.4 and 3.5 Map Iceberg's UnknownType to Spark's NullType in both directions: - TypeToSparkType: UNKNOWN -> NullType (Iceberg to Spark) - SparkTypeToType: NullType -> UnknownType (Spark to Iceberg) This aligns Spark 3.x with the existing Spark 4.x behavior and allows reading v3 tables with unknown-typed columns without throwing UnsupportedOperationException. Spark has supported NullType since 2.x. --- .../apache/iceberg/spark/SparkTypeToType.java | 1 + .../apache/iceberg/spark/TypeToSparkType.java | 1 + .../iceberg/spark/TestSparkSchemaUtil.java | 16 ++++++++++++++++ .../apache/iceberg/spark/SparkTypeToType.java | 1 + .../apache/iceberg/spark/TypeToSparkType.java | 1 + .../iceberg/spark/TestSparkSchemaUtil.java | 16 ++++++++++++++++ 6 files changed, 36 insertions(+) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index 488e2c290ea0..d4e440129af1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -154,6 +154,7 @@ public Type atomic(DataType atomic) { } else if (atomic instanceof DecimalType) { return Types.DecimalType.of( ((DecimalType) atomic).precision(), ((DecimalType) atomic).scale()); + } else if (atomic instanceof BinaryType) { return Types.BinaryType.get(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index 692a89f8a530..704b8d458115 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -39,6 +39,7 @@ import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; +import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType$; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java index 9b5b207a5b6b..0846cf6f1161 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java @@ -24,9 +24,11 @@ import java.util.List; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.expressions.AttributeReference; import org.apache.spark.sql.catalyst.expressions.MetadataAttribute; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.Test; @@ -79,4 +81,18 @@ public void testSchemaConversionWithMetaDataColumnSchema() { } } } + + @Test + public void testUnknownTypeToSpark() { + Schema schema = new Schema(optional(1, "col", Types.UnknownType.get())); + StructType sparkType = SparkSchemaUtil.convert(schema); + assertThat(sparkType.fields()[0].dataType()).isEqualTo(DataTypes.NullType); + } + + @Test + public void testNullTypeToIceberg() { + StructType sparkType = new StructType().add("col", DataTypes.NullType, true); + Type icebergType = SparkSchemaUtil.convert(sparkType).findField("col").type(); + assertThat(icebergType).isEqualTo(Types.UnknownType.get()); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index 488e2c290ea0..d4e440129af1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -154,6 +154,7 @@ public Type atomic(DataType atomic) { } else if (atomic instanceof DecimalType) { return Types.DecimalType.of( ((DecimalType) atomic).precision(), ((DecimalType) atomic).scale()); + } else if (atomic instanceof BinaryType) { return Types.BinaryType.get(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index 692a89f8a530..704b8d458115 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -39,6 +39,7 @@ import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; +import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType$; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java index 4045847d5a4a..b8f436cf2d86 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java @@ -24,10 +24,12 @@ import java.util.List; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.expressions.AttributeReference; import org.apache.spark.sql.catalyst.expressions.MetadataAttribute; import org.apache.spark.sql.catalyst.types.DataTypeUtils; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.Test; @@ -80,4 +82,18 @@ public void testSchemaConversionWithMetaDataColumnSchema() { } } } + + @Test + public void testUnknownTypeToSpark() { + Schema schema = new Schema(optional(1, "col", Types.UnknownType.get())); + StructType sparkType = SparkSchemaUtil.convert(schema); + assertThat(sparkType.fields()[0].dataType()).isEqualTo(DataTypes.NullType); + } + + @Test + public void testNullTypeToIceberg() { + StructType sparkType = new StructType().add("col", DataTypes.NullType, true); + Type icebergType = SparkSchemaUtil.convert(sparkType).findField("col").type(); + assertThat(icebergType).isEqualTo(Types.UnknownType.get()); + } } From b3401450007f44d0fd5f5eff28ced39159c6a223 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 21 Apr 2026 09:46:04 +0800 Subject: [PATCH 03/16] Add .factorypath to .gitignore (#16067) --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 1f5a8efe37f5..98ccfc356d8b 100644 --- a/.gitignore +++ b/.gitignore @@ -58,6 +58,7 @@ coverage.xml # vscode/eclipse files .classpath +.factorypath .project .settings bin/ From 08b0b43aa7bfe81453e5528052abd379309d08c5 Mon Sep 17 00:00:00 2001 From: drexler-sky Date: Tue, 21 Apr 2026 03:38:48 -0700 Subject: [PATCH 04/16] Spark: Replace deprecated registerTempTable with createOrReplaceTempView (#16063) --- .../org/apache/iceberg/spark/actions/TestCreateActions.java | 2 +- .../org/apache/iceberg/spark/actions/TestCreateActions.java | 2 +- .../org/apache/iceberg/spark/actions/TestCreateActions.java | 2 +- .../org/apache/iceberg/spark/actions/TestCreateActions.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index dd751499df30..9b0fecdaae41 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -667,7 +667,7 @@ public void schemaEvolutionTestWithSparkSQL() throws Exception { "CAST(id AS FLOAT) col1", "CAST(id AS STRING) col2", "CAST(id AS INT) col3") - .registerTempTable("tempdata"); + .createOrReplaceTempView("tempdata"); sql("INSERT INTO TABLE %s SELECT * FROM tempdata", tblName); List expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY col0", tblName); List expectedAfterAddColumn = 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 eb89b0a23274..50afb53e0539 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 @@ -662,7 +662,7 @@ public void schemaEvolutionTestWithSparkSQL() throws Exception { "CAST(id AS FLOAT) col1", "CAST(id AS STRING) col2", "CAST(id AS INT) col3") - .registerTempTable("tempdata"); + .createOrReplaceTempView("tempdata"); sql("INSERT INTO TABLE %s SELECT * FROM tempdata", tblName); List expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY col0", tblName); List expectedAfterAddColumn = diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index eb89b0a23274..50afb53e0539 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -662,7 +662,7 @@ public void schemaEvolutionTestWithSparkSQL() throws Exception { "CAST(id AS FLOAT) col1", "CAST(id AS STRING) col2", "CAST(id AS INT) col3") - .registerTempTable("tempdata"); + .createOrReplaceTempView("tempdata"); sql("INSERT INTO TABLE %s SELECT * FROM tempdata", tblName); List expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY col0", tblName); List expectedAfterAddColumn = diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index eb89b0a23274..50afb53e0539 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -662,7 +662,7 @@ public void schemaEvolutionTestWithSparkSQL() throws Exception { "CAST(id AS FLOAT) col1", "CAST(id AS STRING) col2", "CAST(id AS INT) col3") - .registerTempTable("tempdata"); + .createOrReplaceTempView("tempdata"); sql("INSERT INTO TABLE %s SELECT * FROM tempdata", tblName); List expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY col0", tblName); List expectedAfterAddColumn = From dd703abbd4f7f1fd4f0e9992edd4dc17a8971a88 Mon Sep 17 00:00:00 2001 From: Mervyn Lobo Date: Tue, 21 Apr 2026 21:30:24 +0530 Subject: [PATCH 05/16] AWS: Add proxy system property and environment variable configuration for HTTP clients (#15506) --- .../aws/ApacheHttpClientConfigurations.java | 33 +++++++++++++++-- .../iceberg/aws/HttpClientProperties.java | 24 ++++++++++++ ...UrlConnectionHttpClientConfigurations.java | 33 +++++++++++++++-- .../aws/TestHttpClientConfigurations.java | 37 +++++++++++++++++++ docs/docs/aws.md | 10 +++-- 5 files changed, 127 insertions(+), 10 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java b/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java index 3445928d1551..30065c8db510 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java @@ -41,6 +41,8 @@ class ApacheHttpClientConfigurations extends BaseHttpClientConfigurations { private Boolean tcpKeepAliveEnabled; private Boolean useIdleConnectionReaperEnabled; private String proxyEndpoint; + private Boolean proxyUseSystemPropertyValues; + private Boolean proxyUseEnvironmentVariableValues; private ApacheHttpClientConfigurations() {} @@ -82,6 +84,12 @@ private void initialize(Map httpClientProperties) { this.proxyEndpoint = PropertyUtil.propertyAsString( httpClientProperties, HttpClientProperties.PROXY_ENDPOINT, null); + this.proxyUseSystemPropertyValues = + PropertyUtil.propertyAsNullableBoolean( + httpClientProperties, HttpClientProperties.PROXY_USE_SYSTEM_PROPERTY_VALUES); + this.proxyUseEnvironmentVariableValues = + PropertyUtil.propertyAsNullableBoolean( + httpClientProperties, HttpClientProperties.PROXY_USE_ENVIRONMENT_VARIABLE_VALUES); } @VisibleForTesting @@ -113,9 +121,26 @@ void configureApacheHttpClientBuilder(ApacheHttpClient.Builder apacheHttpClientB if (useIdleConnectionReaperEnabled != null) { apacheHttpClientBuilder.useIdleConnectionReaper(useIdleConnectionReaperEnabled); } - if (proxyEndpoint != null) { - apacheHttpClientBuilder.proxyConfiguration( - ProxyConfiguration.builder().endpoint(URI.create(proxyEndpoint)).build()); + configureProxy(apacheHttpClientBuilder); + } + + private void configureProxy(ApacheHttpClient.Builder apacheHttpClientBuilder) { + if (proxyEndpoint != null + || proxyUseSystemPropertyValues != null + || proxyUseEnvironmentVariableValues != null) { + ProxyConfiguration.Builder proxyBuilder = ProxyConfiguration.builder(); + + if (proxyEndpoint != null) { + proxyBuilder.endpoint(URI.create(proxyEndpoint)); + } + if (proxyUseSystemPropertyValues != null) { + proxyBuilder.useSystemPropertyValues(proxyUseSystemPropertyValues); + } + if (proxyUseEnvironmentVariableValues != null) { + proxyBuilder.useEnvironmentVariableValues(proxyUseEnvironmentVariableValues); + } + + apacheHttpClientBuilder.proxyConfiguration(proxyBuilder.build()); } } @@ -138,6 +163,8 @@ protected String generateHttpClientCacheKey() { keyComponents.put("tcpKeepAliveEnabled", tcpKeepAliveEnabled); keyComponents.put("useIdleConnectionReaperEnabled", useIdleConnectionReaperEnabled); keyComponents.put("proxyEndpoint", proxyEndpoint); + keyComponents.put("proxyUseSystemPropertyValues", proxyUseSystemPropertyValues); + keyComponents.put("proxyUseEnvironmentVariableValues", proxyUseEnvironmentVariableValues); return keyComponents.entrySet().stream() .map(entry -> entry.getKey() + "=" + Objects.toString(entry.getValue(), "null")) diff --git a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java index 438ae5bb0431..870d8e23651c 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java @@ -61,6 +61,30 @@ public class HttpClientProperties implements Serializable { */ public static final String PROXY_ENDPOINT = "http-client.proxy-endpoint"; + /** + * Used to enable reading proxy configuration from Java system properties (http.proxyHost, + * http.proxyPort, http.nonProxyHosts, etc.). Default is true. + * + *

For more details, see + * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/ProxyConfiguration.html + * and + * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ProxyConfiguration.html + */ + public static final String PROXY_USE_SYSTEM_PROPERTY_VALUES = + "http-client.proxy-use-system-property-values"; + + /** + * Used to enable reading proxy configuration from environment variables (HTTP_PROXY, HTTPS_PROXY, + * NO_PROXY, etc.). Default is true. + * + *

For more details, see + * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/ProxyConfiguration.html + * and + * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ProxyConfiguration.html + */ + public static final String PROXY_USE_ENVIRONMENT_VARIABLE_VALUES = + "http-client.proxy-use-environment-variable-values"; + /** * Used to configure the connection timeout in milliseconds for {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only diff --git a/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java b/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java index 273baa674804..fbd845852ca9 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java @@ -35,6 +35,8 @@ class UrlConnectionHttpClientConfigurations extends BaseHttpClientConfigurations private Long httpClientUrlConnectionConnectionTimeoutMs; private Long httpClientUrlConnectionSocketTimeoutMs; private String proxyEndpoint; + private Boolean proxyUseSystemPropertyValues; + private Boolean proxyUseEnvironmentVariableValues; private UrlConnectionHttpClientConfigurations() {} @@ -56,6 +58,12 @@ private void initialize(Map httpClientProperties) { this.proxyEndpoint = PropertyUtil.propertyAsString( httpClientProperties, HttpClientProperties.PROXY_ENDPOINT, null); + this.proxyUseSystemPropertyValues = + PropertyUtil.propertyAsNullableBoolean( + httpClientProperties, HttpClientProperties.PROXY_USE_SYSTEM_PROPERTY_VALUES); + this.proxyUseEnvironmentVariableValues = + PropertyUtil.propertyAsNullableBoolean( + httpClientProperties, HttpClientProperties.PROXY_USE_ENVIRONMENT_VARIABLE_VALUES); } @VisibleForTesting @@ -69,9 +77,26 @@ void configureUrlConnectionHttpClientBuilder( urlConnectionHttpClientBuilder.socketTimeout( Duration.ofMillis(httpClientUrlConnectionSocketTimeoutMs)); } - if (proxyEndpoint != null) { - urlConnectionHttpClientBuilder.proxyConfiguration( - ProxyConfiguration.builder().endpoint(URI.create(proxyEndpoint)).build()); + configureProxy(urlConnectionHttpClientBuilder); + } + + private void configureProxy(UrlConnectionHttpClient.Builder urlConnectionHttpClientBuilder) { + if (proxyEndpoint != null + || proxyUseSystemPropertyValues != null + || proxyUseEnvironmentVariableValues != null) { + ProxyConfiguration.Builder proxyBuilder = ProxyConfiguration.builder(); + + if (proxyEndpoint != null) { + proxyBuilder.endpoint(URI.create(proxyEndpoint)); + } + if (proxyUseSystemPropertyValues != null) { + proxyBuilder.useSystemPropertyValues(proxyUseSystemPropertyValues); + } + if (proxyUseEnvironmentVariableValues != null) { + proxyBuilder.useEnvironmentVariablesValues(proxyUseEnvironmentVariableValues); + } + + urlConnectionHttpClientBuilder.proxyConfiguration(proxyBuilder.build()); } } @@ -87,6 +112,8 @@ protected String generateHttpClientCacheKey() { keyComponents.put("connectionTimeoutMs", httpClientUrlConnectionConnectionTimeoutMs); keyComponents.put("socketTimeoutMs", httpClientUrlConnectionSocketTimeoutMs); keyComponents.put("proxyEndpoint", proxyEndpoint); + keyComponents.put("proxyUseSystemPropertyValues", proxyUseSystemPropertyValues); + keyComponents.put("proxyUseEnvironmentVariableValues", proxyUseEnvironmentVariableValues); return keyComponents.entrySet().stream() .map(entry -> entry.getKey() + "=" + Objects.toString(entry.getValue(), "null")) diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java index 0f96ac0f6c82..da73a5c1b5a5 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java @@ -22,6 +22,8 @@ import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; import software.amazon.awssdk.http.apache.ApacheHttpClient; import software.amazon.awssdk.http.apache.ProxyConfiguration; @@ -137,4 +139,39 @@ public void testApacheDefaultConfigurations() { Mockito.verify(spyApacheHttpClientBuilder, Mockito.never()) .proxyConfiguration(Mockito.any(ProxyConfiguration.class)); } + + @ParameterizedTest + @ValueSource( + strings = { + HttpClientProperties.PROXY_USE_SYSTEM_PROPERTY_VALUES, + HttpClientProperties.PROXY_USE_ENVIRONMENT_VARIABLE_VALUES + }) + public void testApacheProxyFlagTriggersProxyConfig(String propertyKey) { + Map properties = Maps.newHashMap(); + properties.put(propertyKey, "false"); + ApacheHttpClient.Builder spy = Mockito.spy(ApacheHttpClient.builder()); + + ApacheHttpClientConfigurations.create(properties).configureApacheHttpClientBuilder(spy); + + Mockito.verify(spy).proxyConfiguration(Mockito.any(ProxyConfiguration.class)); + } + + @ParameterizedTest + @ValueSource( + strings = { + HttpClientProperties.PROXY_USE_SYSTEM_PROPERTY_VALUES, + HttpClientProperties.PROXY_USE_ENVIRONMENT_VARIABLE_VALUES + }) + public void testUrlConnectionProxyFlagTriggersProxyConfig(String propertyKey) { + Map properties = Maps.newHashMap(); + properties.put(propertyKey, "false"); + UrlConnectionHttpClient.Builder spy = Mockito.spy(UrlConnectionHttpClient.builder()); + + UrlConnectionHttpClientConfigurations.create(properties) + .configureUrlConnectionHttpClientBuilder(spy); + + Mockito.verify(spy) + .proxyConfiguration( + Mockito.any(software.amazon.awssdk.http.urlconnection.ProxyConfiguration.class)); + } } diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 1fe867401296..587de402b069 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -705,10 +705,12 @@ For more details of configuration, see sections [URL Connection HTTP Client Conf Configurations for the HTTP client can be set via catalog properties. Below is an overview of available configurations: -| Property | Default | Description | -|----------------------------|---------|------------------------------------------------------------------------------------------------------------| -| http-client.type | apache | Types of HTTP Client.
`urlconnection`: URL Connection HTTP Client
`apache`: Apache HTTP Client | -| http-client.proxy-endpoint | null | An optional proxy endpoint to use for the HTTP client. | +| Property | Default | Description | +|---------------------------------------------------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| http-client.type | apache | Types of HTTP Client.
`urlconnection`: URL Connection HTTP Client
`apache`: Apache HTTP Client | +| http-client.proxy-endpoint | null | An optional proxy endpoint to use for the HTTP client. | +| http-client.proxy-use-system-property-values | null, enabled by default | An optional `true/false` setting that controls whether proxy configuration is read from Java system properties (`http.proxyHost`, `http.proxyPort`, `http.nonProxyHosts`, etc.). | +| http-client.proxy-use-environment-variable-values | null, enabled by default | An optional `true/false` setting that controls whether proxy configuration is read from environment variables (`HTTP_PROXY`, `HTTPS_PROXY`, `NO_PROXY`, etc.). | #### URL Connection HTTP Client Configurations From 70147491589840d222c31f69d65fae32e7e40366 Mon Sep 17 00:00:00 2001 From: kumarpritam863 <148938310+kumarpritam863@users.noreply.github.com> Date: Tue, 21 Apr 2026 23:38:50 +0530 Subject: [PATCH 06/16] Kafka Connect: Do not fail if no partitions assigned (#15955) --------- Co-authored-by: Pritam Kumar Mishra Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../connect/channel/CommitterImpl.java | 72 +++++++++++-------- .../iceberg/connect/channel/Coordinator.java | 32 +++++---- 2 files changed, 62 insertions(+), 42 deletions(-) diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java index 04602a66a5e1..7b2d4a25363d 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java @@ -30,7 +30,6 @@ import org.apache.kafka.clients.admin.ConsumerGroupDescription; import org.apache.kafka.clients.admin.MemberDescription; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; import org.slf4j.Logger; @@ -48,6 +47,7 @@ public class CommitterImpl implements Committer { private KafkaClientFactory clientFactory; private Collection membersWhenWorkerIsCoordinator; private final AtomicBoolean isInitialized = new AtomicBoolean(false); + private String taskId; private void initialize( Catalog icebergCatalog, @@ -58,6 +58,7 @@ private void initialize( this.config = icebergSinkConfig; this.context = sinkTaskContext; this.clientFactory = new KafkaClientFactory(config.kafkaProps()); + this.taskId = config.connectorName() + "-" + config.taskId(); } } @@ -92,16 +93,38 @@ boolean hasLeaderPartition(Collection currentAssignedPartitions) @VisibleForTesting boolean containsFirstPartition( Collection members, Collection partitions) { - // there should only be one task assigned partition 0 of the first topic, - // so elect that one the leader - TopicPartition firstTopicPartition = - members.stream() - .flatMap(member -> member.assignment().topicPartitions().stream()) - .min(new TopicPartitionComparator()) - .orElseThrow( - () -> new ConnectException("No partitions assigned, cannot determine leader")); - - return partitions.contains(firstTopicPartition); + // Determine the first partition across all members to elect the leader + TopicPartition firstTopicPartition = findFirstTopicPartition(members); + + if (firstTopicPartition == null) { + LOG.warn( + "Committer {} found no partitions assigned across all members, cannot determine leader", + taskId); + return false; + } + + boolean containsFirst = partitions.contains(firstTopicPartition); + if (containsFirst) { + LOG.info( + "Committer {} contains the first partition {}, this task is the leader", + taskId, + firstTopicPartition); + } else { + LOG.debug( + "Committer {} does not contain the first partition {}, not the leader", + taskId, + firstTopicPartition); + } + + return containsFirst; + } + + @VisibleForTesting + TopicPartition findFirstTopicPartition(Collection members) { + return members.stream() + .flatMap(member -> member.assignment().topicPartitions().stream()) + .min(new TopicPartitionComparator()) + .orElse(null); } @Override @@ -122,7 +145,7 @@ public void open( Collection addedPartitions) { initialize(icebergCatalog, icebergSinkConfig, sinkTaskContext); if (hasLeaderPartition(addedPartitions)) { - LOG.info("Committer received leader partition. Starting Coordinator."); + LOG.info("Committer {} received leader partition. Starting Coordinator.", taskId); startCoordinator(); } } @@ -141,31 +164,25 @@ public void close(Collection closedPartitions) { // Defensive: close called without prior initialization (should not happen). if (!isInitialized.get()) { - LOG.warn("Close unexpectedly called without partition assignment"); + LOG.warn("Close unexpectedly called on committer {} without partition assignment", taskId); return; } // Empty partitions → task was stopped explicitly. Stop coordinator if running. if (closedPartitions.isEmpty()) { - LOG.info("Task stopped. Closing coordinator."); + LOG.info("Committer {} stopped. Closing coordinator.", taskId); stopCoordinator(); return; } // Normal close: if leader partition is lost, stop coordinator. if (hasLeaderPartition(closedPartitions)) { - LOG.info( - "Committer {}-{} lost leader partition. Stopping coordinator.", - config.connectorName(), - config.taskId()); + LOG.info("Committer {} lost leader partition. Stopping coordinator.", taskId); stopCoordinator(); } // Reset offsets to last committed to avoid data loss. - LOG.info( - "Seeking to last committed offsets for worker {}-{}.", - config.connectorName(), - config.taskId()); + LOG.info("Seeking to last committed offsets for worker {}.", taskId); KafkaUtils.seekToLastCommittedOffsets(context); } @@ -181,9 +198,7 @@ public void save(Collection sinkRecords) { private void processControlEvents() { if (coordinatorThread != null && coordinatorThread.isTerminated()) { throw new NotRunningException( - String.format( - "Coordinator unexpectedly terminated on committer %s-%s", - config.connectorName(), config.taskId())); + String.format("Coordinator unexpectedly terminated on committer %s", taskId)); } if (worker != null) { worker.process(); @@ -192,7 +207,7 @@ private void processControlEvents() { private void startWorker() { if (null == this.worker) { - LOG.info("Starting commit worker {}-{}", config.connectorName(), config.taskId()); + LOG.info("Starting commit worker {}", taskId); SinkWriter sinkWriter = new SinkWriter(catalog, config); worker = new Worker(config, clientFactory, sinkWriter, context); worker.start(); @@ -201,10 +216,7 @@ private void startWorker() { private void startCoordinator() { if (null == this.coordinatorThread) { - LOG.info( - "Task {}-{} elected leader, starting commit coordinator", - config.connectorName(), - config.taskId()); + LOG.info("Task {} elected leader, starting commit coordinator", taskId); Coordinator coordinator = new Coordinator(catalog, config, membersWhenWorkerIsCoordinator, clientFactory, context); coordinatorThread = new CoordinatorThread(coordinator); diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index 068e1e1f6e9c..c986f8afc2eb 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -81,6 +81,7 @@ class Coordinator extends Channel { private final ExecutorService exec; private final CommitState commitState; private volatile boolean terminated; + private final String taskId; Coordinator( Catalog catalog, @@ -110,6 +111,7 @@ class Coordinator extends Channel { .setNameFormat("iceberg-committer" + "-%d") .build()); this.commitState = new CommitState(config); + this.taskId = config.connectorName() + "-" + config.taskId(); } void process() { @@ -119,7 +121,7 @@ void process() { Event event = new Event(config.connectGroupId(), new StartCommit(commitState.currentCommitId())); send(event); - LOG.info("Commit {} initiated", commitState.currentCommitId()); + LOG.info("Coordinator {} initiated commit {}", taskId, commitState.currentCommitId()); } consumeAvailable(POLL_DURATION); @@ -149,7 +151,11 @@ private void commit(boolean partialCommit) { try { doCommit(partialCommit); } catch (Exception e) { - LOG.warn("Commit failed, will try again next cycle", e); + LOG.warn( + "Coordinator {} failed to commit for commit {}, will try again next cycle", + taskId, + commitState.currentCommitId(), + e); } finally { commitState.endCurrentCommit(); } @@ -163,10 +169,9 @@ private void doCommit(boolean partialCommit) { .executeWith(exec) .stopOnFailure() .run( - entry -> { - commitToTable( - entry.getKey(), entry.getValue(), controlTopicOffsets(), validThroughTs); - }); + entry -> + commitToTable( + entry.getKey(), entry.getValue(), controlTopicOffsets(), validThroughTs)); // we should only get here if all tables committed successfully... commitConsumerOffsets(); @@ -179,7 +184,8 @@ private void doCommit(boolean partialCommit) { send(event); LOG.info( - "Commit {} complete, committed to {} table(s), valid-through {}", + "Coordinator {} completed commit {}, committed to {} table(s), valid-through {}", + taskId, commitState.currentCommitId(), commitMap.size(), validThroughTs); @@ -256,13 +262,14 @@ private void commitToTable( .collect(Collectors.toList()); if (terminated) { - throw new ConnectException("Coordinator is terminated, commit aborted"); + throw new ConnectException( + String.format("Coordinator %s is terminated, commit aborted", taskId)); } if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { - LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); + LOG.info( + "Coordinator {} found nothing to commit to table {}, skipping", taskId, tableIdentifier); } else { - String taskId = String.format("%s-%s", config.connectorName(), config.taskId()); if (deleteFiles.isEmpty()) { AppendFiles appendOp = table.newAppend().validateWith(offsetValidator(tableIdentifier, committedOffsets)); @@ -303,7 +310,8 @@ private void commitToTable( send(event); LOG.info( - "Commit complete to table {}, snapshot {}, commit ID {}, valid-through {}", + "Coordinator {} completed commit to table {}, snapshot {}, commit ID {}, valid-through {}", + taskId, tableIdentifier, snapshotId, commitState.currentCommitId(), @@ -372,7 +380,7 @@ private Map parseOffsets(String value) { return Map.of(); } - TypeReference> typeRef = new TypeReference>() {}; + TypeReference> typeRef = new TypeReference<>() {}; try { return MAPPER.readValue(value, typeRef); } catch (IOException e) { From e1091dd211523315313f3c38b63dd8beb1e16ddb Mon Sep 17 00:00:00 2001 From: alpbeysir <94457197+alpbeysir@users.noreply.github.com> Date: Tue, 21 Apr 2026 22:59:33 +0300 Subject: [PATCH 07/16] Core: Use Stream overload for reading response in HTTPClient (#15648) --- .../org/apache/iceberg/rest/HTTPClient.java | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index c359404ec6be..46d9177b9571 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.rest; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectReader; import java.io.IOException; @@ -342,32 +341,23 @@ protected T execute( return null; } - String responseBody = extractResponseBodyAsString(response); - if (!isSuccessful(response)) { // The provided error handler is expected to throw, but a RESTException is thrown if not. + String responseBody = extractResponseBodyAsString(response); throwFailure(response, responseBody, errorHandler); } - if (responseBody == null) { + if (response.getEntity() == null) { throw new RESTException( "Invalid (null) response body for request (expected %s): method=%s, path=%s, status=%d", responseType.getSimpleName(), req.method(), req.path(), response.getCode()); } - try { - ObjectReader reader = objectReaderCache.computeIfAbsent(responseType, mapper::readerFor); - if (parserContext != null && !parserContext.isEmpty()) { - reader = reader.with(parserContext.toInjectableValues()); - } - return reader.readValue(responseBody); - } catch (JsonProcessingException e) { - throw new RESTException( - e, - "Received a success response code of %d, but failed to parse response body into %s", - response.getCode(), - responseType.getSimpleName()); + ObjectReader reader = objectReaderCache.computeIfAbsent(responseType, mapper::readerFor); + if (parserContext != null && !parserContext.isEmpty()) { + reader = reader.with(parserContext.toInjectableValues()); } + return reader.readValue(response.getEntity().getContent()); } catch (IOException e) { throw new RESTException(e, "Error occurred while processing %s request", req.method()); } From 28b6df30043d5a9fa2cbbca852d8d58bfc7deb5d Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 22 Apr 2026 22:33:48 +0900 Subject: [PATCH 08/16] Spark: Fix RoaringBitmap version in runtime-deps.txt (#16076) --- spark/v4.1/spark-runtime/runtime-deps.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/v4.1/spark-runtime/runtime-deps.txt b/spark/v4.1/spark-runtime/runtime-deps.txt index e275e24372af..9a087517cbb0 100644 --- a/spark/v4.1/spark-runtime/runtime-deps.txt +++ b/spark/v4.1/spark-runtime/runtime-deps.txt @@ -36,5 +36,5 @@ org.eclipse.microprofile.openapi:microprofile-openapi-api:4.1.1 org.locationtech.jts:jts-core:1.20.0 org.projectnessie.nessie:nessie-client:0.107.4 org.projectnessie.nessie:nessie-model:0.107.4 -org.roaringbitmap:RoaringBitmap:1.6.13 +org.roaringbitmap:RoaringBitmap:1.6.14 org.threeten:threeten-extra:1.7.1 From 181e7b187c19ddf0e16eb8255e149be8484b2bba Mon Sep 17 00:00:00 2001 From: Anupam Yadav Date: Wed, 22 Apr 2026 06:52:29 -0700 Subject: [PATCH 09/16] Core: Use Idiomatic ThreadLocal cleanup in CommitMetadata (#15284) (#16031) Replace COMMIT_PROPERTIES.set(ImmutableMap.of()) with COMMIT_PROPERTIES.remove() in the finally block of withCommitProperties(). remove() is the recommended cleanup pattern per the ThreadLocal javadoc. Co-authored-by: Anupam Yadav --- .../src/main/java/org/apache/iceberg/spark/CommitMetadata.java | 2 +- .../src/main/java/org/apache/iceberg/spark/CommitMetadata.java | 2 +- .../src/main/java/org/apache/iceberg/spark/CommitMetadata.java | 2 +- .../src/main/java/org/apache/iceberg/spark/CommitMetadata.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index ea400a779235..cb9da3edc678 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -56,7 +56,7 @@ public static R withCommitProperties( ExceptionUtil.castAndThrow(e, exClass); return null; } finally { - COMMIT_PROPERTIES.set(ImmutableMap.of()); + COMMIT_PROPERTIES.remove(); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index ea400a779235..cb9da3edc678 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -56,7 +56,7 @@ public static R withCommitProperties( ExceptionUtil.castAndThrow(e, exClass); return null; } finally { - COMMIT_PROPERTIES.set(ImmutableMap.of()); + COMMIT_PROPERTIES.remove(); } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index ea400a779235..cb9da3edc678 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -56,7 +56,7 @@ public static R withCommitProperties( ExceptionUtil.castAndThrow(e, exClass); return null; } finally { - COMMIT_PROPERTIES.set(ImmutableMap.of()); + COMMIT_PROPERTIES.remove(); } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index ea400a779235..cb9da3edc678 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -56,7 +56,7 @@ public static R withCommitProperties( ExceptionUtil.castAndThrow(e, exClass); return null; } finally { - COMMIT_PROPERTIES.set(ImmutableMap.of()); + COMMIT_PROPERTIES.remove(); } } From b6977d171d19f2f89b76ac249f83e33f408acb3a Mon Sep 17 00:00:00 2001 From: Yingjian Wu <151791653+yingjianwu98@users.noreply.github.com> Date: Wed, 22 Apr 2026 10:13:55 -0700 Subject: [PATCH 10/16] Spark: fix delete from branch for canDeleteWhere where it does not resolve to the correct branch (#15512) --- .../iceberg/spark/extensions/TestDelete.java | 56 +++++++++++++++++++ .../iceberg/spark/source/SparkTable.java | 20 +++++-- 2 files changed, 71 insertions(+), 5 deletions(-) diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 7e0f6207edc9..9e9d751691be 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -1424,6 +1424,62 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx }); } + @TestTemplate + public void testDeleteToWapBranchCanDeleteWhereScansWapBranch() throws NoSuchTableException { + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); + + createAndInitPartitionedTable(); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + + append(tableName, new Employee(1, "hr")); + + spark.conf().set(SparkSQLProperties.WAP_BRANCH, "wap"); + try { + append(tableName, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + + sql("DELETE FROM %s WHERE id = 1", tableName); + + assertThat(sql("SELECT id, dep FROM %s.branch_wap ORDER BY id", tableName)) + .as("DELETE should remove the matching rows from the WAP branch") + .containsExactly(row(0, "hr"), row(2, "hr")); + assertThat(sql("SELECT id, dep FROM %s.branch_main", tableName)) + .as("Main branch must not be modified by a WAP-targeted DELETE") + .containsExactly(row(1, "hr")); + } finally { + spark.conf().unset(SparkSQLProperties.WAP_BRANCH); + } + } + + @TestTemplate + public void testMetadataDeleteToWapBranchCommitsToWapBranch() throws NoSuchTableException { + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); + + createAndInitPartitionedTable(); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + + append(tableName, new Employee(1, "hr"), new Employee(5, "eng")); + + spark.conf().set(SparkSQLProperties.WAP_BRANCH, "wap"); + try { + append(tableName, new Employee(0, "hr"), new Employee(2, "eng")); + + sql("DELETE FROM %s WHERE dep = 'hr'", tableName); + + assertThat(sql("SELECT id, dep FROM %s.branch_wap ORDER BY id", tableName)) + .as("Metadata delete should remove the hr partition on the WAP branch") + .containsExactly(row(2, "eng"), row(5, "eng")); + assertThat(sql("SELECT id, dep FROM %s.branch_main ORDER BY id", tableName)) + .as("Metadata delete must not commit to main when WAP is set") + .containsExactly(row(1, "hr"), row(5, "eng")); + } finally { + spark.conf().unset(SparkSQLProperties.WAP_BRANCH); + } + } + @TestTemplate public void testDeleteWithFilterOnNestedColumn() { createAndInitNestedColumnsTable(); diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 07db8c4ed3fe..80a40d72c8d1 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -49,6 +49,7 @@ import org.apache.iceberg.spark.CommitMetadata; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.spark.SparkV2Filters; import org.apache.iceberg.spark.TimeTravel; @@ -208,11 +209,14 @@ public boolean canDeleteWhere(Predicate[] predicates) { } } - return canDeleteUsingMetadata(deleteExpr); + String scanBranch = + SparkTableUtil.determineReadBranch( + spark(), table(), branch, CaseInsensitiveStringMap.empty()); + return canDeleteUsingMetadata(deleteExpr, scanBranch); } // a metadata delete is possible iff matching files can be deleted entirely - private boolean canDeleteUsingMetadata(Expression deleteExpr) { + private boolean canDeleteUsingMetadata(Expression deleteExpr, String scanBranch) { boolean caseSensitive = SparkUtil.caseSensitive(spark()); if (ExpressionUtil.selectsPartitions(deleteExpr, table(), caseSensitive)) { @@ -227,7 +231,9 @@ private boolean canDeleteUsingMetadata(Expression deleteExpr) { .includeColumnStats() .ignoreResiduals(); - if (snapshot != null) { + if (scanBranch != null) { + scan = scan.useRef(scanBranch); + } else if (snapshot != null) { scan = scan.useSnapshot(snapshot.snapshotId()); } @@ -269,8 +275,12 @@ public void deleteWhere(Predicate[] predicates) { .set("spark.app.id", spark().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr); - if (branch != null) { - deleteFiles.toBranch(branch); + String writeBranch = + SparkTableUtil.determineWriteBranch( + spark(), table(), branch, CaseInsensitiveStringMap.empty()); + + if (writeBranch != null) { + deleteFiles.toBranch(writeBranch); } if (!CommitMetadata.commitProperties().isEmpty()) { From 8ca96649339a81d273eaadfa04b79eee61a4ab1b Mon Sep 17 00:00:00 2001 From: "seokyun.ha" Date: Thu, 23 Apr 2026 03:45:50 +0900 Subject: [PATCH 11/16] Kafka Connect: Support VARIANT when record convert (#15283) * feat: Implement support for VARIANT type in RecordConverter with conversion methods for nested structures --------- Co-authored-by: Brandon Stanley --- .../iceberg/connect/data/RecordConverter.java | 244 ++++++++++++ .../connect/data/TestRecordConverter.java | 367 ++++++++++++++++++ 2 files changed, 611 insertions(+) diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java index 1a57a6444870..51f64a9d4b05 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java @@ -22,23 +22,29 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.math.BigDecimal; +import java.math.BigInteger; import java.math.RoundingMode; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; import java.time.format.DateTimeParseException; import java.time.temporal.Temporal; import java.util.Base64; +import java.util.Collection; +import java.util.Collections; import java.util.Date; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; import org.apache.iceberg.FileFormat; @@ -53,6 +59,7 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Type.PrimitiveType; import org.apache.iceberg.types.Types.DecimalType; @@ -64,6 +71,13 @@ import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.UUIDUtil; +import org.apache.iceberg.variants.ShreddedObject; +import org.apache.iceberg.variants.ValueArray; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantValue; +import org.apache.iceberg.variants.Variants; +import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; @@ -142,6 +156,8 @@ private Object convertValue( return convertTimeValue(value); case TIMESTAMP: return convertTimestampValue(value, (TimestampType) type); + case VARIANT: + return convertVariantValue(value); } throw new UnsupportedOperationException("Unsupported type: " + type.typeId()); } @@ -464,6 +480,234 @@ protected Temporal convertTimestampValue(Object value, TimestampType type) { return convertLocalDateTime(value); } + protected Variant convertVariantValue(Object value) { + if (value instanceof Variant variant) { + return variant; + } + + List sortedFieldNames = + collectFieldNames(value).stream().sorted().collect(Collectors.toList()); + VariantMetadata metadata = Variants.metadata(sortedFieldNames); + return Variant.of(metadata, objectToVariantValue(value, metadata, null)); + } + + /** + * Recursively collects field names from collections, maps, and structs. Returns an empty set for + * null, scalar values, and empty maps, lists, or structs. Map keys must be strings; non-string + * keys cause IllegalArgumentException. + */ + private static Set collectFieldNames(Object value) { + if (value == null) { + return Collections.emptySet(); + } + if (value instanceof Collection collection) { + if (collection.isEmpty()) { + return Collections.emptySet(); + } + Set names = Sets.newHashSet(); + collection.forEach(element -> names.addAll(collectFieldNames(element))); + return names; + } else if (value instanceof Map map) { + if (map.isEmpty()) { + return Collections.emptySet(); + } + Set names = Sets.newHashSet(); + map.forEach( + (key, val) -> { + if (key instanceof String keyStr) { + names.add(keyStr); + names.addAll(collectFieldNames(val)); + } else { + throw new IllegalArgumentException( + "Cannot convert map to variant: keys must be non-null strings, was: " + + (key == null ? "null" : key.getClass().getName())); + } + }); + return names; + } else if (value instanceof Struct struct) { + List fields = struct.schema().fields(); + if (fields.isEmpty()) { + return Collections.emptySet(); + } + Set names = Sets.newHashSet(); + fields.forEach( + field -> { + names.add(field.name()); + names.addAll(collectFieldNames(struct.get(field))); + }); + return names; + } + return Collections.emptySet(); + } + + /** + * Recursively converts a Java object to a VariantValue using the given shared metadata for all + * nested maps. Handles primitives, List (array), and Map (object); map keys become field names. + */ + private static VariantValue objectToVariantValue( + Object value, VariantMetadata metadata, org.apache.kafka.connect.data.Schema schema) { + if (value == null) { + return Variants.ofNull(); + } + VariantValue primitive = primitiveToVariantValue(value, schema); + if (primitive != null) { + return primitive; + } + if (value instanceof Collection collection) { + ValueArray array = Variants.array(); + org.apache.kafka.connect.data.Schema elementSchema = + schema != null ? schema.valueSchema() : null; + for (Object element : collection) { + array.add(objectToVariantValue(element, metadata, elementSchema)); + } + return array; + } + if (value instanceof Map map) { + return mapToVariantValue(map, metadata, schema); + } + if (value instanceof Struct struct) { + ShreddedObject object = Variants.object(metadata); + for (Field field : struct.schema().fields()) { + object.put(field.name(), objectToVariantValue(struct.get(field), metadata, field.schema())); + } + return object; + } + throw new IllegalArgumentException("Cannot convert to variant: " + value.getClass().getName()); + } + + /** Converts a Map to VariantValue; throw IllegalArgumentException if the key is not a string. */ + private static VariantValue mapToVariantValue( + Map map, VariantMetadata metadata, org.apache.kafka.connect.data.Schema schema) { + ShreddedObject object = Variants.object(metadata); + org.apache.kafka.connect.data.Schema mapValueSchema = + schema != null ? schema.valueSchema() : null; + map.forEach( + (key, val) -> { + if (key instanceof String keyStr) { + object.put(keyStr, objectToVariantValue(val, metadata, mapValueSchema)); + } else { + throw new IllegalArgumentException( + "Cannot convert map to variant: keys must be non-null strings, was: " + + (key == null ? "null" : key.getClass().getName())); + } + }); + return object; + } + + /** + * Converts a primitive or primitive-like value to VariantValue; returns null if not supported. + * The optional schema is used to disambiguate java.util.Date which Kafka Connect uses for Date, + * Time, and Timestamp logical types. + */ + private static VariantValue primitiveToVariantValue( + Object value, org.apache.kafka.connect.data.Schema schema) { + if (value instanceof Boolean booleanValue) { + return Variants.of(booleanValue); + } + VariantValue temporal = temporalObjectToVariantValue(value, schema); + if (temporal != null) { + return temporal; + } + if (value instanceof Number number) { + return numberToVariantValue(number); + } + if (value instanceof String stringValue) { + return Variants.of(stringValue); + } + if (value instanceof ByteBuffer byteBuffer) { + return Variants.of(byteBuffer); + } + if (value instanceof byte[] byteArray) { + return Variants.of(ByteBuffer.wrap(byteArray)); + } + if (value instanceof UUID uuid) { + return Variants.ofUUID(uuid); + } + return null; + } + + /** + * Converts java.time values and java.util.Date (with Connect logical type from the optional + * schema) to VariantValue; returns null if the value is not a supported temporal representation. + */ + private static VariantValue temporalObjectToVariantValue( + Object value, org.apache.kafka.connect.data.Schema schema) { + if (value instanceof Instant instant) { + return Variants.ofTimestamptz(DateTimeUtil.microsFromInstant(instant)); + } + if (value instanceof OffsetDateTime offsetDateTime) { + return Variants.ofTimestamptz(DateTimeUtil.microsFromTimestamptz(offsetDateTime)); + } + if (value instanceof ZonedDateTime zonedDateTime) { + return Variants.ofTimestamptz( + DateTimeUtil.microsFromTimestamptz(zonedDateTime.toOffsetDateTime())); + } + if (value instanceof LocalDateTime localDateTime) { + return Variants.ofTimestampntz(DateTimeUtil.microsFromTimestamp(localDateTime)); + } + if (value instanceof LocalDate localDate) { + return Variants.ofDate(DateTimeUtil.daysFromDate(localDate)); + } + if (value instanceof LocalTime localTime) { + return Variants.ofTime(DateTimeUtil.microsFromTime(localTime)); + } + if (value instanceof Date date) { + String logicalName = schema != null ? schema.name() : null; + // Connect represents Timestamp, Time, and Date logical types as java.util.Date at runtime; + // normalize to Instant once, then interpret using the schema logical type name. + Instant connectInstant = date.toInstant(); + if (org.apache.kafka.connect.data.Timestamp.LOGICAL_NAME.equals(logicalName)) { + return Variants.ofTimestamptz(DateTimeUtil.microsFromInstant(connectInstant)); + } + if (org.apache.kafka.connect.data.Time.LOGICAL_NAME.equals(logicalName)) { + LocalTime utcTime = connectInstant.atZone(ZoneOffset.UTC).toLocalTime(); + return Variants.ofTime(DateTimeUtil.microsFromTime(utcTime)); + } + if (org.apache.kafka.connect.data.Date.LOGICAL_NAME.equals(logicalName)) { + return Variants.ofDate(DateTimeUtil.daysFromInstant(connectInstant)); + } + throw new IllegalArgumentException( + "Cannot convert java.util.Date to variant without a recognized logical type schema" + + " (expected Timestamp, Time, or Date but got: " + + logicalName + + ")"); + } + return null; + } + + /** + * Converts a Number to VariantValue; throw IllegalArgumentException if the value is not a + * supported number representation. + */ + private static VariantValue numberToVariantValue(Number number) { + if (number instanceof BigDecimal bigDecimal) { + return Variants.of(bigDecimal); + } + if (number instanceof BigInteger bigInteger) { + return Variants.of(new BigDecimal(bigInteger)); + } + if (number instanceof Integer integer) { + return Variants.of(integer); + } + if (number instanceof Long longValue) { + return Variants.of(longValue); + } + if (number instanceof Float floatValue) { + return Variants.of(floatValue); + } + if (number instanceof Double doubleValue) { + return Variants.of(doubleValue); + } + if (number instanceof Byte byteValue) { + return Variants.of(byteValue); + } + if (number instanceof Short shortValue) { + return Variants.of(shortValue); + } + throw new IllegalArgumentException( + "Cannot convert Number to variant (unknown type): " + number.getClass().getName()); + } + @SuppressWarnings("JavaUtilDate") private OffsetDateTime convertOffsetDateTime(Object value) { if (value instanceof Number) { diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestRecordConverter.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestRecordConverter.java index 45d07f69591b..56a9b6e100ac 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestRecordConverter.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestRecordConverter.java @@ -32,6 +32,7 @@ import java.time.LocalTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.time.ZonedDateTime; import java.time.temporal.Temporal; import java.util.Base64; import java.util.Collection; @@ -74,7 +75,12 @@ import org.apache.iceberg.types.Types.TimeType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; +import org.apache.iceberg.types.Types.VariantType; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.UUIDUtil; +import org.apache.iceberg.variants.PhysicalType; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantValue; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; @@ -152,6 +158,9 @@ public class TestRecordConverter { NestedField.required( 100, "stma", MapType.ofRequired(101, 102, StringType.get(), ID_SCHEMA.asStruct()))); + private static final org.apache.iceberg.Schema VARIANT_SCHEMA = + new org.apache.iceberg.Schema(NestedField.required(1, "v", VariantType.get())); + private static final Schema CONNECT_SCHEMA = SchemaBuilder.struct() .field("i", Schema.INT32_SCHEMA) @@ -881,6 +890,364 @@ public void testEvolveTypeDetectionStructNested() { assertThat(updateMap.get("st.ff").type()).isInstanceOf(DoubleType.class); } + private RecordConverter variantConverter() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(VARIANT_SCHEMA); + return new RecordConverter(table, config); + } + + @Test + public void testConvertVariantValueFromNull() { + Variant variant = variantConverter().convertVariantValue(null); + assertThat(variant).isNotNull(); + assertThat(variant.value().type()).isEqualTo(PhysicalType.NULL); + } + + @Test + public void testConvertVariantValuePassThrough() { + Variant original = variantConverter().convertVariantValue("hello"); + assertThat(variantConverter().convertVariantValue(original)).isSameAs(original); + } + + @Test + public void testConvertVariantValueFromPrimitiveString() { + Variant variant = variantConverter().convertVariantValue("hello"); + assertThat(variant).isNotNull(); + assertThat(variant.metadata()).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.STRING); + assertThat(variant.value().asPrimitive().get()).isEqualTo("hello"); + } + + @Test + public void testConvertVariantValueFromPrimitiveNumber() { + Variant variant = variantConverter().convertVariantValue(123); + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.INT32); + assertThat(variant.value().asPrimitive().get()).isEqualTo(123); + } + + @Test + public void testConvertVariantValueFromBoolean() { + Variant variant = variantConverter().convertVariantValue(true); + assertThat(variant).isNotNull(); + assertThat(variant.value().type()).isEqualTo(PhysicalType.BOOLEAN_TRUE); + assertThat(variant.value().asPrimitive().get()).isEqualTo(true); + } + + @Test + public void testConvertVariantValueFromInstant() { + Instant instant = Instant.parse("2025-04-04T12:34:56.789Z"); + Variant variant = variantConverter().convertVariantValue(instant); + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromInstant(instant)); + } + + @Test + public void testConvertVariantValueFromOffsetDateTime() { + OffsetDateTime odt = OffsetDateTime.parse("2025-04-04T12:34:56.789+09:00"); + Variant variant = variantConverter().convertVariantValue(odt); + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamptz(odt)); + } + + @Test + public void testConvertVariantValueFromZonedDateTime() { + ZonedDateTime zdt = ZonedDateTime.parse("2025-04-04T12:34:56.789-05:00[America/New_York]"); + Variant variant = variantConverter().convertVariantValue(zdt); + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamptz(zdt.toOffsetDateTime())); + } + + @Test + public void testConvertVariantValueFromLocalDateTime() { + LocalDateTime ldt = LocalDateTime.parse("2025-04-04T12:34:56.789"); + Variant variant = variantConverter().convertVariantValue(ldt); + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.TIMESTAMPNTZ); + assertThat(variant.value().asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamp(ldt)); + } + + @Test + public void testConvertVariantValueFromLocalDate() { + LocalDate date = LocalDate.of(2025, 4, 4); + Variant variant = variantConverter().convertVariantValue(date); + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.DATE); + assertThat(variant.value().asPrimitive().get()).isEqualTo(DateTimeUtil.daysFromDate(date)); + } + + @Test + public void testConvertVariantValueFromLocalTime() { + LocalTime time = LocalTime.of(12, 34, 56, 789_000_000); + Variant variant = variantConverter().convertVariantValue(time); + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(0); + assertThat(variant.value().type()).isEqualTo(PhysicalType.TIME); + assertThat(variant.value().asPrimitive().get()).isEqualTo(DateTimeUtil.microsFromTime(time)); + } + + @Test + public void testConvertVariantValueFromList() { + // array with heterogeneous element types (string, int, boolean, double, null, nested array/map, + // java.time primitives). Note: java.util.Date is not supported without Connect logical schema. + Instant instant = Instant.parse("2025-04-04T12:34:56.789Z"); + OffsetDateTime offsetTs = OffsetDateTime.parse("2025-04-04T12:34:56.789+09:00"); + ZonedDateTime zonedTs = ZonedDateTime.parse("2025-04-04T12:34:56.789-05:00[America/New_York]"); + LocalDateTime localTs = LocalDateTime.parse("2025-04-04T12:34:56.789"); + LocalDate localDate = LocalDate.of(2025, 4, 4); + LocalTime localTime = LocalTime.of(12, 34, 56, 789_000_000); + + List input = + Lists.newArrayList( + "a", + 1, + true, + 2.5, + null, + ImmutableList.of("a", "b"), + ImmutableMap.of("key1", "value1", "key2", "value2"), + instant, + offsetTs, + zonedTs, + localTs, + localDate, + localTime); + Variant variant = variantConverter().convertVariantValue(input); + + assertThat(variant).isNotNull(); + assertThat(variant.value().type()).isEqualTo(PhysicalType.ARRAY); + assertThat(variant.value().asArray().numElements()).isEqualTo(13); + + assertThat(variant.value().asArray().get(0).type()).isEqualTo(PhysicalType.STRING); + assertThat(variant.value().asArray().get(0).asPrimitive().get()).isEqualTo("a"); + + assertThat(variant.value().asArray().get(1).type()).isEqualTo(PhysicalType.INT32); + assertThat(variant.value().asArray().get(1).asPrimitive().get()).isEqualTo(1); + + assertThat(variant.value().asArray().get(2).type()).isEqualTo(PhysicalType.BOOLEAN_TRUE); + assertThat(variant.value().asArray().get(2).asPrimitive().get()).isEqualTo(true); + + assertThat(variant.value().asArray().get(3).type()).isEqualTo(PhysicalType.DOUBLE); + assertThat(variant.value().asArray().get(3).asPrimitive().get()).isEqualTo(2.5); + + assertThat(variant.value().asArray().get(4).type()).isEqualTo(PhysicalType.NULL); + + assertThat(variant.value().asArray().get(5).type()).isEqualTo(PhysicalType.ARRAY); + assertThat(variant.value().asArray().get(5).asArray().numElements()).isEqualTo(2); + assertThat(variant.value().asArray().get(5).asArray().get(0).asPrimitive().get()) + .isEqualTo("a"); + assertThat(variant.value().asArray().get(5).asArray().get(1).asPrimitive().get()) + .isEqualTo("b"); + + assertThat(variant.value().asArray().get(6).type()).isEqualTo(PhysicalType.OBJECT); + assertThat(variant.value().asArray().get(6).asObject().numFields()).isEqualTo(2); + assertThat(variant.value().asArray().get(6).asObject().get("key1").asPrimitive().get()) + .isEqualTo("value1"); + assertThat(variant.value().asArray().get(6).asObject().get("key2").asPrimitive().get()) + .isEqualTo("value2"); + + assertThat(variant.value().asArray().get(7).type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asArray().get(7).asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromInstant(instant)); + + assertThat(variant.value().asArray().get(8).type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asArray().get(8).asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamptz(offsetTs)); + + assertThat(variant.value().asArray().get(9).type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asArray().get(9).asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamptz(zonedTs.toOffsetDateTime())); + + assertThat(variant.value().asArray().get(10).type()).isEqualTo(PhysicalType.TIMESTAMPNTZ); + assertThat(variant.value().asArray().get(10).asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamp(localTs)); + + assertThat(variant.value().asArray().get(11).type()).isEqualTo(PhysicalType.DATE); + assertThat(variant.value().asArray().get(11).asPrimitive().get()) + .isEqualTo(DateTimeUtil.daysFromDate(localDate)); + + assertThat(variant.value().asArray().get(12).type()).isEqualTo(PhysicalType.TIME); + assertThat(variant.value().asArray().get(12).asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTime(localTime)); + } + + @Test + public void testConvertVariantValueFromMap() { + // heterogeneous top-level values, nested map, java.time primitives; + // metadata shares one sorted dictionary for the whole tree + Instant instant = Instant.parse("2025-04-04T12:34:56.789Z"); + OffsetDateTime offsetTs = OffsetDateTime.parse("2025-04-04T12:34:56.789+09:00"); + ZonedDateTime zonedTs = ZonedDateTime.parse("2025-04-04T12:34:56.789-05:00[America/New_York]"); + LocalDateTime localTs = LocalDateTime.parse("2025-04-04T12:34:56.789"); + LocalDate localDate = LocalDate.of(2025, 4, 4); + LocalTime localTime = LocalTime.of(12, 34, 56, 789_000_000); + + Map input = Maps.newLinkedHashMap(); + input.put("s", "text"); + input.put("i", 1); + input.put("bool", true); + input.put("d", 2.5); + input.put("n", null); + input.put("hello", ImmutableMap.of("world", 1)); + input.put("tags", ImmutableList.of("a", "b")); + input.put("instant", instant); + input.put("odt", offsetTs); + input.put("zdt", zonedTs); + input.put("ldt", localTs); + input.put("ldate", localDate); + input.put("ltime", localTime); + + Variant variant = variantConverter().convertVariantValue(input); + + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(14); + assertThat(variant.metadata().get(0)).isEqualTo("bool"); + assertThat(variant.metadata().get(1)).isEqualTo("d"); + assertThat(variant.metadata().get(2)).isEqualTo("hello"); + assertThat(variant.metadata().get(3)).isEqualTo("i"); + assertThat(variant.metadata().get(4)).isEqualTo("instant"); + assertThat(variant.metadata().get(5)).isEqualTo("ldate"); + assertThat(variant.metadata().get(6)).isEqualTo("ldt"); + assertThat(variant.metadata().get(7)).isEqualTo("ltime"); + assertThat(variant.metadata().get(8)).isEqualTo("n"); + assertThat(variant.metadata().get(9)).isEqualTo("odt"); + assertThat(variant.metadata().get(10)).isEqualTo("s"); + assertThat(variant.metadata().get(11)).isEqualTo("tags"); + assertThat(variant.metadata().get(12)).isEqualTo("world"); + assertThat(variant.metadata().get(13)).isEqualTo("zdt"); + + assertThat(variant.value().type()).isEqualTo(PhysicalType.OBJECT); + assertThat(variant.value().asObject().numFields()).isEqualTo(13); + + assertThat(variant.value().asObject().get("bool").type()).isEqualTo(PhysicalType.BOOLEAN_TRUE); + assertThat(variant.value().asObject().get("bool").asPrimitive().get()).isEqualTo(true); + + assertThat(variant.value().asObject().get("d").type()).isEqualTo(PhysicalType.DOUBLE); + assertThat(variant.value().asObject().get("d").asPrimitive().get()).isEqualTo(2.5); + + assertThat(variant.value().asObject().get("i").type()).isEqualTo(PhysicalType.INT32); + assertThat(variant.value().asObject().get("i").asPrimitive().get()).isEqualTo(1); + + assertThat(variant.value().asObject().get("n").type()).isEqualTo(PhysicalType.NULL); + + assertThat(variant.value().asObject().get("s").type()).isEqualTo(PhysicalType.STRING); + assertThat(variant.value().asObject().get("s").asPrimitive().get()).isEqualTo("text"); + + VariantValue tags = variant.value().asObject().get("tags"); + assertThat(tags.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(tags.asArray().numElements()).isEqualTo(2); + assertThat(tags.asArray().get(0).asPrimitive().get()).isEqualTo("a"); + assertThat(tags.asArray().get(1).asPrimitive().get()).isEqualTo("b"); + + assertThat(variant.value().asObject().get("instant").type()) + .isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asObject().get("instant").asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromInstant(instant)); + + assertThat(variant.value().asObject().get("odt").type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asObject().get("odt").asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamptz(offsetTs)); + + assertThat(variant.value().asObject().get("zdt").type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(variant.value().asObject().get("zdt").asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamptz(zonedTs.toOffsetDateTime())); + + assertThat(variant.value().asObject().get("ldt").type()).isEqualTo(PhysicalType.TIMESTAMPNTZ); + assertThat(variant.value().asObject().get("ldt").asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTimestamp(localTs)); + + assertThat(variant.value().asObject().get("ldate").type()).isEqualTo(PhysicalType.DATE); + assertThat(variant.value().asObject().get("ldate").asPrimitive().get()) + .isEqualTo(DateTimeUtil.daysFromDate(localDate)); + + assertThat(variant.value().asObject().get("ltime").type()).isEqualTo(PhysicalType.TIME); + assertThat(variant.value().asObject().get("ltime").asPrimitive().get()) + .isEqualTo(DateTimeUtil.microsFromTime(localTime)); + + VariantValue nested = variant.value().asObject().get("hello"); + assertThat(nested.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(nested.asObject().get("world").asPrimitive().get()).isEqualTo(1); + } + + @Test + public void testConvertVariantValueFromStruct() { + // Nested Connect struct: primitives, array, and Timestamp / Time / Date (java.util.Date + + // logical types) + // 2025-04-04 12:34:56.789 UTC (aligned with java.time variant tests) + long tsMillis = 1743770096789L; + long timeMillis = 45296789L; + long dateMillis = 20182L * 86_400_000; + + Schema innerSchema = + SchemaBuilder.struct() + .field("i", Schema.INT32_SCHEMA) + .field("str", Schema.STRING_SCHEMA) + .field("tags", SchemaBuilder.array(Schema.STRING_SCHEMA).build()) + .field("ts", Timestamp.SCHEMA) + .field("t", Time.SCHEMA) + .field("d", org.apache.kafka.connect.data.Date.SCHEMA) + .build(); + Schema outerSchema = + SchemaBuilder.struct().field("inner", innerSchema).field("id", Schema.INT64_SCHEMA).build(); + Struct inner = + new Struct(innerSchema) + .put("i", 1) + .put("str", "world") + .put("tags", ImmutableList.of("a", "b")) + .put("ts", new Date(tsMillis)) + .put("t", new Date(timeMillis)) + .put("d", new Date(dateMillis)); + Struct outer = new Struct(outerSchema).put("inner", inner).put("id", 100L); + + Variant variant = variantConverter().convertVariantValue(outer); + + assertThat(variant).isNotNull(); + assertThat(variant.metadata().dictionarySize()).isEqualTo(8); + assertThat(variant.metadata().get(0)).isEqualTo("d"); + assertThat(variant.metadata().get(1)).isEqualTo("i"); + assertThat(variant.metadata().get(2)).isEqualTo("id"); + assertThat(variant.metadata().get(3)).isEqualTo("inner"); + assertThat(variant.metadata().get(4)).isEqualTo("str"); + assertThat(variant.metadata().get(5)).isEqualTo("t"); + assertThat(variant.metadata().get(6)).isEqualTo("tags"); + assertThat(variant.metadata().get(7)).isEqualTo("ts"); + + assertThat(variant.value().type()).isEqualTo(PhysicalType.OBJECT); + assertThat(variant.value().asObject().get("id").asPrimitive().get()).isEqualTo(100L); + + VariantValue innerVal = variant.value().asObject().get("inner"); + assertThat(innerVal.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(innerVal.asObject().get("i").asPrimitive().get()).isEqualTo(1); + assertThat(innerVal.asObject().get("str").asPrimitive().get()).isEqualTo("world"); + assertThat(innerVal.asObject().get("tags").type()).isEqualTo(PhysicalType.ARRAY); + assertThat(innerVal.asObject().get("tags").asArray().numElements()).isEqualTo(2); + assertThat(innerVal.asObject().get("tags").asArray().get(0).asPrimitive().get()).isEqualTo("a"); + assertThat(innerVal.asObject().get("tags").asArray().get(1).asPrimitive().get()).isEqualTo("b"); + + assertThat(innerVal.asObject().get("ts").type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(innerVal.asObject().get("ts").asPrimitive().get()).isEqualTo(tsMillis * 1000); + + assertThat(innerVal.asObject().get("t").type()).isEqualTo(PhysicalType.TIME); + assertThat(innerVal.asObject().get("t").asPrimitive().get()).isEqualTo(timeMillis * 1000); + + assertThat(innerVal.asObject().get("d").type()).isEqualTo(PhysicalType.DATE); + assertThat(innerVal.asObject().get("d").asPrimitive().get()).isEqualTo(20182); + } + public static Map createMapData() { return ImmutableMap.builder() .put("i", 1) From 77ab506750c16b1bc92a79a1a7b8bea06e8db1f9 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 22 Apr 2026 21:56:42 -0700 Subject: [PATCH 12/16] REST Spec: Clarify identifier uniqueness across tables and views (#15691) * REST: Clarify that identifiers must be unique across all catalog object types Table and view identifiers share the same namespace scope, so a table and a view with the same name in the same namespace are not allowed. The rename and register-view endpoints already enforced this with "already exists as a table or view", but createTable, registerTable, and createView only guarded against same-type conflicts. This change makes all six write operations consistent by using the new CatalogObjectType schema, which enumerates the known object types (table, view) and states the uniqueness invariant explicitly. The 409 conflict descriptions are updated to: - "The identifier is already used by an existing catalog object (see `CatalogObjectType`)" - "The target identifier to rename to is already used by an existing catalog object (see `CatalogObjectType`)" Made-with: Cursor Model: claude-4.6-sonnet-medium-thinking * REST: Regenerate Python code for CatalogObjectType schema addition Made-with: Cursor Model: claude-4.6-sonnet-medium-thinking * Open API: Remove CatalogObjectType and clarify 409 conflict text Drop the unused CatalogObjectType schema and describe identifier conflicts in terms of existing tables or views. Made-with: Cursor Model: GPT-5.2 * update the error msg in the TableAlreadyExistsError and ViewAlreadyExistsError --- open-api/rest-catalog-open-api.yaml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 2ef154f18f26..2435cd43f0e5 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -607,7 +607,7 @@ paths: NamespaceNotFound: $ref: '#/components/examples/NoSuchNamespaceError' 409: - description: Conflict - The table already exists + description: Conflict - The identifier already exists as a table or view content: application/json: schema: @@ -927,7 +927,7 @@ paths: NamespaceNotFound: $ref: '#/components/examples/NoSuchNamespaceError' 409: - description: Conflict - The table already exists + description: Conflict - The identifier already exists as a table or view content: application/json: schema: @@ -1593,7 +1593,7 @@ paths: NamespaceNotFound: $ref: '#/components/examples/NoSuchNamespaceError' 409: - description: Conflict - The view already exists + description: Conflict - The identifier already exists as a table or view content: application/json: schema: @@ -5225,7 +5225,7 @@ components: summary: The requested table identifier already exists value: { "error": { - "message": "The given table already exists", + "message": "The requested table identifier already exists", "type": "AlreadyExistsException", "code": 409 } @@ -5235,7 +5235,7 @@ components: summary: The requested view identifier already exists value: { "error": { - "message": "The given view already exists", + "message": "The requested view identifier already exists", "type": "AlreadyExistsException", "code": 409 } From eaf39d7d3e550945acda297f2e20da42d6085473 Mon Sep 17 00:00:00 2001 From: Bharath Krishna Date: Thu, 23 Apr 2026 09:44:38 -0700 Subject: [PATCH 13/16] Spark 3.4, 3.5, 4.0: Include snapshotId and branch in SparkTable equals and hashCode (#15840) --- .../iceberg/spark/source/SparkTable.java | 17 +++--- .../iceberg/spark/source/TestSparkTable.java | 56 +++++++++++++++++++ .../iceberg/spark/source/SparkTable.java | 17 +++--- .../iceberg/spark/source/TestSparkTable.java | 56 +++++++++++++++++++ .../iceberg/spark/source/SparkTable.java | 17 +++--- .../iceberg/spark/source/TestSparkTable.java | 56 +++++++++++++++++++ 6 files changed, 198 insertions(+), 21 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 353566eb7f34..871ef9355200 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.iceberg.BaseMetadataTable; import org.apache.iceberg.BaseTable; @@ -396,12 +397,13 @@ public void deleteWhere(Predicate[] predicates) { .set("spark.app.id", sparkSession().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr); + String writeBranch = branch; if (SparkTableUtil.wapEnabled(table())) { - branch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); + writeBranch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); } - if (branch != null) { - deleteFiles.toBranch(branch); + if (writeBranch != null) { + deleteFiles.toBranch(writeBranch); } if (!CommitMetadata.commitProperties().isEmpty()) { @@ -424,15 +426,16 @@ public boolean equals(Object other) { return false; } - // use only name in order to correctly invalidate Spark cache SparkTable that = (SparkTable) other; - return icebergTable.name().equals(that.icebergTable.name()); + return icebergTable.name().equals(that.icebergTable.name()) + && Objects.equals(table().uuid(), that.table().uuid()) + && Objects.equals(snapshotId, that.snapshotId) + && Objects.equals(branch, that.branch); } @Override public int hashCode() { - // use only name in order to correctly invalidate Spark cache - return icebergTable.name().hashCode(); + return Objects.hash(icebergTable.name(), table().uuid(), snapshotId, branch); } private static CaseInsensitiveStringMap addSnapshotId( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java index d14b1a52cf82..e3934faa60ce 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java @@ -20,7 +20,9 @@ import static org.assertj.core.api.Assertions.assertThat; +import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -56,4 +58,58 @@ public void testTableEquality() throws NoSuchTableException { assertThat(table1).as("References must be different").isNotSameAs(table2); assertThat(table1).as("Tables must be equivalent").isEqualTo(table2); } + + @TestTemplate + public void testTableInequalityWithDifferentSnapshots() throws NoSuchTableException { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + SparkTable table = (SparkTable) catalog.loadTable(identifier); + + Table icebergTable = validationCatalog.loadTable(tableIdent); + long[] snapshotIds = + icebergTable.history().stream().mapToLong(HistoryEntry::snapshotId).toArray(); + + SparkTable tableAtSnapshot1 = table.copyWithSnapshotId(snapshotIds[0]); + SparkTable tableAtSnapshot2 = table.copyWithSnapshotId(snapshotIds[1]); + + assertThat(tableAtSnapshot1) + .as("Tables at different snapshots must not be equal") + .isNotEqualTo(tableAtSnapshot2); + assertThat(tableAtSnapshot1.hashCode()) + .as("Hash codes should differ for different snapshots") + .isNotEqualTo(tableAtSnapshot2.hashCode()); + } + + @TestTemplate + public void testTableInequalityWithDifferentBranches() throws NoSuchTableException { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + + Table icebergTable = validationCatalog.loadTable(tableIdent); + icebergTable + .manageSnapshots() + .createBranch("testBranch", icebergTable.currentSnapshot().snapshotId()) + .commit(); + + // reload after branch creation so the table sees the new ref + SparkTable table = (SparkTable) catalog.loadTable(identifier); + table.table().refresh(); + + SparkTable tableOnMain = table.copyWithBranch("main"); + SparkTable tableOnBranch = table.copyWithBranch("testBranch"); + + assertThat(tableOnMain) + .as("Tables on different branches must not be equal") + .isNotEqualTo(tableOnBranch); + assertThat(tableOnMain.hashCode()) + .as("Hash codes should differ for different branches") + .isNotEqualTo(tableOnBranch.hashCode()); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 353566eb7f34..871ef9355200 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.iceberg.BaseMetadataTable; import org.apache.iceberg.BaseTable; @@ -396,12 +397,13 @@ public void deleteWhere(Predicate[] predicates) { .set("spark.app.id", sparkSession().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr); + String writeBranch = branch; if (SparkTableUtil.wapEnabled(table())) { - branch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); + writeBranch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); } - if (branch != null) { - deleteFiles.toBranch(branch); + if (writeBranch != null) { + deleteFiles.toBranch(writeBranch); } if (!CommitMetadata.commitProperties().isEmpty()) { @@ -424,15 +426,16 @@ public boolean equals(Object other) { return false; } - // use only name in order to correctly invalidate Spark cache SparkTable that = (SparkTable) other; - return icebergTable.name().equals(that.icebergTable.name()); + return icebergTable.name().equals(that.icebergTable.name()) + && Objects.equals(table().uuid(), that.table().uuid()) + && Objects.equals(snapshotId, that.snapshotId) + && Objects.equals(branch, that.branch); } @Override public int hashCode() { - // use only name in order to correctly invalidate Spark cache - return icebergTable.name().hashCode(); + return Objects.hash(icebergTable.name(), table().uuid(), snapshotId, branch); } private static CaseInsensitiveStringMap addSnapshotId( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java index d14b1a52cf82..e3934faa60ce 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java @@ -20,7 +20,9 @@ import static org.assertj.core.api.Assertions.assertThat; +import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -56,4 +58,58 @@ public void testTableEquality() throws NoSuchTableException { assertThat(table1).as("References must be different").isNotSameAs(table2); assertThat(table1).as("Tables must be equivalent").isEqualTo(table2); } + + @TestTemplate + public void testTableInequalityWithDifferentSnapshots() throws NoSuchTableException { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + SparkTable table = (SparkTable) catalog.loadTable(identifier); + + Table icebergTable = validationCatalog.loadTable(tableIdent); + long[] snapshotIds = + icebergTable.history().stream().mapToLong(HistoryEntry::snapshotId).toArray(); + + SparkTable tableAtSnapshot1 = table.copyWithSnapshotId(snapshotIds[0]); + SparkTable tableAtSnapshot2 = table.copyWithSnapshotId(snapshotIds[1]); + + assertThat(tableAtSnapshot1) + .as("Tables at different snapshots must not be equal") + .isNotEqualTo(tableAtSnapshot2); + assertThat(tableAtSnapshot1.hashCode()) + .as("Hash codes should differ for different snapshots") + .isNotEqualTo(tableAtSnapshot2.hashCode()); + } + + @TestTemplate + public void testTableInequalityWithDifferentBranches() throws NoSuchTableException { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + + Table icebergTable = validationCatalog.loadTable(tableIdent); + icebergTable + .manageSnapshots() + .createBranch("testBranch", icebergTable.currentSnapshot().snapshotId()) + .commit(); + + // reload after branch creation so the table sees the new ref + SparkTable table = (SparkTable) catalog.loadTable(identifier); + table.table().refresh(); + + SparkTable tableOnMain = table.copyWithBranch("main"); + SparkTable tableOnBranch = table.copyWithBranch("testBranch"); + + assertThat(tableOnMain) + .as("Tables on different branches must not be equal") + .isNotEqualTo(tableOnBranch); + assertThat(tableOnMain.hashCode()) + .as("Hash codes should differ for different branches") + .isNotEqualTo(tableOnBranch.hashCode()); + } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 1ee9e9b08074..6f0f992f1c20 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.iceberg.BaseMetadataTable; import org.apache.iceberg.BaseTable; @@ -438,12 +439,13 @@ public void deleteWhere(Predicate[] predicates) { .set("spark.app.id", sparkSession().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr); + String writeBranch = branch; if (SparkTableUtil.wapEnabled(table())) { - branch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); + writeBranch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); } - if (branch != null) { - deleteFiles.toBranch(branch); + if (writeBranch != null) { + deleteFiles.toBranch(writeBranch); } if (!CommitMetadata.commitProperties().isEmpty()) { @@ -466,15 +468,16 @@ public boolean equals(Object other) { return false; } - // use only name in order to correctly invalidate Spark cache SparkTable that = (SparkTable) other; - return icebergTable.name().equals(that.icebergTable.name()); + return icebergTable.name().equals(that.icebergTable.name()) + && Objects.equals(table().uuid(), that.table().uuid()) + && Objects.equals(snapshotId, that.snapshotId) + && Objects.equals(branch, that.branch); } @Override public int hashCode() { - // use only name in order to correctly invalidate Spark cache - return icebergTable.name().hashCode(); + return Objects.hash(icebergTable.name(), table().uuid(), snapshotId, branch); } private static CaseInsensitiveStringMap addSnapshotId( diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java index d14b1a52cf82..e3934faa60ce 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java @@ -20,7 +20,9 @@ import static org.assertj.core.api.Assertions.assertThat; +import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -56,4 +58,58 @@ public void testTableEquality() throws NoSuchTableException { assertThat(table1).as("References must be different").isNotSameAs(table2); assertThat(table1).as("Tables must be equivalent").isEqualTo(table2); } + + @TestTemplate + public void testTableInequalityWithDifferentSnapshots() throws NoSuchTableException { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + SparkTable table = (SparkTable) catalog.loadTable(identifier); + + Table icebergTable = validationCatalog.loadTable(tableIdent); + long[] snapshotIds = + icebergTable.history().stream().mapToLong(HistoryEntry::snapshotId).toArray(); + + SparkTable tableAtSnapshot1 = table.copyWithSnapshotId(snapshotIds[0]); + SparkTable tableAtSnapshot2 = table.copyWithSnapshotId(snapshotIds[1]); + + assertThat(tableAtSnapshot1) + .as("Tables at different snapshots must not be equal") + .isNotEqualTo(tableAtSnapshot2); + assertThat(tableAtSnapshot1.hashCode()) + .as("Hash codes should differ for different snapshots") + .isNotEqualTo(tableAtSnapshot2.hashCode()); + } + + @TestTemplate + public void testTableInequalityWithDifferentBranches() throws NoSuchTableException { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + + Table icebergTable = validationCatalog.loadTable(tableIdent); + icebergTable + .manageSnapshots() + .createBranch("testBranch", icebergTable.currentSnapshot().snapshotId()) + .commit(); + + // reload after branch creation so the table sees the new ref + SparkTable table = (SparkTable) catalog.loadTable(identifier); + table.table().refresh(); + + SparkTable tableOnMain = table.copyWithBranch("main"); + SparkTable tableOnBranch = table.copyWithBranch("testBranch"); + + assertThat(tableOnMain) + .as("Tables on different branches must not be equal") + .isNotEqualTo(tableOnBranch); + assertThat(tableOnMain.hashCode()) + .as("Hash codes should differ for different branches") + .isNotEqualTo(tableOnBranch.hashCode()); + } } From 361ad5a8e885641d0e2cce8412e1a702a2078edb Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 23 Apr 2026 19:33:09 +0200 Subject: [PATCH 14/16] Core, Spark: Verify that TRUNCATE removes orphaned DVs (#16078) --- .../org/apache/iceberg/TestDeleteFiles.java | 71 +++++++++++++++++++ .../iceberg/spark/sql/TestDeleteFrom.java | 38 ++++++++++ 2 files changed, 109 insertions(+) diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index 68e5fa8b560e..d7cdd5c5d884 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -615,6 +615,10 @@ public void removingDataFileByExpressionAlsoRemovesDV() { .containsEntry(SnapshotSummary.REPLACED_MANIFESTS_COUNT, "2"); assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); + assertThat(deleteSnap.summary()) + .containsEntry(SnapshotSummary.REMOVED_DVS_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_DELETE_FILES_PROP, "1"); + validateDeleteManifest( deleteSnap.deleteManifests(table.io()).get(0), dataSeqs(1L, 1L), @@ -658,6 +662,10 @@ public void removingDataFileByPathAlsoRemovesDV() { .containsEntry(SnapshotSummary.REPLACED_MANIFESTS_COUNT, "2"); assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); + assertThat(deleteSnap.summary()) + .containsEntry(SnapshotSummary.REMOVED_DVS_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_DELETE_FILES_PROP, "1"); + validateDeleteManifest( deleteSnap.deleteManifests(table.io()).get(0), dataSeqs(1L, 1L), @@ -667,6 +675,69 @@ public void removingDataFileByPathAlsoRemovesDV() { statuses(ManifestEntry.Status.DELETED, ManifestEntry.Status.EXISTING)); } + @TestTemplate + public void removingDataFilesWhenTruncatingAlsoRemovesDVs() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + DeleteFile dv1 = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-1-deletes.puffin") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket=0") + .withRecordCount(5) + .withReferencedDataFile(DATA_FILE_BUCKET_0_IDS_0_2.location()) + .withContentOffset(4) + .withContentSizeInBytes(6) + .build(); + + DeleteFile dv2 = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-2-deletes.puffin") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket=0") + .withRecordCount(5) + .withReferencedDataFile(DATA_FILE_BUCKET_0_IDS_8_10.location()) + .withContentOffset(4) + .withContentSizeInBytes(6) + .build(); + + commit( + table, + table + .newRowDelta() + .addRows(DATA_FILE_BUCKET_0_IDS_0_2) + .addRows(DATA_FILE_BUCKET_0_IDS_8_10) + .addDeletes(dv1) + .addDeletes(dv2), + branch); + + Snapshot snapshot = latestSnapshot(table, branch); + assertThat(snapshot.sequenceNumber()).isEqualTo(1); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(1); + + // deleting by row filter should also remove the orphaned dv1 from delete manifests. When a + // table is truncated via TRUNCATE, the row filter is sent as Expressions.alwaysTrue() + commit(table, table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()), branch); + + Snapshot deleteSnap = latestSnapshot(table, branch); + assertThat(deleteSnap.sequenceNumber()).isEqualTo(2); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(2); + + assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); + assertThat(deleteSnap.summary()) + .containsEntry(SnapshotSummary.REMOVED_DVS_PROP, "2") + .containsEntry(SnapshotSummary.REMOVED_DELETE_FILES_PROP, "2"); + + validateDeleteManifest( + deleteSnap.deleteManifests(table.io()).get(0), + dataSeqs(1L, 1L), + fileSeqs(1L, 1L), + ids(deleteSnap.snapshotId(), deleteSnap.snapshotId()), + files(dv1, dv2), + statuses(Status.DELETED, Status.DELETED)); + } + private static ByteBuffer longToBuffer(long value) { return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); } diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index 536d568003cf..02c5ecd66b80 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -23,6 +23,7 @@ import java.util.List; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -185,4 +186,41 @@ public void testDeleteFromTablePartitionedByVarbinary() { ImmutableList.of(row(1L, new byte[] {-29, -68, -47})), sql("SELECT * FROM %s where data = X'e3bcd1'", tableName)); } + + @TestTemplate + public void truncateWithDVs() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg TBLPROPERTIES ('format-version'='3','write.delete.mode'='merge-on-read')", + tableName); + List records = + ImmutableList.of( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.coalesce(1).writeTo(tableName).append(); + + assertThat(sql("SELECT * FROM %s ORDER BY id", tableName)) + .containsExactly(row(1L, "a"), row(2L, "b"), row(3L, "c")); + + sql("DELETE FROM %s WHERE id = 1", tableName); + assertThat(validationCatalog.loadTable(tableIdent).currentSnapshot().summary()) + .containsEntry(SnapshotSummary.ADDED_DVS_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_POS_DELETES_PROP, "1"); + + sql("DELETE FROM %s WHERE id = 2", tableName); + // DVs have been merged into single file + assertThat(validationCatalog.loadTable(tableIdent).currentSnapshot().summary()) + .containsEntry(SnapshotSummary.ADDED_DVS_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_DVS_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_POS_DELETES_PROP, "2"); + + assertThat(sql("SELECT * FROM %s ORDER BY id", tableName)).containsExactly(row(3L, "c")); + + sql("TRUNCATE TABLE %s", tableName); + assertThat(validationCatalog.loadTable(tableIdent).currentSnapshot().summary()) + .containsEntry(SnapshotSummary.REMOVED_DVS_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_POS_DELETES_PROP, "2"); + + assertThat(sql("SELECT * FROM %s ORDER BY id", tableName)).isEmpty(); + } } From 435a02ec3f85689a4df67cff02d286ce5cb4a03e Mon Sep 17 00:00:00 2001 From: Kurtis Wright Date: Thu, 23 Apr 2026 20:24:55 +0000 Subject: [PATCH 15/16] Spark: Add unknown type support to Spark 3.4 and 3.5 Map Iceberg's UnknownType to Spark's NullType in both directions (TypeToSparkType: UNKNOWN -> NullType; SparkTypeToType: NullType -> UnknownType). Filter NullType-backed fields from Parquet/ORC writers so UnknownType columns produce only nulls. Aligns Spark 3.x with the existing Spark 4.x behavior from #13445. Tests added (mirrored from #13445 to v3.4 and v3.5): - AvroDataTestBase: unk field in SUPPORTED_PRIMITIVES, plus testUnknownNestedLevel, testUnknownListType, testUnknownMapType - TestSparkOrcReader: testUnknownListType, testUnknownMapType overrides - TestSparkParquetReader: testUnknownListType, testUnknownMapType overrides - TestSparkRecordOrcReaderWriter: testUnknownListType, testUnknownMapType overrides - TestORCDataFrameWrite: testUnknownListType, testUnknownMapType overrides - TestParquetDataFrameWrite: testUnknownListType, testUnknownMapType overrides - TestParquetScan: testUnknownListType, testUnknownMapType overrides - ScanTestBase.writeAndValidate: opt into format version 3 when the schema contains UnknownType --- .../spark/PruneColumnsWithoutReordering.java | 2 + .../apache/iceberg/spark/TypeToSparkType.java | 1 - .../data/ParquetWithSparkSchemaVisitor.java | 31 ++++++---- .../iceberg/spark/data/SparkOrcWriter.java | 21 +++++-- .../spark/data/SparkParquetWriters.java | 38 ++++++++++--- .../vectorized/VectorizedSparkOrcReaders.java | 2 + .../iceberg/spark/data/AvroDataTestBase.java | 56 +++++++++++++++++-- .../spark/data/TestSparkOrcReader.java | 17 ++++++ .../spark/data/TestSparkParquetReader.java | 16 ++++++ .../data/TestSparkRecordOrcReaderWriter.java | 17 ++++++ .../iceberg/spark/source/ScanTestBase.java | 8 ++- .../spark/source/TestORCDataFrameWrite.java | 24 ++++++++ .../source/TestParquetDataFrameWrite.java | 24 ++++++++ .../iceberg/spark/source/TestParquetScan.java | 18 ++++++ .../spark/PruneColumnsWithoutReordering.java | 2 + .../apache/iceberg/spark/TypeToSparkType.java | 1 - .../data/ParquetWithSparkSchemaVisitor.java | 31 ++++++---- .../iceberg/spark/data/SparkOrcWriter.java | 21 +++++-- .../spark/data/SparkParquetWriters.java | 39 ++++++++++--- .../iceberg/spark/data/AvroDataTestBase.java | 56 +++++++++++++++++-- .../spark/data/TestSparkOrcReader.java | 17 ++++++ .../spark/data/TestSparkParquetReader.java | 16 ++++++ .../data/TestSparkRecordOrcReaderWriter.java | 17 ++++++ .../iceberg/spark/source/ScanTestBase.java | 8 ++- .../spark/source/TestORCDataFrameWrite.java | 24 ++++++++ .../source/TestParquetDataFrameWrite.java | 24 ++++++++ .../iceberg/spark/source/TestParquetScan.java | 18 ++++++ 27 files changed, 485 insertions(+), 64 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java index fbd21f737450..fec413ca079a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java @@ -41,6 +41,7 @@ import org.apache.spark.sql.types.IntegerType$; import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -238,5 +239,6 @@ public Type primitive(Type.PrimitiveType primitive) { .put(TypeID.STRING, ImmutableSet.of(StringType$.class)) .put(TypeID.FIXED, ImmutableSet.of(BinaryType$.class)) .put(TypeID.BINARY, ImmutableSet.of(BinaryType$.class)) + .put(TypeID.UNKNOWN, ImmutableSet.of(NullType$.class)) .buildOrThrow(); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index 704b8d458115..d33632bbbd54 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -36,7 +36,6 @@ import org.apache.spark.sql.types.IntegerType$; import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.MapType$; -import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.NullType$; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java index d74a76f94e87..2a2eef198b76 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java @@ -31,6 +31,7 @@ import org.apache.parquet.schema.Type.Repetition; import org.apache.spark.sql.types.ArrayType; import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.MapType; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -181,21 +182,27 @@ private static T visitField( private static List visitFields( StructType struct, GroupType group, ParquetWithSparkSchemaVisitor visitor) { - StructField[] sFields = struct.fields(); - Preconditions.checkArgument( - sFields.length == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); - for (int i = 0; i < sFields.length; i += 1) { - Type field = group.getFields().get(i); - StructField sField = sFields[i]; - Preconditions.checkArgument( - field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.name())), - "Structs do not match: field %s != %s", - field.getName(), - sField.name()); - results.add(visitField(sField, field, visitor)); + + int fieldIndex = 0; + for (StructField sField : struct.fields()) { + if (sField.dataType() != DataTypes.NullType) { + Type field = group.getFields().get(fieldIndex); + Preconditions.checkArgument( + field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.name())), + "Structs do not match: field %s != %s", + field.getName(), + sField.name()); + results.add(visitField(sField, field, visitor)); + + fieldIndex += 1; + } } + // All the group fields should have been visited + Preconditions.checkArgument( + fieldIndex == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); + return results; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index 6b799e677bf4..6fc8849c82b2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -20,6 +20,8 @@ import java.io.Serializable; import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.Stream; import javax.annotation.Nullable; import org.apache.iceberg.FieldMetrics; @@ -77,7 +79,7 @@ public OrcValueWriter record( TypeDescription record, List names, List> fields) { - return new InternalRowWriter(fields, record.getChildren()); + return new InternalRowWriter(fields, iStruct, record.getChildren()); } @Override @@ -133,12 +135,16 @@ public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, TypeDescriptio private static class InternalRowWriter extends GenericOrcWriters.StructWriter { private final List> fieldGetters; - InternalRowWriter(List> writers, List orcTypes) { - super(writers); + InternalRowWriter( + List> writers, Types.StructType iStruct, List orcTypes) { + super(iStruct, writers); this.fieldGetters = Lists.newArrayListWithExpectedSize(orcTypes.size()); - for (TypeDescription orcType : orcTypes) { - fieldGetters.add(createFieldGetter(orcType)); + Map idToType = + orcTypes.stream().collect(Collectors.toMap(ORCSchemaUtil::fieldId, s -> s)); + + for (Types.NestedField iField : iStruct.fields()) { + fieldGetters.add(createFieldGetter(idToType.get(iField.fieldId()))); } } @@ -149,6 +155,11 @@ protected Object get(InternalRow struct, int index) { } static FieldGetter createFieldGetter(TypeDescription fieldType) { + // In the case of an UnknownType + if (fieldType == null) { + return (row, ordinal) -> null; + } + final FieldGetter fieldGetter; switch (fieldType.getCategory()) { case BOOLEAN: diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index f4ae6114c8ab..a1ed6c66f337 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -26,6 +26,7 @@ import java.util.NoSuchElementException; import java.util.Optional; import java.util.UUID; +import java.util.stream.IntStream; import org.apache.iceberg.Schema; import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; import org.apache.iceberg.parquet.ParquetValueWriter; @@ -55,6 +56,7 @@ import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType; import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -94,15 +96,18 @@ public ParquetValueWriter message( public ParquetValueWriter struct( StructType sStruct, GroupType struct, List> fieldWriters) { List fields = struct.getFields(); - StructField[] sparkFields = sStruct.fields(); List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); - List sparkTypes = Lists.newArrayList(); for (int i = 0; i < fields.size(); i += 1) { writers.add(newOption(struct.getType(i), fieldWriters.get(i))); - sparkTypes.add(sparkFields[i].dataType()); } - return new InternalRowWriter(writers, sparkTypes); + StructField[] sFields = sStruct.fields(); + DataType[] types = new DataType[sFields.length]; + for (int i = 0; i < sFields.length; i += 1) { + types[i] = sFields[i].dataType(); + } + + return new InternalRowWriter(writers, types); } @Override @@ -566,14 +571,33 @@ public Map.Entry next() { private static class InternalRowWriter extends ParquetValueWriters.StructWriter { private final DataType[] types; - private InternalRowWriter(List> writers, List types) { - super(writers); - this.types = types.toArray(new DataType[0]); + private InternalRowWriter(List> writers, DataType[] types) { + super(writerToFieldIndex(types, writers.size()), writers); + this.types = types; } @Override protected Object get(InternalRow struct, int index) { return struct.get(index, types[index]); } + + /** Returns a mapping from writer index to field index, skipping Unknown columns. */ + private static int[] writerToFieldIndex(DataType[] types, int numWriters) { + if (null == types) { + return IntStream.rangeClosed(0, numWriters).toArray(); + } + + // value writer index to record field index + int[] indexes = new int[numWriters]; + int writerIndex = 0; + for (int pos = 0; pos < types.length; pos += 1) { + if (!(types[pos] instanceof NullType)) { + indexes[writerIndex] = pos; + writerIndex += 1; + } + } + + return indexes; + } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index 8dceb075e604..4f324239881e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -465,6 +465,8 @@ public ColumnVector convert( DeletedColumnVector deletedVector = new DeletedColumnVector(field.type()); deletedVector.setValue(new boolean[batchSize]); fieldVectors.add(deletedVector); + } else if (field.type().equals(Types.UnknownType.get())) { + fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, null)); } else { fieldVectors.add( fieldConverters diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java index 0db6a65fd394..45053c1a4f1f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; @@ -108,8 +109,8 @@ protected boolean supportsRowLineage() { required(114, "dec_9_0", Types.DecimalType.of(9, 0)), // int encoded required(115, "dec_11_2", Types.DecimalType.of(11, 2)), // long encoded required(116, "dec_20_5", Types.DecimalType.of(20, 5)), // requires padding - required(117, "dec_38_10", Types.DecimalType.of(38, 10)) // Spark's maximum precision - ); + required(117, "dec_38_10", Types.DecimalType.of(38, 10)), // Spark's maximum precision + optional(118, "unk", Types.UnknownType.get())); @TempDir protected Path temp; @@ -120,10 +121,13 @@ public void testSimpleStruct() throws IOException { @Test public void testStructWithRequiredFields() throws IOException { + List supportedPrimitives = + SUPPORTED_PRIMITIVES.fields().stream() + .filter(f -> f.type().typeId() != Type.TypeID.UNKNOWN) + .collect(Collectors.toList()); writeAndValidate( TypeUtil.assignIncreasingFreshIds( - new Schema( - Lists.transform(SUPPORTED_PRIMITIVES.fields(), Types.NestedField::asRequired)))); + new Schema(Lists.transform(supportedPrimitives, Types.NestedField::asRequired)))); } @Test @@ -603,4 +607,48 @@ public void testRowLineage() throws Exception { record.copy(Map.of("id", 4L, "data", "d", "_row_id", 1_001L)), record.copy(Map.of("id", 5L, "data", "e")))); } + + @Test + public void testUnknownNestedLevel() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(1, "id", LongType.get()), + optional( + 2, + "nested", + Types.StructType.of( + required(20, "int", Types.IntegerType.get()), + optional(21, "unk", Types.UnknownType.get())))); + + writeAndValidate(schema); + } + + @Test + public void testUnknownListType() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional(1, "data", ListType.ofOptional(2, Types.UnknownType.get()))); + + writeAndValidate(schema); + } + + @Test + public void testUnknownMapType() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional( + 1, + "data", + MapType.ofOptional(2, 3, Types.StringType.get(), Types.UnknownType.get()))); + + writeAndValidate(schema); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java index 546a44fc77bb..a1f71848b14e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -106,4 +107,20 @@ private void writeAndValidateRecords(Schema schema, Iterable expect private Iterator batchesToRows(Iterator batches) { return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index 9ae8b8cbe530..993dc868bba8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -249,4 +249,20 @@ public void testMissingRequiredWithoutDefault() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Missing required field: missing_str"); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java index 8e1f860085c6..3c88db139e47 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -152,4 +153,20 @@ private static void assertEqualsUnsafe( .isFalse(); assertThat(actualIter.hasNext()).as("Actual iterator should not have any extra rows").isFalse(); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java index c368c4a815fe..21381e824e89 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java @@ -41,6 +41,7 @@ import org.apache.iceberg.spark.data.AvroDataTestBase; import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; @@ -95,11 +96,12 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw HadoopTables tables = new HadoopTables(CONF); // If V3 spec features are used, set the format version to 3 - Map tableProperties = + boolean requiresV3 = writeSchema.columns().stream() .anyMatch(f -> f.initialDefaultLiteral() != null || f.writeDefaultLiteral() != null) - ? ImmutableMap.of(TableProperties.FORMAT_VERSION, "3") - : ImmutableMap.of(); + || TypeUtil.find(writeSchema, t -> t.typeId() == Type.TypeID.UNKNOWN) != null; + Map tableProperties = + requiresV3 ? ImmutableMap.of(TableProperties.FORMAT_VERSION, "3") : ImmutableMap.of(); Table table = tables.create( writeSchema, PartitionSpec.unpartitioned(), tableProperties, location.toString()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java index 35be6423ee23..e833e9f54ba2 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java @@ -18,9 +18,13 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.spark.SparkException; +import org.junit.jupiter.api.Test; public class TestORCDataFrameWrite extends DataFrameWriteTestBase { @Override @@ -30,4 +34,24 @@ protected void configureTable(Table table) { .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.ORC.toString()) .commit(); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java index 90a9ac48a486..c24d92ef30af 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java @@ -18,9 +18,13 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.spark.SparkException; +import org.junit.jupiter.api.Test; public class TestParquetDataFrameWrite extends DataFrameWriteTestBase { @Override @@ -30,4 +34,24 @@ protected void configureTable(Table table) { .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.toString()) .commit(); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java index 6b9ec85b7f0b..6056f1a7929d 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; @@ -37,6 +38,7 @@ import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; public class TestParquetScan extends ScanTestBase { protected boolean vectorized() { @@ -84,4 +86,20 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw super.writeAndValidate(writeSchema, expectedSchema); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java index fbd21f737450..fec413ca079a 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java @@ -41,6 +41,7 @@ import org.apache.spark.sql.types.IntegerType$; import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -238,5 +239,6 @@ public Type primitive(Type.PrimitiveType primitive) { .put(TypeID.STRING, ImmutableSet.of(StringType$.class)) .put(TypeID.FIXED, ImmutableSet.of(BinaryType$.class)) .put(TypeID.BINARY, ImmutableSet.of(BinaryType$.class)) + .put(TypeID.UNKNOWN, ImmutableSet.of(NullType$.class)) .buildOrThrow(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index 704b8d458115..d33632bbbd54 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -36,7 +36,6 @@ import org.apache.spark.sql.types.IntegerType$; import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.MapType$; -import org.apache.spark.sql.types.NullType$; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.NullType$; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java index 9480385d5452..e11a85d538a6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java @@ -31,6 +31,7 @@ import org.apache.parquet.schema.Type.Repetition; import org.apache.spark.sql.types.ArrayType; import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.MapType; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -173,21 +174,27 @@ private static T visitField( private static List visitFields( StructType struct, GroupType group, ParquetWithSparkSchemaVisitor visitor) { - StructField[] sFields = struct.fields(); - Preconditions.checkArgument( - sFields.length == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); - for (int i = 0; i < sFields.length; i += 1) { - Type field = group.getFields().get(i); - StructField sField = sFields[i]; - Preconditions.checkArgument( - field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.name())), - "Structs do not match: field %s != %s", - field.getName(), - sField.name()); - results.add(visitField(sField, field, visitor)); + + int fieldIndex = 0; + for (StructField sField : struct.fields()) { + if (sField.dataType() != DataTypes.NullType) { + Type field = group.getFields().get(fieldIndex); + Preconditions.checkArgument( + field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.name())), + "Structs do not match: field %s != %s", + field.getName(), + sField.name()); + results.add(visitField(sField, field, visitor)); + + fieldIndex += 1; + } } + // All the group fields should have been visited + Preconditions.checkArgument( + fieldIndex == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); + return results; } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index 6b799e677bf4..6fc8849c82b2 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -20,6 +20,8 @@ import java.io.Serializable; import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.Stream; import javax.annotation.Nullable; import org.apache.iceberg.FieldMetrics; @@ -77,7 +79,7 @@ public OrcValueWriter record( TypeDescription record, List names, List> fields) { - return new InternalRowWriter(fields, record.getChildren()); + return new InternalRowWriter(fields, iStruct, record.getChildren()); } @Override @@ -133,12 +135,16 @@ public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, TypeDescriptio private static class InternalRowWriter extends GenericOrcWriters.StructWriter { private final List> fieldGetters; - InternalRowWriter(List> writers, List orcTypes) { - super(writers); + InternalRowWriter( + List> writers, Types.StructType iStruct, List orcTypes) { + super(iStruct, writers); this.fieldGetters = Lists.newArrayListWithExpectedSize(orcTypes.size()); - for (TypeDescription orcType : orcTypes) { - fieldGetters.add(createFieldGetter(orcType)); + Map idToType = + orcTypes.stream().collect(Collectors.toMap(ORCSchemaUtil::fieldId, s -> s)); + + for (Types.NestedField iField : iStruct.fields()) { + fieldGetters.add(createFieldGetter(idToType.get(iField.fieldId()))); } } @@ -149,6 +155,11 @@ protected Object get(InternalRow struct, int index) { } static FieldGetter createFieldGetter(TypeDescription fieldType) { + // In the case of an UnknownType + if (fieldType == null) { + return (row, ordinal) -> null; + } + final FieldGetter fieldGetter; switch (fieldType.getCategory()) { case BOOLEAN: 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 58be7f610c81..a1ed6c66f337 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 @@ -26,6 +26,7 @@ import java.util.NoSuchElementException; import java.util.Optional; import java.util.UUID; +import java.util.stream.IntStream; import org.apache.iceberg.Schema; import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; import org.apache.iceberg.parquet.ParquetValueWriter; @@ -55,6 +56,7 @@ import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType; import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -94,14 +96,18 @@ public ParquetValueWriter message( public ParquetValueWriter struct( StructType sStruct, GroupType struct, List> fieldWriters) { List fields = struct.getFields(); - StructField[] sparkFields = sStruct.fields(); List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); - List sparkTypes = Lists.newArrayList(); for (int i = 0; i < fields.size(); i += 1) { writers.add(newOption(struct.getType(i), fieldWriters.get(i))); - sparkTypes.add(sparkFields[i].dataType()); } - return new InternalRowWriter(writers, sparkTypes); + + StructField[] sFields = sStruct.fields(); + DataType[] types = new DataType[sFields.length]; + for (int i = 0; i < sFields.length; i += 1) { + types[i] = sFields[i].dataType(); + } + + return new InternalRowWriter(writers, types); } @Override @@ -565,14 +571,33 @@ public Map.Entry next() { private static class InternalRowWriter extends ParquetValueWriters.StructWriter { private final DataType[] types; - private InternalRowWriter(List> writers, List types) { - super(writers); - this.types = types.toArray(new DataType[0]); + private InternalRowWriter(List> writers, DataType[] types) { + super(writerToFieldIndex(types, writers.size()), writers); + this.types = types; } @Override protected Object get(InternalRow struct, int index) { return struct.get(index, types[index]); } + + /** Returns a mapping from writer index to field index, skipping Unknown columns. */ + private static int[] writerToFieldIndex(DataType[] types, int numWriters) { + if (null == types) { + return IntStream.rangeClosed(0, numWriters).toArray(); + } + + // value writer index to record field index + int[] indexes = new int[numWriters]; + int writerIndex = 0; + for (int pos = 0; pos < types.length; pos += 1) { + if (!(types[pos] instanceof NullType)) { + indexes[writerIndex] = pos; + writerIndex += 1; + } + } + + return indexes; + } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java index 0db6a65fd394..45053c1a4f1f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; @@ -108,8 +109,8 @@ protected boolean supportsRowLineage() { required(114, "dec_9_0", Types.DecimalType.of(9, 0)), // int encoded required(115, "dec_11_2", Types.DecimalType.of(11, 2)), // long encoded required(116, "dec_20_5", Types.DecimalType.of(20, 5)), // requires padding - required(117, "dec_38_10", Types.DecimalType.of(38, 10)) // Spark's maximum precision - ); + required(117, "dec_38_10", Types.DecimalType.of(38, 10)), // Spark's maximum precision + optional(118, "unk", Types.UnknownType.get())); @TempDir protected Path temp; @@ -120,10 +121,13 @@ public void testSimpleStruct() throws IOException { @Test public void testStructWithRequiredFields() throws IOException { + List supportedPrimitives = + SUPPORTED_PRIMITIVES.fields().stream() + .filter(f -> f.type().typeId() != Type.TypeID.UNKNOWN) + .collect(Collectors.toList()); writeAndValidate( TypeUtil.assignIncreasingFreshIds( - new Schema( - Lists.transform(SUPPORTED_PRIMITIVES.fields(), Types.NestedField::asRequired)))); + new Schema(Lists.transform(supportedPrimitives, Types.NestedField::asRequired)))); } @Test @@ -603,4 +607,48 @@ public void testRowLineage() throws Exception { record.copy(Map.of("id", 4L, "data", "d", "_row_id", 1_001L)), record.copy(Map.of("id", 5L, "data", "e")))); } + + @Test + public void testUnknownNestedLevel() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(1, "id", LongType.get()), + optional( + 2, + "nested", + Types.StructType.of( + required(20, "int", Types.IntegerType.get()), + optional(21, "unk", Types.UnknownType.get())))); + + writeAndValidate(schema); + } + + @Test + public void testUnknownListType() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional(1, "data", ListType.ofOptional(2, Types.UnknownType.get()))); + + writeAndValidate(schema); + } + + @Test + public void testUnknownMapType() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional( + 1, + "data", + MapType.ofOptional(2, 3, Types.StringType.get(), Types.UnknownType.get()))); + + writeAndValidate(schema); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java index 3b68a830b088..3fcfe6845c99 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -107,4 +108,20 @@ private void writeAndValidateRecords(Schema schema, Iterable expect private Iterator batchesToRows(Iterator batches) { return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index 328dcaa0014c..bc4b77059d43 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -248,4 +248,20 @@ public void testMissingRequiredWithoutDefault() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Missing required field: missing_str"); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java index bf738be59cb8..634327a81d86 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -150,4 +151,20 @@ private static void assertEqualsUnsafe( assertThat(expectedIter).as("Expected iterator should not have any extra rows.").isExhausted(); assertThat(actualIter).as("Actual iterator should not have any extra rows.").isExhausted(); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java index 39ea25ae6f54..aa8966877021 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java @@ -41,6 +41,7 @@ import org.apache.iceberg.spark.data.AvroDataTestBase; import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; @@ -95,11 +96,12 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw HadoopTables tables = new HadoopTables(CONF); // If V3 spec features are used, set the format version to 3 - Map tableProperties = + boolean requiresV3 = writeSchema.columns().stream() .anyMatch(f -> f.initialDefaultLiteral() != null || f.writeDefaultLiteral() != null) - ? ImmutableMap.of(TableProperties.FORMAT_VERSION, "3") - : ImmutableMap.of(); + || TypeUtil.find(writeSchema, t -> t.typeId() == Type.TypeID.UNKNOWN) != null; + Map tableProperties = + requiresV3 ? ImmutableMap.of(TableProperties.FORMAT_VERSION, "3") : ImmutableMap.of(); Table table = tables.create( writeSchema, PartitionSpec.unpartitioned(), tableProperties, location.toString()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java index 35be6423ee23..892e260f66f0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java @@ -18,9 +18,13 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.spark.SparkException; +import org.junit.jupiter.api.Test; public class TestORCDataFrameWrite extends DataFrameWriteTestBase { @Override @@ -30,4 +34,24 @@ protected void configureTable(Table table) { .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.ORC.toString()) .commit(); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java index 90a9ac48a486..c24d92ef30af 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java @@ -18,9 +18,13 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.spark.SparkException; +import org.junit.jupiter.api.Test; public class TestParquetDataFrameWrite extends DataFrameWriteTestBase { @Override @@ -30,4 +34,24 @@ protected void configureTable(Table table) { .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.toString()) .commit(); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java index c0dee43d6de1..8b567bcaf11e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; @@ -37,6 +38,7 @@ import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; public class TestParquetScan extends ScanTestBase { protected boolean vectorized() { @@ -83,4 +85,20 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw super.writeAndValidate(writeSchema, expectedSchema); } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } } From 873cfe77fba8bbdfc2a91e66a932b4666979c7ff Mon Sep 17 00:00:00 2001 From: Kurtis Wright Date: Fri, 24 Apr 2026 20:37:08 +0000 Subject: [PATCH 16/16] Spark 3.4: Fix unknown type tests by calling configureTable in ScanTestBase The v3.4 ScanTestBase was missing the configureTable(table) call that was added to v3.5/v4.0 by #12546. Without it, TestAvroDataFrameWrite and TestORCDataFrameWrite wrote through the default (Parquet) format, producing a misleading Parquet error instead of the format's own error for UnknownType in lists/maps. Mirror v4.0 by: - invoking configureTable(table) in ScanTestBase.writeAndValidate - updating TestORCDataFrameWrite's expected messages to the ORC errors --- .../java/org/apache/iceberg/spark/source/ScanTestBase.java | 1 + .../apache/iceberg/spark/source/TestORCDataFrameWrite.java | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java index 21381e824e89..aa8966877021 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java @@ -105,6 +105,7 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw Table table = tables.create( writeSchema, PartitionSpec.unpartitioned(), tableProperties, location.toString()); + configureTable(table); // Important: use the table's schema for the rest of the test // When tables are created, the column ids are reassigned. diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java index e833e9f54ba2..892e260f66f0 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java @@ -42,7 +42,7 @@ public void testUnknownListType() { .isInstanceOf(SparkException.class) .cause() .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + .hasMessageStartingWith("Cannot create ListType with unknown element type"); } @Test @@ -52,6 +52,6 @@ public void testUnknownMapType() { .isInstanceOf(SparkException.class) .cause() .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + .hasMessageStartingWith("Cannot create MapType with unknown value type"); } }