-
Notifications
You must be signed in to change notification settings - Fork 2.4k
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] Spark reader fails to read hudi table created using 1.0.0-beta2 version #12068
Comments
There is a check when if (tableConfig.populateMetaFields()) {
HoodieRecord.RECORD_KEY_METADATA_FIELD
} else {
val keyFields = tableConfig.getRecordKeyFields.get()
checkState(keyFields.length == 1)
keyFields.head
} |
@danny0405 - are you saying that I need to set hoodie.populate.meta.fields=true during the bulk insert operation? |
yes, if your primary key fields are multiple. |
@danny0405 - When I enabled that for the table that had multiple fields in the record key, I notice that the bulk insert operation is taking an unreasonably long time. Something that took 10 ish minutes before this change ran for over 2 hours and failed with the following exception:
was this expected? I'm not operating with a lot of data for this test, but if the bulk insert operation takes exponentially higher time with this config, it would not be something that we can use. |
It looks like a Hadoop error, is there any cues related to Hudi specifically? The non-metadata field write should be faster but should not be that long, 2 ~ 3x performance gap is expected there. |
No - given that it executes for over 2 hours, I would assume that it is stemming from something within hudi. I see this |
Could you please share your spark-submit/pyspark command here. I can see you have mentioned Spark version is 3.3 and in the above error mentioned it is pointed spark3. |
I have tested the following sample code and worked without any issues. Cluster Details:
spark-shell \
--jars packaging/hudi-spark-bundle/target/hudi-spark3.5-bundle_2.12-1.0.0-beta2.jar \
--conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' \
--conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog' \
--conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' \
--conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar' \
--conf spark.ui.port=14040 import scala.collection.JavaConversions._
import org.apache.spark.sql.SaveMode._
import org.apache.hudi.DataSourceReadOptions._
import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.table.HoodieTableConfig._
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig._
import org.apache.hudi.keygen.constant.KeyGeneratorOptions._
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.config.HoodieIndexConfig
import org.apache.hudi.common.config.HoodieStorageConfig
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode
import spark.implicits._
val columns = Seq("ts","uuid","rider","driver","fare","city")
val data =
Seq((1695159649087L,"334e26e9-8355-45cc-97c6-c31daf0df330","rider-A","driver-K",19.10,"san_francisco"),
(1695091554788L,"e96c4396-3fad-413a-a942-4cb36106d721","rider-C","driver-M",27.70 ,"san_francisco"),
(1695046462179L,"9909a8b1-2d15-4d3d-8ec9-efc48c536a00","rider-D","driver-L",33.90 ,"san_francisco"),
(1695516137016L,"e3cf430c-889d-4015-bc98-59bdce1e530c","rider-F","driver-P",34.15,"sao_paulo" ),
(1695115999911L,"c8abbe79-8d89-47ea-b4ce-4d224bae5bfa","rider-J","driver-T",17.85,"chennai"));
var inserts = spark.createDataFrame(data).toDF(columns:_*)
val tableName = "trips_table"
val basePath = "file:///tmp/trips_table"
val bulkWriteOptions: Map[String, String] = Map(
DataSourceWriteOptions.OPERATION.key() -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.TABLE_TYPE.key() -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME.key() -> "snappy",
HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key() -> "2147483648",
"hoodie.parquet.small.file.limit" -> "1073741824",
HoodieTableConfig.POPULATE_META_FIELDS.key() -> "false",
HoodieWriteConfig.BULK_INSERT_SORT_MODE.key() -> BulkInsertSortMode.GLOBAL_SORT.name(),
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key() -> "true",
HoodieIndexConfig.INDEX_TYPE.key() -> "RECORD_INDEX",
DataSourceWriteOptions.META_SYNC_ENABLED.key() -> "false",
"hoodie.metadata.record.index.enable" -> "true",
"hoodie.metadata.enable" -> "true",
"hoodie.datasource.write.hive_style_partitioning" -> "true",
"hoodie.clustering.inline" -> "true",
"hoodie.clustering.plan.strategy.target.file.max.bytes" -> "2147483648",
"hoodie.clustering.plan.strategy.small.file.limit" -> "1073741824",
"hoodie.datasource.write.partitionpath.field" -> "city",
"hoodie.datasource.write.recordkey.field" -> "uuid",
"hoodie.datasource.write.precombine.field" -> "ts",
"hoodie.table.name" -> tableName
)
inserts.write.format("hudi").
options(bulkWriteOptions).
mode(Overwrite).
save(basePath)
val tripsDF = spark.read.format("hudi").load(basePath)
tripsDF.show(false) |
Hi @rangareddy. The packages I use are Are you suggesting that 1.0.0-beta2 should only be used with spark 3.5? ( For managing other dependencies, we're using EMR 6.x which does not have spark 3.5 and supports up till spark 3.4). I also tried to use the 3.4 spark bundle and got the same exact error while trying to read the table. So it's reproducible on my end. |
I have tested once again with Spark 3.4 and Hudi 1.0.0-beta, and it has worked. Could you please try the above example code (downloading/building spark3.4) and see if you are able to replicate the issue. I have attached the logs for your reference. |
Hi @rangareddy - Sure. I will also try that with the code you attached. However, I made a change to it since I'm using ComplexKeyGenerator with multiple fields in the record key. Here's how I start my spark-shell For EMR 6.11.0:
For EMR 6.15.0
Here's the full code I used:
here's the logs with Spark 3.3. on EMR 6.11.0:
Then I used EMR 6.15.0 with spark 3.4 and got the same exception:
I hope this helps clarify on the issue that I'm facing in my environment. However, I would also like to add that if I switch to using SimpleKeyGenerator and make the record key as Is your recommendation that we should use SimpleKeyGenerator and a single field in the record key since there is a bug in the ComplexKeyGenerator / record key with multiple fields? |
After investigating the issue, I discovered that the problem lies in the configuration setting HoodieTableConfig.POPULATE_META_FIELDS.key() being set to "false". This setting is only compatible with the SimpleKeyGenerator. Solution:
The following article will help you understand what virtual keys are and when to set the HoodieTableConfig.POPULATE_META_FIELDS.key() parameter value to false or true. https://medium.com/@simpsons/virtual-keys-with-apache-hudi-848020ee377d Note: Go through the "Why Virtual keys?" section. |
@dataproblems Are we all good on this, can this be closed? |
@dataproblems As discussed on slack closing this. We are already discussing the other issue here - #12116 |
@ad1happy2go - yes! Since this was an issue with enabling the meta fields, it can be closed. |
Describe the problem you faced
I'm creating a hudi table using bulk insert operation and the reader of the table fails with
IllegalStateException
.A clear and concise description of the problem.
To Reproduce
Steps to reproduce the behavior:
Expected behavior
I should be able to read the data back into a dataframe with no exceptions.
Environment Description
Hudi version : 1.0.0-beta2
Spark version : 3.3.2
Hive version :
Hadoop version : 3.3.3
Storage (HDFS/S3/GCS..) : S3
Running on Docker? (yes/no) : no
Additional context
Here are the hudi options I'm using for the bulk insert:
Here's the hoodie.properties from the table that was generated using 1.0.0-beta2
Stacktrace
The text was updated successfully, but these errors were encountered: