You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "d-tw (via GitHub)" <gi...@apache.org> on 2023/06/05 12:21:15 UTC

[GitHub] [iceberg] d-tw opened a new issue, #7771: Non-nullable columns marked as nullable during table creation

d-tw opened a new issue, #7771:
URL: https://github.com/apache/iceberg/issues/7771

   ### Apache Iceberg version
   
   1.2.1 (latest release)
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   When creating an iceberg table via Spark using a Glue catalog, the non-nullability of columns is not preserved - all columns are stored as nullable.
   I don't know if this is a bug, or a config issue, but I'm using a vanilla config and couldn't find anything in the docs.
   
   Launching spark as follows:
   
   ```sh
   /opt/spark/bin/pyspark --packages "org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.0.0,software.amazon.awssdk:bundle:2.17.178,software.amazon.awssdk:url-connection-client:2.17.178" \
       --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
       --conf spark.sql.catalog.my_catalog.warehouse=/unused-warehouse \
       --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
       --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \
       --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \
       --conf spark.sql.catalog.spark_catalog.type=hive \
       --conf spark.hadoop.fs.s3a.access.key=$AWS_ACCESS_KEY_ID \
       --conf spark.hadoop.fs.s3a.secret.key=$AWS_SECRET_ACCESS_KEY \
       --conf "spark.driver.extraJavaOptions=-Dsoftware.amazon.awssdk.http.service.impl=software.amazon.awssdk.http.urlconnection.UrlConnectionSdkHttpService -Dderby.system.home=/tmp/derby -Divy.cache.dir=/tmp -Divy.home=/tmp" \
       --conf "spark.executor.extraJavaOptions=-Dsoftware.amazon.awssdk.http.service.impl=software.amazon.awssdk.http.urlconnection.UrlConnectionSdkHttpService -Dderby.system.home=/tmp/derby -Divy.cache.dir=/tmp -Divy.home=/tmp"
   ```
   
   
   Running the following commands:
   
   ```py
   from pyspark.sql.types import StructType, StructField, StringType
   
   schema = StructType([StructField('foo', StringType(), nullable=False)])
   
   df = spark.createDataFrame([], schema=schema)
   
   df.printSchema()
   # root
   # |-- foo: string (nullable = false)
   
   df.writeTo('my_table_with_non_null_fields').create()
   
   spark.table('my_table_with_non_null_fields').printSchema()
   # root
   # |-- foo: string (nullable = true)
   ```
   
   When I check the contents of the Glue catalog, I find:
   
   ```json
   "Columns": [
       {
           "Name": "foo",
           "Type": "string",
           "Parameters": {
               "iceberg.field.current": "true",
               "iceberg.field.id": "1",
               "iceberg.field.optional": "true"
           }
       }
   ]
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] Non-nullable columns marked as nullable during table creation [iceberg]

Posted by "zhongyujiang (via GitHub)" <gi...@apache.org>.
zhongyujiang commented on issue #7771:
URL: https://github.com/apache/iceberg/issues/7771#issuecomment-2026461665

   > Do you know if this feature will added anytime soon?
   
   @va-bo-101  I'm not sure, I think I can investigate more on this later and file a PR to support it, so the community can review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] Non-nullable columns marked as nullable during table creation [iceberg]

Posted by "zhongyujiang (via GitHub)" <gi...@apache.org>.
zhongyujiang commented on issue #7771:
URL: https://github.com/apache/iceberg/issues/7771#issuecomment-2022611858

   @vbmarsexpress I just took a look at the latest code of Iceberg SparkCatalog and found that the Iceberg data source does not yet support this feature in Spark. So  I think we are currently unable to achieve this. Maybe we can bring this in. 
   cc: @aokolnychyi 
   
   > However, I see that the parquet files have the expected schema with the right annotation(required/optional).
   
   Are you saying that the fields are optional in the table schema, but required in the Parquet schema? I think this is because the schema of Parquet file is dervied from Spark DataFrame instead of Iceberg table schema, this should be fine.
   
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] Non-nullable columns marked as nullable during table creation [iceberg]

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi closed issue #7771: Non-nullable columns marked as nullable during table creation
URL: https://github.com/apache/iceberg/issues/7771


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] Non-nullable columns marked as nullable during table creation [iceberg]

Posted by "va-bo-101 (via GitHub)" <gi...@apache.org>.
va-bo-101 commented on issue #7771:
URL: https://github.com/apache/iceberg/issues/7771#issuecomment-2023865578

   @zhongyujiang, that is correct. When using the above data frame approach to create iceberg tables, the parquet schema says a field is required, but the iceberg table metadata says the field is not required, irrespective of the schema of that field. 
   
   > Maybe we can bring this in.
   Do you know if this feature will added anytime soon? 
   
   
   On a related note, we can mitigate this if we are using a direct create table command using spark-sql. But there is one limitation with spark-sql's create statement; it does not allow setting not-null on array-element fields and map-value fields. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] Non-nullable columns marked as nullable during table creation [iceberg]

Posted by "vamsibokam (via GitHub)" <gi...@apache.org>.
vamsibokam commented on issue #7771:
URL: https://github.com/apache/iceberg/issues/7771#issuecomment-2022070117

   @zhongyujiang, Could you elaborate on how to preserve the nullability constraints using the above [change](https://github.com/apache/spark/pull/41070) mentioned above? Should we set some kind of config property or table property to have the right nullability constraint in the final iceberg table?
   
   A little more context: I am using spark(3.5.1) and iceberg-spark(1.5.0), which should have the above change. I am seeing this issue when creating iceberg tables using Pyspark like above. However, I see that the parquet files have the expected schema with the right annotation(required/optional). 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang commented on issue #7771: Non-nullable columns marked as nullable during table creation

Posted by "zhongyujiang (via GitHub)" <gi...@apache.org>.
zhongyujiang commented on issue #7771:
URL: https://github.com/apache/iceberg/issues/7771#issuecomment-1580708249

   @d-tw I think this is because Spark intentionally uses a nullable schema when running CTAS/RTAS,  see this [PR](https://github.com/apache/spark/pull/25536) for more context. And Spark recently [introduced](https://github.com/apache/spark/pull/41070) a way to allow users reserve nullability, should be helpful for your case.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


Re: [I] Non-nullable columns marked as nullable during table creation [iceberg]

Posted by "zhongyujiang (via GitHub)" <gi...@apache.org>.
zhongyujiang commented on issue #7771:
URL: https://github.com/apache/iceberg/issues/7771#issuecomment-2029673825

   @va-bo-101 Just created #10074 to fix this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org