parisni opened a new issue, #5488:
URL: https://github.com/apache/hudi/issues/5488
hudi 0.11.0
spark 3.2.1
when hive_sync then `read.table("table_name")` raise an error
`pyspark.sql.utils.AnalysisException: Table does not support reads`.
The error does't raise when ` --conf
'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog'`
is not set.
```python
pyspark --packages org.apache.hudi:hudi-spark3.2-bundle_2.12:0.11.0
--conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' --conf
'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog'
sc.setLogLevel("WARN")
dataGen = sc._jvm.org.apache.hudi.QuickstartUtils.DataGenerator()
inserts = sc._jvm.org.apache.hudi.QuickstartUtils.convertToStringList(
dataGen.generateInserts(10)
)
from pyspark.sql.functions import expr
df = spark.read.json(spark.sparkContext.parallelize(inserts, 10)).withColumn(
"part", expr("'foo'")
)
tableName = "test_hudi_pyspark"
basePath = f"/tmp/{tableName}"
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": "upsert",
"hoodie.datasource.write.precombine.field": "ts",
"hoodie.upsert.shuffle.parallelism": 2,
"hoodie.insert.shuffle.parallelism": 2,
"hoodie.datasource.hive_sync.database": "default",
"hoodie.datasource.hive_sync.table": tableName,
"hoodie.datasource.hive_sync.mode": "hms",
"hoodie.datasource.hive_sync.enable": "true",
"hoodie.datasource.hive_sync.partition_fields": "part",
"hoodie.datasource.hive_sync.partition_extractor_class":
"org.apache.hudi.hive.MultiPartKeysValueExtractor",
}
(df.write.format("hudi").options(**hudi_options).mode("overwrite").save(basePath))
spark.read.format("hudi").load(basePath).count()
spark.table("default.test_hudi_pyspark").count()
ERROR: pyspark.sql.utils.AnalysisException: Table does not support reads:
default.test_hudi_pyspark
```
I debugged it a bit and the hudi catalog for load table uses the
super.loadTable which is not aware of hudi ?
```scala
override def loadTable(ident: Identifier): Table = {
try {
super.loadTable(ident) match {
case v1: V1Table if sparkAdapter.isHoodieTable(v1.catalogTable) =>
HoodieInternalV2Table(
spark,
v1.catalogTable.location.toString,
catalogTable = Some(v1.catalogTable),
tableIdentifier = Some(ident.toString))
case o => o // this case is used
}
} catch {
case e: Exception =>
throw e
}
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]