-
Notifications
You must be signed in to change notification settings - Fork 2.9k
/
FileOutStream.java
259 lines (236 loc) · 8.67 KB
/
FileOutStream.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
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
/*
* Licensed to the University of California, Berkeley 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 tachyon.client.file;
import java.io.IOException;
import java.io.OutputStream;
import java.util.LinkedList;
import java.util.List;
import com.google.common.base.Preconditions;
import tachyon.annotation.PublicApi;
import tachyon.client.Cancelable;
import tachyon.client.ClientContext;
import tachyon.client.ClientOptions;
import tachyon.client.FileSystemMasterClient;
import tachyon.client.TachyonStorageType;
import tachyon.client.UnderStorageType;
import tachyon.client.block.BlockStoreContext;
import tachyon.client.block.BufferedBlockOutStream;
import tachyon.underfs.UnderFileSystem;
import tachyon.util.io.PathUtils;
import tachyon.worker.WorkerClient;
/**
* Provides a streaming API to write a file. This class wraps the BlockOutStreams for each of the
* blocks in the file and abstracts the switching between streams. The backing streams can write to
* Tachyon space in the local machine or remote machines. If the
* {@link tachyon.client.UnderStorageType} is PERSIST, another stream will write the data to the
* under storage system.
*/
@PublicApi
public final class FileOutStream extends OutputStream implements Cancelable {
private final long mFileId;
private final long mBlockSize;
private final TachyonStorageType mTachyonStorageType;
private final UnderStorageType mUnderStorageType;
private final FileSystemContext mContext;
private final OutputStream mUnderStorageOutputStream;
private final String mUnderStorageFile;
private final WorkerClient mWorkerClient;
private boolean mCanceled;
private boolean mClosed;
private String mHostname;
private boolean mShouldCacheCurrentBlock;
private BufferedBlockOutStream mCurrentBlockOutStream;
private List<BufferedBlockOutStream> mPreviousBlockOutStreams;
/**
* Creates a new file output stream.
*
* @param fileId the file id
* @param options the client options
* @throws IOException if an I/O error occurs
*/
public FileOutStream(long fileId, ClientOptions options) throws IOException {
mFileId = fileId;
mBlockSize = options.getBlockSize();
mTachyonStorageType = options.getTachyonStorageType();
mUnderStorageType = options.getUnderStorageType();
mContext = FileSystemContext.INSTANCE;
mPreviousBlockOutStreams = new LinkedList<BufferedBlockOutStream>();
if (mUnderStorageType.isPersist()) {
mWorkerClient = BlockStoreContext.INSTANCE.acquireWorkerClient();
String sessionUnderStorageFolder = mWorkerClient.getSessionUfsTempFolder();
mUnderStorageFile = PathUtils.concatPath(sessionUnderStorageFolder, mFileId);
UnderFileSystem underStorageClient =
UnderFileSystem.get(mUnderStorageFile, ClientContext.getConf());
underStorageClient.mkdirs(sessionUnderStorageFolder, true);
mUnderStorageOutputStream = underStorageClient.create(mUnderStorageFile, (int) mBlockSize);
} else {
mWorkerClient = null;
mUnderStorageFile = null;
mUnderStorageOutputStream = null;
}
mClosed = false;
mCanceled = false;
mHostname = options.getHostname();
mShouldCacheCurrentBlock = mTachyonStorageType.isStore();
}
@Override
public void cancel() throws IOException {
mCanceled = true;
close();
}
@Override
public void close() throws IOException {
if (mClosed) {
return;
}
if (mCurrentBlockOutStream != null) {
mPreviousBlockOutStreams.add(mCurrentBlockOutStream);
}
Boolean canComplete = false;
if (mUnderStorageType.isPersist()) {
if (mCanceled) {
// TODO(yupeng): Handle this special case in under storage integrations.
mUnderStorageOutputStream.close();
UnderFileSystem underFsClient =
UnderFileSystem.get(mUnderStorageFile, ClientContext.getConf());
underFsClient.delete(mUnderStorageFile, false);
} else {
mUnderStorageOutputStream.flush();
mUnderStorageOutputStream.close();
try {
// TODO(yupeng): Investigate if this RPC can be moved to master.
mWorkerClient.addCheckpoint(mFileId);
} finally {
BlockStoreContext.INSTANCE.releaseWorkerClient(mWorkerClient);
}
canComplete = true;
}
}
if (mTachyonStorageType.isStore()) {
try {
if (mCanceled) {
for (BufferedBlockOutStream bos : mPreviousBlockOutStreams) {
bos.cancel();
}
} else {
for (BufferedBlockOutStream bos : mPreviousBlockOutStreams) {
bos.close();
}
canComplete = true;
}
} catch (IOException ioe) {
handleCacheWriteException(ioe);
}
}
if (canComplete) {
FileSystemMasterClient masterClient = mContext.acquireMasterClient();
try {
masterClient.completeFile(mFileId);
} finally {
mContext.releaseMasterClient(masterClient);
}
}
mClosed = true;
}
@Override
public void flush() throws IOException {
// TODO(yupeng): Handle flush for Tachyon storage stream as well.
if (mUnderStorageType.isPersist()) {
mUnderStorageOutputStream.flush();
}
}
@Override
public void write(int b) throws IOException {
if (mShouldCacheCurrentBlock) {
try {
if (mCurrentBlockOutStream == null || mCurrentBlockOutStream.remaining() == 0) {
getNextBlock();
}
mCurrentBlockOutStream.write(b);
} catch (IOException ioe) {
handleCacheWriteException(ioe);
}
}
if (mUnderStorageType.isPersist()) {
mUnderStorageOutputStream.write(b);
}
}
@Override
public void write(byte[] b) throws IOException {
write(b, 0, b.length);
}
@Override
public void write(byte[] b, int off, int len) throws IOException {
Preconditions.checkArgument(b != null, "Buffer is null");
Preconditions.checkArgument(off >= 0 && len >= 0 && len + off <= b.length,
String.format("Buffer length (%d), offset(%d), len(%d)", b.length, off, len));
if (mShouldCacheCurrentBlock) {
try {
int tLen = len;
int tOff = off;
while (tLen > 0) {
if (mCurrentBlockOutStream == null || mCurrentBlockOutStream.remaining() == 0) {
getNextBlock();
}
long currentBlockLeftBytes = mCurrentBlockOutStream.remaining();
if (currentBlockLeftBytes >= tLen) {
mCurrentBlockOutStream.write(b, tOff, tLen);
tLen = 0;
} else {
mCurrentBlockOutStream.write(b, tOff, (int) currentBlockLeftBytes);
tOff += currentBlockLeftBytes;
tLen -= currentBlockLeftBytes;
}
}
} catch (IOException ioe) {
handleCacheWriteException(ioe);
}
}
if (mUnderStorageType.isPersist()) {
mUnderStorageOutputStream.write(b, off, len);
}
}
private void getNextBlock() throws IOException {
if (mCurrentBlockOutStream != null) {
Preconditions.checkState(mCurrentBlockOutStream.remaining() <= 0,
"The current block still has space left, no need to get new block");
mPreviousBlockOutStreams.add(mCurrentBlockOutStream);
}
if (mTachyonStorageType.isStore()) {
mCurrentBlockOutStream =
mContext.getTachyonBlockStore().getOutStream(getNextBlockId(), mBlockSize, mHostname);
mShouldCacheCurrentBlock = true;
}
}
private long getNextBlockId() throws IOException {
FileSystemMasterClient masterClient = mContext.acquireMasterClient();
try {
return masterClient.getNewBlockIdForFile(mFileId);
} finally {
mContext.releaseMasterClient(masterClient);
}
}
private void handleCacheWriteException(IOException ioe) throws IOException {
if (!mUnderStorageType.isPersist()) {
// TODO(yupeng): Handle this exception better.
throw new IOException("Fail to cache: " + ioe.getMessage(), ioe);
}
// TODO(yupeng): Handle this error.
if (mCurrentBlockOutStream != null) {
mShouldCacheCurrentBlock = false;
mCurrentBlockOutStream.cancel();
}
}
}