xintongsong commented on code in PR #21508: URL: https://github.com/apache/flink/pull/21508#discussion_r1064353993
########## flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AzureBlobFsRecoverableDataOutputStream.java: ########## @@ -0,0 +1,303 @@ +/* + * 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.BaseHadoopFsRecoverableFsDataOutputStream; +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 BaseHadoopFsRecoverableFsDataOutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(AzureBlobFsRecoverableDataOutputStream.class); + private static final String RENAME = ".rename"; + + // Not final to override in tests + public static int minBufferLength = 2097152; Review Comment: We use `@VisibleForTesting` for fields that have broader visibility then needed for testing purpose. ########## flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/BaseHadoopFsRecoverableFsDataOutputStream.java: ########## @@ -0,0 +1,87 @@ +/* + * 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.runtime.fs.hdfs; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; +import org.apache.flink.core.fs.RecoverableWriter; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +/** Base class for ABFS and Hadoop recoverable stream. */ +@Internal +public abstract class BaseHadoopFsRecoverableFsDataOutputStream + extends RecoverableFsDataOutputStream { + + protected FileSystem fs; + + protected Path targetFile; + + protected Path tempFile; + + protected FSDataOutputStream out; + + // In ABFS outputstream we need to add this to the current pos + protected long initialFileSize = 0; + + public long getPos() throws IOException { + return out.getPos(); + } + + @Override + public void write(int b) throws IOException { + out.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + out.write(b, off, len); + } + + @Override + public abstract void flush() throws IOException; + + @Override + public void sync() throws IOException { + out.hflush(); + out.hsync(); + } Review Comment: 1. IIUC, @anoopsjohn 's comment was that you can call `hsync` without calling `hflush` in `sync()` for ABFS. It was not about ABFS should call `hsync` in `flush()`. Though the two methods do the same thing, according to him. 2. Not needing to call `hflush` before `hsync` sounds to be a specialty of ABFS. We probably should make `AzureBlobFsRecoverableDataOutputStream` override the `flush/sync` method, rather than make the method abstract in the base class. ########## flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableSerializer.java: ########## @@ -33,7 +33,7 @@ @Internal class HadoopRecoverableSerializer implements SimpleVersionedSerializer<HadoopFsRecoverable> { - static final HadoopRecoverableSerializer INSTANCE = new HadoopRecoverableSerializer(); + public static final HadoopRecoverableSerializer INSTANCE = new HadoopRecoverableSerializer(); Review Comment: No longer needed. -- 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]
