Skip to content

Conversation

@pramodsatya
Copy link
Contributor

@pramodsatya pramodsatya commented May 20, 2025

Description

Adds hive configs hive.read-formats and hive.write-formats to configure the file formats supported by hive connector for read and write operations respectively.

Motivation and Context

Presto C++ only supports reading of tables with DWRF, ORC and PARQUET formats, and writing to tables with DWRF and PARQUET formats, with the hive connector. Using these hive configs will allow to fail-fast at coordinator when attempting to read from and write to tables with unsupported file formats in Presto C++.
Currently attempting to read from tables with unsupported file formats in Presto C++ fails at the worker:

it != readerFactories().end() ReaderFactory is not registered for format text

Release Notes

== RELEASE NOTES ==
Hive Connector Changes
* Adds :ref:`connector/hive:Hive Configuration Properties` `hive.read-formats` and `hive.write-formats` to allow users to set file formats supported for read and write operations by hive connector.

@prestodb-ci prestodb-ci added the from:IBM PR from IBM label May 20, 2025
@pramodsatya pramodsatya marked this pull request as ready for review May 20, 2025 15:20
@pramodsatya pramodsatya requested a review from a team as a code owner May 20, 2025 15:20
@pramodsatya pramodsatya requested a review from jaystarshot May 20, 2025 15:20
@prestodb-ci prestodb-ci requested review from a team, pdabre12 and sh-shamsan and removed request for a team May 20, 2025 15:20
@pramodsatya pramodsatya requested review from a team, aditi-pandit, nishithakbhaskaran and tdcmeehan and removed request for a team, pdabre12 and sh-shamsan May 20, 2025 15:20
@aditi-pandit
Copy link
Contributor

@pramodsatya : Thanks for this code. Should we add a check for the file formats applicable at the Writer side as well ? Native execution only supports DWRF and Parquet writers.

@pramodsatya pramodsatya force-pushed the hive_rd_fmt branch 3 times, most recently from 3f1bcac to 7f731d8 Compare May 26, 2025 23:28
@pramodsatya pramodsatya changed the title [native] Fail-fast for file formats unsupported by hive connector Add hive configs for supported read and write formats May 27, 2025
@pramodsatya
Copy link
Contributor Author

pramodsatya commented May 27, 2025

Thanks for the feedback @tdcmeehan, @aditi-pandit . Added hive configs for supported read and write formats, and validated read/write operations fail for unsupported formats when these configs are set. Could you please take another look?

Copy link
Contributor

@aditi-pandit aditi-pandit left a comment

Choose a reason for hiding this comment

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

Thanks @pramodsatya

@steveburnett
Copy link
Contributor

Should we have documentation for these new properties? https://github.com/prestodb/presto/blob/master/presto-docs/src/main/sphinx/presto_cpp/properties.rst

steveburnett
steveburnett previously approved these changes Jun 3, 2025
Copy link
Contributor

@steveburnett steveburnett left a comment

Choose a reason for hiding this comment

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

LGTM! (docs)

Pull branch, local doc build, looks good. Thanks!

@pramodsatya pramodsatya requested a review from tdcmeehan June 4, 2025 20:13
Comment on lines 212 to 216
``hive.read-formats`` Comma separated list of file formats supported for reads
from tables.

``hive.write-formats`` Comma separated list of file formats supported for writes
to tables.
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you mention that the default behavior is to allow all built-in support of read and write formats?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added this to the doc, could you please take another look?

Copy link
Contributor

@aditi-pandit aditi-pandit left a comment

Choose a reason for hiding this comment

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

@pramodsatya : This code looks good. Though can you write e2e tests with queries ... maybe set read/write formats to only parquet in the queryRunner and show that reading.writing any other format fails.

@pramodsatya
Copy link
Contributor Author

@pramodsatya : This code lookso good. Though can you write e2e tests with queries ... maybe set read/write formats to only parquet in the queryRunner and show that reading.writing any other format fails.

Thanks @aditi-pandit, added tests for these configs with different file formats. Could you please take another look?

@steveburnett
Copy link
Contributor

Suggest adding a link to the doc in the release note entry, like so:

== RELEASE NOTES ==

Hive Connector Changes
* Adds :ref:`connector/hive:Hive Configuration Properties` `hive.read-formats` and `hive.write-formats` to allow users to set file formats supported for read and write operations by hive connector.

session.getRuntimeStats());
Table table = layout.getTable(metastore, metastoreContext);

if (!readFormats.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Don't think we should delay checking the tablelayout for storageFormat until split scheduling. We want to avoid scheduling the fragment at all. Is it possible to do this earlier in BasePlanFragmenter
https://github.com/prestodb/presto/blob/master/presto-main-base/src/main/java/com/facebook/presto/sql/planner/BasePlanFragmenter.java#L249 ? The TableLayout is available at this point.

@tdcmeehan : wdyt ?

SchemaTableName tableName = ((HiveTableHandle) tableHandle).getSchemaTableName();
Table table = metastore.getTable(metastoreContext, tableName.getSchemaName(), tableName.getTableName())
.orElseThrow(() -> new TableNotFoundException(tableName));
HiveStorageFormat tableStorageFormat = extractHiveStorageFormat(table);
Copy link
Contributor

Choose a reason for hiding this comment

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

Even this code is called during scheduling stage executions... Its worthwhile only if the check is done sooner in logical planning or BasePlanFragmenter.

I think extractHiveStorageFormat from a table can be called much sooner. Can you investigate further ?

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

Labels

from:IBM PR from IBM

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants