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

Error in query: Cannot partition by nested column: meta.source / java.lang.IllegalArgumentException: Can't find preCombineKey meta.lastUpdated in root #5388

Closed
santoshsb opened this issue Apr 21, 2022 · 6 comments · Fixed by #5517
Assignees
Labels

Comments

@santoshsb
Copy link

Hi Team,

Environment: AWS
emr-6.5.0
Applications: Hive 3.1.2, Tez 0.9.2, Presto 0.261, Spark 3.1.2

I have created a hudi table using spark/spark-session using the following code
`val hudiOptions = Map[String,String](
HoodieWriteConfig.TABLE_NAME -> "patient_hudi",
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> "COPY_ON_WRITE",
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "id",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "meta.source",
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "meta.lastUpdated"
)

readJson.write
.format("org.apache.hudi")
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.options(hudiOptions)
.mode(SaveMode.Overwrite)
.save("s3a://xyz/data/hudi/patient/")`

Now I want to use the spark-sql and load the data from this table for manipulation, followed the documentation to create a new table using existing HUDI table and ran the following query
spark-sql> create table patient_sql using hudi options (primaryKey = 'id', preCombineField = 'meta.lastUpdated') partitioned by (meta.source) location 'xyz/data/hudi/patient/';

And getting the following error

Error in query: Cannot partition by nested column: meta.source

Is partitioning by nested column not supported while using spark-sql or am I missing something here.

And when I take the partitioned by clause from the query (just to test) the following error is thrown,

java.lang.IllegalArgumentException: Can't find preCombineKey meta.lastUpdated in root |-- _hoodie_commit_time: string (nullable = true) |-- _hoodie_commit_seqno: string (nullable = true) |-- _hoodie_record_key: string (nullable = true) |-- _hoodie_partition_path: string (nullable = true) |-- _hoodie_file_name: string (nullable = true) |-- address: array (nullable = true) | |-- element: struct (containsNull = true) | | |-- city: string (nullable = true) | | |-- country: string (nullable = true) | | |-- extension: array (nullable = true) | | | |-- element: struct (containsNull = true) | | | | |-- extension: array (nullable = true) | | | | | |-- element: struct (containsNull = true) | | | | | | |-- url: string (nullable = true) | | | | | | |-- valueDecimal: double (nullable = true) | | | | |-- url: string (nullable = true) | | |-- line: array (nullable = true) | | | |-- element: string (containsNull = true) | | |-- postalCode: string (nullable = true) | | |-- state: string (nullable = true) |-- birthDate: string (nullable = true) |-- communication: array (nullable = true) | |-- element: struct (containsNull = true) | | |-- language: struct (nullable = true) | | | |-- coding: array (nullable = true) | | | | |-- element: struct (containsNull = true) | | | | | |-- code: string (nullable = true) | | | | | |-- display: string (nullable = true) | | | | | |-- system: string (nullable = true) | | | |-- text: string (nullable = true) |-- extension: array (nullable = true) | |-- element: struct (containsNull = true) | | |-- extension: array (nullable = true) | | | |-- element: struct (containsNull = true) | | | | |-- url: string (nullable = true) | | | | |-- valueCoding: struct (nullable = true) | | | | | |-- code: string (nullable = true) | | | | | |-- display: string (nullable = true) | | | | | |-- system: string (nullable = true) | | | | |-- valueString: string (nullable = true) | | |-- url: string (nullable = true) | | |-- valueAddress: struct (nullable = true) | | | |-- city: string (nullable = true) | | | |-- country: string (nullable = true) | | | |-- state: string (nullable = true) | | |-- valueCode: string (nullable = true) | | |-- valueDecimal: double (nullable = true) | | |-- valueString: string (nullable = true) |-- gender: string (nullable = true) |-- id: string (nullable = true) |-- identifier: array (nullable = true) | |-- element: struct (containsNull = true) | | |-- system: string (nullable = true) | | |-- type: struct (nullable = true) | | | |-- coding: array (nullable = true) | | | | |-- element: struct (containsNull = true) | | | | | |-- code: string (nullable = true) | | | | | |-- display: string (nullable = true) | | | | | |-- system: string (nullable = true) | | | |-- text: string (nullable = true) | | |-- value: string (nullable = true) |-- managingOrganization: struct (nullable = true) | |-- reference: string (nullable = true) | |-- type: string (nullable = true) |-- maritalStatus: struct (nullable = true) | |-- coding: array (nullable = true) | | |-- element: struct (containsNull = true) | | | |-- code: string (nullable = true) | | | |-- display: string (nullable = true) | | | |-- system: string (nullable = true) | |-- text: string (nullable = true) |-- meta: struct (nullable = true) | |-- extension: array (nullable = true) | | |-- element: struct (containsNull = true) | | | |-- url: string (nullable = true) | | | |-- valueString: string (nullable = true) | |-- lastUpdated: string (nullable = true) | |-- source: string (nullable = true) | |-- versionId: string (nullable = true) |-- multipleBirthBoolean: boolean (nullable = true) |-- multipleBirthInteger: long (nullable = true) |-- name: array (nullable = true) | |-- element: struct (containsNull = true) | | |-- family: string (nullable = true) | | |-- given: array (nullable = true) | | | |-- element: string (containsNull = true) | | |-- prefix: array (nullable = true) | | | |-- element: string (containsNull = true) | | |-- use: string (nullable = true) |-- resourceType: string (nullable = true) |-- telecom: array (nullable = true) | |-- element: struct (containsNull = true) | | |-- system: string (nullable = true) | | |-- use: string (nullable = true) | | |-- value: string (nullable = true) |-- text: struct (nullable = true) | |-- div: string (nullable = true) | |-- status: string (nullable = true) . at org.apache.hudi.common.util.ValidationUtils.checkArgument(ValidationUtils.java:40) at org.apache.spark.sql.hudi.HoodieOptionConfig$.validateTable(HoodieOptionConfig.scala:209) at org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.parseSchemaAndConfigs(HoodieCatalogTable.scala:214) at org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.initHoodieTable(HoodieCatalogTable.scala:156) at org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.run(CreateHoodieTableCommand.scala:67) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68) at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79) at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:230) at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3751) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:107) at org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:232) at org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:110) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:135) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:107) at org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:232) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:135) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:253) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:134) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:68) at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3749) at org.apache.spark.sql.Dataset.<init>(Dataset.scala:230) at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:101) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:98) at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:618) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:613) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:650) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:67) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:381) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1(SparkSQLCLIDriver.scala:500) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1$adapted(SparkSQLCLIDriver.scala:494) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:494) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:284) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:959) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1047) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1056) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Thanks for the help.

@santoshsb santoshsb changed the title [SUPPORT]: Error in query: Cannot partition by nested column: meta.source / java.lang.IllegalArgumentException: Can't find preCombineKey meta.lastUpdated in root Error in query: Cannot partition by nested column: meta.source / java.lang.IllegalArgumentException: Can't find preCombineKey meta.lastUpdated in root Apr 25, 2022
@codope
Copy link
Member

codope commented Apr 26, 2022

@santoshsb EMR 6.5 comes with Hudi 0.9. There were two issues that were resolved recently. One was released in Hudi 0.10.1 and another should be out in 0.11.0 in the next few days. Can you give the latest master a try?
Related issues
#4308
#5379

@codope codope added this to Awaiting Triage in GI Tracker Board via automation Apr 26, 2022
@codope codope moved this from Awaiting Triage to awaiting ack triaged in GI Tracker Board Apr 26, 2022
@santoshsb
Copy link
Author

@codope thanks for the response, am working on getting the latest master code. Just downloaded the source code will build and deploy soon.

@yihua yihua added spark Issues related to spark spark-sql priority:critical production down; pipelines stalled; Need help asap. labels Apr 26, 2022
@yihua
Copy link
Contributor

yihua commented Apr 29, 2022

@santoshsb Following up on this, have you tried the latest master and see if it solves the problem for you?

@yihua yihua moved this from awaiting ack triaged to User Action in GI Tracker Board Apr 29, 2022
@santoshsb
Copy link
Author

santoshsb commented Apr 29, 2022

@yihua while checking on the latest master for this fix, we ran into this other issue #5452. Will probably simplify the schema and see if it avoids this #5452 issue and try the above one for now.

@xushiyan xushiyan added writer-core Issues relating to core transactions/write actions and removed core-writer-flow labels May 18, 2022
@xushiyan
Copy link
Member

xushiyan commented May 18, 2022

@santoshsb this is caused by some discrepancies btw spark sql and spark data source option. we should fix this in 0.11.1. Spark data source allows using nested field for primary key and precombine field. but spark sql does some validation which restricts it. The validation logic is here (to be relaxed)
#5517

But nested field is not supported as partition field due to spark sql restriction.

cc @yihua @XuQianJin-Stars

@xushiyan xushiyan moved this from User Action to Triaged in GI Tracker Board May 18, 2022
@xushiyan xushiyan added priority:blocker and removed priority:critical production down; pipelines stalled; Need help asap. writer-core Issues relating to core transactions/write actions labels May 18, 2022
GI Tracker Board automation moved this from Triaged to Done May 22, 2022
@santoshsb
Copy link
Author

Thank you.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
Development

Successfully merging a pull request may close this issue.

4 participants