[
https://issues.apache.org/jira/browse/HADOOP-15515?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16503930#comment-16503930
]
Chris Douglas commented on HADOOP-15515:
----------------------------------------
I looked more closely at the synchronization, and there may be some edge cases
that aren't covered. Specifically:
* if {{close}} is holding the lock on {{openFileStreams}} while cleaning up, if
another thread is waiting on its monitor then it may add elements to the
collection after the filesystem is closed.
* In {{close}}, exceptions while closing streams may prevent the intended
behavior. While {{close}} should be
[idempotent|https://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html#close--]
(so a closed stream won't put this in an unrecoverable state), if any stream
throws an exception, then subsequent entries will not be closed. The intended
behavior is probably closer to {{IOUtils::cleanup}}, which swallows
{{IOException}}.
* The lock order (i.e., hold {{openFileStreams}} while closing a stream, or
hold one at at time) should be documented.
* {{openFileStreams}} can be a final field
> adl.AdlFilesystem.close() doesn't release locks on open files
> -------------------------------------------------------------
>
> Key: HADOOP-15515
> URL: https://issues.apache.org/jira/browse/HADOOP-15515
> Project: Hadoop Common
> Issue Type: Bug
> Components: fs/adl
> Affects Versions: 2.7.3
> Environment: HDInsight on MS Azure:
>
> Hadoop 2.7.3.2.6.2.25-1
> Subversion [email protected]:hortonworks/hadoop.git -r
> 1ceeb58bb3bb5904df0cbb7983389bcaf2ffd0b6
> Compiled by jenkins on 2017-11-29T15:28Z
> Compiled with protoc 2.5.0
> From source with checksum 90b73c4c185645c1f47b61f942230
> This command was run using
> /usr/hdp/2.6.2.25-1/hadoop/hadoop-common-2.7.3.2.6.2.25-1.jar
> Reporter: Jay Hankinson
> Assignee: Vishwajeet Dusane
> Priority: Major
> Attachments: HDFS-13344-001.patch, HDFS-13344-002.patch
>
>
> If you write to a file on and Azure ADL filesystem and close the file system
> but not the file before the process exits, the next time you try open the
> file for append it fails with:
> Exception in thread "main" 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.).
> [a67c6b32-e78b-4852-9fac-142a3e2ba963][2018-03-22T20:54:08.3520940-07:00]
> The following moves local file to HDFS if it doesn't exist or appends it's
> contents if it does:
>
> {code:java}
> public void addFile(String source, String dest, Configuration conf) throws
> IOException {
> FileSystem fileSystem = FileSystem.get(conf);
> // Get the filename out of the file path
> String filename = source.substring(source.lastIndexOf('/') +
> 1,source.length());
> // Create the destination path including the filename.
> if (dest.charAt(dest.length() - 1) != '/')
> { dest = dest + "/" + filename; }
> else {
> dest = dest + filename;
> }
> // Check if the file already exists
> Path path = new Path(dest);
> FSDataOutputStream out;
> if (fileSystem.exists(path)) {
> System.out.println("File " + dest + " already exists appending");
> out = fileSystem.append(path);
> } else {
> out = fileSystem.create(path);
> }
> // Create a new file and write data to it.
> InputStream in = new BufferedInputStream(new FileInputStream(new File(
> source)));
> byte[] b = new byte[1024];
> int numBytes = 0;
> while ((numBytes = in.read(b)) > 0) {
> out.write(b, 0, numBytes);
> }
> // Close the file system not the file
> in.close();
> //out.close();
> fileSystem.close();
> }
> {code}
> If "dest" is an adl:// location, invoking the function a second time (after
> the process has exited) it raises the error. If it's a regular hdfs:// file
> system, it doesn't as all the locks are released. The same exception is also
> raised if a subsequent append is done using: hdfs dfs -appendToFile.
> As I can't see a way to force lease recovery in this situation, this seems
> like a bug. org.apache.hadoop.fs.adl.AdlFileSystem inherits close() from
> org.apache.hadoop.fs.FileSystem
> [https://hadoop.apache.org/docs/r3.0.0/api/org/apache/hadoop/fs/adl/AdlFileSystem.html]
> Which states:
> Close this FileSystem instance. Will release any held locks. This does not
> seem to be the case
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]