|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.flink.runtime.webmonitor; |
| 19 | + |
| 20 | +import org.apache.flink.api.common.JobID; |
| 21 | +import org.apache.flink.api.common.time.Time; |
| 22 | +import org.apache.flink.client.deployment.application.ApplicationRunner; |
| 23 | +import org.apache.flink.client.program.PackagedProgram; |
| 24 | +import org.apache.flink.configuration.Configuration; |
| 25 | +import org.apache.flink.runtime.concurrent.Executors; |
| 26 | +import org.apache.flink.runtime.dispatcher.DispatcherGateway; |
| 27 | +import org.apache.flink.runtime.rest.handler.HandlerRequest; |
| 28 | +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; |
| 29 | +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; |
| 30 | +import org.apache.flink.runtime.webmonitor.handlers.JarIdPathParameter; |
| 31 | +import org.apache.flink.runtime.webmonitor.handlers.JarRunHandler; |
| 32 | +import org.apache.flink.runtime.webmonitor.handlers.JarRunMessageParameters; |
| 33 | +import org.apache.flink.runtime.webmonitor.handlers.JarRunRequestBody; |
| 34 | +import org.apache.flink.runtime.webmonitor.handlers.JarUploadHandler; |
| 35 | + |
| 36 | +import org.junit.Rule; |
| 37 | +import org.junit.Test; |
| 38 | +import org.junit.rules.TemporaryFolder; |
| 39 | + |
| 40 | +import java.nio.file.Files; |
| 41 | +import java.nio.file.Path; |
| 42 | +import java.nio.file.Paths; |
| 43 | +import java.util.Collection; |
| 44 | +import java.util.Collections; |
| 45 | +import java.util.HashMap; |
| 46 | +import java.util.IdentityHashMap; |
| 47 | +import java.util.List; |
| 48 | +import java.util.Map; |
| 49 | +import java.util.Set; |
| 50 | +import java.util.concurrent.CompletableFuture; |
| 51 | +import java.util.concurrent.TimeUnit; |
| 52 | + |
| 53 | +import static org.junit.Assert.assertEquals; |
| 54 | + |
| 55 | +/** Tests for the {@link WebSubmissionExtension}. */ |
| 56 | +public class WebSubmissionExtensionTest { |
| 57 | + |
| 58 | + private static final String JAR_NAME = "output-test-program.jar"; |
| 59 | + |
| 60 | + @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); |
| 61 | + |
| 62 | + @Test |
| 63 | + public void applicationsRunInSeparateThreads() throws Exception { |
| 64 | + final Path tempDir = temporaryFolder.getRoot().toPath(); |
| 65 | + |
| 66 | + final Path uploadDir = Files.createDirectories(tempDir.resolve("uploadDir")); |
| 67 | + // create a copy because the upload handler moves uploaded jars (because it assumes it to be |
| 68 | + // a temporary file) |
| 69 | + final Path jarFile = |
| 70 | + Files.copy( |
| 71 | + Paths.get(System.getProperty("targetDir")).resolve(JAR_NAME), |
| 72 | + tempDir.resolve("app.jar")); |
| 73 | + |
| 74 | + final DispatcherGateway dispatcherGateway = new TestingDispatcherGateway.Builder().build(); |
| 75 | + |
| 76 | + final ThreadCapturingApplicationRunner threadCapturingApplicationRunner = |
| 77 | + new ThreadCapturingApplicationRunner(); |
| 78 | + |
| 79 | + final WebSubmissionExtension webSubmissionExtension = |
| 80 | + new WebSubmissionExtension( |
| 81 | + new Configuration(), |
| 82 | + () -> CompletableFuture.completedFuture(dispatcherGateway), |
| 83 | + Collections.emptyMap(), |
| 84 | + new CompletableFuture<>(), |
| 85 | + uploadDir, |
| 86 | + Executors.directExecutor(), |
| 87 | + Time.of(5, TimeUnit.SECONDS), |
| 88 | + () -> threadCapturingApplicationRunner); |
| 89 | + |
| 90 | + final String jarPath = uploadJar(webSubmissionExtension, jarFile, dispatcherGateway); |
| 91 | + final String jarId = Paths.get(jarPath).getFileName().toString(); |
| 92 | + |
| 93 | + final JarRunHandler jarRunHandler = webSubmissionExtension.getJarRunHandler(); |
| 94 | + |
| 95 | + final Map<String, String> pathParameters = new HashMap<>(); |
| 96 | + pathParameters.put(JarIdPathParameter.KEY, jarId); |
| 97 | + final HandlerRequest<JarRunRequestBody, JarRunMessageParameters> runRequest = |
| 98 | + new HandlerRequest( |
| 99 | + new JarRunRequestBody(), |
| 100 | + new JarRunMessageParameters(), |
| 101 | + pathParameters, |
| 102 | + Collections.emptyMap()); |
| 103 | + |
| 104 | + // run several applications in sequence, and verify that each thread is unique |
| 105 | + int numApplications = 20; |
| 106 | + for (int i = 0; i < numApplications; i++) { |
| 107 | + jarRunHandler.handleRequest(runRequest, dispatcherGateway).get(); |
| 108 | + } |
| 109 | + assertEquals(numApplications, threadCapturingApplicationRunner.getThreads().size()); |
| 110 | + } |
| 111 | + |
| 112 | + private static String uploadJar( |
| 113 | + WebSubmissionExtension extension, Path jarFile, DispatcherGateway dispatcherGateway) |
| 114 | + throws Exception { |
| 115 | + final JarUploadHandler jarUploadHandler = extension.getJarUploadHandler(); |
| 116 | + |
| 117 | + final HandlerRequest<EmptyRequestBody, EmptyMessageParameters> uploadRequest = |
| 118 | + new HandlerRequest<>( |
| 119 | + EmptyRequestBody.getInstance(), |
| 120 | + EmptyMessageParameters.getInstance(), |
| 121 | + Collections.emptyMap(), |
| 122 | + Collections.emptyMap(), |
| 123 | + Collections.singletonList(jarFile.toFile())); |
| 124 | + |
| 125 | + return jarUploadHandler.handleRequest(uploadRequest, dispatcherGateway).get().getFilename(); |
| 126 | + } |
| 127 | + |
| 128 | + private static class ThreadCapturingApplicationRunner implements ApplicationRunner { |
| 129 | + |
| 130 | + private final Set<Thread> threads = Collections.newSetFromMap(new IdentityHashMap<>()); |
| 131 | + |
| 132 | + @Override |
| 133 | + public List<JobID> run( |
| 134 | + DispatcherGateway dispatcherGateway, |
| 135 | + PackagedProgram program, |
| 136 | + Configuration configuration) { |
| 137 | + threads.add(Thread.currentThread()); |
| 138 | + return Collections.singletonList(new JobID()); |
| 139 | + } |
| 140 | + |
| 141 | + public Collection<Thread> getThreads() { |
| 142 | + return threads; |
| 143 | + } |
| 144 | + } |
| 145 | +} |
0 commit comments