Github user StefanRRichter commented on a diff in the pull request:
https://github.com/apache/flink/pull/3522#discussion_r113682224
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStream.java
---
@@ -0,0 +1,152 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.fs.Path;
+import
org.apache.flink.runtime.state.CheckpointMetadataStreamFactory.CheckpointMetadataOutputStream;
+import
org.apache.flink.runtime.state.CheckpointMetadataStreamFactory.StreamHandleAndPointer;
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link CheckpointMetadataOutputStream} that writes into a file and
+ * returns a {@link StreamStateHandle} upon closing.
+ */
+public final class FsCheckpointMetadataOutputStream extends
CheckpointMetadataOutputStream {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(FsCheckpointMetadataOutputStream.class);
+
+ //
------------------------------------------------------------------------
+
+ private final FSDataOutputStream out;
+
+ private final Path path;
+
+ private final FileSystem fileSystem;
+
+ private volatile boolean closed;
+
+
+ public FsCheckpointMetadataOutputStream(FileSystem fileSystem, Path
path) throws IOException {
+ this.fileSystem = checkNotNull(fileSystem);
+ this.path = checkNotNull(path);
+
+ this.out = fileSystem.create(path, WriteMode.NO_OVERWRITE);
+ }
+
+ //
------------------------------------------------------------------------
+ // I/O
+ //
------------------------------------------------------------------------
+
+ @Override
+ public final void write(int b) throws IOException {
+ out.write(b);
+ }
+
+ @Override
+ public final 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 flush() throws IOException {
+ out.flush();
+ }
+
+ @Override
+ public void sync() throws IOException {
+ out.sync();
+ }
+
+ //
------------------------------------------------------------------------
+ // Closing
+ //
------------------------------------------------------------------------
+
+ public boolean isClosed() {
+ return closed;
+ }
+
+ @Override
+ public void close() {
+ if (!closed) {
--- End diff --
Either the `volatile` on `closed` is not required or this does not give
full thread safety and should be replaced by `AtomicBoolean`.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---