ramkrish86 commented on code in PR #21508:
URL: https://github.com/apache/flink/pull/21508#discussion_r1060557482


##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.flink.fs.azurefs;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.runtime.fs.hdfs.HadoopFsRecoverable;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An implementation of the {@link RecoverableFsDataOutputStream} for 
AzureBlob's file system
+ * abstraction.
+ */
+@Internal
+public class AzureBlobFsRecoverableDataOutputStream extends 
RecoverableFsDataOutputStream {
+
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(AzureBlobFsRecoverableDataOutputStream.class);
+
+    private final FileSystem fs;
+
+    private final Path targetFile;
+
+    private final Path tempFile;
+
+    private final FSDataOutputStream out;
+
+    // Not final to override in tests
+    public static int minBufferLength = 2097152;
+
+    // init to 0. When ever recovery is done add this to the pos.
+    private long initialFileSize = 0;
+
+    AzureBlobFsRecoverableDataOutputStream(FileSystem fs, Path targetFile, 
Path tempFile)
+            throws IOException {
+        this.fs = checkNotNull(fs);
+        this.targetFile = checkNotNull(targetFile);
+        LOG.debug("The targetFile is {}", targetFile.getName());
+        this.tempFile = checkNotNull(tempFile);
+        LOG.debug("The tempFile is {}", tempFile.getName());
+        this.out = fs.create(tempFile);
+    }
+
+    AzureBlobFsRecoverableDataOutputStream(FileSystem fs, HadoopFsRecoverable 
recoverable)
+            throws IOException {
+        this.fs = checkNotNull(fs);
+        this.targetFile = checkNotNull(recoverable.targetFile());
+        this.tempFile = checkNotNull(recoverable.tempFile());
+        long len = fs.getFileStatus(tempFile).getLen();
+        LOG.info("The recoverable offset is {} and the file len is {}", 
recoverable.offset(), len);
+        // Happens when we recover from a previously committed offset. 
Otherwise this is not
+        // really needed
+        if (len > recoverable.offset()) {
+            truncate(fs, recoverable);
+        } else if (len < recoverable.offset()) {
+            LOG.error(
+                    "Temp file length {} is less than the expected recoverable 
offset {}",
+                    len,
+                    recoverable.offset());
+            throw new IOException(
+                    "Unable to create recoverable outputstream as length of 
file "
+                            + len
+                            + " is less than "
+                            + "recoverable offset "
+                            + recoverable.offset());
+        }
+        out = fs.append(tempFile);
+        if (out.getPos() == 0) {
+            // In ABFS when we try to append we don't account for the initial 
file size like we do
+            // in DFS.
+            // So we explicitly store this and when we do a persist call we 
make use of it.
+            // This we have raised a bug in ABFS hadoop driver side. Once 
fixed this will not be
+            // needed. So it should be ok to put this in side the 'if' check.
+            initialFileSize = fs.getFileStatus(tempFile).getLen();
+        }
+        LOG.debug("Created a new OS for appending {}", tempFile);
+    }
+
+    private void truncate(FileSystem fs, HadoopFsRecoverable recoverable) 
throws IOException {
+        Path renameTempPath = new Path(tempFile.toString() + ".rename");
+        try {
+            LOG.info(
+                    "Creating the temp rename file {} for truncating the 
tempFile {}",
+                    renameTempPath,
+                    tempFile);
+            FSDataOutputStream fsDataOutputStream = fs.create(renameTempPath);
+            LOG.info("Opening the tempFile {} for truncate", tempFile);
+            FSDataInputStream fsDis = fs.open(tempFile);
+            // 2 MB buffers. TODO : Make this configurable
+            long remaining = recoverable.offset();
+            byte[] buf = null;
+            long dataWritten = 0;
+            int readBytes = -1;
+            while (remaining != 0) {
+                if (minBufferLength < remaining) {
+                    buf = new byte[minBufferLength];
+                } else {
+                    buf = new byte[(int) remaining];
+                }
+                readBytes = fsDis.read(buf, 0, buf.length);
+                if (readBytes != -1) {
+                    remaining -= readBytes;
+                    LOG.info("Bytes remaining to read {}", remaining);
+                    fsDataOutputStream.write(buf, 0, readBytes);
+                    dataWritten += readBytes;
+                    LOG.info("Successfully wrote {} bytes of data", 
dataWritten);
+                } else {
+                    LOG.debug("Reached the end of the file");
+                    remaining = 0;
+                }
+            }
+            // TODO : Support intermediate flush?
+            LOG.info("Closing the temp rename file {}", renameTempPath);
+            fsDataOutputStream.close();
+            if (fsDis != null) {
+                LOG.debug("Closing the input stream");
+                fsDis.close();
+            }
+        } catch (IOException e) {
+            LOG.error(
+                    "Unable to recover. Exception while trying to truncate the 
temp file {}",
+                    tempFile);
+            // We cannot recover. This we can control if user does not want 
this??
+            throw e;
+        }
+        try {
+            LOG.info("Deleting the actual temp file {}", tempFile);
+            fs.delete(tempFile, false);
+        } catch (IOException e) {
+            LOG.error("Unable to recover. Error while deleting the temp file 
{}", tempFile);
+            // unable to recover.
+            throw e;
+        }
+        rename(fs, renameTempPath);

Review Comment:
   In our testing with DefaultRollingpolicy - the recovery is not really 
possible as the inprogress that was marked would have already been committed in 
the next checkpoint. Only with policy as Checkpoint based rolling policy we can 
achieve this . So basically when moving to an older checkpoint - I believe  yes 
if we fail in a delete and rename step it is a non recoverable error but we can 
always move to latest checkpoint. But am not very sure how this recovery will 
be used in case of FS based sinks. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to