Skip to content

Commit

Permalink
HADOOP-13327 Output Stream Specification. (#2587)
Browse files Browse the repository at this point in the history
This defines what output streams and especially those which implement
Syncable are meant to do, and documents where implementations (HDFS; S3)
don't. With tests.

The file:// FileSystem now supports Syncable if an application calls
FileSystem.setWriteChecksum(false) before creating a file -checksumming
and Syncable.hsync() are incompatible.

Contributed by Steve Loughran.
  • Loading branch information
steveloughran authored Feb 10, 2021
1 parent a8bd516 commit 798df6d
Show file tree
Hide file tree
Showing 33 changed files with 1,561 additions and 99 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.impl.StoreImplementationUtils;

import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Expand Down Expand Up @@ -312,10 +313,7 @@ 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);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.io.EOFException;
import java.io.FileDescriptor;
import java.io.IOException;
import java.util.StringJoiner;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
Expand Down Expand Up @@ -153,4 +154,12 @@ public boolean hasCapability(final String capability) {
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(in);
}

@Override
public String toString() {
return new StringJoiner(", ",
BufferedFSInputStream.class.getSimpleName() + "[", "]")
.add("in=" + in)
.toString();
}
}
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 @@ -46,6 +46,7 @@
import org.apache.hadoop.util.Progressable;

import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;

/****************************************************************
* Abstract Checksumed FileSystem.
Expand Down Expand Up @@ -479,12 +480,15 @@ public IOStatistics getIOStatistics() {

/**
* Probe the inner stream for a capability.
*
* Syncable operations are rejected before being passed down.
* @param capability string to query the stream support for.
* @return true if a capability is known to be supported.
*/
@Override
public boolean hasCapability(final String capability) {
if (isProbeForSyncable(capability)) {
return false;
}
return datas.hasCapability(capability);
}
}
Expand Down
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.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
Expand Down Expand Up @@ -237,10 +238,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;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
Expand Down Expand Up @@ -126,10 +127,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 @@ -43,10 +43,12 @@
import java.util.Locale;
import java.util.Optional;
import java.util.StringTokenizer;
import java.util.concurrent.atomic.AtomicLong;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.impl.StoreImplementationUtils;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
Expand Down Expand Up @@ -137,8 +139,13 @@ class LocalFSFileInputStream extends FSInputStream implements
STREAM_READ_SKIP_BYTES)
.build();

/** Reference to the bytes read counter for slightly faster counting. */
private final AtomicLong bytesRead;

public LocalFSFileInputStream(Path f) throws IOException {
fis = new FileInputStream(pathToFile(f));
bytesRead = ioStatistics.getCounterReference(
STREAM_READ_BYTES);
}

@Override
Expand All @@ -161,8 +168,8 @@ public boolean seekToNewSource(long targetPos) throws IOException {
return false;
}

/*
* Just forward to the fis
/**
* Just forward to the fis.
*/
@Override
public int available() throws IOException { return fis.available(); }
Expand All @@ -178,7 +185,7 @@ public int read() throws IOException {
if (value >= 0) {
this.position++;
statistics.incrementBytesRead(1);
ioStatistics.incrementCounter(STREAM_READ_BYTES);
bytesRead.addAndGet(1);
}
return value;
} catch (IOException e) { // unexpected exception
Expand All @@ -196,7 +203,7 @@ public int read(byte[] b, int off, int len) throws IOException {
if (value > 0) {
this.position += value;
statistics.incrementBytesRead(value);
ioStatistics.incrementCounter(STREAM_READ_BYTES, value);
bytesRead.addAndGet(value);
}
return value;
} catch (IOException e) { // unexpected exception
Expand Down Expand Up @@ -285,7 +292,7 @@ public FSDataInputStream open(PathHandle fd, int bufferSize)
* For create()'s FSOutputStream.
*********************************************************/
final class LocalFSFileOutputStream extends OutputStream implements
IOStatisticsSource, StreamCapabilities {
IOStatisticsSource, StreamCapabilities, Syncable {
private FileOutputStream fos;

/**
Expand Down Expand Up @@ -354,6 +361,21 @@ public void write(int b) throws IOException {
}
}

@Override
public void hflush() throws IOException {
flush();
}

/**
* HSync calls sync on fhe file descriptor after a local flush() call.
* @throws IOException failure
*/
@Override
public void hsync() throws IOException {
flush();
fos.getFD().sync();
}

@Override
public boolean hasCapability(String capability) {
// a bit inefficient, but intended to make it easier to add
Expand All @@ -362,7 +384,7 @@ public boolean hasCapability(String capability) {
case StreamCapabilities.IOSTATISTICS:
return true;
default:
return false;
return StoreImplementationUtils.isProbeForSyncable(capability);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,11 @@
public interface StreamCapabilities {
/**
* Stream hflush capability implemented by {@link Syncable#hflush()}.
*
* Use the {@link #HSYNC} probe to check for the support of Syncable;
* it's that presence of {@code hsync()} which matters.
*/
@Deprecated
String HFLUSH = "hflush";

/**
Expand Down
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 operations.
* Consult the Hadoop filesystem specification for the definition of the
* semantics of these operations.
*/
@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,96 @@
/*
* 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 probe capability being for {@link StreamCapabilities#HSYNC}
* or {@link StreamCapabilities#HFLUSH}
* {@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 isProbeForSyncable(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 in input stream
* @param capability capability to probe for
* @return true if the stream declares that it supports the capability.
*/
public static boolean hasCapability(InputStream in, String capability) {
return objectHasCapability(in, capability);
}

}
Loading

0 comments on commit 798df6d

Please sign in to comment.