[ 
https://issues.apache.org/jira/browse/IMPALA-13655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17912049#comment-17912049
 ] 

ASF subversion and git services commented on IMPALA-13655:
----------------------------------------------------------

Commit fa570e8ea74ece419fb68aa9d123d3433f439d12 in impala's branch 
refs/heads/master from Noemi Pap-Takacs
[ https://gitbox.apache.org/repos/asf?p=impala.git;h=fa570e8ea ]

IMPALA-13655: UPDATE redundantly accumulates memory in HDFS WRITER

When IcebergUpdateImpl created the table sink it didn't set
'inputIsClustered' to true. Therefore HdfsTableSink expected
random input and kept the output writers open for every partition,
which resulted in high memory consumption and potentially an
OOM error when the number of partitions are high.

Since we actually sort the rows before the sink we can set
'inputIsClustered' to true, which means HdfsTableSink can write
files one by one, because whenever it gets a row that belongs
to a new partition it knows that it can close the current output
writer, and open a new one.

Testing:
 - e2e regression test

Change-Id: I9bad335cc946364fc612e8aaf90858eaabd7c4af
Reviewed-on: http://gerrit.cloudera.org:8080/22325
Reviewed-by: Impala Public Jenkins <[email protected]>
Tested-by: Impala Public Jenkins <[email protected]>


> UPDATE redundantly accumulates memory in HDFS WRITER
> ----------------------------------------------------
>
>                 Key: IMPALA-13655
>                 URL: https://issues.apache.org/jira/browse/IMPALA-13655
>             Project: IMPALA
>          Issue Type: Bug
>          Components: Frontend
>            Reporter: Noemi Pap-Takacs
>            Assignee: Noemi Pap-Takacs
>            Priority: Major
>              Labels: impala-iceberg
>             Fix For: Impala 4.5.0
>
>
> When we have an Iceberg table that has lots of partitions, and we want to 
> update lots of values in the table, it will use much more memory than needed.
> Repro steps:
> {noformat}
> create table tmp_ice_tpch
> partitioned by spec(truncate(500, l_orderkey))
> stored by iceberg as
> select * from tpch.lineitem;
> UPDATE TABLE tmp_ice_tpch SET l_partkey=l_partkey+1;
> # We likely get a Memory Limit Exceeded error here{noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to