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

Hudi uniform support #2333

Merged
merged 27 commits into from Mar 29, 2024

Conversation

the-other-tim-brown
Copy link
Contributor

@the-other-tim-brown the-other-tim-brown commented Nov 26, 2023

Which Delta project/connector is this regarding?

  • Spark
  • Standalone
  • Flink
  • Kernel
  • Other (Uniform)

Description

  • This change aims to add support for Hudi in Uniform
  • The changes were mostly copied from OneTable which has a working version of Delta to Hudi already

How was this patch tested?

Some basic tests are added

Does this PR introduce any user-facing changes?

Yes, this allows users to expose their Delta tables as Hudi

@the-other-tim-brown
Copy link
Contributor Author

This is my first time contributing to Delta Lake and I need some assistance on a few points:

  1. How do I use intellij to run and debug tests? I run into kerberos issues still after many hours of trying to get this environment setup. Is there a quickstart guide I've missed?
  2. How do I run a test suite against a particular version of spark/scala. Hudi only supports up to spark 3.4 as of the 0.14.0 release so we won't be able to run with 3.5 in the tests.
  3. General guidance on UniForm integration. I've just copied over what exists for Iceberg and currently put the code in the same module. I'm guessing we'll want to create a similar module for Hudi if your goal is to keep the dependencies lean and only include them if required.

@vkorukanti
Copy link
Collaborator

This is my first time contributing to Delta Lake and I need some assistance on a few points:

  1. How do I use intellij to run and debug tests? I run into kerberos issues still after many hours of trying to get this environment setup. Is there a quickstart guide I've missed?

These are the instructions that work for me. Can you tell bit more about the Kerberos issue?

  1. How do I run a test suite against a particular version of spark/scala. Hudi only supports up to spark 3.4 as of the 0.14.0 release so we won't be able to run with 3.5 in the tests.

@lzlfred ^

  1. General guidance on UniForm integration. I've just copied over what exists for Iceberg and currently put the code in the same module. I'm guessing we'll want to create a similar module for Hudi if your goal is to keep the dependencies lean and only include them if required.

@lzlfred ^

@the-other-tim-brown
Copy link
Contributor Author

This is my first time contributing to Delta Lake and I need some assistance on a few points:

  1. How do I use intellij to run and debug tests? I run into kerberos issues still after many hours of trying to get this environment setup. Is there a quickstart guide I've missed?

These are the instructions that work for me. Can you tell bit more about the Kerberos issue?

I run into this error which seems like it may be a version issue?

An exception or error caused a run to abort: 'void org.apache.hadoop.security.HadoopKerberosName.setRuleMechanism(java.lang.String)' 
java.lang.NoSuchMethodError: 'void org.apache.hadoop.security.HadoopKerberosName.setRuleMechanism(java.lang.String)'
	at org.apache.hadoop.security.HadoopKerberosName.setConfiguration(HadoopKerberosName.java:84)
	at org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:315)
	at org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:300)
	at org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:575)
	at org.apache.spark.util.Utils$.$anonfun$getCurrentUserName$1(Utils.scala:2416)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.util.Utils$.getCurrentUserName(Utils.scala:2416)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:334)
	at org.apache.spark.SparkContext$.getOrCreate(SparkContext.scala:2888)
	at org.apache.spark.sql.SparkSession$Builder.$anonfun$getOrCreate$2(SparkSession.scala:1099)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:1093)
	at org.apache.spark.sql.delta.ConvertToHudiSuite.createSparkSession(ConvertToHudiSuite.scala:105)
	at org.apache.spark.sql.delta.ConvertToHudiSuite.beforeAll(ConvertToHudiSuite.scala:43)```

@Tagar
Copy link
Contributor

Tagar commented Nov 28, 2023

java.lang.NoSuchMethodError: 'void org.apache.hadoop.security.HadoopKerberosName.setRuleMechanism

Not sure how this was tested, perhaps hadoop and hadoop-common both need to be pinned to the same version, something like 3.3.6

@lzlfred
Copy link
Contributor

lzlfred commented Nov 29, 2023

@the-other-tim-brown Thanks for putting this together. Delta appreciate this effort and lets work together to add the support.

  1. the current master & delta 3.0 branch is on Spark 3.5; I am not aware of any way to build and run with another Spark version.

  2. Agree... We should have a separate module for Hudi.

@the-other-tim-brown
Copy link
Contributor Author

@the-other-tim-brown Thanks for putting this together. Delta appreciate this effort and lets work together to add the support.

  1. the current master & delta 3.0 branch is on Spark 3.5; I am not aware of any way to build and run with another Spark version.
  2. Agree... We should have a separate module for Hudi.
  1. If we are ok just comparing the files in the current view of the table, we can avoid bringing in the hudi spark bundle which will solve this issue. Is that ok?

  2. I will need help with this, this was my first time using sbt and I'm not sure what all needs to be shaded vs not.

@lzlfred
Copy link
Contributor

lzlfred commented Dec 5, 2023

If we are ok just comparing the files in the current view of the table, we can avoid bringing in the hudi spark bundle which will solve this issue. Is that ok?

Yes sure. at this point we do not need to read the hudi table by Spark yet.

I will need help with this, this was my first time using sbt and I'm not sure what all needs to be shaded vs not.

myself is not a sbt expert either... i will circle this back to the committers and see.

@lzlfred
Copy link
Contributor

lzlfred commented Feb 14, 2024

without being able to review this in detail yet, here are a few high level feedbacks:

  1. we should make hudi in its own package/module to avoid fat jars that contains dependencies of multiple formats.
  2. we should make sure hudi metadata dir is whitelisted in Delta's vacuum logic, which deletes everything Not on its white list or not tracked by delta log. code pointer: https://github.com/delta-io/delta/blob/master/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala#L217
  3. I suggest we should add more unit tests such as doing insert/update/delete with Delta, and then read with hudi to see file lists matches. More unit tests including schema evolution, partitioning, statistics conversion can come in future.

I will review this in more detail in the next few weeks. Thanks for the effort !

Copy link
Contributor

@lzlfred lzlfred left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks great ! plz do

  1. clean up and make separate Hudi package as previously mentioned.
  2. do a grep and fix other Iceberg occurrences mostly in comments.

import scala.collection.JavaConverters._

/**
* Used to prepare (convert) and then commit a set of Delta actions into the Iceberg table located
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* Used to prepare (convert) and then commit a set of Delta actions into the Iceberg table located
* Used to prepare (convert) and then commit a set of Delta actions into the Hudi table metadata located

* at the same path as [[postCommitSnapshot]]
*
*
* @param conf Configuration for Iceberg Hadoop interactions.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* @param conf Configuration for Iceberg Hadoop interactions.
* @param conf Configuration for Hudi Hadoop interactions.

*
*
* @param conf Configuration for Iceberg Hadoop interactions.
* @param postCommitSnapshot Latest Delta snapshot associated with this Iceberg commit.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* @param postCommitSnapshot Latest Delta snapshot associated with this Iceberg commit.
* @param postCommitSnapshot Latest Delta snapshot associated with this Hudi commit.

*
* @param conf Configuration for Iceberg Hadoop interactions.
* @param postCommitSnapshot Latest Delta snapshot associated with this Iceberg commit.
* @param tableOp How to instantiate the underlying Iceberg table. Defaults to WRITE_TABLE.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* @param tableOp How to instantiate the underlying Iceberg table. Defaults to WRITE_TABLE.
* @param tableOp How to instantiate the underlying Hudi table. Defaults to WRITE_TABLE.

import scala.util.control.NonFatal

object HudiConverter {

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: remove empty line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed

if (!UniversalFormat.hudiEnabled(snapshotToConvert.metadata)) {
return None
}
txn.catalogTable match {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Iceberg almost always require a catalog so we had this requirement. If thats not true for Hudi, I would suggest to release this contraint, otherwise it wont work if we access/write to the Delta table by path.

However on the other hand I get the fact that we need to pass in the tableName to Hudi table creator.

Can we add a TODO here saying requiring a catalogTable is a constraint that we can release in the future ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't require a catalog for Hudi. I just figured if there is a name present we can use it when creating the table.

!UniversalFormat.hudiEnabled(postCommitSnapshot.metadata)) {
return
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

extra line.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

}

private def runArchiver(table: HoodieJavaTable[_ <: HoodieAvroPayload],
config: HoodieWriteConfig,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ident... plz ident 4 space from private.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

}

private def markInstantsAsCleaned(table: HoodieJavaTable[_],
writeConfig: HoodieWriteConfig,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ident

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

}

private def getWriteConfig(schema: Schema,
numCommitsToKeep: Int,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indent

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@the-other-tim-brown
Copy link
Contributor Author

looks great ! plz do

  1. clean up and make separate Hudi package as previously mentioned.
  2. do a grep and fix other Iceberg occurrences mostly in comments.

@lzlfred I've moved the code into a separate package but I do not have any experience with setting up a new sbt module. After moving the code, the iceberg and hudi modules fail to compile. I've pushed an update so you or someone on your team can help debug this.

build.sbt Show resolved Hide resolved
@the-other-tim-brown the-other-tim-brown changed the title [WIP] Hudi uniform support Hudi uniform support Mar 8, 2024
Copy link
Contributor

@lzlfred lzlfred left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what a epic feature ! thanks for all efforts.

* removed from the wait queue. Only one snapshot is queued at any point of time.
*
*/
override def enqueueSnapshotForConversion(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ideally we want to consolidate all these async logic with iceberg conversion... I will take that part and I am not asking for a TODO from this PR.. .

isNullable)
// TODO: Add List and Map support: https://github.com/delta-io/delta/issues/2738
case ArrayType(elementType, containsNull) =>
throw new UnsupportedOperationException("UniForm doesn't support Array columns")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: say Uniform Hudi

throw new UnsupportedOperationException("UniForm doesn't support Array columns")

case MapType(keyType, valueType, valueContainsNull) =>
throw new UnsupportedOperationException("UniForm doesn't support Map columns")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above.

case atomicType: AtomicType => convertAtomic(atomicType, isNullable)

case other =>
throw new UnsupportedOperationException(s"Cannot convert Delta type $other to Iceberg")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
throw new UnsupportedOperationException(s"Cannot convert Delta type $other to Iceberg")
throw new UnsupportedOperationException(s"Cannot convert Delta type $other to Hudi")

Copy link
Collaborator

@scottsand-db scottsand-db left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SBT related changes LGTM


package org.apache.spark.sql.delta.hooks

import org.apache.spark.sql.SparkSession
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

idk why scala style does not report error, while plz follow

/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala

and make import to be

import org.apache.spark.sql.delta.{OptimisticTransactionImpl, Snapshot, UniversalFormat}
import org.apache.spark.sql.delta.actions.Action
import org.apache.spark.sql.delta.metering.DeltaLogging

import org.apache.spark.sql.SparkSession

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.types.{ArrayType, NullType, MapType}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

plz swap NullType and MapType to be alphabetical.

@scottsand-db scottsand-db merged commit 9028303 into delta-io:master Mar 29, 2024
7 checks passed
@the-other-tim-brown the-other-tim-brown deleted the hudi-uniform-support branch March 29, 2024 16:27
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

5 participants