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

Transaction log JSON formatting issue when writing data via Python bindings #1017

Closed
tobyych opened this issue Dec 14, 2022 — with Slack · 2 comments · Fixed by #1054
Closed

Transaction log JSON formatting issue when writing data via Python bindings #1017

tobyych opened this issue Dec 14, 2022 — with Slack · 2 comments · Fixed by #1054
Labels
binding/python Issues for the Python package bug Something isn't working

Comments

Copy link

tobyych commented Dec 14, 2022

hi there, i am using deltalake 0.6.4 python bindings to try writing delta tables to a S3-compatible storage.

i noticed an issue that the transaction log generated is not entirely the same as the example in https://github.com/delta-io/delta/blob/master/PROTOCOL.md (refer to the image attached, left one is the log generated by deltalake package, right one is the example on GitHub)

image

this caused the data query engine Trino (version: 397)
not able to read the delta log properly.

the relevant part of the error message is:
Caused by: java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.IOException: com.fasterxml.jackson.databind.exc.MismatchedInputException: Cannot deserialize value of type java.lang.String from Array value (token JsonToken.START_ARRAY)
at [Source: (String)"{"commitInfo":{"clientVersion":"delta-rs.0.5.0","operation":"delta-rs.Write","operationParameters":{"mode":"Append","partitionBy":[],"predicate":null},"timestamp":1670980236192}}"; line: 1, column: 131] (through reference chain: io.trino.plugin.deltalake.transactionlog.DeltaLakeTransactionLogEntry["commitInfo"]->io.trino.plugin.deltalake.transactionlog.CommitInfoEntry["operationParameters"]->java.util.LinkedHashMap["partitionBy"])

may i know if there is any method to work around the issue?

Slack Message

@rtyler rtyler added bug Something isn't working binding/python Issues for the Python package labels Dec 14, 2022
@rtyler
Copy link
Member

rtyler commented Dec 16, 2022

As I mentioned in Slack, I think it's not that big of a deal for us to add some quotes around the partitionBy. I think the Trinio plugin also has a bug here because commitInfo is optional in the protocol and therefore I don't believe it should be considered to have a required and agreed upon schema

@sfilimonov-exos
Copy link

delta-spark also fails to read such a table with unquoted partitionBy.

Code to reproduce:

import pandas as pd
from delta import DeltaTable
from delta import configure_spark_with_delta_pip
from deltalake.writer import write_deltalake
from pyspark.sql import SparkSession

spark = configure_spark_with_delta_pip(SparkSession.builder.appName("lo").master("local[*]")) \
    .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") \
    .getOrCreate()

table = "table_12345"

df = pd.DataFrame(data=[("one", 2), ("two", 2)], columns=["name", "val"])

write_deltalake(table, df)
# First write doesn't add partitionBy to commit info - thus need to write one more time 
write_deltalake(table, df, mode="append")

dt = DeltaTable.forPath(spark, table)
dt.toDF().show()  # The table itself can be read without an issue.
dt.history().show()  # But not the history
delta-spark 2.2.0
deltalake 0.6.4
22/12/16 11:43:03 ERROR Executor: Exception in task 9.0 in stage 6.0 (TID 62)
com.fasterxml.jackson.databind.exc.MismatchedInputException: Cannot deserialize value of type `java.lang.String` from Array value (token `JsonToken.START_ARRAY`)
 at [Source: (String)"{"commitInfo":{"clientVersion":"delta-rs.0.5.0","operation":"delta-rs.Write","operationParameters":{"mode":"Append","partitionBy":[],"predicate":null},"timestamp":1671183764077}}"; line: 1, column: 131] (through reference chain: org.apache.spark.sql.delta.actions.SingleAction["commitInfo"]->org.apache.spark.sql.delta.actions.CommitInfo["operationParameters"]->com.fasterxml.jackson.module.scala.deser.GenericMapFactoryDeserializerResolver$BuilderWrapper["partitionBy"])

wjones127 pushed a commit that referenced this issue Jan 17, 2023
# Description
Currently writing "operationParameters" in commit info is misaligned
with delta io connector.


[Here](https://github.com/delta-io/delta/blob/36a7edb8cf507e713700ba827c5fb5ad32b9163e/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala#L695)
the sample of structure which is used in delta io.

So the goal of this PR is to align with delta io approach and the PR do
two thins: convert all values to string and delete keys with null
values.

# Related Issue(s)
Closes [issue #1017](#1017)

Co-authored-by: Ilya Moshkov <ilya.moshkov@exosfinancial.com>
chitralverma pushed a commit to chitralverma/delta-rs that referenced this issue Mar 17, 2023
# Description
Currently writing "operationParameters" in commit info is misaligned
with delta io connector.


[Here](https://github.com/delta-io/delta/blob/36a7edb8cf507e713700ba827c5fb5ad32b9163e/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala#L695)
the sample of structure which is used in delta io.

So the goal of this PR is to align with delta io approach and the PR do
two thins: convert all values to string and delete keys with null
values.

# Related Issue(s)
Closes [issue delta-io#1017](delta-io#1017)

Co-authored-by: Ilya Moshkov <ilya.moshkov@exosfinancial.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
binding/python Issues for the Python package bug Something isn't working
Projects
None yet
3 participants