bgaborg commented on a change in pull request #575: HADOOP-13327 Output Stream 
Specification
URL: https://github.com/apache/hadoop/pull/575#discussion_r264215533
 
 

 ##########
 File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/outputstream.md
 ##########
 @@ -0,0 +1,857 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+<!-- MACRO{toc|fromDepth=1|toDepth=3} -->
+
+# Output: `OutputStream`, `Syncable` and `StreamCapabilities`
+
+
+With the exception of `FileSystem.copyFromLocalFile()`,
+all API operations which write data to a filesystem in Hadoop do so
+through the Java "OutputStreams" API. More specifically, they do
+so through `OutputStream` subclasses obtained through calls to
+`FileSystem.create()`, `FileSystem.append()`,
+or `FSDataOutputStreamBuilder.build()`.
+
+These all return instances of `FSDataOutputStream`, through which data
+can be written through various `write()` methods.
+After a stream's `close()` method is called, all data written to the
+stream MUST BE persisted to the fileysystem and visible to oll other
+clients attempting to read data from that path via `FileSystem.open()`.
+
+As well as operations to write the data, Hadoop's Output Streams
+provide methods to flush buffered data back to the filesystem,
+so as to ensure that the data is reliably persisted and/or visible
+to other callers. This is done via the `Syncable` interface. It was
+originally intended that the presence of this interface could be interpreted
+as a guarantee that the stream supported it's methods, but this has proven
+impossible to guarantee as the static nature of the interface is incompatible
+with filesystems whose syncability semantics may vary on a store/path basis.
+As an example, erasure coded files in HDFS have
+A new interface, `StreamCapabilities` has been implemented to allow callers to 
probe the exact capabilities of a stream, even transitively
+through a chain of streams.
+
+* HDFS's primary stream implementation is
+`org.apache.hadoop.hdfs.DFSOutputStream`.
+* The subclass `org.apache.hadoop.hdfs.DFSStripedOutputStream` supports erasure
+coding: it removes the `Syncable` behaviors from the base class.
+* The output streams `org.apache.hadoop.fs.FSOutputSummer` and
+`org.apache.hadoop.fs.ChecksumFileSystem.ChecksumFSOutputSummer`
+contain the underlying checksummed output stream used by
+both HDFS and the "file" filesystems.
+
+
+## Output Stream Model
+
+For this specification, an output stream can be viewed as a list of bytes
+stored in in the client
+
+```python
+buffer: List[byte]`
+```
+
+A flag, `open` tracks whether the stream is open: after the stream
+is closed no more data may be written to it:
+
+```python
+open: bool
+buffer: List[byte]
+```
+
+The destination path of the stream, `path` can be tracked to form a triple
+`Path, open, buffer`
+
+
+```python
+Stream = (path, open, buffer)
+```
+
+(Immediately) after `Syncable` operations which flush data to the filesystem,
+the data at the stream's destination path must match that of
+`buffer`. That is, the following condition holds:
+
+```python
+FS'.Files(path) == buffer
+```
+
+Any client reading the data at the path will see the new data.
+The two sync operations, `hflush()` and `hsync()` differ in their durability
+guarantees, not visibility of data.
+
+
+### State of Stream and filesystem after `Filesystem.create()`
+
+
+The output stream returned by a `FileSystem.create(path)` call contains no
+data:
+
+```python
+Stream' = (path, true, [])
+```
+
+The filesystem `FS'` must contain a 0-byte file at the path:
+
+```python
+data(FS', path) == []
+```
+
+Accordingly, the the initial state of `Stream'.buffer` is implicitly
+consistent with the data at the filesystem.
+
+
+*Object Stores*: the 0-byte empty file may not exist in the filesystem.
+
+### State of Stream and filesystem after `Filesystem.append()`
+
+The output stream returned from a call of
+ `FileSystem.append(path, buffersize, progress)`,
+can be modelled as a stream whose `buffer` is intialized to that of
+the original file:
+
+```python
+Stream' = (path, true, data(FS, path))
+```
+
+####  Persisting data
+
+When the stream writes data back to its store, be it in any
+supported flush operation, in the `close()` operation, or at any other
+time the stream chooses to do so, the contents of the file
+are replaced with the current buffer
+
+```python
+Stream' = (path, true, buffer)
+FS' = FS where data(FS', path) == buffer
+```
+
+After a call to `close()`, the stream is closed for all operations other
+than `close()`; they MAY fail with `IOException` or `RuntimeException`.
+
+```python
+Stream' =  (path, false, [])
+```
+
+The `close()` operation must be idempotent with the sole attempt to write the
+data made in the first invocation.
+
+1. If `close()` succeeds, subsequent calls are no-ops.
+1. If `close()` fails, again, subsequent calls are no-ops. They MAY rethrow
+the previous exception, but they MUST NOT retry the write.
+
+<!--  ============================================================= -->
+<!--  CLASS: FSDataOutputStream -->
+<!--  ============================================================= -->
+
+# Class `FSDataOutputStream`
+
+
+```java
+public class FSDataOutputStream
+  extends DataOutputStream
+  implements Syncable, CanSetDropBehind, StreamCapabilities {
+ // ...
+}
+```
+
+The `FileSystem.create()`, `FileSystem.append()` and
+`FSDataOutputStreamBuilder.build()` calls return an instance
+of a class `FSDataOutputStream`, a subclass of `java.io.OutputStream`.
+
+The base class wraps an `OutputStream` instance, one which may implement 
`Streamable`,
+`CanSetDropBehind` and `StreamCapabilities`.
+
+This document covers the requirements of such implementations.
+
+HDFS's `FileSystem` implementation, `DistributedFileSystem`, returns an 
instance
+of `HdfsDataOutputStream`. This implementation has at least two behaviors
+which are not explicitly declared by the base Java implmentation
+
+1. Writes are synchronized: more than one thread can write to the same
+output stream. This is a use pattern which HBase relies on.
+
+1. `OutputStream.flush()` is a no-op when the file is closed. Apache Druid
+has made such a call on this in the past
+[HADOOP-14346](https://issues.apache.org/jira/browse/HADOOP-14346).
+
+
+As the HDFS implementation is considered the de-facto specification of
+the FileSystem APIs, the fact that `write()` is thread-safe is significant.
+
+For compatibility, not only must other FS clients be thread-safe,
+but new HDFS featues, such as encryption and Erasure Coding must also
+implement consistent behavior with the core HDFS output stream.
+
+Put differently:
+
+*It isn't enough for Output Streams to implement the core semantics
+of `java.io.OutputStream`: they need to implement the extra semantics
+of `HdfsDataOutputStream`. Failure to do so must be considered regressions*
+
+The concurrent `write()` call is the most significant tightening of
+the Java specification.
+
+
+## Class `java.io.OutputStream`
+
+
+A Java `OutputStream` allows applications to write a sequence of bytes to a 
destination.
+In a Hadoop filesystem, that destination is the data under a path in the 
filesystem.
+
+
+```java
+public abstract class OutputStream implements Closeable, Flushable {
+  public abstract void write(int b) throws IOException;
+  public void write(byte b[]) throws IOException;
+  public void write(byte b[], int off, int len) throws IOException;
+  public void flush() throws IOException ;
+  public void close() throws IOException;
+}
+```
+
+### <a name="write(data: int)"></a>`write(Stream, data)`
+
+Writes a byte of data to the stream.
+
+#### Preconditions
+
+```python
+Stream.open else raise ClosedChannelException, PathIOException, IOException
+```
+
+The exception `java.nio.channels.ClosedChannelExceptionn` is
+raised in the HDFS output streams when trying to write to a closed file.
+Ths exception does not include the destination path; and
+`Exception.getMessage()` is `null`. It is therefore of limited value in stack
+traces. Implementors may wish to raise exceptions with more detail, such
+as a `PathIOException`.
+
+
+#### Postconditions
+
+The buffer has the lower 8 bits of the data argument appended to it.
+
+```python
+Stream'.buffer = Stream.buffer + [data & 0xff]
+```
+
+There may be an explicit limit on the size of cached data, or an implicit
+limit based by the available capacity of the destination filesystem.
+When a limit is reached, `write()` SHOULD fail with an `IOException`.
+
+### <a name="write(buffer,offset,len)"></a>`write(Stream, byte[] data, int 
offset, int len)`
+
+
+#### Preconditions
+
+The preconditions are all defined in `OutputStream.write()`
+
+```python
+Stream.open else raise ClosedChannelException, PathIOException, IOException
+data != null else raise NullPointerException
+offset >= 0 else raise IndexOutOfBoundsException
+len >= 0 else raise IndexOutOfBoundsException
+offset < data.length else raise IndexOutOfBoundsException
+offset + len < data.length else raise IndexOutOfBoundsException
+```
+
+There may be an explicit limit on the size of cached data, or an implicit
+limit based by the available capacity of the destination filesystem.
+When a limit is reached, `write()` SHOULD fail with an `IOException`.
+
+After the operation has returned, the buffer may be re-used. The outcome
+of updates to the buffer while the `write()` operation is in progress is 
undefined.
+
+#### Postconditions
+
+```python
+Stream'.buffer = Stream.buffer + data[offset...(offset + len)]
+```
+
+### <a name="write(buffer)"></a>`write(byte[] data)`
+
+This is defined as the equivalent of:
+
+```python
+write(data, 0, data.length)
+```
+
+### <a name="flush()"></a>`flush()`
+
+Requests that the data is flushed. The specification of `ObjectStream.flush()`
+declares that this SHOULD write data to the "intended destination".
+
+It explicitly precludes any guarantees about durability.
+
+For that reason, this document doesn't provide any normative
+specifications of behaviour.
+
+
+#### Preconditions
+
+
+```python
+Stream.open else raise IOException
+```
+
+#### Postconditions
+
+None.
+
+If the implementation chooses to implement a stream-flushing operation,
+the data may be saved to the file system such that it becomes visible to
+others"
+
+```python
+FS' = FS where data(FS, path) == buffer
+```
+
+Some applications have been known to call `flush()` on a closed stream
+on the assumption that it is harmless. Implementations MAY choose to
+support this behaviour.
+
+### <a name="close"></a>`close()`
+
+The `close()` operation saves all data to the filesystem and
+releases any resources used for writing data.
+
+The `close()` call is expected to block
+until the write has completed (as with `Syncable.hflush()`), possibly
+until it has been written to durable storage (as HDFS does).
+
+After `close()` completes, the data in a file MUST be visible and consistent
+with the data most recently written. The metadata of the file MUST be 
consistent
+with the data and the write history itself (i.e. any modification time fields
+updated).
+
+After `close()` is invoked, all subsequent `write()` calls on the stream
+MUST fail with an `IOException`.
+
+
+Any locking/leaseholding mechanism is also required to release its lock/lease.
+
+```python
+Stream'.open = false
+FS' = FS where data(FS, path) == buffer
+```
+
+The `close()` call MAY fail during its operation.
+
+1. Callers of the API MUST expect for some calls to fail and SHOULD code 
appropriately.
+Catching and swallowing exceptions, while common, is not always the ideal 
solution.
+1. Even after a failure, `close()` MUST place the stream into a closed state.
+Follow-on calls to `close()` are ignored, and calls to other methods
+rejected. That is: caller's cannot be expected to call `close()` repeatedly
+until it succeeds.
+1. The duration of the `call()` operation is undefined. Operations which rely
+on acknowledgements from remote systems to meet the persistence guarantees
+implicitly have to await these acknowledgements. Some Object Store output 
streams
+upload the entire data file in the `close()` operation. This can take a large 
amount
+of time. The fact that many user applications assume that `close()` is both 
fast
+and does not fail means that this behavior is dangerous.
+
+Recommendations for safe use by callers
+
+* Do plan for exceptions being raised, either in catching and logging or
+by throwing the exception further up. Catching and silently swallowing 
exceptions
+may hide serious problems.
+* Heartbeat operations SHOULD take place on a separate thread, so that a long
+delay in `close()` does not block the thread so long that the heartbeat times
+out.
+
+
+## Interface `StreamCapabilities`
+
+```java
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+```
+
+The `StreamCapabilities` interface exists to allow callers to dynamically
+determine the behavior of a stream.
+
+
+The reference Implementation of this interface is
+ `org.apache.hadoop.hdfs.DFSOutputStream`
+
+```java
+public boolean hasCapability(String capability) {
 
 Review comment:
   In the implementation of DFSOutputStream there's
   ```
     @Override
     public boolean hasCapability(String capability) {
       switch (StringUtils.toLowerCase(capability)) {
       case StreamCapabilities.HSYNC:
       case StreamCapabilities.HFLUSH:
         return true;
       default:
         return false;
       }
     }
   ```
   
   Why do you use **if** here?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to