/
RetryHandlingBlockWorkerClient.java
376 lines (343 loc) · 13.8 KB
/
RetryHandlingBlockWorkerClient.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
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
/*
* 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;
import alluxio.AbstractThriftClient;
import alluxio.Configuration;
import alluxio.Constants;
import alluxio.PropertyKey;
import alluxio.RuntimeConstants;
import alluxio.client.block.options.LockBlockOptions;
import alluxio.client.resource.LockBlockResource;
import alluxio.exception.AlluxioException;
import alluxio.exception.ExceptionMessage;
import alluxio.exception.UfsBlockAccessTokenUnavailableException;
import alluxio.exception.WorkerOutOfSpaceException;
import alluxio.exception.status.AlluxioStatusException;
import alluxio.metrics.MetricsSystem;
import alluxio.retry.CountingRetry;
import alluxio.retry.ExponentialBackoffRetry;
import alluxio.retry.RetryPolicy;
import alluxio.retry.TimeoutRetry;
import alluxio.thrift.AlluxioTException;
import alluxio.thrift.BlockWorkerClientService;
import alluxio.util.ThreadFactoryUtils;
import alluxio.util.network.NetworkAddressUtils;
import alluxio.wire.LockBlockResult;
import alluxio.wire.ThriftUtils;
import alluxio.wire.WorkerNetAddress;
import com.codahale.metrics.Counter;
import com.google.common.base.Preconditions;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import javax.annotation.concurrent.ThreadSafe;
/**
* The client talks to a block worker server. It keeps sending keep alive message to the worker
* server.
*
* Note: Every client instance is associated with a session which is usually created for each block
* stream. So be careful when reusing this client for multiple blocks.
*/
@ThreadSafe
public final class RetryHandlingBlockWorkerClient
extends AbstractThriftClient<BlockWorkerClientService.Client> implements BlockWorkerClient {
private static final Logger LOG = LoggerFactory.getLogger(RetryHandlingBlockWorkerClient.class);
private static final ScheduledExecutorService HEARTBEAT_POOL = Executors.newScheduledThreadPool(
Configuration.getInt(PropertyKey.USER_BLOCK_WORKER_CLIENT_THREADS),
ThreadFactoryUtils.build("block-worker-heartbeat-%d", true));
private static final ExecutorService HEARTBEAT_CANCEL_POOL = Executors.newFixedThreadPool(5,
ThreadFactoryUtils.build("block-worker-heartbeat-cancel-%d", true));
private final BlockWorkerThriftClientPool mClientPool;
private final BlockWorkerThriftClientPool mClientHeartbeatPool;
// Tracks the number of active heartbeat close requests.
private static final AtomicInteger NUM_ACTIVE_SESSIONS = new AtomicInteger(0);
private final Long mSessionId;
// This is the address of the data server on the worker.
private final InetSocketAddress mWorkerDataServerAddress;
private final WorkerNetAddress mWorkerNetAddress;
private final InetSocketAddress mRpcAddress;
private ScheduledFuture<?> mHeartbeat;
/**
* Factory method for {@link RetryHandlingBlockWorkerClient}.
*
* @param clientPool the client pool
* @param clientHeartbeatPool the client pool for heartbeat
* @param workerNetAddress the worker address to connect to
* @param sessionId the session id to use, this should be unique
* @throws IOException if it fails to register the session with the worker specified
*/
protected static RetryHandlingBlockWorkerClient create(BlockWorkerThriftClientPool clientPool,
BlockWorkerThriftClientPool clientHeartbeatPool, WorkerNetAddress workerNetAddress,
Long sessionId) throws IOException {
RetryHandlingBlockWorkerClient client =
new RetryHandlingBlockWorkerClient(clientPool, clientHeartbeatPool, workerNetAddress,
sessionId);
client.init();
return client;
}
private RetryHandlingBlockWorkerClient(
BlockWorkerThriftClientPool clientPool,
BlockWorkerThriftClientPool clientHeartbeatPool,
WorkerNetAddress workerNetAddress, final Long sessionId) {
mClientPool = clientPool;
mClientHeartbeatPool = clientHeartbeatPool;
mWorkerNetAddress = Preconditions.checkNotNull(workerNetAddress, "workerNetAddress");
mRpcAddress = NetworkAddressUtils.getRpcPortSocketAddress(workerNetAddress);
mWorkerDataServerAddress = NetworkAddressUtils.getDataPortSocketAddress(workerNetAddress);
mSessionId = sessionId;
}
private void init() throws IOException {
if (mSessionId != null) {
// Register the session before any RPCs for this session start.
ExponentialBackoffRetry retryPolicy =
new ExponentialBackoffRetry(BASE_SLEEP_MS, MAX_SLEEP_MS, RPC_MAX_NUM_RETRY);
try {
sessionHeartbeat(retryPolicy);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
// The heartbeat is scheduled to run in a fixed rate. The heartbeat won't consume a thread
// from the pool while it is not running.
mHeartbeat = HEARTBEAT_POOL.scheduleAtFixedRate(new Runnable() {
@Override
public void run() {
try {
sessionHeartbeat(new CountingRetry(0));
} catch (InterruptedException e) {
// Do nothing.
} catch (Exception e) {
LOG.warn("Failed to heartbeat for session {}", mSessionId, e.getMessage());
}
}
}, Configuration.getInt(PropertyKey.USER_HEARTBEAT_INTERVAL_MS),
Configuration.getInt(PropertyKey.USER_HEARTBEAT_INTERVAL_MS), TimeUnit.MILLISECONDS);
NUM_ACTIVE_SESSIONS.incrementAndGet();
}
}
@Override
public BlockWorkerClientService.Client acquireClient() throws IOException {
try {
return mClientPool.acquire();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
@Override
public void releaseClient(BlockWorkerClientService.Client client) {
mClientPool.release(client);
}
@Override
public void close() {
if (mHeartbeat != null) {
HEARTBEAT_CANCEL_POOL.submit(new Runnable() {
@Override
public void run() {
mHeartbeat.cancel(true);
mHeartbeat = null;
NUM_ACTIVE_SESSIONS.decrementAndGet();
}
});
}
}
@Override
public WorkerNetAddress getWorkerNetAddress() {
return mWorkerNetAddress;
}
@Override
public void accessBlock(final long blockId) throws IOException {
retryRPC(new RpcCallable<Void, BlockWorkerClientService.Client>() {
@Override
public Void call(BlockWorkerClientService.Client client) throws TException {
client.accessBlock(blockId);
return null;
}
});
}
@Override
public void cacheBlock(final long blockId) throws IOException, AlluxioException {
retryRPC(new RpcCallableThrowsAlluxioTException<Void, BlockWorkerClientService.Client>() {
@Override
public Void call(BlockWorkerClientService.Client client)
throws AlluxioTException, TException {
client.cacheBlock(getSessionId(), blockId);
return null;
}
});
}
@Override
public void cancelBlock(final long blockId) throws IOException, AlluxioException {
retryRPC(new RpcCallableThrowsAlluxioTException<Void, BlockWorkerClientService.Client>() {
@Override
public Void call(BlockWorkerClientService.Client client)
throws AlluxioTException, TException {
client.cancelBlock(getSessionId(), blockId);
return null;
}
});
}
@Override
public InetSocketAddress getDataServerAddress() {
return mWorkerDataServerAddress;
}
@Override
public long getSessionId() {
Preconditions.checkNotNull(mSessionId, "Session ID is accessed when it is not supported");
return mSessionId;
}
@Override
public LockBlockResource lockBlock(final long blockId, final LockBlockOptions options)
throws IOException, AlluxioException {
LockBlockResult result = retryRPC(
new RpcCallableThrowsAlluxioTException<LockBlockResult, BlockWorkerClientService.Client>() {
@Override
public LockBlockResult call(BlockWorkerClientService.Client client)
throws AlluxioTException, TException {
return ThriftUtils
.fromThrift(client.lockBlock(blockId, getSessionId(), options.toThrift()));
}
});
return new LockBlockResource(this, result, blockId);
}
@Override
public LockBlockResource lockUfsBlock(final long blockId, final LockBlockOptions options)
throws IOException, AlluxioException {
int retryInterval = Constants.SECOND_MS;
RetryPolicy retryPolicy = new TimeoutRetry(Configuration
.getLong(PropertyKey.USER_UFS_BLOCK_OPEN_TIMEOUT_MS), retryInterval);
do {
LockBlockResource resource = lockBlock(blockId, options);
if (resource.getResult().getLockBlockStatus().ufsTokenNotAcquired()) {
LOG.debug("Failed to acquire a UFS read token because of contention for block {} with "
+ "LockBlockOptions {}", blockId, options);
} else {
return resource;
}
} while (retryPolicy.attemptRetry());
throw new UfsBlockAccessTokenUnavailableException(
ExceptionMessage.UFS_BLOCK_ACCESS_TOKEN_UNAVAILABLE, blockId, options.getUfsPath());
}
@Override
public boolean promoteBlock(final long blockId) throws IOException, AlluxioException {
return retryRPC(
new RpcCallableThrowsAlluxioTException<Boolean, BlockWorkerClientService.Client>() {
@Override
public Boolean call(BlockWorkerClientService.Client client)
throws AlluxioTException, TException {
return client.promoteBlock(blockId);
}
});
}
@Override
public void removeBlock(final long blockId) throws IOException, AlluxioException {
retryRPC(new RpcCallableThrowsAlluxioTException<Void, BlockWorkerClientService.Client>() {
@Override
public Void call(BlockWorkerClientService.Client client)
throws AlluxioTException, TException {
client.removeBlock(blockId);
return null;
}
});
}
@Override
public String requestBlockLocation(final long blockId, final long initialBytes,
final int writeTier) throws IOException {
try {
return retryRPC(
new RpcCallableThrowsAlluxioTException<String, BlockWorkerClientService.Client>() {
@Override
public String call(BlockWorkerClientService.Client client)
throws AlluxioTException, TException {
return client.requestBlockLocation(getSessionId(), blockId, initialBytes, writeTier);
}
});
} catch (WorkerOutOfSpaceException e) {
throw new IOException(ExceptionMessage.CANNOT_REQUEST_SPACE
.getMessageWithUrl(RuntimeConstants.ALLUXIO_DEBUG_DOCS_URL, mRpcAddress, blockId));
} catch (AlluxioException e) {
throw new IOException(e);
}
}
@Override
public boolean requestSpace(final long blockId, final long requestBytes) throws IOException {
try {
boolean success = retryRPC(
new RpcCallableThrowsAlluxioTException<Boolean, BlockWorkerClientService.Client>() {
@Override
public Boolean call(BlockWorkerClientService.Client client)
throws AlluxioTException, TException {
return client.requestSpace(getSessionId(), blockId, requestBytes);
}
});
if (!success) {
throw new IOException(ExceptionMessage.CANNOT_REQUEST_SPACE
.getMessageWithUrl(RuntimeConstants.ALLUXIO_DEBUG_DOCS_URL, mRpcAddress, blockId));
}
return true;
} catch (AlluxioException e) {
throw new IOException(e);
}
}
@Override
public boolean unlockBlock(final long blockId) throws IOException {
return retryRPC(new RpcCallable<Boolean, BlockWorkerClientService.Client>() {
@Override
public Boolean call(BlockWorkerClientService.Client client) throws TException {
return client.unlockBlock(blockId, getSessionId());
}
});
}
/**
* sessionHeartbeat is not retried because it is supposed to be called periodically.
*
* @throws IOException if it fails to heartbeat
* @throws InterruptedException if heartbeat is interrupted
*/
@Override
public void sessionHeartbeat(RetryPolicy retryPolicy) throws IOException, InterruptedException {
TException exception;
do {
BlockWorkerClientService.Client client = mClientHeartbeatPool.acquire();
try {
client.sessionHeartbeat(mSessionId, null);
Metrics.BLOCK_WORKER_HEATBEATS.inc();
return;
} catch (AlluxioTException e) {
AlluxioStatusException se = AlluxioStatusException.fromThrift(e);
LOG.warn(se.getMessage());
throw new IOException(se);
} catch (TException e) {
client.getOutputProtocol().getTransport().close();
exception = e;
LOG.warn(e.getMessage());
} finally {
mClientHeartbeatPool.release(client);
}
} while (retryPolicy.attemptRetry());
Preconditions.checkNotNull(exception);
throw new IOException(exception);
}
/**
* Metrics related to the {@link RetryHandlingBlockWorkerClient}.
*/
public static final class Metrics {
private static final Counter BLOCK_WORKER_HEATBEATS =
MetricsSystem.clientCounter("BlockWorkerHeartbeats");
private Metrics() {} // prevent instantiation
}
}