[ 
https://issues.apache.org/jira/browse/HIVE-24851?focusedWorklogId=562304&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-562304
 ]

ASF GitHub Bot logged work on HIVE-24851:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 08/Mar/21 11:18
            Start Date: 08/Mar/21 11:18
    Worklog Time Spent: 10m 
      Work Description: pvary commented on a change in pull request #2040:
URL: https://github.com/apache/hive/pull/2040#discussion_r589348565



##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java
##########
@@ -88,22 +88,38 @@ public AvroGenericRecordReader(JobConf job, FileSplit 
split, Reporter reporter)
       gdr.setExpected(latest);
     }
 
-    if (split.getLength() == 0) {
-      this.isEmptyInput = true;
-      this.start = 0;
-      this.reader = null;
+    org.apache.avro.file.FileReader<GenericRecord> nonFinalReader = null;
+    try {
+      if (split.getLength() == 0) {
+        this.isEmptyInput = true;
+        this.start = 0;
+        this.reader = null;
+      } else {
+        this.isEmptyInput = false;
+        nonFinalReader = new DataFileReader<GenericRecord>(new 
FsInput(split.getPath(), job), gdr);
+        this.reader = nonFinalReader;
+        this.reader.sync(split.getStart());
+        this.start = reader.tell();

Review comment:
       Ok. Do I understand correctly that the problem is not with the 
`DataFileReader` creation, but some other initialization in the 
`AvroGenericRecordReader`? Basically what we want to do is make sure that we 
free up the resources if the `AvroGenericRecordReader` creation is unsuccessful?
   
   Then I do not understand the importance of the `nonFinalReader`. Could we 
just set the reader at creation? Later, if it is set but there is an exception 
then just close it before rethrowing the exception?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 562304)
    Time Spent: 2.5h  (was: 2h 20m)

> resources leak on exception in AvroGenericRecordReader constructor
> ------------------------------------------------------------------
>
>                 Key: HIVE-24851
>                 URL: https://issues.apache.org/jira/browse/HIVE-24851
>             Project: Hive
>          Issue Type: Bug
>    Affects Versions: 3.1.2, 4.0.0
>            Reporter: Lukasz Osipiuk
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2.5h
>  Remaining Estimate: 0h
>
> AvroGenericRecordReader constructor creates an instance of FileReader but 
> lacks proper exception handling, and reader is not closed on the failure path.
> This results in leaking of underlying resources (e.g. S3 connections).
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to