/
FileStateHandle.java
144 lines (121 loc) · 4.18 KB
/
FileStateHandle.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
/*
* 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.FSDataInputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.AbstractCloseableHandle;
import org.apache.flink.runtime.state.StreamStateHandle;
import java.io.IOException;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* {@link StreamStateHandle} for state that was written to a file stream. The written data is
* identifier by the file path. The state can be read again by calling {@link #openInputStream()}.
*/
public class FileStateHandle extends AbstractCloseableHandle implements StreamStateHandle {
private static final long serialVersionUID = 350284443258002355L;
/** The path to the file in the filesystem, fully describing the file system */
private final Path filePath;
/** The size of the state in the file */
private final long stateSize;
/** Cached file system handle */
private transient FileSystem fs;
/**
* Creates a new file state for the given file path.
*
* @param filePath The path to the file that stores the state.
*/
public FileStateHandle(Path filePath, long stateSize) {
checkArgument(stateSize >= -1);
this.filePath = checkNotNull(filePath);
this.stateSize = stateSize;
}
/**
* Gets the path where this handle's state is stored.
*
* @return The path where this handle's state is stored.
*/
public Path getFilePath() {
return filePath;
}
@Override
public FSDataInputStream openInputStream() throws IOException {
ensureNotClosed();
FSDataInputStream inputStream = getFileSystem().open(filePath);
registerCloseable(inputStream);
return inputStream;
}
/**
* Discard the state by deleting the file that stores the state. If the parent directory
* of the state is empty after deleting the state file, it is also deleted.
*
* @throws Exception Thrown, if the file deletion (not the directory deletion) fails.
*/
@Override
public void discardState() throws Exception {
getFileSystem().delete(filePath, false);
// send a call to delete the checkpoint directory containing the file. This will
// fail (and be ignored) when some files still exist
try {
getFileSystem().delete(filePath.getParent(), false);
} catch (IOException ignored) {}
}
/**
* Returns the file size in bytes.
*
* @return The file size in bytes.
* @throws IOException Thrown if the file system cannot be accessed.
*/
@Override
public long getStateSize() throws IOException {
return stateSize;
}
/**
* Gets the file system that stores the file state.
*
* @return The file system that stores the file state.
* @throws IOException Thrown if the file system cannot be accessed.
*/
private FileSystem getFileSystem() throws IOException {
if (fs == null) {
fs = FileSystem.get(filePath.toUri());
}
return fs;
}
// ------------------------------------------------------------------------
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (!(o instanceof FileStateHandle)) {
return false;
}
FileStateHandle that = (FileStateHandle) o;
return filePath.equals(that.filePath);
}
@Override
public int hashCode() {
return filePath.hashCode();
}
@Override
public String toString() {
return String.format("File State: %s [%d bytes]", filePath, stateSize);
}
}