Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-13518

hdfsHSync()/hdfsFileClose() doesn't release lease with libhdfs on Azure ADL

    XMLWordPrintableJSON

    Details

    • Type: Bug
    • Status: Open
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: 2.7.3
    • Fix Version/s: None
    • Component/s: fs/adl, libhdfs
    • Labels:
      None
    • Environment:

      Description

      If hdfsFileClose() is called on a file opened for write/append directly after hdfsHSync(), without further data being written to the file, the file doesn't close and the lease isn't released. In the ADL
      output trace, with hdfsHSync() immediately before hdfsFileClose() I see:
       
      18/04/26 23:31:11 DEBUG store.HttpTransport: HTTPRequest,Succeeded,cReqId:607c6051-2ef3-4e57-9b20-caea70f22a31.0,lat:43,err:,Reqlen:14,Resplen:0,token_ns:3700,sReqId:40b2a8df-c9b5-4eff-b84e-c1346f4fdb88,path:/tmp/testfile.txt,qp:op=APPEND&append=true&syncFlag=METADATA&filesessionid=76b19d5b-f3ad-4400-a742-2f7996450859&leaseid=76b19d5b-f3ad-4400-a742-2f7996450859&offset=28&api-version=2016-11-01
       
      but nothing else. I also get an error if I try to re-open the file:
       
      18/04/26 23:37:28 DEBUG store.HttpTransport: HTTPRequest,Failed,cReqId:f216b9d6-3e35-4166-92cf-a7c4e2389dcd.0,lat:9,err:HTTP400(IllegalArgumentException),Reqlen:0,Resplen:357,token_ns:13001,sReqId:4b9e1e11-3f61-46a6-b461-3e5b180266c5,path:/tmp/testfile.txt,qp:op=APPEND&append=true&syncFlag=DATA&filesessionid=3197c37c-356d-491a-a902-aa19b6d8a374&leaseid=3197c37c-356d-491a-a902-aa19b6d8a374&api-version=2016-11-01
      hdfsOpenFile(/tmp/testfile.txt): FileSystem#append((Lorg/apache/hadoop/fs/Path;)Lorg/apache/hadoop/fs/FSDataOutputStream error:
      java.io.IOException: APPEND failed with error 0x83090a16 (Failed to perform the requested operation because the file is currently open in write mode by another user or process.). [4b9e1e11-3f61-46a6-b461-3e5b180266c5][2018-04-26T16:37:28.6549201-07:00]
      at com.microsoft.azure.datalake.store.ADLStoreClient.getRemoteException(ADLStoreClient.java:1142)
      at com.microsoft.azure.datalake.store.ADLStoreClient.getExceptionFromResponse(ADLStoreClient.java:1106)
      at com.microsoft.azure.datalake.store.ADLStoreClient.getAppendStream(ADLStoreClient.java:294)
      at org.apache.hadoop.fs.adl.AdlFileSystem.append(AdlFileSystem.java:398)
      at org.apache.hadoop.fs.FileSystem.append(FileSystem.java:1187)
       
      Appending to the file is then prevented for the next 8 or so minutes until the lease expires. If hdfsWrite()/Flush() is called after hdfsHSync() and before hdfsFileClose() I see the APPEND() with syncFlag=CLOSE as expected.
       
      18/04/26 23:37:53 DEBUG store.HttpTransport: HTTPRequest,Succeeded,cReqId:ecd19d36-b535-4dd5-8c49-1588a2a60947.0,lat:124,err:,Reqlen:14,Resplen:0,token_ns:4700,sReqId:4a6a2c94-ab31-47b2-aae8-1e4250974d27,path:/tmp/testfile.txt,qp:op=APPEND&append=true&syncFlag=METADATA&filesessionid=21adf26e-3208-40fc-803a-1c18e1914090&leaseid=21adf26e-3208-40fc-803a-1c18e1914090&offset=0&api-version=2016-11-01
      18/04/26 23:37:53 DEBUG store.HttpTransport: HTTPRequest,Succeeded,cReqId:08708059-6396-4af9-a4da-d7d60c44041b.0,lat:43,err:,Reqlen:14,Resplen:0,token_ns:3900,sReqId:a4b5f419-f87f-4bed-99ed-8ad1d24c9ab7,path:/tmp/testfile.txt,qp:op=APPEND&append=true&syncFlag=CLOSE&filesessionid=21adf26e-3208-40fc-803a-1c18e1914090&leaseid=21adf26e-3208-40fc-803a-1c18e1914090&offset=14&api-version=2016-11-01
       
       libhdfs repro case attached.

        Attachments

        1. hdfssimple.c
          1 kB
          Jay Hankinson

          Activity

            People

            • Assignee:
              Unassigned
              Reporter:
              ukjay75 Jay Hankinson
            • Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

              • Created:
                Updated: