delta icon indicating copy to clipboard operation
delta copied to clipboard

[Spark] Resolves #1679 issue glue catalog

Open calixtofelipe opened this issue 1 year ago • 10 comments

Which Delta project/connector is this regarding?

  • [x] Spark

Description

  • This PR resolves issue 1 in the issue #1679
  • ** Attention **
    • only the issue 1 has been solved in this PR
    • The issue 2 still unsolved in the #1679
  • The PR will change how the catalog schema is saved in Hive Metastore.
  • Technical details:
    • Added a new boolean parameter spark.databricks.delta.fixSchema.GlueCatalog in DeltaSqlConf.
    • When this parameter is true, then in the class CreateDeltaTableCommand: 1 - In cleanupTableDefinition function, the schema will be updated with the table schema 2 - In updateCatalog function, after create a table in the catalog it will update the table schema using a session catalog function (alterTableDataSchema)
  • Describe why we need the change.
  • When we are using AWS Glue Catalog, the catalog can't recognize the schema, issue #1679 . This PR will solve the problem.

How was this patch tested?

I created 2 tests in DeltaTableBuilderSuite: "Test schema external table delta glue catalog conf activated" "Test schema delta glue catalog conf activated" These tests just check if managed and external table will create the schema correctly when the parameter activated. But the solution was tested in AWS glue catalog, creating. the tables and check in glue catalog if the table has the right schema and check if Athena can read the table.

Follow the 2 ways we can create tables after this solution: Managed table: The database location needs to be informed in the database catalog configuration.

# set the conf ("spark.databricks.delta.fixSchema.GlueCatalog", "true")
df_products.coalesce(1).write \
    .format("delta") \
    .option("mergeSchema", True) \
    .mode("overwrite") \
    .saveAsTable("database_name.table_name")

External table

# set the conf ("spark.databricks.delta.fixSchema.GlueCatalog", "true")
df_products.coalesce(1).write \
    .format("delta") \
    .option("mergeSchema", True) \
    .option("path", "s3://bucket_name/table_folder_name") \
    .mode("overwrite") \
    .save()

spark.catalog.createExternalTable(tableName="database_name.table_name", path="s3://bucket_name/table_folder_name")

Does this PR introduce any user-facing changes?

No.

calixtofelipe avatar Nov 18 '23 19:11 calixtofelipe

@calixtofelipe Thanks! It seems that your PR looks very similar to my original PR (https://github.com/delta-io/delta/pull/1579) but at that time it introduced another issue. Let me share something we observed in the past soon.

moomindani avatar Nov 20 '23 02:11 moomindani

@calixtofelipe Thanks! It seems that your PR looks very similar to my original PR (#1579) but at that time it introduced another issue. Let me share something we observed in the past soon.

Hey @moomindani, in the original PR you changed the provider='delta' to 'parquet' and it generate the another issue because a lot of other places will check the provider (E.g: impact time travel capability). This new PR still change the schema but I didn't change the provider as you did in the original PR. I added the command to alter the metadata and this command will update the Hive metastore successfully without overwrite the schema to empty as apache/spark project does when we execute createTable function. So, as we are keeping the provider='delta' the will keep all the delta capabilities. thanks for replying and helping.

calixtofelipe avatar Nov 20 '23 21:11 calixtofelipe

Thanks for clarifying it. I confirmed that your PR won't cause the same issue that I experienced. BTW, if I understand it correctly, your PR solves only the Issue 1 described in https://github.com/delta-io/delta/issues/1679, not the Issue 2. Is it correct?

moomindani avatar Nov 21 '23 05:11 moomindani

Thanks for clarifying it. I confirmed that your PR won't cause the same issue that I experienced. BTW, if I understand it correctly, your PR solves only the Issue 1 described in #1679, not the Issue 2. Is it correct?

Yes, only issue 1 has been resolved. Regarding issue 2, I understand that it generates a poor user experience, as you described in the original PR. However, since we can create the database with the location, it is not as critical as issue 1. If I have some time I will try summit a PR to apache/spark project to fix the issue 1 as you described in the original PR too, the error is generated there. Thanks again for replaying and helping to analyze this issues.

calixtofelipe avatar Nov 21 '23 12:11 calixtofelipe

I agree, Issue 1 is more critical than Issue 2. Thanks for clarifying it. It may be better to explicitly mention that this PR solves Issue 1 of https://github.com/delta-io/delta/issues/1679 in the overview.

moomindani avatar Nov 22 '23 05:11 moomindani

I agree, Issue 1 is more critical than Issue 2. Thanks for clarifying it. It may be better to explicitly mention that this PR solves Issue 1 of #1679 in the overview.

I totally agree. I added the comment in the PR description and edited my commend in the issue. Thanks again for helping.

calixtofelipe avatar Nov 22 '23 12:11 calixtofelipe

Any timeline as to when this will be merged?

juliangordon avatar Feb 22 '24 11:02 juliangordon

Hi @calixtofelipe, which conf are you using to run it on AWS Glue?

I mean not only spark.conf spark.databricks.delta.fixSchema.GlueCatalog. Additional argument such as --extra-py-files and --extra-jars

lucabem avatar Feb 23 '24 13:02 lucabem

Hi @calixtofelipe, which conf are you using to run it on AWS Glue?

I mean not only spark.conf spark.databricks.delta.fixSchema.GlueCatalog. Additional argument such as --extra-py-files and --extra-jars

Yes, @lucabem, after you build a delta package from this branch, you should set it as a delta-package via extra-jars. Make sure that in your Spark session, you are calling the right delta package.

calixtofelipe avatar Jun 02 '24 08:06 calixtofelipe