<p>The core behavior of <code>FSDataInputStream</code> is defined by <code>java.io.DataInputStream</code>, with extensions that add key assumptions to the system.</p>
<olstyle="list-style-type: decimal">
<li>The source is a local or remote filesystem.</li>
<li>The stream being read references a finite array of bytes.</li>
<li>The length of the data does not change during the read process.</li>
<li>The contents of the data does not change during the process.</li>
<li>The source file remains present during the read process.</li>
<li>Callers may use <code>Seekable.seek()</code> to offsets within the array of bytes, with future reads starting at this offset.</li>
<li>The cost of forward and backward seeks is low.</li>
<li>There is no requirement for the stream implementation to be thread-safe.</li>
<li>BUT, if a stream implements <ahref="#PositionedReadable">PositionedReadable</a>, “positioned reads” MUST be thread-safe.</li>
</ol>
<p>Files are opened via <code>FileSystem.open(p)</code>, which, if successful, returns:</p>
<divclass="source">
<divclass="source">
<pre>result = FSDataInputStream(0, FS.Files[p])
</pre></div></div>
<p>The stream can be modeled as:</p>
<divclass="source">
<divclass="source">
<pre>FSDIS = (pos, data[], isOpen)
</pre></div></div>
<p>with access functions:</p>
<divclass="source">
<divclass="source">
<pre>pos(FSDIS)
data(FSDIS)
isOpen(FSDIS)
</pre></div></div>
<p><b>Implicit invariant</b>: the size of the data stream equals the size of the file as returned by <code>FileSystem.getFileStatus(Path p)</code></p>
<p>Implementations SHOULD be robust against failure. If an inner stream is closed, it should be checked for being <code>null</code> first.</p>
</li>
<li>
<p>Implementations SHOULD NOT raise <code>IOException</code> exceptions (or any other exception) during this operation. Client applications often ignore these, or may fail unexpectedly.</p>
<p>Read <code>length</code> bytes of data into the destination buffer, starting at offset <code>offset</code>. The source of the data is the current position of the stream, as implicitly set in <code>pos</code>.</p><section>
<h4><aname="Preconditions"></a>Preconditions</h4>
<divclass="source">
<divclass="source">
<pre>isOpen(FSDIS)
buffer != null else raise NullPointerException
length >= 0
offset < len(buffer)
length <= len(buffer) - offset
pos >= 0 else raise EOFException, IOException
</pre></div></div>
<p>Exceptions that may be raised on precondition failure are</p>
<divclass="source">
<divclass="source">
<pre>InvalidArgumentException
ArrayIndexOutOfBoundsException
RuntimeException
</pre></div></div>
<p>Not all filesystems check the <code>isOpen</code> state.</p></section><section>
<p>The <code>java.io</code> API states that if the amount of data to be read (i.e. <code>length</code>) then the call must block until the amount of data available is greater than zero —that is, until there is some data. The call is not required to return when the buffer is full, or indeed block until there is no data left in the stream.</p>
<p>That is, rather than <code>l</code> being simply defined as <code>min(length, len(data)-length)</code>, it strictly is an integer in the range <code>1..min(length, len(data)-length)</code>. While the caller may expect as much of the buffer as possible to be filled in, it is within the specification for an implementation to always return a smaller number, perhaps only ever 1 byte.</p>
<p>What is critical is that unless the destination buffer size is 0, the call must block until at least one byte is returned. Thus, for any data source of length greater than zero, repeated invocations of this <code>read()</code> operation will eventually read all the data.</p></section></section><section>
<p>If the operation is supported, the file SHOULD be open:</p>
<divclass="source">
<divclass="source">
<pre>isOpen(FSDIS)
</pre></div></div>
<p>Some filesystems do not perform this check, relying on the <code>read()</code> contract to reject reads on a closed stream (e.g. <code>RawLocalFileSystem</code>).</p>
<p>A <code>seek(0)</code> MUST always succeed, as the seek position must be positive and less than the length of the Stream:</p>
<divclass="source">
<divclass="source">
<pre>s > 0 and ((s==0) or ((s < len(data)))) else raise [EOFException, IOException]
</pre></div></div>
<p>Some FileSystems do not raise an exception if this condition is not met. They instead return -1 on any <code>read()</code> operation where, at the time of the read, <code>len(data(FSDIS)) < pos(FSDIS)</code>.</p>
<p>After a failed seek, the value of <code>pos(FSDIS)</code> may change. As an example, seeking past the EOF may move the read position to the end of the file, <i>as well as raising an <code>EOFException</code>.</i></p></section><section>
<p>There is an implicit invariant: a seek to the current position is a no-op</p>
<divclass="source">
<divclass="source">
<pre>seek(getPos())
</pre></div></div>
<p>Implementations may recognise this operation and bypass all other precondition checks, leaving the input stream unchanged.</p>
<p>The most recent connectors to object stores all implement some form of “lazy-seek”: the <code>seek()</code> call may appear to update the stream, and the value of <code>getPos()</code> is updated, but the file is not opened/reopenend until data is actually read. Implementations of lazy seek MUST still validate the new seek position against the known length of the file. However the state of the file (i.e. does it exist, what its current length is) does not need to be refreshed at this point. The fact that a file has been deleted or truncated may not surface until that <code>read()</code> call.</p></section></section><section>
<p>This operation instructs the source to retrieve <code>data[]</code> from a different source from the current source. This is only relevant if the filesystem supports multiple replicas of a file and there is more than 1 replica of the data at offset <code>offset</code>.</p><section>
<h4><aname="Preconditions"></a>Preconditions</h4>
<p>Not all subclasses implement this operation, and instead either raise an exception or return <code>False</code>.</p>
<p>If the operation is supported and there is a new location for the data:</p>
<divclass="source">
<divclass="source">
<pre> FSDIS' = (pos, data', true)
result = True
</pre></div></div>
<p>The new data is the original data (or an updated version of it, as covered in the Consistency section below), but the block containing the data at <code>offset</code> is sourced from a different replica.</p>
<p>If there is no other copy, <code>FSDIS</code> is not updated; the response indicates this:</p>
<divclass="source">
<divclass="source">
<pre> result = False
</pre></div></div>
<p>Outside of test methods, the primary use of this method is in the {{FSInputChecker}} class, which can react to a checksum error in a read by attempting to source the data elsewhere. If a new source can be found it attempts to reread and recheck that portion of the file.</p></section></section><section>
<p>This operation instructs the source to release any system resources they are currently holding on to, such as buffers, sockets, file descriptors, etc. Any subsequent IO operation will likely have to reacquire these resources. Unbuffering is useful in situation where streams need to remain open, but no IO operation is expected from the stream in the immediate future (examples include file handle cacheing).</p><section>
<h4><aname="Preconditions"></a>Preconditions</h4>
<p>Not all subclasses implement this operation. In addition to implementing <code>CanUnbuffer</code>. Subclasses must implement the <code>StreamCapabilities</code> interface and <code>StreamCapabilities.hasCapability(UNBUFFER)</code> must return true. If a subclass implements <code>CanUnbuffer</code> but does not report the functionality via <code>StreamCapabilities</code> then the call to <code>unbuffer</code> does nothing. If a subclass reports that it does implement <code>UNBUFFER</code>, but does not implement the <code>CanUnbuffer</code> interface, an <code>UnsupportedOperationException</code> is thrown.</p>
<p>The majority of subclasses that do not implement this operation simply do nothing.</p>
<p>If the operation is supported, <code>unbuffer</code> releases any and all system resources associated with the stream. The exact list of what these resources are is generally implementation dependent, however, in general, it may include buffers, sockets, file descriptors, etc.</p></section></section></section><section>
<p>The <code>PositionedReadable</code> operations supply “positioned reads” (“pread”). They provide the ability to read data into a buffer from a specific position in the data stream. Positioned reads equate to a <ahref="#Seekable.seek"><code>Seekable.seek</code></a> at a particular offset followed by a <ahref="#InputStream.read.buffer"><code>InputStream.read(buffer[], offset, length)</code></a>, only there is a single method invocation, rather than <code>seek</code> then <code>read</code>, and two positioned reads can <i>optionally</i> run concurrently over a single instance of a <code>FSDataInputStream</code> stream.</p>
<p>The interface declares positioned reads thread-safe (some of the implementations do not follow this guarantee).</p>
<p>Any positional read run concurrent with a stream operation — e.g. <ahref="#Seekable.seek"><code>Seekable.seek</code></a>, <ahref="#Seekable.getPos"><code>Seekable.getPos()</code></a>, and <ahref="#InputStream.read"><code>InputStream.read()</code></a>— MUST run in isolation; there must not be mutual interference.</p>
<p>Concurrent positional reads and stream operations MUST be serializable; one may block the other so they run in series but, for better throughput and ‘liveness’, they SHOULD run concurrently.</p>
<p>Given two parallel positional reads, one at <code>pos1</code> for <code>len1</code> into buffer <code>dest1</code>, and another at <code>pos2</code> for <code>len2</code> into buffer <code>dest2</code>, AND given a concurrent, stream read run after a seek to <code>pos3</code>, the resultant buffers MUST be filled as follows, even if the reads happen to overlap on the underlying stream:</p>
<p>Note that implementations are not required to be atomic; the intermediate state of the operation (the change in the value of <code>getPos()</code>) may be visible.</p><section>
<p>Not all <code>FSDataInputStream</code> implementations support these operations. Those that do not implement <code>Seekable.seek()</code> do not implement the <code>PositionedReadable</code> interface.</p>
<p>This could be considered obvious: if a stream is not <code>Seekable</code>, a client cannot seek to a location. It is also a side effect of the base class implementation, which uses <code>Seekable.seek()</code>.</p>
<p><b>Implicit invariant</b>: for all <code>PositionedReadable</code> operations, the value of <code>pos</code> is unchanged at the end of the operation</p>
<p>For any operations that fail, the contents of the destination <code>buffer</code> are undefined. Implementations may overwrite part or all of the buffer before reporting a failure.</p></section><section>
<li>A return value of -1 means that the stream had no more available data.</li>
<li>An invocation with <code>length==0</code> implicitly does not read any data; implementations may short-cut the operation and omit any IO. In such instances, checks for the stream being at the end of the file may be omitted.</li>
<li>If an IO exception occurs during the read operation(s), the final state of <code>buffer</code> is undefined.</li>
<p>If an IO exception occurs during the read operation(s), the final state of <code>buffer</code> is undefined.</p>
<p>If there is not enough data in the input stream to satisfy the requests, the final state of <code>buffer</code> is undefined.</p></section><section>
<p>Read fully data for a list of ranges asynchronously. The default implementation iterates through the ranges, tries to coalesce the ranges based on values of <code>minSeekForVectorReads</code> and <code>maxReadSizeForVectorReads</code> and then read each merged ranges synchronously, but the intent is sub classes can implement efficient implementation. Reading in both direct and heap byte buffers are supported. Also, clients are encouraged to use <code>WeakReferencedElasticByteBufferPool</code> for allocating buffers such that even direct buffers are garbage collected when they are no longer referenced.</p>
<p>The position returned by <code>getPos()</code> after <code>readVectored()</code> is undefined.</p>
<p>If a file is changed while the <code>readVectored()</code> operation is in progress, the output is undefined. Some ranges may have old data, some may have new, and some may have both.</p>
<p>While a <code>readVectored()</code> operation is in progress, normal read api calls may block.</p>
<p>Note: Don’t use direct buffers for reading from ChecksumFileSystem as that may lead to memory fragmentation explained in HADOOP-18296.</p><section>
<p>The smallest reasonable seek. Two ranges won’t be merged together if the difference between end of first and start of next range is more than this value.</p></section><section>
<p>Maximum number of bytes which can be read in one go after merging the ranges. Two ranges won’t be merged if the combined data to be read is more than this value. Essentially setting this to 0 will disable the merging of ranges.</p></section></section><section>
<h2><aname="Consistency"></a>Consistency</h2>
<ul>
<li>All readers, local and remote, of a data stream FSDIS provided from a <code>FileSystem.open(p)</code> are expected to receive access to the data of <code>FS.Files[p]</code> at the time of opening.</li>
<li>If the underlying data is changed during the read process, these changes MAY or MAY NOT be visible.</li>
<li>Such changes that are visible MAY be partially visible.</li>
</ul>
<p>At time t0</p>
<divclass="source">
<divclass="source">
<pre>FSDIS0 = FS'read(p) = (0, data0[])
</pre></div></div>
<p>At time t1</p>
<divclass="source">
<divclass="source">
<pre>FS' = FS' where FS'.Files[p] = data1
</pre></div></div>
<p>From time <code>t >= t1</code>, the value of <code>FSDIS0</code> is undefined.</p>
<p>It may be unchanged</p>
<divclass="source">
<divclass="source">
<pre>FSDIS0.data == data0
forall l in len(FSDIS0.data):
FSDIS0.read() == data0[l]
</pre></div></div>
<p>It may pick up the new data</p>
<divclass="source">
<divclass="source">
<pre>FSDIS0.data == data1
forall l in len(FSDIS0.data):
FSDIS0.read() == data1[l]
</pre></div></div>
<p>It may be inconsistent, such that a read of an offset returns data from either of the datasets</p>
<divclass="source">
<divclass="source">
<pre>forall l in len(FSDIS0.data):
(FSDIS0.read(l) == data0[l]) or (FSDIS0.read(l) == data1[l]))
</pre></div></div>
<p>That is, every value read may be from the original or updated file.</p>
<p>It may also be inconsistent on repeated reads of same offset, that is at time <code>t2 > t1</code>:</p>
<divclass="source">
<divclass="source">
<pre>r2 = FSDIS0.read(l)
</pre></div></div>
<p>While at time <code>t3 > t2</code>:</p>
<divclass="source">
<divclass="source">
<pre>r3 = FSDIS0.read(l)
</pre></div></div>
<p>It may be that <code>r3 != r2</code>. (That is, some of the data my be cached or replicated, and on a subsequent read, a different version of the file’s contents are returned).</p>
<p>Similarly, if the data at the path <code>p</code>, is deleted, this change MAY or MAY not be visible during read operations performed on <code>FSDIS0</code>.</p></section>