Skip to content
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

Conversation

steveloughran
Copy link
Contributor

Specification of OutputStream and Syncable

with

  • RawLocalFileSystem to implement Syncable
  • Consistent use of StreamCapabilities everywhere

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

@steveloughran
Copy link
Contributor Author

  • does not address final comments in that review
  • no ITest runs (I don't think it needs it; will look at again)

@steveloughran steveloughran force-pushed the HADOOP-13327-outputstream-and-syncable branch from 2e520ca to f54d260 Compare January 21, 2021 11:51
@bgaborg bgaborg self-requested a review January 21, 2021 14:43
@joshelser
Copy link
Member

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

Copy link
Member

@joshelser joshelser left a 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!

@steveloughran
Copy link
Contributor Author

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

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.

@steveloughran
Copy link
Contributor Author

checkstyle nits

./hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java:512:          if (c == -1) {:24: Must have at least one statement. [EmptyBlock]
./hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java:70:import org.apache.hadoop.util.StringUtils;:8: Unused import - org.apache.hadoop.util.StringUtils. [UnusedImports]
./hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobAppendStream.java:30:import java.util.Locale;:8: Unused import - java.util.Locale. [UnusedImports]
./hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SyncableDataOutputStream.java:31:import org.apache.hadoop.classification.InterfaceAudience;:1: Duplicate import to line 28 - org.apache.hadoop.classification.InterfaceAudience. [RedundantImport]

@joshelser
Copy link
Member

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.

Yep, for sure. This is a nice improvement.

@apache apache deleted a comment from hadoop-yetus Jan 28, 2021
@apache apache deleted a comment from hadoop-yetus Jan 28, 2021
@apache apache deleted a comment from hadoop-yetus Jan 29, 2021
@steveloughran
Copy link
Contributor Author

checkstyles and whitespace. The javac changes are all from deprecating hflush capability

Copy link
Contributor

@mehakmeet mehakmeet left a 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
Copy link
Contributor

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

Copy link
Contributor

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.
Copy link
Contributor

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;
}
}
Copy link
Contributor

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(
Copy link
Contributor

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

Copy link
Contributor Author

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;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

unused import.

@steveloughran
Copy link
Contributor Author

FWIW, here are the IOStats logged on the runs

ABFS (which supports hsync)

.dfs.core.windows.net/stevel-testing/test/testSyncable?action=flush&retainUncommittedData=false&position=2&close=true&timeout=90
2021-02-08 12:27:48,642 INFO  [JUnit-testSyncable]: contract.AbstractContractCreateTest (AbstractContractCreateTest.java:validateSyncableSemantics(586)) - IOStatistics counters=((queue_shrunk_ops=1) (time_spent_task_wait.failures=0) (write_current_buffer_ops=2) (bytes_upload_failed=0) (bytes_upload=2) (time_spent_on_put_request.failures=0) (time_spent_on_put_request=2) (time_spent_task_wait=0) (bytes_upload_successfully=2));
gauges=();
minimums=((time_spent_task_wait.failures.min=-1) (time_spent_on_put_request.min=28) (time_spent_task_wait.min=-1) (time_spent_on_put_request.failures.min=-1));
maximums=((time_spent_on_put_request.failures.max=-1) (time_spent_task_wait.max=-1) (time_spent_on_put_request.max=36) (time_spent_task_wait.failures.max=-1));
means=((time_spent_task_wait.mean=(samples=0, sum=0, mean=0.0000)) (time_spent_on_put_request.failures.mean=(samples=0, sum=0, mean=0.0000)) (time_spent_task_wait.failures.mean=(samples=0, sum=0, mean=0.0000)) (time_spent_on_put_request.mean=(samples=2, sum=64, mean=32.0000)));

s3a

2021-02-08 12:29:05,294 [JUnit-testSyncable] WARN  contract.AbstractContractCreateTest (AbstractContractCreateTest.java:validateSyncableSemantics(575)) - Output file was not created; this is an object store with different visibility semantics
2021-02-08 12:29:05,500 [JUnit-testSyncable] INFO  contract.AbstractContractCreateTest (AbstractContractCreateTest.java:validateSyncableSemantics(586)) - IOStatistics counters=((stream_write_bytes=2) (stream_write_queue_duration=0) (action_executor_acquired=1) (stream_write_block_uploads=1) (stream_write_exceptions=0) (stream_write_exceptions_completing_upload=0) (stream_write_total_time=0) (stream_write_total_data=2) (action_executor_acquired.failures=0));
gauges=((stream_write_block_uploads_data_pending=0) (stream_write_block_uploads_pending=1));
minimums=((action_executor_acquired.failures.min=-1) (action_executor_acquired.min=0));
maximums=((action_executor_acquired.max=0) (action_executor_acquired.failures.max=-1));
means=((action_executor_acquired.failures.mean=(samples=0, sum=0, mean=0.0000)) (action_executor_acquired.mean=(samples=1, sum=0, mean=0.0000)));

Copy link
Contributor

@mukund-thakur mukund-thakur left a 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
Copy link
Contributor

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
Copy link
Contributor

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.
Copy link
Contributor

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?

Copy link
Contributor Author

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.


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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

update

Copy link
Contributor Author

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
Copy link
Contributor

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
Copy link
Contributor

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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

data or date?

Copy link
Contributor Author

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.
Copy link
Contributor

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
@steveloughran
Copy link
Contributor Author

done the final comments and then rebased; unless yetus really blows up will merge in

@steveloughran steveloughran force-pushed the HADOOP-13327-outputstream-and-syncable branch from e96ba69 to adf17d0 Compare February 9, 2021 21:33
@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 40s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 markdownlint 0m 0s markdownlint was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 0m 0s test4tests The patch appears to include 11 new or modified test files.
_ trunk Compile Tests _
+0 🆗 mvndep 14m 1s Maven dependency ordering for branch
+1 💚 mvninstall 26m 18s trunk passed
+1 💚 compile 27m 42s trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04
+1 💚 compile 23m 17s trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01
+1 💚 checkstyle 4m 51s trunk passed
+1 💚 mvnsite 7m 8s trunk passed
+1 💚 shadedclient 30m 6s branch has no errors when building and testing our client artifacts.
+1 💚 javadoc 5m 4s trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 6m 11s trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01
+0 🆗 spotbugs 0m 53s Used deprecated FindBugs config; considering switching to SpotBugs.
+1 💚 findbugs 11m 36s trunk passed
_ Patch Compile Tests _
+0 🆗 mvndep 0m 48s Maven dependency ordering for patch
+1 💚 mvninstall 4m 24s the patch passed
+1 💚 compile 20m 16s the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04
-1 ❌ javac 20m 16s /diff-compile-javac-root-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04.txt root-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 generated 11 new + 2026 unchanged - 0 fixed = 2037 total (was 2026)
+1 💚 compile 18m 2s the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01
-1 ❌ javac 18m 2s /diff-compile-javac-root-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01.txt root-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu120.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu120.04-b01 generated 11 new + 1919 unchanged - 0 fixed = 1930 total (was 1919)
-0 ⚠️ checkstyle 3m 55s /diff-checkstyle-root.txt root: The patch generated 1 new + 183 unchanged - 7 fixed = 184 total (was 190)
+1 💚 mvnsite 6m 37s the patch passed
-1 ❌ whitespace 0m 0s /whitespace-eol.txt The patch has 2 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply
+1 💚 xml 0m 6s The patch has no ill-formed XML file.
+1 💚 shadedclient 13m 16s patch has no errors when building and testing our client artifacts.
+1 💚 javadoc 5m 7s the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04
+1 💚 javadoc 6m 12s the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01
+1 💚 findbugs 12m 13s the patch passed
_ Other Tests _
+1 💚 unit 17m 16s hadoop-common in the patch passed.
+1 💚 unit 2m 37s hadoop-hdfs-client in the patch passed.
-1 ❌ unit 191m 0s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 unit 2m 16s hadoop-aws in the patch passed.
+1 💚 unit 2m 23s hadoop-azure in the patch passed.
+1 💚 unit 1m 22s hadoop-azure-datalake in the patch passed.
+1 💚 asflicense 1m 9s The patch does not generate ASF License warnings.
460m 25s
Reason Tests
Failed junit tests hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2587/4/artifact/out/Dockerfile
GITHUB PR #2587
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint xml
uname Linux 8d9eac62d9ae 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 4625616
Default Java Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2587/4/testReport/
Max. process+thread count 3757 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-tools/hadoop-aws hadoop-tools/hadoop-azure hadoop-tools/hadoop-azure-datalake U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2587/4/console
versions git=2.25.1 maven=3.6.3 findbugs=4.0.6
Powered by Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@steveloughran steveloughran merged commit 798df6d into apache:trunk Feb 10, 2021
asfgit pushed a commit that referenced this pull request Feb 10, 2021
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
@ndimiduk
Copy link
Member

Thank you for this effort, @steveloughran !

@steveloughran
Copy link
Contributor Author

@ndimiduk this will let you run HBase against local FS now as if you turn off the checksumming the WAL will be synced properly...

@ndimiduk
Copy link
Member

No one should do this. Only want it for stable tests.

@steveloughran
Copy link
Contributor Author

No one should do this. Only want it for stable tests.

that and to challenge sqllite for dominance in the embedded-database market

jojochuang pushed a commit to jojochuang/hadoop that referenced this pull request May 23, 2023
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
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants