-
Notifications
You must be signed in to change notification settings - Fork 2.9k
/
BlockInStream.java
254 lines (234 loc) · 10.3 KB
/
BlockInStream.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
/*
* 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.client.block.stream;
import alluxio.Configuration;
import alluxio.PropertyKey;
import alluxio.Seekable;
import alluxio.client.BoundedStream;
import alluxio.client.Locatable;
import alluxio.client.PositionedReadable;
import alluxio.client.block.AlluxioBlockStore;
import alluxio.client.block.BlockWorkerClient;
import alluxio.client.block.options.LockBlockOptions;
import alluxio.client.file.FileSystemContext;
import alluxio.client.file.options.InStreamOptions;
import alluxio.client.resource.LockBlockResource;
import alluxio.exception.AlluxioException;
import alluxio.proto.dataserver.Protocol;
import alluxio.util.CommonUtils;
import alluxio.util.network.NetworkAddressUtils;
import alluxio.wire.LockBlockResult;
import alluxio.wire.WorkerNetAddress;
import com.google.common.base.Preconditions;
import com.google.common.io.Closer;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import javax.annotation.concurrent.NotThreadSafe;
/**
* Provides a stream API to read a block from Alluxio. An instance extending this class can be
* obtained by calling {@link AlluxioBlockStore#getInStream}. Multiple
* {@link BlockInStream}s can be opened for a block.
*
* This class provides the same methods as a Java {@link InputStream} with additional methods from
* Alluxio Stream interfaces.
*
* Block lock ownership:
* The read lock of the block is acquired when the stream is created and released when the
* stream is closed.
*/
@NotThreadSafe
public class BlockInStream extends FilterInputStream implements BoundedStream, Seekable,
PositionedReadable, Locatable {
/** Helper to manage closeables. */
private final Closer mCloser;
private final BlockWorkerClient mBlockWorkerClient;
private final boolean mLocal;
private final PacketInStream mInputStream;
/**
* Creates an instance of local {@link BlockInStream} that reads from local worker.
*
* @param blockId the block ID
* @param blockSize the block size
* @param workerNetAddress the worker network address
* @param context the file system context
* @param options the options
* @throws IOException if it fails to create an instance
* @return the {@link BlockInStream} created
*/
// TODO(peis): Use options idiom (ALLUXIO-2579).
public static BlockInStream createLocalBlockInStream(long blockId, long blockSize,
WorkerNetAddress workerNetAddress, FileSystemContext context, InStreamOptions options)
throws IOException {
Closer closer = Closer.create();
try {
BlockWorkerClient blockWorkerClient =
closer.register(context.createBlockWorkerClient(workerNetAddress));
LockBlockResource lockBlockResource =
closer.register(blockWorkerClient.lockBlock(blockId, LockBlockOptions.defaults()));
PacketInStream inStream = closer.register(PacketInStream
.createLocalPacketInStream(lockBlockResource.getResult().getBlockPath(), blockId,
blockSize));
blockWorkerClient.accessBlock(blockId);
return new BlockInStream(inStream, blockWorkerClient, closer, options);
} catch (AlluxioException | IOException e) {
CommonUtils.closeQuietly(closer);
throw CommonUtils.castToIOException(e);
}
}
/**
* Creates an instance of remote {@link BlockInStream} that reads from a remote worker.
*
* @param blockId the block ID
* @param blockSize the block size
* @param workerNetAddress the worker network address
* @param context the file system context
* @param options the options
* @throws IOException if it fails to create an instance
* @return the {@link BlockInStream} created
*/
// TODO(peis): Use options idiom (ALLUXIO-2579).
public static BlockInStream createRemoteBlockInStream(long blockId, long blockSize,
WorkerNetAddress workerNetAddress, FileSystemContext context, InStreamOptions options)
throws IOException {
Closer closer = Closer.create();
try {
BlockWorkerClient blockWorkerClient =
closer.register(context.createBlockWorkerClient(workerNetAddress));
LockBlockResource lockBlockResource =
closer.register(blockWorkerClient.lockBlock(blockId, LockBlockOptions.defaults()));
PacketInStream inStream = closer.register(PacketInStream
.createNettyPacketInStream(context, blockWorkerClient.getDataServerAddress(), blockId,
lockBlockResource.getResult().getLockId(), blockWorkerClient.getSessionId(),
blockSize, false, Protocol.RequestType.ALLUXIO_BLOCK));
blockWorkerClient.accessBlock(blockId);
return new BlockInStream(inStream, blockWorkerClient, closer, options);
} catch (AlluxioException | IOException e) {
CommonUtils.closeQuietly(closer);
throw CommonUtils.castToIOException(e);
}
}
/**
* Creates an instance of {@link BlockInStream}.
*
* This method keeps polling the block worker until the block is cached to Alluxio or
* it successfully acquires a UFS read token with a timeout.
* (1) If the block is cached to Alluxio after polling, it returns {@link BlockInStream}
* to read the block from Alluxio storage.
* (2) If a UFS read token is acquired after polling, it returns {@link BlockInStream}
* to read the block from an Alluxio worker that reads the block from UFS.
* (3) If the polling times out, an {@link IOException} with cause
* {@link alluxio.exception.UfsBlockAccessTokenUnavailableException} is thrown.
*
* @param context the file system context
* @param ufsPath the UFS path
* @param blockId the block ID
* @param blockSize the block size
* @param blockStart the position at which the block starts in the file
* @param alluxioMountPoint the mount point of the file in Alluxio
* @param mountTableVersion the version of Alluxio mount table
* @param workerNetAddress the worker network address
* @param options the options
* @throws IOException if it fails to create an instance
* @return the {@link BlockInStream} created
*/
// TODO(peis): Use options idiom (ALLUXIO-2579).
public static BlockInStream createUfsBlockInStream(FileSystemContext context, String ufsPath,
long blockId, long blockSize, long blockStart, String alluxioMountPoint,
long mountTableVersion,
WorkerNetAddress workerNetAddress, InStreamOptions options) throws IOException {
Closer closer = Closer.create();
try {
BlockWorkerClient blockWorkerClient =
closer.register(context.createBlockWorkerClient(workerNetAddress));
LockBlockOptions lockBlockOptions =
LockBlockOptions.defaults().setUfsPath(ufsPath).setOffset(blockStart)
.setBlockSize(blockSize).setMaxUfsReadConcurrency(options.getMaxUfsReadConcurrency())
.setAlluxioMountPoint(alluxioMountPoint).setMountTableVersion(mountTableVersion);
LockBlockResult lockBlockResult =
closer.register(blockWorkerClient.lockUfsBlock(blockId, lockBlockOptions)).getResult();
PacketInStream inStream;
if (lockBlockResult.getLockBlockStatus().blockInAlluxio()) {
boolean local = blockWorkerClient.getDataServerAddress().getHostName()
.equals(NetworkAddressUtils.getClientHostName());
if (local && Configuration.getBoolean(PropertyKey.USER_SHORT_CIRCUIT_ENABLED)) {
inStream = closer.register(PacketInStream
.createLocalPacketInStream(lockBlockResult.getBlockPath(), blockId, blockSize));
} else {
inStream = closer.register(PacketInStream
.createNettyPacketInStream(context, blockWorkerClient.getDataServerAddress(), blockId,
lockBlockResult.getLockId(), blockWorkerClient.getSessionId(), blockSize, false,
Protocol.RequestType.ALLUXIO_BLOCK));
}
blockWorkerClient.accessBlock(blockId);
} else {
Preconditions.checkState(lockBlockResult.getLockBlockStatus().ufsTokenAcquired());
inStream = closer.register(PacketInStream
.createNettyPacketInStream(context, blockWorkerClient.getDataServerAddress(), blockId,
lockBlockResult.getLockId(), blockWorkerClient.getSessionId(), blockSize,
!options.getAlluxioStorageType().isStore(), Protocol.RequestType.UFS_BLOCK));
}
return new BlockInStream(inStream, blockWorkerClient, closer, options);
} catch (AlluxioException | IOException e) {
CommonUtils.closeQuietly(closer);
throw CommonUtils.castToIOException(e);
}
}
@Override
public void close() throws IOException {
mCloser.close();
}
@Override
public long remaining() {
return mInputStream.remaining();
}
@Override
public void seek(long pos) throws IOException {
mInputStream.seek(pos);
}
@Override
public int positionedRead(long pos, byte[] b, int off, int len) throws IOException {
return mInputStream.positionedRead(pos, b, off, len);
}
@Override
public InetSocketAddress location() {
return mBlockWorkerClient.getDataServerAddress();
}
@Override
public boolean isLocal() {
return mLocal;
}
/**
* @return whether this stream is reading directly from a local file
*/
public boolean isShortCircuit() {
return mInputStream.isShortCircuit();
}
/**
* Creates an instance of {@link BlockInStream}.
*
* @param inputStream the packet inputstream
* @param blockWorkerClient the block worker client
* @param closer the closer registered with closable resources open so far
* @param options the options
*/
protected BlockInStream(PacketInStream inputStream, BlockWorkerClient blockWorkerClient,
Closer closer, InStreamOptions options) {
super(inputStream);
mInputStream = inputStream;
mBlockWorkerClient = blockWorkerClient;
mCloser = closer;
mCloser.register(mInputStream);
mLocal = blockWorkerClient.getDataServerAddress().getHostName()
.equals(NetworkAddressUtils.getClientHostName());
}
}