Skip to content

[SUPPORT] Querying Hudi Table via Presto Hive Connector Errors out when having DecimalType Column.  #9494

@rahil-c

Description

@rahil-c

Issue Summary

Versions of apps

  • Hudi 0.13.1

  • Presto 0.281

  • Spark 3.4.0

  • glue enabled

  • When using EMR 6.12.0 with spark to create the hudi table, and presto to query with aws Glue as the catalog, hitting the following issue below. Creating a regular parquet table with decimal type col and querying works fine, whereas with hudi format seeing an issue (not sure if its truly presto related)

config json for enabling glue on spark and presto when creating emr cluster

[
  {
    "Classification": "spark-hive-site",
    "Properties": {
      "hive.metastore.client.factory.class": "com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory"
    }
  },
  {
      "Classification": "hive-site",
      "Properties": {
        "hive.metastore.client.factory.class": "com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory",
        "hive.metastore.schema.verification": "false"
      }
    },
    {
        "Classification": "presto-connector-hive",
        "Properties": {
          "hive.metastore": "glue"
        }
      }

]


Exception/Stack Trace


java.lang.UnsupportedOperationException: com.facebook.presto.common.type.ShortDecimalType
    at com.facebook.presto.common.type.AbstractType.writeSlice(AbstractType.java:146)
    at com.facebook.presto.parquet.reader.BinaryColumnReader.readValue(BinaryColumnReader.java:55)
    at com.facebook.presto.parquet.reader.AbstractColumnReader.lambda$readValues$0(AbstractColumnReader.java:169)
    at com.facebook.presto.parquet.reader.AbstractColumnReader.processValues(AbstractColumnReader.java:223)
    at com.facebook.presto.parquet.reader.AbstractColumnReader.readValues(AbstractColumnReader.java:168)
    at com.facebook.presto.parquet.reader.AbstractColumnReader.readNext(AbstractColumnReader.java:148)
    at com.facebook.presto.parquet.reader.ParquetReader.readPrimitive(ParquetReader.java:390)
    at com.facebook.presto.parquet.reader.ParquetReader.readColumnChunk(ParquetReader.java:557)
    at com.facebook.presto.parquet.reader.ParquetReader.readBlock(ParquetReader.java:540)
    at com.facebook.presto.hive.parquet.ParquetPageSource$ParquetBlockLoader.load(ParquetPageSource.java:230)
    at com.facebook.presto.hive.parquet.ParquetPageSource$ParquetBlockLoader.load(ParquetPageSource.java:208)
    at com.facebook.presto.common.block.LazyBlock.assureLoaded(LazyBlock.java:313)
    at com.facebook.presto.common.block.LazyBlock.getLoadedBlock(LazyBlock.java:304)
    at com.facebook.presto.common.Page.getLoadedPage(Page.java:314)
    at com.facebook.presto.operator.TableScanOperator.getOutput(TableScanOperator.java:282)
    at com.facebook.presto.operator.Driver.processInternal(Driver.java:468)
    at com.facebook.presto.operator.Driver.lambda$processFor$10(Driver.java:333)
    at com.facebook.presto.operator.Driver.tryWithLock(Driver.java:782)
    at com.facebook.presto.operator.Driver.processFor(Driver.java:326)
    at com.facebook.presto.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1097)
    at com.facebook.presto.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:165)
    at com.facebook.presto.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:603)
    at com.facebook.presto.$gen.Presto_0_281_amzn_0____20230814_195449_1.run(Unknown Source)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:750)


Repro Steps

  1. Create hudi table via EMR Spark

pyspark --jars /usr/lib/hudi/hudi-spark-bundle.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"


from pyspark.sql.types import *
from decimal import Decimal

test_data =[
        ("100", "2015-01-01", Decimal(1123231231.12)),
        ("101", "2015-01-01", Decimal(1123231231.12)),
        ("102", "2015-01-01", Decimal(1123231231.12)),
        ("103", "2015-01-01", Decimal(1123231231.12)),
        ("104", "2015-01-02", Decimal(1123231231.12)),
        ("105", "2015-01-02", Decimal(1123231231.12)),
    ]


schema = StructType(
    [
        StructField("id", StringType()),
        StructField("creation_date", StringType()),
        StructField("decimal_col", DecimalType(18, 2)),
    ]
)

# Create a DataFrame
inputDF =  spark.createDataFrame(data = test_data,
                           schema = schema)

# Specify common DataSourceWriteOptions in the single hudiOptions variable
hudiOptions = {
'hoodie.table.name': 'presto_hudi_table',
'hoodie.datasource.write.recordkey.field': 'id',
'hoodie.datasource.write.partitionpath.field': 'creation_date',
'hoodie.datasource.write.precombine.field': 'last_update_time',
'hoodie.datasource.hive_sync.enable': 'true',
'hoodie.datasource.hive_sync.table': 'presto_hudi_table',
'hoodie.datasource.hive_sync.partition_fields': 'creation_date',
'hoodie.datasource.hive_sync.partition_extractor_class': 'org.apache.hudi.hive.MultiPartKeysValueExtractor'
}

# Write a DataFrame as a Hudi dataset
inputDF.write \
.format('org.apache.hudi') \
.option('hoodie.datasource.write.operation', 'insert') \
.options(**hudiOptions) \
.mode('overwrite') \
.save('s3://my-bucket/emr-hive-table/myhudidataset/')

  1. Query the Hudi table with EMR Presto

presto-cli --catalog awsdatacatalog

use default;

presto:default> select * from presto_hudi_table;

Query 20230817_192637_00006_4kyqe, FAILED, 2 nodes
Splits: 18 total, 0 done (0.00%)
[Latency: client-side: 0:07, server-side: 0:07] [0 rows, 0B] [0 rows/s, 0B/s]

Query 20230817_192637_00006_4kyqe failed: com.facebook.presto.common.type.ShortDecimalType

presto:default> describe presto_hudi_table;
         Column         |     Type      |     Extra     | Comment
------------------------+---------------+---------------+---------
 _hoodie_commit_time    | varchar       |               |
 _hoodie_commit_seqno   | varchar       |               |
 _hoodie_record_key     | varchar       |               |
 _hoodie_partition_path | varchar       |               |
 _hoodie_file_name      | varchar       |               |
 id                     | varchar       |               |
 decimal_col            | decimal(18,2) |               |
 creation_date          | varchar       | partition key |
(8 rows)

Query 20230817_192818_00007_4kyqe, FINISHED, 3 nodes
Splits: 36 total, 36 done (100.00%)
[Latency: client-side: 387ms, server-side: 379ms] [8 rows, 701B] [21 rows/s, 1.81KB/s]

presto:default>

cc @codope @yihua

Metadata

Metadata

Assignees

Labels

Type

No type

Projects

Status

✅ Done

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions