-
Notifications
You must be signed in to change notification settings - Fork 8.9k
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
HADOOP-13327 Output Stream Specification. #2587
HADOOP-13327 Output Stream Specification. #2587
Conversation
|
2e520ca
to
f54d260
Compare
Tagging a few HBase folks who may be interested in this as well, @ndimiduk @saintstack @Apache9 @busbey given the previous work on apache/hbase#1408 and apache/hbase#1597. Steve had mentioned to me that he was thinking of HBase and us being able to more safely put WALs onto file:// with these changes (not having to just disable the checks with LocalFileSystem). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
All makes sense to me. Tried my best to read (and not just skim) outputstream.md. Excellent work. Thanks for bringing to my attention, Steve!
you are still going to need some robust storage, RAID-1+ or similar, but we are getting sync all the way through, and you can query the streams to make sure they say they support it -including local fs. |
checkstyle nits
|
Yep, for sure. This is a nice improvement. |
checkstyles and whitespace. The javac changes are all from deprecating hflush capability |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1, pending by some nits. I have already reviewed the .md in the previous pr(now closed) and was satisfied.
try { | ||
out.hflush(); | ||
if (!supportsFlush) { | ||
// hsync not ignored |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
little confusing comment, maybe we can change this to "hysc tests not ignored"?
LOG.info("Expecting files under {} to have supportsSync={}" | ||
+ " and supportsFlush={}", | ||
path, supportsSync, supportsFlush); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: blank line.
LOG.info("Successfully read synced data on a new reader {}", in); | ||
} | ||
} else { | ||
// np sync. Let's do a flush and see what happens. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: typo in "np" -> "no"
if (!isSupported(IS_BLOBSTORE)) { | ||
throw e; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe add a LOG.warn() about the FileNotFoundException if it is an object store?
public LocalFSFileInputStream(Path f) throws IOException { | ||
fis = new FileInputStream(pathToFile(f)); | ||
bytesRead = ioStatistics.getCounterReference( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No issues here, just confirming if these changes aren't part of a different PR by mistake(IOStats PR).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is a followon to IOStats...it was actually blocked on the buffer between row local fs output and FSDataOutput:
org.apache.hadoop.fs.statistics.BufferedIOStatisticsOutputStream
- needed for hsync passthrough to the File output stream. We're now generating IOStats for the local FS too.
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FileSystem; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
unused import.
FWIW, here are the IOStats logged on the runs ABFS (which supports hsync)
s3a
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM +1, pending some nits.
Awesome work on the docs. @Steve
* 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 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
input stream
## Output Stream Model | ||
|
||
For this specification, an output stream can be viewed as a list of bytes | ||
stored in the client -the `hsync()` and `hflush()` operations the actions |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
are the actions?
FS'.Files(path) == buffer | ||
``` | ||
|
||
Any client reading the data at the path MUST see the new data. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As per the java doc on hsync and hflush,they are about the guarantees on data being actually writted to disk or final storage. So just a question here - How it is made sure that data will be visible after close call for the readers?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
up to the implementation. Close() must pass all its data to the shared FS.
Now, if you want some fun, look at NFS client side caching. Dates from the era of diskless sun workstations and was optimised for short lived files which would only be used by the workstations, so copying over a 1MB/s shared ethernet to an even slower shared HDD would hurt the rest of the cluster.
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/outputstream.md
Outdated
Show resolved
Hide resolved
|
||
Forwarding this to a full flush across a distributed filesystem, or worse, | ||
a distant object store, is very inefficient. | ||
Filesystem clients which do uprate a `flush()` to an `hflush()` will eventually |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
update
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I went with "upgrade" in the end
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 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
close() not call()?
it MAY be durable. That is: it does not have to be persisted, merely guaranteed | ||
to be consistently visible to all clients attempting to open a new stream reading | ||
data at the path. | ||
1. `Syncable.hsync()` MUST transmit the data as per `hflush` the data and persist |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
remove the second "the data" ?
to is not something everyone expects, nor convenient for any program trying | ||
to pick up updated data in a file being written. Most visible is the length | ||
of a file returned in the various `list` commands and `getFileStatus` —this | ||
is often out of data. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
data or date?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
date. will fix.
By default, HDFS does not sync data to disk when a stream is closed; it will | ||
be asynchronously saved to disk. | ||
|
||
This does not mean that users do not expect it. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This one is the best :)
This PR removes the changes related to S3A output stream lifecycle, so only covers the specification of Syncable and ensures that StreamCapabilities passes all the way through to the final implementation classes. All streams which implement Syncable hsync/hflush declare this in their stream capabilities Change-Id: I82b16a8e0965f34eb0c42504da43e8fbeabcb68c
…lities... ...even if the inner stream says "yes" Change-Id: Ie401232a23fbc05ae40baf1700fe1cf2ab80a42a
Verify that if you call LocalFileSystem.setWriteChecksum(false) then the streams you get back support Syncable. This now allows applications to sync to the local FS by calling getLocal(), disabling checksums and then creating files. + docs reviewed, hdfs issued moved to one place, and "why implementors must not relay flush to hflush". * abfs.xml declares support for hflush/hsync Change-Id: I6f1b19a00877a26a1f3a53ce15ca2a1cff205ded
Change-Id: I3f9fa4c277d3c6975856274b898d56d1103bd742
Change-Id: I2f2fe20ab317b8836f0ba460e8d2ab95762a5a3b
Plus * validate getFileStatus() behavior, allow for FS to not immediately update it * declare that stream impls MUST implement and then reject syncable so that apps calling it don't get mislead. Change-Id: Id53f48d53652963e6d9f8f6e322a524c89106b27
if all is good on yetus, will merge Change-Id: I23344abdf040649985c12222fef71e57c1ae7fcf
done the final comments and then rebased; unless yetus really blows up will merge in |
e96ba69
to
adf17d0
Compare
💔 -1 overall
This message was automatically generated. |
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. Change-Id: I892d768de6268f4dd6f175b3fe3b7e5bcaa91194
Thank you for this effort, @steveloughran ! |
@ndimiduk this will let you run HBase against local FS now as if you turn off the checksumming the WAL will be synced properly... |
No one should do this. Only want it for stable tests. |
that and to challenge sqllite for dominance in the embedded-database market |
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. Conflicts: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java Change-Id: I892d768de6268f4dd6f175b3fe3b7e5bcaa91194
Specification of OutputStream and Syncable
with
This is a rebase of #2102. Because the RawLocalOutputStream is now wrapped by
BufferedIOStatisticsOutputStream
, which does passthrough of stream capabilities and the Syncable API, the tests which were failing there should now work