xintongsong commented on code in PR #21508:
URL: https://github.com/apache/flink/pull/21508#discussion_r1051919235
##########
flink-filesystems/flink-azure-fs-hadoop/pom.xml:
##########
@@ -185,13 +185,6 @@ under the License.
</relocation>
</relocations>
<filters>
- <filter>
-
<artifact>org.apache.flink:flink-hadoop-fs</artifact>
-
<excludes>
-
<exclude>org/apache/flink/runtime/util/HadoopUtils</exclude>
-
<exclude>org/apache/flink/runtime/fs/hdfs/HadoopRecoverable*</exclude>
-
</excludes>
- </filter>
Review Comment:
Just trying to understand, why do we need this change?
##########
flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java:
##########
@@ -37,7 +37,7 @@
public class HadoopFileSystem extends FileSystem {
/** The wrapped Hadoop File System. */
- private final org.apache.hadoop.fs.FileSystem fs;
+ protected final org.apache.hadoop.fs.FileSystem fs;
Review Comment:
This change is unnecessary. You can use `getHadoopFileSystem()` where it's
needed.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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;
+
+ public AzureBlobFsRecoverableDataOutputStream(FileSystem fs, Path
targetFile, Path tempFile)
Review Comment:
Can be package private.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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;
+
+ public 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);
+ }
+
+ public 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());
+ }
+ // 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.
+ initialFileSize = fs.getFileStatus(tempFile).getLen();
+ out = fs.append(tempFile);
+ 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;
+ while (remaining != 0) {
+ if (minBufferLength < remaining) {
+ buf = new byte[minBufferLength];
+ } else {
+ buf = new byte[(int) remaining];
+ }
+ fsDis.read(buf, 0, buf.length);
+ remaining -= buf.length;
+ LOG.info("Bytes remaining to read {}", remaining);
+ fsDataOutputStream.write(buf);
+ dataWritten += buf.length;
+ LOG.info("Successfully wrote {} bytes of data {}",
dataWritten);
Review Comment:
2 placeholders while only 1 argument is provided.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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;
+
+ public 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);
+ }
+
+ public 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());
+ }
+ // 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.
+ initialFileSize = fs.getFileStatus(tempFile).getLen();
+ out = fs.append(tempFile);
+ 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;
+ while (remaining != 0) {
+ if (minBufferLength < remaining) {
+ buf = new byte[minBufferLength];
+ } else {
+ buf = new byte[(int) remaining];
+ }
+ fsDis.read(buf, 0, buf.length);
Review Comment:
Better to check the actual number of bytes being read.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobRecoverableWriter.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.runtime.fs.hdfs.HadoopFsRecoverable;
+import org.apache.flink.runtime.fs.hdfs.HadoopRecoverableSerializer;
+import org.apache.flink.runtime.fs.hdfs.HadoopRecoverableWriter;
+
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/** Recoverable writer for AzureBlob file system. */
+public class AzureBlobRecoverableWriter extends HadoopRecoverableWriter {
+ /**
+ * Creates a new Recoverable writer.
+ *
+ * @param fs The AzureBlob file system on which the writer operates.
+ */
+ public AzureBlobRecoverableWriter(FileSystem fs) {
+ super(fs);
+ }
+
+ protected void checkSupportedFSSchemes(org.apache.hadoop.fs.FileSystem fs)
{
+ // This writer is only supported on a subset of file systems
+ if (!("abfs".equalsIgnoreCase(fs.getScheme())
+ || "abfss".equalsIgnoreCase(fs.getScheme()))) {
+ throw new UnsupportedOperationException(
+ "Recoverable writers on AzureBlob are only supported for
ABFS");
+ }
+ }
+
+ @Override
+ protected RecoverableFsDataOutputStream getRecoverableFsDataOutputStream(
+ org.apache.hadoop.fs.Path targetFile, org.apache.hadoop.fs.Path
tempFile)
+ throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream(fs, targetFile,
tempFile);
+ }
+
+ @Override
+ public RecoverableFsDataOutputStream recover(ResumeRecoverable
recoverable) throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream(fs,
(HadoopFsRecoverable) recoverable);
+ }
+
+ @Override
+ public boolean requiresCleanupOfRecoverableState() {
+ return false;
+ }
+
+ @Override
+ public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws
IOException {
+ return false;
+ }
+
+ @Override
+ public RecoverableFsDataOutputStream.Committer
recoverForCommit(CommitRecoverable recoverable)
+ throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream.ABFSCommitter(
+ fs, (HadoopFsRecoverable) recoverable);
+ }
+
+ @Override
+ public SimpleVersionedSerializer<CommitRecoverable>
getCommitRecoverableSerializer() {
+ @SuppressWarnings("unchecked")
+ SimpleVersionedSerializer<CommitRecoverable> typedSerializer =
+ (SimpleVersionedSerializer<CommitRecoverable>)
+ (SimpleVersionedSerializer<?>)
HadoopRecoverableSerializer.INSTANCE;
+
+ return typedSerializer;
+ }
+
+ @Override
+ public SimpleVersionedSerializer<ResumeRecoverable>
getResumeRecoverableSerializer() {
+ @SuppressWarnings("unchecked")
+ SimpleVersionedSerializer<ResumeRecoverable> typedSerializer =
+ (SimpleVersionedSerializer<ResumeRecoverable>)
+ (SimpleVersionedSerializer<?>)
HadoopRecoverableSerializer.INSTANCE;
+ return typedSerializer;
+ }
+
+ @Override
+ public boolean supportsResume() {
+ return true;
+ }
Review Comment:
And then we won't need the visibility change for
`HadoopRecoverableSerializer`.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFileSystem.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.RecoverableWriter;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
+
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/** Wraps the hadoop's file system to create AzureBlobFileSystem. */
+@Internal
+class AzureBlobFileSystem extends HadoopFileSystem {
+ /**
+ * Wraps the given Hadoop File System object as a Flink File System
object. The given Hadoop
+ * file system object is expected to be initialized already.
+ *
+ * @param hadoopFileSystem The Azure Blob FileSystem that will be used
under the hood.
+ */
+ public AzureBlobFileSystem(FileSystem hadoopFileSystem) {
Review Comment:
Can be package private.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobRecoverableWriter.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.runtime.fs.hdfs.HadoopFsRecoverable;
+import org.apache.flink.runtime.fs.hdfs.HadoopRecoverableSerializer;
+import org.apache.flink.runtime.fs.hdfs.HadoopRecoverableWriter;
+
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/** Recoverable writer for AzureBlob file system. */
+public class AzureBlobRecoverableWriter extends HadoopRecoverableWriter {
+ /**
+ * Creates a new Recoverable writer.
+ *
+ * @param fs The AzureBlob file system on which the writer operates.
+ */
+ public AzureBlobRecoverableWriter(FileSystem fs) {
+ super(fs);
+ }
+
+ protected void checkSupportedFSSchemes(org.apache.hadoop.fs.FileSystem fs)
{
+ // This writer is only supported on a subset of file systems
+ if (!("abfs".equalsIgnoreCase(fs.getScheme())
+ || "abfss".equalsIgnoreCase(fs.getScheme()))) {
+ throw new UnsupportedOperationException(
+ "Recoverable writers on AzureBlob are only supported for
ABFS");
+ }
+ }
+
+ @Override
+ protected RecoverableFsDataOutputStream getRecoverableFsDataOutputStream(
+ org.apache.hadoop.fs.Path targetFile, org.apache.hadoop.fs.Path
tempFile)
+ throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream(fs, targetFile,
tempFile);
+ }
+
+ @Override
+ public RecoverableFsDataOutputStream recover(ResumeRecoverable
recoverable) throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream(fs,
(HadoopFsRecoverable) recoverable);
+ }
+
+ @Override
+ public boolean requiresCleanupOfRecoverableState() {
+ return false;
+ }
+
+ @Override
+ public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws
IOException {
+ return false;
+ }
Review Comment:
```suggestion
```
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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;
+
+ public 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);
+ }
+
+ public AzureBlobFsRecoverableDataOutputStream(FileSystem fs,
HadoopFsRecoverable recoverable)
Review Comment:
Can be package private.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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 {
Review Comment:
This class is very similar to `HadoopRecoverableFsDataOutputStream`. The
only differences are in the initialization and the committer implementation. I
wonder if they can be further deduplicated. I'm particularly interested in
hiding `HadoopFsRecoverable` from this submodule.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobRecoverableWriter.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.runtime.fs.hdfs.HadoopFsRecoverable;
+import org.apache.flink.runtime.fs.hdfs.HadoopRecoverableSerializer;
+import org.apache.flink.runtime.fs.hdfs.HadoopRecoverableWriter;
+
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/** Recoverable writer for AzureBlob file system. */
+public class AzureBlobRecoverableWriter extends HadoopRecoverableWriter {
+ /**
+ * Creates a new Recoverable writer.
+ *
+ * @param fs The AzureBlob file system on which the writer operates.
+ */
+ public AzureBlobRecoverableWriter(FileSystem fs) {
+ super(fs);
+ }
+
+ protected void checkSupportedFSSchemes(org.apache.hadoop.fs.FileSystem fs)
{
+ // This writer is only supported on a subset of file systems
+ if (!("abfs".equalsIgnoreCase(fs.getScheme())
+ || "abfss".equalsIgnoreCase(fs.getScheme()))) {
+ throw new UnsupportedOperationException(
+ "Recoverable writers on AzureBlob are only supported for
ABFS");
+ }
+ }
+
+ @Override
+ protected RecoverableFsDataOutputStream getRecoverableFsDataOutputStream(
+ org.apache.hadoop.fs.Path targetFile, org.apache.hadoop.fs.Path
tempFile)
+ throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream(fs, targetFile,
tempFile);
+ }
+
+ @Override
+ public RecoverableFsDataOutputStream recover(ResumeRecoverable
recoverable) throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream(fs,
(HadoopFsRecoverable) recoverable);
+ }
+
+ @Override
+ public boolean requiresCleanupOfRecoverableState() {
+ return false;
+ }
+
+ @Override
+ public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws
IOException {
+ return false;
+ }
+
+ @Override
+ public RecoverableFsDataOutputStream.Committer
recoverForCommit(CommitRecoverable recoverable)
+ throws IOException {
+ return new AzureBlobFsRecoverableDataOutputStream.ABFSCommitter(
+ fs, (HadoopFsRecoverable) recoverable);
+ }
+
+ @Override
+ public SimpleVersionedSerializer<CommitRecoverable>
getCommitRecoverableSerializer() {
+ @SuppressWarnings("unchecked")
+ SimpleVersionedSerializer<CommitRecoverable> typedSerializer =
+ (SimpleVersionedSerializer<CommitRecoverable>)
+ (SimpleVersionedSerializer<?>)
HadoopRecoverableSerializer.INSTANCE;
+
+ return typedSerializer;
+ }
+
+ @Override
+ public SimpleVersionedSerializer<ResumeRecoverable>
getResumeRecoverableSerializer() {
+ @SuppressWarnings("unchecked")
+ SimpleVersionedSerializer<ResumeRecoverable> typedSerializer =
+ (SimpleVersionedSerializer<ResumeRecoverable>)
+ (SimpleVersionedSerializer<?>)
HadoopRecoverableSerializer.INSTANCE;
+ return typedSerializer;
+ }
+
+ @Override
+ public boolean supportsResume() {
+ return true;
+ }
Review Comment:
```suggestion
```
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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;
+
+ public 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);
+ }
+
+ public 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());
+ }
+ // 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.
+ initialFileSize = fs.getFileStatus(tempFile).getLen();
+ out = fs.append(tempFile);
+ 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;
+ while (remaining != 0) {
+ if (minBufferLength < remaining) {
+ buf = new byte[minBufferLength];
+ } else {
+ buf = new byte[(int) remaining];
+ }
+ fsDis.read(buf, 0, buf.length);
+ remaining -= buf.length;
+ LOG.info("Bytes remaining to read {}", remaining);
+ fsDataOutputStream.write(buf);
+ dataWritten += buf.length;
+ LOG.info("Successfully wrote {} bytes of data {}",
dataWritten);
+ }
+ // TODO : Support intermediate flush?
+ LOG.info("Closing the temp rename file {}", renameTempPath);
+ fsDataOutputStream.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);
+ }
+
+ private void rename(FileSystem fs, Path renameTempPath) throws IOException
{
+ LOG.info("Renaming the temp rename file {} back to tempFile {}",
renameTempPath, tempFile);
+ try {
+ boolean result = fs.rename(renameTempPath, tempFile);
+ if (!result) {
+ LOG.error(
+ "Unable to recover. Rename operation failed {} to {}",
+ renameTempPath,
+ tempFile);
+ throw new IOException("Unable to recover. Rename operation
failed");
+ } else {
+ LOG.info("Rename was successful");
+ }
+ } catch (IOException e) {
+ LOG.error(
+ "Unable to recover. Renaming of tempFile did not happen
after truncating {} to {}",
+ renameTempPath,
+ tempFile);
+ throw e;
+ }
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ out.write(b, off, len);
+ }
+
+ @Override
+ public long getPos() throws IOException {
+ return out.getPos();
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ out.write(b);
+ }
+
+ @Override
+ public void flush() throws IOException {
+ out.hflush();
+ }
+
+ @Override
+ public void sync() throws IOException {
+ out.hflush();
+ out.hsync();
+ }
+
+ @Override
+ public RecoverableWriter.ResumeRecoverable persist() throws IOException {
+ sync();
+ return new HadoopFsRecoverable(targetFile, tempFile, getPos() +
initialFileSize);
+ }
+
+ @Override
+ public Committer closeForCommit() throws IOException {
+ final long pos = getPos();
+ close();
+ return new AzureBlobFsRecoverableDataOutputStream.ABFSCommitter(
+ fs, new HadoopFsRecoverable(targetFile, tempFile, pos +
initialFileSize));
+ }
+
+ @Override
+ public void close() throws IOException {
+ out.close();
+ }
+
+ // ------------------------------------------------------------------------
+ // Committer
+ // ------------------------------------------------------------------------
+
+ /**
+ * Implementation of a committer for the Hadoop File System abstraction.
This implementation
+ * commits by renaming the temp file to the final file path. The temp file
is truncated before
+ * renaming in case there is trailing garbage data.
+ */
+ static class ABFSCommitter implements Committer {
+
+ private final FileSystem fs;
+ private final HadoopFsRecoverable recoverable;
+
+ ABFSCommitter(FileSystem fs, HadoopFsRecoverable recoverable) {
+ this.fs = checkNotNull(fs);
+ this.recoverable = checkNotNull(recoverable);
+ }
+
+ @Override
+ public void commit() throws IOException {
+ final Path src = recoverable.tempFile();
+ final Path dest = recoverable.targetFile();
+ final long expectedLength = recoverable.offset();
+ FileStatus srcStatus = null;
+ try {
+ srcStatus = fs.getFileStatus(src);
+ } catch (FileNotFoundException fnfe) {
+ // srcStatus will be null
+ } catch (IOException e) {
+ throw new IOException("Cannot clean commit: Staging file does
not exist.");
+ }
+ if (srcStatus != null) {
+ LOG.debug(
+ "The srcStatus is {} and exp length is {}",
+ srcStatus.getLen(),
+ expectedLength);
+ if (srcStatus.getLen() != expectedLength) {
+ LOG.error(
+ "The src file {} with length {} does not match the
expected length {}",
+ src,
+ srcStatus.getLen(),
+ expectedLength);
+ throw new IOException(
+ "The src file "
+ + src
+ + " with length "
+ + srcStatus.getLen()
+ + " "
+ + "does not match the expected length "
+ + expectedLength);
+ }
+ try {
+ fs.rename(src, dest);
+ } catch (IOException e) {
+ throw new IOException(
+ "Committing file by rename failed: " + src + " to
" + dest, e);
+ }
+ } else if (!fs.exists(dest)) {
+ // neither exists - that can be a sign of
+ // - (1) a serious problem (file system loss of data)
+ // - (2) a recovery of a savepoint that is some time old and
the users
+ // removed the files in the meantime.
+
+ // TODO how to handle this?
+ // We probably need an option for users whether this should
log,
+ // or result in an exception or unrecoverable exception
Review Comment:
I think we should explicitly throw an exception here, because the data
cannot be committed.
##########
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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;
+
+ public 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);
+ }
+
+ public 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());
+ }
+ // 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.
+ initialFileSize = fs.getFileStatus(tempFile).getLen();
+ out = fs.append(tempFile);
+ 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);
Review Comment:
This means that, despite the src and dest files are on the same remote file
system, all the data being copied have to go through the local system where
this piece of code is executed. I wonder if there's a better way doing this.
(Maybe not, just trying to double-check on this.)
--
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]