Skip to content

HADOOP-13327 Output Stream Specification. #1694

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.impl.StoreImplementationUtils;

import com.google.common.base.Preconditions;

Expand Down Expand Up @@ -308,9 +309,6 @@ private void freeBuffers() {

@Override
public boolean hasCapability(String capability) {
if (out instanceof StreamCapabilities) {
return ((StreamCapabilities) out).hasCapability(capability);
}
return false;
return StoreImplementationUtils.hasCapability(out, capability);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,6 @@ public interface CanSetDropBehind {
* UnsupportedOperationException If this stream doesn't support
* setting the drop-behind.
*/
public void setDropBehind(Boolean dropCache)
void setDropBehind(Boolean dropCache)
throws IOException, UnsupportedOperationException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.impl.StoreImplementationUtils;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.util.IdentityHashStore;

Expand Down Expand Up @@ -234,10 +235,7 @@ public void unbuffer() {

@Override
public boolean hasCapability(String capability) {
if (in instanceof StreamCapabilities) {
return ((StreamCapabilities) in).hasCapability(capability);
}
return false;
return StoreImplementationUtils.hasCapability(in, capability);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.impl.StoreImplementationUtils;

/** Utility that wraps a {@link OutputStream} in a {@link DataOutputStream}.
*/
Expand Down Expand Up @@ -122,10 +123,7 @@ public OutputStream getWrappedStream() {

@Override
public boolean hasCapability(String capability) {
if (wrappedStream instanceof StreamCapabilities) {
return ((StreamCapabilities) wrappedStream).hasCapability(capability);
}
return false;
return StoreImplementationUtils.hasCapability(wrappedStream, capability);
}

@Override // Syncable
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,8 @@
*/
@InterfaceAudience.LimitedPrivate({"HDFS"})
@InterfaceStability.Unstable
abstract public class FSOutputSummer extends OutputStream {
abstract public class FSOutputSummer extends OutputStream implements
StreamCapabilities {
// data checksum
private final DataChecksum sum;
// internal buffer for storing data before it is checksumed
Expand Down Expand Up @@ -254,4 +255,9 @@ protected synchronized void setChecksumBufSize(int size) {
protected synchronized void resetChecksumBufSize() {
setChecksumBufSize(sum.getBytesPerChecksum() * BUFFER_NUM_CHUNKS);
}

@Override
public boolean hasCapability(String capability) {
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,20 +23,24 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

/** This interface for flush/sync operation. */
/**
* This is the interface for flush/sync operation.
* Consult the Hadoop filesystem specification for the definition of the
* semantics of this operation.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceStability.Stable
public interface Syncable {

/** Flush out the data in client's user buffer. After the return of
* this call, new readers will see the data.
* @throws IOException if any error occurs
*/
public void hflush() throws IOException;
void hflush() throws IOException;

/** Similar to posix fsync, flush out the data in client's user buffer
* all the way to the disk device (but the disk may have it in its cache).
* @throws IOException if error occurs
*/
public void hsync() throws IOException;
void hsync() throws IOException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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
*/

package org.apache.hadoop.fs.impl;

import java.io.InputStream;
import java.io.OutputStream;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.StreamCapabilities;

import static org.apache.hadoop.fs.StreamCapabilities.HFLUSH;
import static org.apache.hadoop.fs.StreamCapabilities.HSYNC;

/**
* Utility classes to help implementing filesystems and streams.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class StoreImplementationUtils {

private StoreImplementationUtils() {
}

/**
* Check the supplied capabilities for being those required for full
* {@code Syncable.hsync()} and {@code Syncable.hflush()} functionality.
* @param capability capability string.
* @return true if either refers to one of the Syncable operations.
*/
public static boolean supportsSyncable(String capability) {
return capability.equalsIgnoreCase(HSYNC) ||
capability.equalsIgnoreCase((HFLUSH));
}

/**
* Probe for an object having a capability; returns true
* if the stream implements {@link StreamCapabilities} and its
* {@code hasCapabilities()} method returns true for the capability.
* This is a package private method intended to provided a common
* implementation for input and output streams.
* {@link StreamCapabilities#hasCapability(String)} call is for public use.
* @param object object to probe.
* @param capability capability to probe for
* @return true if the object implements stream capabilities and
* declares that it supports the capability.
*/
static boolean objectHasCapability(Object object, String capability) {
if (object instanceof StreamCapabilities) {
return ((StreamCapabilities) object).hasCapability(capability);
}
return false;
}

/**
* Probe for an output stream having a capability; returns true
* if the stream implements {@link StreamCapabilities} and its
* {@code hasCapabilities()} method returns true for the capability.
* @param out output stream
* @param capability capability to probe for
* @return true if the stream declares that it supports the capability.
*/
public static boolean hasCapability(OutputStream out, String capability) {
return objectHasCapability(out, capability);
}

/**
* Probe for an input stream having a capability; returns true
* if the stream implements {@link StreamCapabilities} and its
* {@code hasCapabilities()} method returns true for the capability.
* @param out output stream
* @param capability capability to probe for
* @return true if the stream declares that it supports the capability.
*/
public static boolean hasCapability(InputStream out, String capability) {
return objectHasCapability(out, capability);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -603,11 +603,15 @@ and MAY be a `RuntimeException` or subclass. For instance, HDFS may raise a `Inv

result = FSDataOutputStream

A zero byte file must exist at the end of the specified path, visible to all.

The updated (valid) FileSystem must contains all the parent directories of the path, as created by `mkdirs(parent(p))`.

The result is `FSDataOutputStream`, which through its operations may generate new filesystem states with updated values of
`FS.Files[p]`

The behavior of the returned stream is covered in [Output](outputstream.html).

#### Implementation Notes

* Some implementations split the create into a check for the file existing
Expand All @@ -616,10 +620,18 @@ The result is `FSDataOutputStream`, which through its operations may generate ne
clients creating files with `overwrite==true` to fail if the file is created
by another client between the two tests.

* S3A, Swift and potentially other Object Stores do not currently change the FS state
* S3A, Swift and potentially other Object Stores do not currently change the `FS` state
until the output stream `close()` operation is completed.
This MAY be a bug, as it allows >1 client to create a file with `overwrite==false`,
and potentially confuse file/directory logic
This is a significant difference between the behavior of object stores
and that of filesystems, as it allows >1 client to create a file with `overwrite==false`,
and potentially confuse file/directory logic. In particular, using `create()` to acquire
an exclusive lock on a file (whoever creates the file without an error is considered
the holder of the lock) may not not a safe algorithm to use when working with object stores.

* Object stores may create an empty file as a marker when a file is created.
However, object stores with `overwrite=true` semantics may not implement this atomically,
so creating files with `overwrite=false` cannot be used as an implicit exclusion
mechanism between processes.

* The Local FileSystem raises a `FileNotFoundException` when trying to create a file over
a directory, hence it is listed as an exception that MAY be raised when
Expand All @@ -631,6 +643,8 @@ this precondition fails.

Make a `FSDataOutputStreamBuilder` to specify the parameters to create a file.

The behavior of the returned stream is covered in [Output](outputstream.html).

#### Implementation Notes

`createFile(p)` returns a `FSDataOutputStreamBuilder` only and does not make
Expand All @@ -656,17 +670,21 @@ Implementations without a compliant call SHOULD throw `UnsupportedOperationExcep

#### Postconditions

FS
FS' = FS
result = FSDataOutputStream

Return: `FSDataOutputStream`, which can update the entry `FS.Files[p]`
by appending data to the existing list.

The behavior of the returned stream is covered in [Output](outputstream.html).

### `FSDataOutputStreamBuilder appendFile(Path p)`

Make a `FSDataOutputStreamBuilder` to specify the parameters to append to an
existing file.

The behavior of the returned stream is covered in [Output](outputstream.html).

#### Implementation Notes

`appendFile(p)` returns a `FSDataOutputStreamBuilder` only and does not make
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ HDFS as these are commonly expected by Hadoop client applications.
1. [Notation](notation.html)
1. [Model](model.html)
1. [FileSystem class](filesystem.html)
1. [OutputStream, Syncable and `StreamCapabilities`](outputstream.html)
1. [FSDataInputStream class](fsdatainputstream.html)
1. [PathCapabilities interface](pathcapabilities.html)
1. [FSDataOutputStreamBuilder class](fsdataoutputstreambuilder.html)
Expand Down
Loading