-
Notifications
You must be signed in to change notification settings - Fork 2.9k
/
UnderFileSystemBlockReader.java
360 lines (326 loc) · 13.5 KB
/
UnderFileSystemBlockReader.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
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
/*
* The Alluxio Open Foundation licenses this work under the Apache License, version 2.0
* (the "License"). You may not use this work except in compliance with the License, which is
* available at www.apache.org/licenses/LICENSE-2.0
*
* This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
* either express or implied, as more fully set forth in the License.
*
* See the NOTICE file distributed with this work for information regarding copyright ownership.
*/
package alluxio.worker.block;
import alluxio.AlluxioURI;
import alluxio.Configuration;
import alluxio.PropertyKey;
import alluxio.StorageTierAssoc;
import alluxio.WorkerStorageTierAssoc;
import alluxio.exception.AlluxioException;
import alluxio.exception.BlockAlreadyExistsException;
import alluxio.exception.BlockDoesNotExistException;
import alluxio.exception.InvalidWorkerStateException;
import alluxio.exception.PreconditionMessage;
import alluxio.exception.status.AlluxioStatusException;
import alluxio.underfs.UfsManager;
import alluxio.underfs.UfsManager.UfsInfo;
import alluxio.underfs.UnderFileSystem;
import alluxio.util.network.NetworkAddressUtils;
import alluxio.worker.block.io.BlockReader;
import alluxio.worker.block.io.BlockWriter;
import alluxio.worker.block.meta.UnderFileSystemBlockMeta;
import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.channels.ReadableByteChannel;
import javax.annotation.concurrent.NotThreadSafe;
/**
* This class implements a {@link BlockReader} to read a block directly from UFS, and
* optionally cache the block to the Alluxio worker if the whole block it is read.
*/
@NotThreadSafe
public final class UnderFileSystemBlockReader implements BlockReader {
private static final Logger LOG = LoggerFactory.getLogger(UnderFileSystemBlockReader.class);
/** An object storing the mapping of tier aliases to ordinals. */
private final StorageTierAssoc mStorageTierAssoc = new WorkerStorageTierAssoc();
/** The initial size of the block allocated in Alluxio storage when the block is cached. */
private final long mInitialBlockSize;
/** The block metadata for the UFS block. */
private final UnderFileSystemBlockMeta mBlockMeta;
/** The Local block store. It is used to interact with Alluxio. */
private final BlockStore mLocalBlockStore;
/** The input stream to read from UFS. */
private InputStream mUnderFileSystemInputStream;
/** The mount point URI of the UFS we are reading from. */
private AlluxioURI mUfsMountPointUri;
/** The block writer to write the block to Alluxio. */
private BlockWriter mBlockWriter;
/** If set, the reader is closed and should not be used afterwards. */
private boolean mClosed;
/** The manager for different ufs. */
private final UfsManager mUfsManager;
/** The manager for all ufs instream */
private final UfsInputStreamManager mUfsInstreamManager;
/**
* The position of mUnderFileSystemInputStream (if not null) is blockStart + mInStreamPos.
* When mUnderFileSystemInputStream is not set, this is set to -1 (an invalid state) when
* mUnderFileSystemInputStream is null. Check mUnderFileSystemInputStream directly to see whether
* that is valid instead of relying on this invalid state of the position to be safe.
*/
private long mInStreamPos;
/**
* Creates an instance of {@link UnderFileSystemBlockReader} and initializes it with a reading
* offset.
*
* @param blockMeta the block meta
* @param offset the position within the block to start the read
* @param localBlockStore the Local block store
* @param ufsManager the manager of ufs
* @param ufsInstreamManager the manager of ufs instreams
* @return the block reader
* @throws BlockDoesNotExistException if the UFS block does not exist in the UFS block store
*/
public static UnderFileSystemBlockReader create(UnderFileSystemBlockMeta blockMeta, long offset,
BlockStore localBlockStore, UfsManager ufsManager, UfsInputStreamManager ufsInstreamManager)
throws BlockDoesNotExistException, IOException {
UnderFileSystemBlockReader ufsBlockReader =
new UnderFileSystemBlockReader(blockMeta, localBlockStore, ufsManager, ufsInstreamManager);
ufsBlockReader.init(offset);
return ufsBlockReader;
}
/**
* Creates an instance of {@link UnderFileSystemBlockReader}.
*
* @param blockMeta the block meta
* @param localBlockStore the Local block store
* @param ufsManager the manager of ufs
* @param ufsInstreamManager the manager of ufs instreams
*/
private UnderFileSystemBlockReader(UnderFileSystemBlockMeta blockMeta, BlockStore localBlockStore,
UfsManager ufsManager, UfsInputStreamManager ufsInstreamManager) {
mInitialBlockSize = Configuration.getBytes(PropertyKey.WORKER_FILE_BUFFER_SIZE);
mBlockMeta = blockMeta;
mLocalBlockStore = localBlockStore;
mInStreamPos = -1;
mUfsManager = ufsManager;
mUfsInstreamManager = ufsInstreamManager;
}
/**
* Initializes the reader. This is only called in the factory method.
*
* @param offset the position within the block to start the read
* @throws BlockDoesNotExistException if the UFS block does not exist in the UFS block store
*/
private void init(long offset) throws BlockDoesNotExistException, IOException {
UnderFileSystem ufs = mUfsManager.get(mBlockMeta.getMountId()).getUfs();
ufs.connectFromWorker(
NetworkAddressUtils.getConnectHost(NetworkAddressUtils.ServiceType.WORKER_RPC));
updateUnderFileSystemInputStream(offset);
updateBlockWriter(offset);
}
@Override
public ReadableByteChannel getChannel() {
throw new UnsupportedOperationException("UFSFileBlockReader#getChannel is not supported");
}
@Override
public long getLength() {
return mBlockMeta.getBlockSize();
}
@Override
public ByteBuffer read(long offset, long length) throws IOException {
Preconditions.checkState(!mClosed);
updateUnderFileSystemInputStream(offset);
updateBlockWriter(offset);
long bytesToRead = Math.min(length, mBlockMeta.getBlockSize() - offset);
if (bytesToRead <= 0) {
return ByteBuffer.allocate(0);
}
byte[] data = new byte[(int) bytesToRead];
int bytesRead = 0;
Preconditions.checkNotNull(mUnderFileSystemInputStream);
while (bytesRead < bytesToRead) {
int read;
try {
read = mUnderFileSystemInputStream.read(data, bytesRead, (int) (bytesToRead - bytesRead));
} catch (IOException e) {
throw AlluxioStatusException.fromIOException(e);
}
if (read == -1) {
break;
}
bytesRead += read;
}
mInStreamPos += bytesRead;
// We should always read the number of bytes as expected since the UFS file length (hence block
// size) should be always accurate.
Preconditions
.checkState(bytesRead == bytesToRead, PreconditionMessage.NOT_ENOUGH_BYTES_READ.toString(),
bytesRead, bytesToRead, mBlockMeta.getUnderFileSystemPath());
if (mBlockWriter != null && mBlockWriter.getPosition() < mInStreamPos) {
try {
Preconditions.checkState(mBlockWriter.getPosition() >= offset);
mLocalBlockStore.requestSpace(mBlockMeta.getSessionId(), mBlockMeta.getBlockId(),
mInStreamPos - mBlockWriter.getPosition());
ByteBuffer buffer = ByteBuffer.wrap(data, (int) (mBlockWriter.getPosition() - offset),
(int) (mInStreamPos - mBlockWriter.getPosition()));
mBlockWriter.append(buffer.duplicate());
} catch (Exception e) {
LOG.warn("Failed to cache data read from UFS (on read()): {}", e.getMessage());
cancelBlockWriter();
}
}
return ByteBuffer.wrap(data, 0, bytesRead);
}
/**
* This interface is supposed to be used for sequence block reads.
*
* @param buf the byte buffer
* @return the number of bytes read, -1 if it reaches EOF and none was read
*/
@Override
public int transferTo(ByteBuf buf) throws IOException {
Preconditions.checkState(!mClosed);
if (mUnderFileSystemInputStream == null) {
return -1;
}
if (mBlockMeta.getBlockSize() <= mInStreamPos) {
return -1;
}
// Make a copy of the state to keep track of what we have read in this transferTo call.
ByteBuf bufCopy = null;
if (mBlockWriter != null) {
bufCopy = buf.duplicate();
bufCopy.readerIndex(bufCopy.writerIndex());
}
int bytesToRead =
(int) Math.min(buf.writableBytes(), mBlockMeta.getBlockSize() - mInStreamPos);
int bytesRead = buf.writeBytes(mUnderFileSystemInputStream, bytesToRead);
if (bytesRead <= 0) {
return bytesRead;
}
mInStreamPos += bytesRead;
if (mBlockWriter != null && bufCopy != null) {
try {
bufCopy.writerIndex(buf.writerIndex());
while (bufCopy.readableBytes() > 0) {
mLocalBlockStore.requestSpace(mBlockMeta.getSessionId(), mBlockMeta.getBlockId(),
mInStreamPos - mBlockWriter.getPosition());
mBlockWriter.append(bufCopy);
}
} catch (Exception e) {
LOG.warn("Failed to cache data read from UFS (on transferTo()): {}", e.getMessage());
cancelBlockWriter();
}
}
return bytesRead;
}
/**
* Closes the block reader. After this, this block reader should not be used anymore.
* This is recommended to be called after the client finishes reading the block. It is usually
* triggered when the client unlocks the block.
*/
@Override
public void close() throws IOException {
if (mClosed) {
return;
}
try {
// This aborts the block if the block is not fully read.
updateBlockWriter(mBlockMeta.getBlockSize());
if (mUnderFileSystemInputStream != null) {
mUfsInstreamManager.checkIn(mUnderFileSystemInputStream);
mUnderFileSystemInputStream = null;
}
if(mBlockWriter!=null) {
mBlockWriter.close();
}
} finally {
mClosed = true;
}
}
@Override
public boolean isClosed() {
return mClosed;
}
/**
* @return the mount point URI of the UFS that this reader is currently reading from
*/
public AlluxioURI getUfsMountPointUri() {
return mUfsMountPointUri;
}
/**
* Updates the UFS input stream given an offset to read.
*
* @param offset the read offset within the block
*/
private void updateUnderFileSystemInputStream(long offset) throws IOException {
if ((mUnderFileSystemInputStream != null) && offset != mInStreamPos) {
mUfsInstreamManager.checkIn(mUnderFileSystemInputStream);
mUnderFileSystemInputStream = null;
mInStreamPos = -1;
}
if (mUnderFileSystemInputStream == null && offset < mBlockMeta.getBlockSize()) {
UfsInfo ufsInfo = mUfsManager.get(mBlockMeta.getMountId());
UnderFileSystem ufs = ufsInfo.getUfs();
mUfsMountPointUri = ufsInfo.getUfsMountPointUri();
mUnderFileSystemInputStream = mUfsInstreamManager.checkOut(ufs,
mBlockMeta.getUnderFileSystemPath(), mBlockMeta.getOffset() + offset);
mInStreamPos = offset;
}
}
/**
* Closes the current block writer, cleans up its temp block and sets it to null.
*/
private void cancelBlockWriter() throws IOException {
if (mBlockWriter == null) {
return;
}
try {
mBlockWriter.close();
mBlockWriter = null;
mLocalBlockStore.abortBlock(mBlockMeta.getSessionId(), mBlockMeta.getBlockId());
} catch (BlockDoesNotExistException e) {
// This can only happen when the session is expired.
LOG.warn("Block {} does not exist when being aborted. The session may have expired.",
mBlockMeta.getBlockId());
} catch (BlockAlreadyExistsException | InvalidWorkerStateException | IOException e) {
// We cannot skip the exception here because we need to make sure that the user of this
// reader does not commit the block if it fails to abort the block.
throw AlluxioStatusException.fromCheckedException(e);
}
}
/**
* Updates the block writer given an offset to read. If the offset is beyond the current
* position of the block writer, the block writer will be aborted.
*
* @param offset the read offset
*/
private void updateBlockWriter(long offset) throws IOException {
if (mBlockWriter != null && offset > mBlockWriter.getPosition()) {
cancelBlockWriter();
}
try {
if (mBlockWriter == null && offset == 0 && !mBlockMeta.isNoCache()) {
BlockStoreLocation loc = BlockStoreLocation.anyDirInTier(mStorageTierAssoc.getAlias(0));
mLocalBlockStore.createBlock(mBlockMeta.getSessionId(), mBlockMeta.getBlockId(), loc,
mInitialBlockSize);
mBlockWriter = mLocalBlockStore.getBlockWriter(
mBlockMeta.getSessionId(), mBlockMeta.getBlockId());
}
} catch (BlockAlreadyExistsException e) {
// This can happen when there are concurrent UFS readers who are all trying to cache to block.
LOG.debug(
"Failed to update block writer for UFS block [blockId: {}, ufsPath: {}, offset: {}]."
+ "Concurrent UFS readers may be caching the same block.",
mBlockMeta.getBlockId(), mBlockMeta.getUnderFileSystemPath(), offset, e);
mBlockWriter = null;
} catch (IOException | AlluxioException e) {
LOG.warn(
"Failed to update block writer for UFS block [blockId: {}, ufsPath: {}, offset: {}]: {}",
mBlockMeta.getBlockId(), mBlockMeta.getUnderFileSystemPath(), offset, e.getMessage());
mBlockWriter = null;
}
}
}