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

[SUPPORT] table comments not fully supported #7531

Open
parisni opened this issue Dec 21, 2022 · 6 comments · May be fixed by #8683
Open

[SUPPORT] table comments not fully supported #7531

parisni opened this issue Dec 21, 2022 · 6 comments · May be fixed by #8683
Labels
priority:minor everything else; usability gaps; questions; feature reqs schema-and-data-types spark Issues related to spark

Comments

@parisni
Copy link
Contributor

parisni commented Dec 21, 2022

Hudi 12.1

When upsert spark DF with comments metadata, then it is present un the Avro shema commited. Also if enabled it is propagated in HMS.
But spark datasource likely omit them while reading. As a result they are hidden when reading from spark

@yihua
Copy link
Contributor

yihua commented Dec 21, 2022

@parisni Thanks for raising this issue. Could you provide more details and reproducible steps? When saying spark DF with comments metadata, do you mean the schema associated with the dataframe has the comments?

@yihua yihua added priority:minor everything else; usability gaps; questions; feature reqs spark Issues related to spark schema-and-data-types labels Dec 21, 2022
@parisni
Copy link
Contributor Author

parisni commented Dec 21, 2022 via email

@parisni
Copy link
Contributor Author

parisni commented Dec 22, 2022

@yihua reproductible example

# add uuid column with comment foo bar
from pyspark.sql.types import StructType, StructField, StringType, IntegerType
data = [
    (1, "f5c2ebfd-f57b-4ff3-ac5c-f30674037b21", "A", "BC", "C"),
    (2, "f5c2ebfd-f57b-4ff3-ac5c-f30674037b22", "A", "BC", "C"),
    (3, "f5c2ebfd-f57b-4ff3-ac5c-f30674037b21", "A", "BC", "C"),
    (4, "f5c2ebfd-f57b-4ff3-ac5c-f30674037b22", "A", "BC", "C"),
]

schema = StructType(
    [
        StructField("uuid", IntegerType(), True, {"comment": "foo bar"}),
        StructField("user_id", StringType(), True),
        StructField("col1", StringType(), True),
        StructField("ts", StringType(), True),
        StructField("part", StringType(), True),
    ]
)
df = spark.createDataFrame(data=data, schema=schema)


tableName = "test_hudi_comment"
basePath = f"/tmp/hudi/"

hudi_options = {
    "hoodie.table.name": tableName,
    "hoodie.datasource.write.recordkey.field": "uuid",
    "hoodie.datasource.write.partitionpath.field": "part",
    "hoodie.datasource.write.table.name": tableName,
    "hoodie.datasource.write.operation": "insert",
    "hoodie.datasource.write.precombine.field": "ts",
    "hoodie.upsert.shuffle.parallelism": 1,
    "hoodie.insert.shuffle.parallelism": 1,
    "hoodie.datasource.hive_sync.enable": "false",
}
(df.write.format("hudi").options(**hudi_options).mode("append").save(basePath))
spark.read.format("hudi").load(basePath).registerTempTable("foo")
spark.sql("desc extended foo").show()

# there is no foo bar on the hudi side
+--------------------+---------+-------+
|            col_name|data_type|comment|
+--------------------+---------+-------+
| _hoodie_commit_time|   string|   null|
|_hoodie_commit_seqno|   string|   null|
|  _hoodie_record_key|   string|   null|
|_hoodie_partition...|   string|   null|
|   _hoodie_file_name|   string|   null|
|                uuid|      int|   null|
|             user_id|   string|   null|
|                col1|   string|   null|
|                  ts|   string|   null|
|                part|   string|   null|
+--------------------+---------+-------+

# the avro has foo bar doc
  "partitionToWriteStats" : {
    "C" : [ {
      "fileId" : "e90400c1-5311-4fdc-83f2-757326c7560d-0",
      "path" : "C/e90400c1-5311-4fdc-83f2-757326c7560d-0_0-17-36_20221222095833220.parquet",
      "prevCommit" : "null",
      "numWrites" : 4,
      "numDeletes" : 0,
      "numUpdateWrites" : 0,
      "numInserts" : 4,
      "totalWriteBytes" : 435614,
      "totalWriteErrors" : 0,
      "tempPath" : null,
      "partitionPath" : "C",
      "totalLogRecords" : 0,
      "totalLogFilesCompacted" : 0,
      "totalLogSizeCompacted" : 0,
      "totalUpdatedRecordsCompacted" : 0,
      "totalLogBlocks" : 0,
      "totalCorruptLogBlock" : 0,
      "totalRollbackBlocks" : 0,
      "fileSizeInBytes" : 435614,
      "minEventTime" : null,
      "maxEventTime" : null
    } ]
  },
  "compacted" : false,
  "extraMetadata" : {
    "schema" : "{\"type\":\"record\",\"name\":\"test_hudi_comment_record\",\"namespace\":\"hoodie.test_hudi_comment\",\"fields\":[{\"name\":\"uuid\",\"type\":[\"null\",\"int\"],\"doc\":\"foo bar\",\"default\":null},{\"name\":\"user_id\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"col1\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"ts\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"part\",\"type\":[\"null\",\"string\"],\"default\":null}]}"
  },
  "operationType" : "INSERT"
}


@xushiyan
Copy link
Member

xushiyan commented Jan 7, 2023

@jonvex can you look into this please? looks like some config fixes should resolve it

@jonvex
Copy link
Contributor

jonvex commented Jan 11, 2023

Verified this issue and created a Jira ticket

@codope
Copy link
Member

codope commented Mar 29, 2023

Tracked in HUDI-5533

@parisni parisni linked a pull request May 10, 2023 that will close this issue
4 tasks
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
priority:minor everything else; usability gaps; questions; feature reqs schema-and-data-types spark Issues related to spark
Projects
Status: 🏁 Triaged
Development

Successfully merging a pull request may close this issue.

5 participants