forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
/
BlobLibraryCacheManager.java
499 lines (419 loc) · 18.3 KB
/
BlobLibraryCacheManager.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
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
/*
* 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.execution.librarycache;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.runtime.blob.PermanentBlobService;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkUserCodeClassLoader;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.UserCodeClassLoader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
import java.io.IOException;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* Provides facilities to download a set of libraries (typically JAR files) for a job from a {@link
* PermanentBlobService} and create a class loader with references to them.
*/
@ThreadSafe
public class BlobLibraryCacheManager implements LibraryCacheManager {
private static final Logger LOG = LoggerFactory.getLogger(BlobLibraryCacheManager.class);
// --------------------------------------------------------------------------------------------
/** The global lock to synchronize operations. */
private final Object lockObject = new Object();
/** Registered entries per job. */
@GuardedBy("lockObject")
private final Map<JobID, LibraryCacheEntry> cacheEntries = new HashMap<>();
/** The blob service to download libraries. */
@GuardedBy("lockObject")
private final PermanentBlobService blobService;
private final ClassLoaderFactory classLoaderFactory;
// --------------------------------------------------------------------------------------------
public BlobLibraryCacheManager(
PermanentBlobService blobService, ClassLoaderFactory classLoaderFactory) {
this.blobService = checkNotNull(blobService);
this.classLoaderFactory = checkNotNull(classLoaderFactory);
}
@Override
public ClassLoaderLease registerClassLoaderLease(JobID jobId) {
synchronized (lockObject) {
return cacheEntries
.computeIfAbsent(jobId, jobID -> new LibraryCacheEntry(jobId))
.obtainLease();
}
}
/**
* Gets the number of tasks holding {@link ClassLoader} references for the given job.
*
* @param jobId ID of a job
* @return number of reference holders
*/
int getNumberOfReferenceHolders(JobID jobId) {
synchronized (lockObject) {
LibraryCacheEntry entry = cacheEntries.get(jobId);
return entry == null ? 0 : entry.getReferenceCount();
}
}
/**
* Returns the number of registered jobs that this library cache manager handles.
*
* @return number of jobs (irrespective of the actual number of tasks per job)
*/
int getNumberOfManagedJobs() {
synchronized (lockObject) {
return cacheEntries.size();
}
}
@Override
public void shutdown() {
synchronized (lockObject) {
for (LibraryCacheEntry entry : cacheEntries.values()) {
entry.releaseClassLoader();
}
cacheEntries.clear();
}
}
// --------------------------------------------------------------------------------------------
/** Customize classLoader factory. */
@FunctionalInterface
public interface ClassLoaderFactory {
URLClassLoader createClassLoader(URL[] libraryURLs);
}
/** Default ClassLoader factory. */
public static class DefaultClassLoaderFactory implements ClassLoaderFactory {
/** The resolve order to use when creating a {@link ClassLoader}. */
protected final FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder;
/**
* List of patterns for classes that should always be resolved from the parent ClassLoader,
* if possible.
*/
protected final String[] alwaysParentFirstPatterns;
/** Class loading exception handler. */
protected final Consumer<Throwable> classLoadingExceptionHandler;
/** Test if classloader is used outside of job. */
protected final boolean checkClassLoaderLeak;
protected DefaultClassLoaderFactory(
FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
Consumer<Throwable> classLoadingExceptionHandler,
boolean checkClassLoaderLeak) {
this.classLoaderResolveOrder = classLoaderResolveOrder;
this.alwaysParentFirstPatterns = alwaysParentFirstPatterns;
this.classLoadingExceptionHandler = classLoadingExceptionHandler;
this.checkClassLoaderLeak = checkClassLoaderLeak;
}
@Override
public URLClassLoader createClassLoader(URL[] libraryURLs) {
return FlinkUserCodeClassLoaders.create(
classLoaderResolveOrder,
libraryURLs,
getParentClassLoader(),
alwaysParentFirstPatterns,
classLoadingExceptionHandler,
checkClassLoaderLeak);
}
protected ClassLoader getParentClassLoader() {
return FlinkUserCodeClassLoaders.class.getClassLoader();
}
}
public static ClassLoaderFactory defaultClassLoaderFactory(
FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
@Nullable FatalErrorHandler fatalErrorHandlerJvmMetaspaceOomError,
boolean checkClassLoaderLeak) {
Consumer<Throwable> exceptionHandler =
createClassLoadingExceptionHandler(fatalErrorHandlerJvmMetaspaceOomError);
ServiceLoader<ClassLoaderFactoryBuilder> classLoaderService =
ServiceLoader.load(ClassLoaderFactoryBuilder.class);
Iterator<ClassLoaderFactoryBuilder> factoryIt = classLoaderService.iterator();
ClassLoaderFactoryBuilder factory = null;
while (factoryIt.hasNext()) {
factory = factoryIt.next();
if (factory.isCompatible()) {
return factory.buildServerLoaderFactory(
classLoaderResolveOrder,
alwaysParentFirstPatterns,
exceptionHandler,
checkClassLoaderLeak);
}
}
return new DefaultClassLoaderFactory(
classLoaderResolveOrder,
alwaysParentFirstPatterns,
exceptionHandler,
checkClassLoaderLeak);
}
private static Consumer<Throwable> createClassLoadingExceptionHandler(
@Nullable FatalErrorHandler fatalErrorHandlerJvmMetaspaceOomError) {
return fatalErrorHandlerJvmMetaspaceOomError != null
? classLoadingException -> {
if (ExceptionUtils.isMetaspaceOutOfMemoryError(classLoadingException)) {
fatalErrorHandlerJvmMetaspaceOomError.onFatalError(classLoadingException);
}
}
: FlinkUserCodeClassLoader.NOOP_EXCEPTION_HANDLER;
}
// --------------------------------------------------------------------------------------------
private final class LibraryCacheEntry {
private final JobID jobId;
@GuardedBy("lockObject")
private int referenceCount;
@GuardedBy("lockObject")
@Nullable
private ResolvedClassLoader resolvedClassLoader;
@GuardedBy("lockObject")
private boolean isReleased;
private LibraryCacheEntry(JobID jobId) {
this.jobId = jobId;
referenceCount = 0;
this.resolvedClassLoader = null;
this.isReleased = false;
}
private UserCodeClassLoader getOrResolveClassLoader(
Collection<PermanentBlobKey> libraries, Collection<URL> classPaths)
throws IOException {
synchronized (lockObject) {
verifyIsNotReleased();
if (resolvedClassLoader == null) {
resolvedClassLoader =
new ResolvedClassLoader(
createUserCodeClassLoader(jobId, libraries, classPaths),
libraries,
classPaths);
} else {
resolvedClassLoader.verifyClassLoader(libraries, classPaths);
}
return resolvedClassLoader;
}
}
@GuardedBy("lockObject")
private URLClassLoader createUserCodeClassLoader(
JobID jobId,
Collection<PermanentBlobKey> requiredJarFiles,
Collection<URL> requiredClasspaths)
throws IOException {
try {
final URL[] libraryURLs =
new URL[requiredJarFiles.size() + requiredClasspaths.size()];
int count = 0;
// add URLs to locally cached JAR files
for (PermanentBlobKey key : requiredJarFiles) {
libraryURLs[count] = blobService.getFile(jobId, key).toURI().toURL();
++count;
}
// add classpaths
for (URL url : requiredClasspaths) {
libraryURLs[count] = url;
++count;
}
return classLoaderFactory.createClassLoader(libraryURLs);
} catch (Exception e) {
// rethrow or wrap
ExceptionUtils.tryRethrowIOException(e);
throw new IOException(
"Library cache could not register the user code libraries.", e);
}
}
@GuardedBy("lockObject")
public int getReferenceCount() {
return referenceCount;
}
@GuardedBy("lockObject")
private DefaultClassLoaderLease obtainLease() {
verifyIsNotReleased();
referenceCount += 1;
return DefaultClassLoaderLease.create(this);
}
private void release() {
synchronized (lockObject) {
if (isReleased) {
return;
}
if (referenceCount > 0) {
referenceCount -= 1;
}
if (referenceCount == 0) {
releaseClassLoader();
cacheEntries.remove(jobId);
}
}
}
@GuardedBy("lockObject")
private void releaseClassLoader() {
if (resolvedClassLoader != null) {
resolvedClassLoader.releaseClassLoader();
resolvedClassLoader = null;
}
isReleased = true;
}
@GuardedBy("lockObject")
private void verifyIsNotReleased() {
Preconditions.checkState(
!isReleased, "The LibraryCacheEntry has already been released.");
}
}
private static final class DefaultClassLoaderLease
implements LibraryCacheManager.ClassLoaderLease {
private final LibraryCacheEntry libraryCacheEntry;
private boolean isClosed;
private DefaultClassLoaderLease(LibraryCacheEntry libraryCacheEntry) {
this.libraryCacheEntry = libraryCacheEntry;
this.isClosed = false;
}
@Override
public UserCodeClassLoader getOrResolveClassLoader(
Collection<PermanentBlobKey> requiredJarFiles, Collection<URL> requiredClasspaths)
throws IOException {
verifyIsNotClosed();
return libraryCacheEntry.getOrResolveClassLoader(requiredJarFiles, requiredClasspaths);
}
private void verifyIsNotClosed() {
Preconditions.checkState(!isClosed, "The ClassLoaderHandler has already been closed.");
}
@Override
public void release() {
if (isClosed) {
return;
}
isClosed = true;
libraryCacheEntry.release();
}
private static DefaultClassLoaderLease create(LibraryCacheEntry libraryCacheEntry) {
return new DefaultClassLoaderLease(libraryCacheEntry);
}
}
private static final class ResolvedClassLoader implements UserCodeClassLoader {
private final URLClassLoader classLoader;
/**
* Set of BLOB keys used for a previous job/task registration.
*
* <p>The purpose of this is to make sure, future registrations do not differ in content as
* this is a contract of the {@link BlobLibraryCacheManager}.
*/
private final Set<PermanentBlobKey> libraries;
/**
* Set of class path URLs used for a previous job/task registration.
*
* <p>The purpose of this is to make sure, future registrations do not differ in content as
* this is a contract of the {@link BlobLibraryCacheManager}.
*/
private final Set<String> classPaths;
private final Map<String, Runnable> releaseHooks;
private ResolvedClassLoader(
URLClassLoader classLoader,
Collection<PermanentBlobKey> requiredLibraries,
Collection<URL> requiredClassPaths) {
this.classLoader = classLoader;
// NOTE: do not store the class paths, i.e. URLs, into a set for performance reasons
// see http://findbugs.sourceforge.net/bugDescriptions.html#DMI_COLLECTION_OF_URLS
// -> alternatively, compare their string representation
this.classPaths = new HashSet<>(requiredClassPaths.size());
for (URL url : requiredClassPaths) {
classPaths.add(url.toString());
}
this.libraries = new HashSet<>(requiredLibraries);
this.releaseHooks = new HashMap<>();
}
@Override
public ClassLoader asClassLoader() {
return classLoader;
}
@Override
public void registerReleaseHookIfAbsent(String releaseHookName, Runnable releaseHook) {
releaseHooks.putIfAbsent(releaseHookName, releaseHook);
}
private void verifyClassLoader(
Collection<PermanentBlobKey> requiredLibraries,
Collection<URL> requiredClassPaths) {
// Make sure the previous registration referred to the same libraries and class paths.
// NOTE: the original collections may contain duplicates and may not already be Set
// collections with fast checks whether an item is contained in it.
// lazy construction of a new set for faster comparisons
if (libraries.size() != requiredLibraries.size()
|| !new HashSet<>(requiredLibraries).containsAll(libraries)) {
throw new IllegalStateException(
"The library registration references a different set of library BLOBs than"
+ " previous registrations for this job:\nold:"
+ libraries
+ "\nnew:"
+ requiredLibraries);
}
// lazy construction of a new set with String representations of the URLs
if (classPaths.size() != requiredClassPaths.size()
|| !requiredClassPaths.stream()
.map(URL::toString)
.collect(Collectors.toSet())
.containsAll(classPaths)) {
throw new IllegalStateException(
"The library registration references a different set of library BLOBs than"
+ " previous registrations for this job:\nold:"
+ classPaths
+ "\nnew:"
+ requiredClassPaths);
}
}
/**
* Release the class loader to ensure any file descriptors are closed and the cached
* libraries are deleted immediately.
*/
private void releaseClassLoader() {
runReleaseHooks();
try {
classLoader.close();
} catch (IOException e) {
LOG.warn(
"Failed to release user code class loader for "
+ Arrays.toString(libraries.toArray()));
}
}
private void runReleaseHooks() {
Set<Map.Entry<String, Runnable>> hooks = releaseHooks.entrySet();
if (!hooks.isEmpty()) {
for (Map.Entry<String, Runnable> hookEntry : hooks) {
try {
LOG.debug("Running class loader shutdown hook: {}.", hookEntry.getKey());
hookEntry.getValue().run();
} catch (Throwable t) {
LOG.warn(
"Failed to run release hook '{}' for user code class loader.",
hookEntry.getValue(),
t);
}
}
releaseHooks.clear();
}
}
}
}