Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Non-nullable columns implies invariants but invariants not enabled in v3,7 #2882

Open
wahani opened this issue Sep 13, 2024 · 17 comments
Open
Labels
bug Something isn't working

Comments

@wahani
Copy link

wahani commented Sep 13, 2024

Environment

Delta-rs version: python-0.16.0+ (example works until 0.15.3, also tested with 0.19.2)

Binding: Python

Environment:

  • Cloud provider:
  • OS: MacOS Monterey
  • Other:
    • pyspark: 3.5.0
    • pandas: 2.2.2
    • pyarrow: 17.0.0

Bug

What happened:

Beginning with version python-0.16.0+ I receive a DeltaTableFeatureException when reading a delta table with a local spark session. This is related to the newly introduced handling of timezones. In 0.15.3 the write operation will create the following protocol {"protocol":{"minReaderVersion":1,"minWriterVersion":2}} with 0.16.0 and onward we will see {"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["timestampNtz"],"writerFeatures":["timestampNtz"]}}. However, when reading the delta table using a local spark session, I will get the following error message:

py4j.protocol.Py4JJavaError: An error occurred while calling o33.load.
: org.apache.spark.sql.delta.DeltaTableFeatureException: [DELTA_FEATURES_PROTOCOL_METADATA_MISMATCH] Unable to operate on this table because the following table features are enabled in metadata but not listed in protocol: invariants.
        at org.apache.spark.sql.delta.DeltaErrorsBase.tableFeatureMismatchException(DeltaErrors.scala:2281)
        at org.apache.spark.sql.delta.DeltaErrorsBase.tableFeatureMismatchException$(DeltaErrors.scala:2278)
        at org.apache.spark.sql.delta.DeltaErrors$.tableFeatureMismatchException(DeltaErrors.scala:3382)
        at org.apache.spark.sql.delta.DeltaLog.assertTableFeaturesMatchMetadata(DeltaLog.scala:436)
        at org.apache.spark.sql.delta.Snapshot.init(Snapshot.scala:247)
        at org.apache.spark.sql.delta.Snapshot.<init>(Snapshot.scala:531)
        at org.apache.spark.sql.delta.SnapshotManagement.$anonfun$createSnapshot$2(SnapshotManagement.scala:634)
        at org.apache.spark.sql.delta.SnapshotManagement.createSnapshotFromGivenOrEquivalentLogSegment(SnapshotManagement.scala:796)
        at org.apache.spark.sql.delta.SnapshotManagement.createSnapshotFromGivenOrEquivalentLogSegment$(SnapshotManagement.scala:782)
        at org.apache.spark.sql.delta.DeltaLog.createSnapshotFromGivenOrEquivalentLogSegment(DeltaLog.scala:74)
        at org.apache.spark.sql.delta.SnapshotManagement.createSnapshot(SnapshotManagement.scala:627)
        at org.apache.spark.sql.delta.SnapshotManagement.createSnapshot$(SnapshotManagement.scala:618)
        at org.apache.spark.sql.delta.DeltaLog.createSnapshot(DeltaLog.scala:74)
        at org.apache.spark.sql.delta.SnapshotManagement.$anonfun$getSnapshotForLogSegmentInternal$1(SnapshotManagement.scala:1043)
        at scala.Option.map(Option.scala:230)
        at org.apache.spark.sql.delta.SnapshotManagement.getSnapshotForLogSegmentInternal(SnapshotManagement.scala:1036)
        at org.apache.spark.sql.delta.SnapshotManagement.getSnapshotForLogSegmentInternal$(SnapshotManagement.scala:1031)
        at org.apache.spark.sql.delta.DeltaLog.getSnapshotForLogSegmentInternal(DeltaLog.scala:74)
        at org.apache.spark.sql.delta.SnapshotManagement.getUpdatedSnapshot(SnapshotManagement.scala:1012)
        at org.apache.spark.sql.delta.SnapshotManagement.getUpdatedSnapshot$(SnapshotManagement.scala:1003)
        at org.apache.spark.sql.delta.DeltaLog.getUpdatedSnapshot(DeltaLog.scala:74)
        at org.apache.spark.sql.delta.SnapshotManagement.$anonfun$getSnapshotAtInit$2(SnapshotManagement.scala:583)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile(DeltaLogging.scala:168)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile$(DeltaLogging.scala:166)
        at org.apache.spark.sql.delta.DeltaLog.recordFrameProfile(DeltaLog.scala:74)
        at org.apache.spark.sql.delta.SnapshotManagement.$anonfun$getSnapshotAtInit$1(SnapshotManagement.scala:573)
        at org.apache.spark.sql.delta.SnapshotManagement.withSnapshotLockInterruptibly(SnapshotManagement.scala:78)
        at org.apache.spark.sql.delta.SnapshotManagement.withSnapshotLockInterruptibly$(SnapshotManagement.scala:75)
        at org.apache.spark.sql.delta.DeltaLog.withSnapshotLockInterruptibly(DeltaLog.scala:74)
        at org.apache.spark.sql.delta.SnapshotManagement.getSnapshotAtInit(SnapshotManagement.scala:573)
        at org.apache.spark.sql.delta.SnapshotManagement.getSnapshotAtInit$(SnapshotManagement.scala:572)
        at org.apache.spark.sql.delta.DeltaLog.getSnapshotAtInit(DeltaLog.scala:74)
        at org.apache.spark.sql.delta.SnapshotManagement.$init$(SnapshotManagement.scala:69)
        at org.apache.spark.sql.delta.DeltaLog.<init>(DeltaLog.scala:80)
        at org.apache.spark.sql.delta.DeltaLog$.$anonfun$apply$4(DeltaLog.scala:853)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:323)
        at org.apache.spark.sql.delta.DeltaLog$.$anonfun$apply$3(DeltaLog.scala:848)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile(DeltaLogging.scala:168)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile$(DeltaLogging.scala:166)
        at org.apache.spark.sql.delta.DeltaLog$.recordFrameProfile(DeltaLog.scala:651)
        at org.apache.spark.sql.delta.metering.DeltaLogging.$anonfun$recordDeltaOperationInternal$1(DeltaLogging.scala:136)
        at com.databricks.spark.util.DatabricksLogging.recordOperation(DatabricksLogging.scala:128)
        at com.databricks.spark.util.DatabricksLogging.recordOperation$(DatabricksLogging.scala:117)
        at org.apache.spark.sql.delta.DeltaLog$.recordOperation(DeltaLog.scala:651)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordDeltaOperationInternal(DeltaLogging.scala:135)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordDeltaOperation(DeltaLogging.scala:125)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordDeltaOperation$(DeltaLogging.scala:115)
        at org.apache.spark.sql.delta.DeltaLog$.recordDeltaOperation(DeltaLog.scala:651)
        at org.apache.spark.sql.delta.DeltaLog$.createDeltaLog$1(DeltaLog.scala:847)
        at org.apache.spark.sql.delta.DeltaLog$.$anonfun$apply$5(DeltaLog.scala:866)
        at com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4792)
        at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
        at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
        at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
        at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2257)
        at com.google.common.cache.LocalCache.get(LocalCache.java:4000)
        at com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4789)
        at org.apache.spark.sql.delta.DeltaLog$.getDeltaLogFromCache$1(DeltaLog.scala:865)
        at org.apache.spark.sql.delta.DeltaLog$.apply(DeltaLog.scala:875)
        at org.apache.spark.sql.delta.DeltaLog$.forTable(DeltaLog.scala:751)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.$anonfun$deltaLog$1(DeltaTableV2.scala:92)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2$.withEnrichedUnsupportedTableException(DeltaTableV2.scala:367)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.deltaLog$lzycompute(DeltaTableV2.scala:92)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.deltaLog(DeltaTableV2.scala:90)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.$anonfun$initialSnapshot$4(DeltaTableV2.scala:145)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.$anonfun$initialSnapshot$1(DeltaTableV2.scala:145)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2$.withEnrichedUnsupportedTableException(DeltaTableV2.scala:367)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.initialSnapshot$lzycompute(DeltaTableV2.scala:144)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.initialSnapshot(DeltaTableV2.scala:124)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.toBaseRelation$lzycompute(DeltaTableV2.scala:236)
        at org.apache.spark.sql.delta.catalog.DeltaTableV2.toBaseRelation(DeltaTableV2.scala:234)
        at org.apache.spark.sql.delta.sources.DeltaDataSource.$anonfun$createRelation$5(DeltaDataSource.scala:250)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile(DeltaLogging.scala:168)
        at org.apache.spark.sql.delta.metering.DeltaLogging.recordFrameProfile$(DeltaLogging.scala:166)
        at org.apache.spark.sql.delta.sources.DeltaDataSource.recordFrameProfile(DeltaDataSource.scala:49)
        at org.apache.spark.sql.delta.sources.DeltaDataSource.createRelation(DeltaDataSource.scala:209)
        at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:346)
        at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:229)
        at org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:211)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:211)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:186)
        at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:566)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:374)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
        at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
        at java.base/java.lang.Thread.run(Thread.java:829)

Adding 'invariants' to the writerFeatures manually will resolve the issue. So far I haven't been able to find an option to add the feature explicitly using the python bindings. E.g. by providing table configuration like "delta.minReaderVersion" or "delta.writerFeatures" - they seem to be ignored or not correct.

What you expected to happen:

I can write timestamp data with deltalake and read it back in using spark without any additional configuration (0.15.3 behavior).

How to reproduce it:

import pyarrow as pa
from deltalake import write_deltalake
from pandas import DataFrame
from pyspark.sql import SparkSession

delta_table_path = "delta-table"

schema = pa.schema(
    [
        pa.field("id", pa.int16(), False),
        pa.field("values", pa.float64(), False),
        pa.field("date", pa.timestamp("us"), True),
    ]
)

write_deltalake(
    delta_table_path,
    data=DataFrame({"id": [1, 2], "values": [2, 1], "date": [None, None]}),
    schema=schema,
    configuration={"delta.minReaderVersion": "1", "delta.minWriterVersion": "2"} # silently ignored
)

spark = (
    SparkSession.builder.appName("failing-delta-load")
    .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
    .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog")
    .config("spark.jars", "delta-spark_2.12-3.2.0.jar,delta-storage-3.2.0.jar")
    .getOrCreate()
)

df = spark.read.format("delta").load(delta_table_path)

