HIVE-27458: Support HCatStorer and HCatLoader for Iceberg tables
Changes Made: HIVE-27458: Support multiple OutputCommitter in a single Vertex : Reason: There can be more than one OutputCommitter per vertex. If a job writes to 2 Iceberg tables in the same vertex, there can be 2 commit tasks happening in same vertex. In that case the first committer after finishing the commitTask will remove all the writers from the WritersRegistry that belongs to this task. WritersRegistry holds all writers in a concurrentMap with taskAttempt id as key - https://github.com/apache/hive/blob/9da7488179e7c69d986dbc8a6654a5c3dc6c0210/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterRegistry.java#L29 To fix this:
- Added a new setting "iceberg.output.id" to differentiate between different output committers so the commitTask will only remove writer belonging to this OutputCommitter.
- For existing Hive jobs it won't cause any issue as the id will be null and still the key will use (taskId, null) as key which is similar to existing flow.
HIVE-27458: Support HCatStorer and HCatLoader for Iceberg tables :
Reason: The Pig Reader available in Iceberg can only read Parquet tables using Hadoop catalog. There is no way to read tables in Glue or Hive catalog via Pig. Also there is no way to write data via Pig.
- There is not much change required apart from setting the write operation which right now will always be OTHER as delete/update or not supported in Pig.
- "iceberg.mr.output.tables" setting was missing as it is set in configureJob. This can be set in the configureOutputProperties along with the "iceberg.mr.operation.type" property.
- configureOutputProperties can be called multiple times. The implementation should take care of populating similar properties. So to avoid adding the same table name again and again, added a check to see if the outputTable name is already present in the config.
why you don't use the iceberg catalog? what is the use case for using hcatalog? cc. @deniskuzZ
Could you please fill out the details and the reasoning. I do not understand what is the goal of this modification.
why you don't use the iceberg catalog? what is the use case for using hcatalog? cc. @deniskuzZ
HCatStorer and HCatLoader are used to read and write Hive/Glue tables via Apache Pig. Right now there is no other catalog to read Iceberg tables via Apache Pig.
The Pig Reader available in Iceberg can only read Parquet tables using Hadoop catalog. There is no way to read tables in Glue or Hive catalog via Pig. Also there is no way to write data via Pig.
HCatStorer and HCatLoader needs only minor changes to support Iceberg table. It needs to call the storageHandler's configureJobConf() to setup "iceberg.mr.output.tables" setting and use the HiveIcebergOutputCommitter to commit job in TezAM (Hive does it in MoveTask).
The second commit - Support multiple OutputCommitter in a single Vertex is to address presence of multiple committer in single vertex. There is a possibility that one vertex can write to more than one table. In that case The first committer might complete the write and remove the writer for the table in WritersRegistry (it is a shared static variable). To differentiate between different committer I have introduced the "iceberg.output.id" setting. For traditional hive jobs it would be null and it shouldn't be a problem.
This pull request has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs. Feel free to reach out on the [email protected] list if the patch is in need of reviews.
hi @maswin, thanks for the contribution, and apologies that the ticket slipped by me. In general PR looks good to me. If you are still interested in this functionality, please rebase the PR.
@maswin, FYI iceberg-pig was removed from iceberg
0280885ac95bdf763556a84bb9d7c6fd9c8c5e2a Pig: Remove iceberg-pig (#11380)
so please let us know if you plan on using Hive's iceberg-handler instead
In general PR looks good to me. If you are still interested in this functionality, please rebase the PR.
Done. Rebased the PR
so please let us know if you plan on using Hive's iceberg-handler instead
Yes, Hive's iceberg-handler is used.
Number of TestHiveIcebergInserts.testMultiTableInsert tests failed with commit manifest file not found exception. Probably related to WriteKey
@maswin, I think we'll get the HIVE-27458: Support multiple OutputCommitter in a single Vertex situation in Hive for the multi-insert statement scenario: https://github.com/apache/hive/blob/89e7d4a31b32317188f91aed8ce30e0d36600acc/ql/src/test/results/clientpositive/llap/multi_insert.q.out#L50-L99
would be better if we resolve that part under separate JIRA
Quality Gate passed
Issues
23 New issues
0 Accepted issues
Measures
0 Security Hotspots
0.0% Coverage on New Code
0.0% Duplication on New Code
This PR introduces number of test failures
This pull request has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs. Feel free to reach out on the [email protected] list if the patch is in need of reviews.
@maswin, this PR would be closed soon unless you plan to complete it.
This pull request has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs. Feel free to reach out on the [email protected] list if the patch is in need of reviews.







