[
https://issues.apache.org/jira/browse/HIVE-26319?focusedWorklogId=785165&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-785165
]
ASF GitHub Bot logged work on HIVE-26319:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 27/Jun/22 16:04
Start Date: 27/Jun/22 16:04
Worklog Time Spent: 10m
Work Description: kasakrisz commented on code in PR #3362:
URL: https://github.com/apache/hive/pull/3362#discussion_r907549481
##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java:
##########
@@ -127,14 +133,23 @@ public void commitTask(TaskAttemptContext
originalContext) throws IOException {
.run(output -> {
Table table =
HiveIcebergStorageHandler.table(context.getJobConf(), output);
if (table != null) {
- HiveIcebergWriter writer = writers.get(output);
+ Collection<DataFile> dataFiles = Lists.newArrayList();
+ Collection<DeleteFile> deleteFiles = Lists.newArrayList();
String fileForCommitLocation =
generateFileForCommitLocation(table.location(), jobConf,
- attemptID.getJobID(), attemptID.getTaskID().getId());
- if (writer != null) {
- createFileForCommit(writer.files(), fileForCommitLocation,
table.io());
- } else {
+ attemptID.getJobID(), attemptID.getTaskID().getId());
+ if (writers.get(output) != null) {
+ for (HiveIcebergWriter writer : writers.get(output)) {
+ if (writer != null) {
+ dataFiles.addAll(writer.files().dataFiles());
+ deleteFiles.addAll(writer.files().deleteFiles());
Review Comment:
Reverted this.
##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java:
##########
@@ -162,53 +177,96 @@ public void abortTask(TaskAttemptContext originalContext)
throws IOException {
TaskAttemptContext context =
TezUtil.enrichContextWithAttemptWrapper(originalContext);
// Clean up writer data from the local store
- Map<String, HiveIcebergWriter> writers =
WriterRegistry.removeWriters(context.getTaskAttemptID());
+ Map<String, List<HiveIcebergWriter>> writerMap =
WriterRegistry.removeWriters(context.getTaskAttemptID());
// Remove files if it was not done already
- if (writers != null) {
- for (HiveIcebergWriter writer : writers.values()) {
- writer.close(true);
+ if (writerMap != null) {
+ for (List<HiveIcebergWriter> writerList : writerMap.values()) {
+ for (HiveIcebergWriter writer : writerList) {
+ writer.close(true);
+ }
}
}
}
+ @Override
+ public void commitJob(JobContext originalContext) throws IOException {
+ commitJobs(Collections.singletonList(originalContext));
+ }
+
+ private static class OutputTable {
Review Comment:
Added.
Issue Time Tracking
-------------------
Worklog Id: (was: 785165)
Time Spent: 6h (was: 5h 50m)
> Iceberg integration: Perform update split early
> -----------------------------------------------
>
> Key: HIVE-26319
> URL: https://issues.apache.org/jira/browse/HIVE-26319
> Project: Hive
> Issue Type: Improvement
> Components: File Formats
> Reporter: Krisztian Kasa
> Assignee: Krisztian Kasa
> Priority: Major
> Labels: pull-request-available
> Fix For: 4.0.0
>
> Time Spent: 6h
> Remaining Estimate: 0h
>
> Extend update split early to iceberg tables like in HIVE-21160 for native
> acid tables
--
This message was sent by Atlassian Jira
(v8.20.7#820007)