for row in df.collect():
    print(row)

Thanks for any help or guidance with this.

@wahani wahani added the bug Something isn't working label Sep 13, 2024
@ion-elgreco
Copy link
Collaborator

For some reason the protocol requires invariants to be enabled even if it's V7. I honestly don't see why this is required since the idea behind table features is that you can opt in

@wahani
Copy link
Author

wahani commented Sep 13, 2024

@ion-elgreco I agree. Meanwhile I also created the same table using pyspark and see no difference in the protocol it creates, so maybe it is something else.

@ion-elgreco
Copy link
Collaborator

@ion-elgreco I agree. Meanwhile I also created the same table using pyspark and see no difference in the protocol it creates, so maybe it is something else.

Can you share the first json log of both tables, can just paste them as code or share as gists

@ion-elgreco
Copy link
Collaborator

@wahani once this lands: #2712, you can manually add invariants to a table ;)

@wahani
Copy link
Author

wahani commented Sep 16, 2024

@ion-elgreco I agree. Meanwhile I also created the same table using pyspark and see no difference in the protocol it creates, so maybe it is something else.

Can you share the first json log of both tables, can just paste them as code or share as gists

python-0.19.1

{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["timestampNtz"],"writerFeatures":["timestampNtz"]}}
{"metaData":{"id":"6950fa52-78c5-4e69-8c62-14d6da440d58","name":null,"description":null,"format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"short\",\"nullable\":false,\"metadata\":{}},{\"name\":\"values\",\"type\":\"double\",\"nullable\":false,\"metadata\":{}},{\"name\":\"date\",\"type\":\"timestamp_ntz\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1726471364355,"configuration":{}}}
{"add":{"path":"part-00001-57ecdc29-2b6c-4c64-805d-537216102e83-c000.snappy.parquet","partitionValues":{},"size":1179,"modificationTime":1726471364356,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"values\":1.0,\"id\":1},\"maxValues\":{\"values\":2.0,\"id\":2},\"nullCount\":{\"id\":0,\"date\":2,\"values\":0}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}}
{"commitInfo":{"timestamp":1726471364356,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists"},"clientVersion":"delta-rs.0.19.1"}}

python-0.15.3

{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"metaData":{"id":"ee5190ed-3786-4db9-b518-6b97d343a975","name":null,"description":null,"format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"short\",\"nullable\":false,\"metadata\":{}},{\"name\":\"values\",\"type\":\"double\",\"nullable\":false,\"metadata\":{}},{\"name\":\"date\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1726471484655,"configuration":{}}}
{"add":{"path":"0-f28acb3f-a592-43f7-b163-1e6cc6740429-0.parquet","partitionValues":{},"size":1172,"modificationTime":1726471484654,"dataChange":true,"stats":"{\"numRecords\": 2, \"minValues\": {\"id\": 1, \"values\": 1.0}, \"maxValues\": {\"id\": 2, \"values\": 2.0}, \"nullCount\": {\"id\": 0, \"values\": 0, \"date\": 2}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}}
{"commitInfo":{"timestamp":1726471484655,"operation":"CREATE TABLE","operationParameters":{"location":"file:///.../zz-delta-table","protocol":"{\"minReaderVersion\":1,\"minWriterVersion\":2}","mode":"ErrorIfExists","metadata":"{\"configuration\":{},\"createdTime\":1726471484655,\"description\":null,\"format\":{\"options\":{},\"provider\":\"parquet\"},\"id\":\"ee5190ed-3786-4db9-b518-6b97d343a975\",\"name\":null,\"partitionColumns\":[],\"schemaString\":\"{\\\"type\\\":\\\"struct\\\",\\\"fields\\\":[{\\\"name\\\":\\\"id\\\",\\\"type\\\":\\\"short\\\",\\\"nullable\\\":false,\\\"metadata\\\":{}},{\\\"name\\\":\\\"values\\\",\\\"type\\\":\\\"double\\\",\\\"nullable\\\":false,\\\"metadata\\\":{}},{\\\"name\\\":\\\"date\\\",\\\"type\\\":\\\"timestamp\\\",\\\"nullable\\\":true,\\\"metadata\\\":{}}]}\"}"},"clientVersion":"delta-rs.0.17.0"}}

pyspark-3.5.0

{"commitInfo":{"timestamp":1726471612932,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"3","numOutputRows":"2","numOutputBytes":"2247"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.2.0","txnId":"c2be61a6-4831-4b3b-8599-937b00ce781c"}}
{"metaData":{"id":"49fd7cb1-1ec1-4cd0-9415-5686380e8f66","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"values\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}},{\"name\":\"date\",\"type\":\"timestamp_ntz\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1726471610014}}
{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["timestampNtz"],"writerFeatures":["timestampNtz"]}}
{"add":{"path":"part-00004-2e0632a4-9306-4fb7-9eb8-80caa199d2dc-c000.snappy.parquet","partitionValues":{},"size":887,"modificationTime":1726471612873,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":1,\"values\":2.0},\"maxValues\":{\"id\":1,\"values\":2.0},\"nullCount\":{\"id\":0,\"values\":0,\"date\":1}}"}}
{"add":{"path":"part-00009-9b8d1a42-dfa9-43b2-9c77-7550d83e0754-c000.snappy.parquet","partitionValues":{},"size":887,"modificationTime":1726471612873,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"id\":2,\"values\":1.0},\"maxValues\":{\"id\":2,\"values\":1.0},\"nullCount\":{\"id\":0,\"values\":0,\"date\":1}}"}}

@ion-elgreco
Copy link
Collaborator

@wahani there is actually no difference between spark-delta and delta-rs here (python-0.15.3 you have to ignore though since that was never valid).

Are you sure your table does not have delta.invariants somewhere?

@wahani
Copy link
Author

wahani commented Sep 18, 2024

@ion-elgreco well it is just the code I provide that reproduces the error. Strictly speaking I would say the nullable property of a column is an invariant, but I don't know how those are handled really, or how those are represented other than in the metadata.

As I wrote, I think the problem must be something else then the protocol, it was just the error message from pyspark pointing into that direction.

@ion-elgreco
Copy link
Collaborator

@wahani if you create a table with nullable only columns, it does work then?

@wahani
Copy link
Author

wahani commented Sep 18, 2024

@ion-elgreco Seems to be the right direction. I can confirm with

schema = pa.schema(
    [
        pa.field("id", pa.int16(), True),
        pa.field("values", pa.float64(), True),
        pa.field("date", pa.timestamp("us"), True),
    ]
)

it does work. Tested now with python-0.20.0.

@ion-elgreco
Copy link
Collaborator

Ok so nullable columns implies invariants I guess, will take a look at this in couple weeks

@wahani
Copy link
Author

wahani commented Sep 18, 2024

To further complicate things, also adding a timezone to the timestamp data but keeping the nullable constraint works. I.e. schema

schema = pa.schema(
    [
        pa.field("id", pa.int16(), False),
        pa.field("values", pa.float64(), False),
        pa.field("date", pa.timestamp("us", "UTC"), True),
    ]
)

will not result in an error with pyspark. This is also how I resolved the issue in our codebase.

@ion-elgreco
Copy link
Collaborator

@wahani that creates a table without table features so that's why it works because then invariants is always enabled in lower than 3,7

@wahani
Copy link
Author

wahani commented Sep 18, 2024

Thanks for clarifying.

@ion-elgreco ion-elgreco changed the title Creating delta table with timestampNtz will create spark-incompatible delta protocol Non-nullable columns implies invariants but invariants not enabled in v3,7 Sep 18, 2024
@ion-elgreco
Copy link
Collaborator

@wahani can you please recreate that spark table with nullable False, the previous one you shared had nullable is True on all cols, so it's not entirely clear to me how the logs will look like then

@wahani
Copy link
Author

wahani commented Oct 1, 2024

Sure. With the schema:

schema = pa.schema(
    [
        pa.field("id", pa.int16(), False),
        pa.field("values", pa.float64(), False),
        pa.field("date", pa.timestamp("us", "UTC"), False),
    ]
)

we get the following metadata:

{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"metaData":{"id":"20dc260d-15ba-4271-bb18-152d971bf71e","name":null,"description":null,"format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"short\",\"nullable\":false,\"metadata\":{}},{\"name\":\"values\",\"type\":\"double\",\"nullable\":false,\"metadata\":{}},{\"name\":\"date\",\"type\":\"timestamp\",\"nullable\":false,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1727786433560,"configuration":{}}}
{"add":{"path":"part-00001-2d622120-57a6-4e1f-b433-ad0947430aff-c000.snappy.parquet","partitionValues":{},"size":1310,"modificationTime":1727786433600,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"date\":\"2024-10-01T14:40:33.545349Z\",\"values\":1.0,\"id\":1},\"maxValues\":{\"id\":2,\"values\":2.0,\"date\":\"2024-10-01T14:40:33.545355Z\"},\"nullCount\":{\"id\":0,\"date\":0,\"values\":0}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}}
{"commitInfo":{"timestamp":1727786433601,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists"},"operationMetrics":{"execution_time_ms":46,"num_added_files":1,"num_added_rows":2,"num_partitions":0,"num_removed_files":0},"clientVersion":"delta-rs.0.20.0"}}

@ion-elgreco
Copy link
Collaborator

ion-elgreco commented Oct 1, 2024

Sure. With the schema:

schema = pa.schema(
    [
        pa.field("id", pa.int16(), False),
        pa.field("values", pa.float64(), False),
        pa.field("date", pa.timestamp("us", "UTC"), False),
    ]
)

we get the following metadata:

{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"metaData":{"id":"20dc260d-15ba-4271-bb18-152d971bf71e","name":null,"description":null,"format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"short\",\"nullable\":false,\"metadata\":{}},{\"name\":\"values\",\"type\":\"double\",\"nullable\":false,\"metadata\":{}},{\"name\":\"date\",\"type\":\"timestamp\",\"nullable\":false,\"metadata\":{}}]}","partitionColumns":[],"createdTime":1727786433560,"configuration":{}}}
{"add":{"path":"part-00001-2d622120-57a6-4e1f-b433-ad0947430aff-c000.snappy.parquet","partitionValues":{},"size":1310,"modificationTime":1727786433600,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"date\":\"2024-10-01T14:40:33.545349Z\",\"values\":1.0,\"id\":1},\"maxValues\":{\"id\":2,\"values\":2.0,\"date\":\"2024-10-01T14:40:33.545355Z\"},\"nullCount\":{\"id\":0,\"date\":0,\"values\":0}}","tags":null,"deletionVector":null,"baseRowId":null,"defaultRowCommitVersion":null,"clusteringProvider":null}}
{"commitInfo":{"timestamp":1727786433601,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists"},"operationMetrics":{"execution_time_ms":46,"num_added_files":1,"num_added_rows":2,"num_partitions":0,"num_removed_files":0},"clientVersion":"delta-rs.0.20.0"}}

You need to re-create it with spark ;)

@Kimahriman
Copy link
Contributor

Kimahriman commented Oct 9, 2024

Yes Delta-Spark automatically creates (incorrect) "not-null invariants" for non-nullable columns, enabling the "invariant" feature: https://github.com/delta-io/delta/blob/8e0b133f46f641941ad15ed8cbe7c2d1cc777a5b/spark/src/main/scala/org/apache/spark/sql/delta/constraints/Invariants.scala#L73. I've been trying to get the arguably incorrect handling of nested not-null columns fixed for a while: delta-io/delta#860

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

No branches or pull requests

3 participants