[
https://issues.apache.org/jira/browse/FLINK-10963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16707288#comment-16707288
]
ASF GitHub Bot commented on FLINK-10963:
----------------------------------------
dawidwys commented on a change in pull request #7161:
[FLINK-10963][fs-connector, s3] Cleanup tmp S3 objects uploaded as backups of
in-progress files.
URL: https://github.com/apache/flink/pull/7161#discussion_r238276552
##########
File path:
flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java
##########
@@ -86,41 +84,17 @@ RecoverableMultiPartUpload
recoverRecoverableUpload(S3Recoverable recoverable) t
incompletePart);
}
- @VisibleForTesting
- Optional<File> downloadLastDataChunk(S3Recoverable recoverable) throws
IOException {
+ private Optional<File> recoverInProgressPart(S3Recoverable recoverable)
throws IOException {
- final String objectName = recoverable.incompleteObjectName();
- if (objectName == null) {
+ final String objectKey = recoverable.incompleteObjectName();
+ if (objectKey == null) {
return Optional.empty();
}
// download the file (simple way)
- final RefCountedFile fileAndStream =
tmpFileSupplier.apply(null);
- final File file = fileAndStream.getFile();
-
- long numBytes = 0L;
-
- try (
- final OffsetAwareOutputStream outStream =
fileAndStream.getStream();
- final org.apache.hadoop.fs.FSDataInputStream
inStream =
- fs.open(new
org.apache.hadoop.fs.Path('/' + objectName))
- ) {
- final byte[] buffer = new byte[32 * 1024];
-
- int numRead;
- while ((numRead = inStream.read(buffer)) > 0) {
- outStream.write(buffer, 0, numRead);
- numBytes += numRead;
- }
- }
-
- // some sanity checks
- if (numBytes != file.length() || numBytes !=
fileAndStream.getStream().getLength()) {
- throw new IOException(String.format("Error recovering
writer: " +
- "Downloading the last
data chunk file gives incorrect length. " +
- "File=%d bytes,
Stream=%d bytes",
- file.length(), numBytes));
- }
+ final RefCountedFile refCountedFile =
tmpFileSupplier.apply(null);
+ final File file = refCountedFile.getFile();
+ final long numBytes = s3AccessHelper.getObject(objectKey, file);
Review comment:
I don't fully understand the difference, but you've changed the stream you
use to download the file from S3, previously you wrote through
`fileAndStream.getStream`, now you write through `new
FileOutputStream(fileAndStream.getFile())` Is there any difference? Is it ok to
change it?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> Cleanup small objects uploaded to S3 as independent objects
> -----------------------------------------------------------
>
> Key: FLINK-10963
> URL: https://issues.apache.org/jira/browse/FLINK-10963
> Project: Flink
> Issue Type: Sub-task
> Components: filesystem-connector
> Affects Versions: 1.7.0
> Reporter: Kostas Kloudas
> Assignee: Kostas Kloudas
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.7.1
>
>
> The S3 {{RecoverableWriter}} uses the Multipart Upload (MPU) Feature of S3 in
> order to upload the different part files. This means that a large part is
> split in chunks of at least 5MB which are uploaded independently, whenever
> each one of them is ready.
> This 5MB minimum size requires special handling of parts that are less than
> 5MB when a checkpoint barrier arrives. These small files are uploaded as
> independent objects (not associated with an active MPU). This way, when Flink
> needs to restore, it simply downloads them and resumes writing to them.
> These small objects are currently not cleaned up, thus leading to wasted
> space on S3.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)