Skip to content

Commit

Permalink
[FLINK-21108][web] Add custom netty HTTP request inbound/outbound han…
Browse files Browse the repository at this point in the history
…dlers

Closes apache#16463
  • Loading branch information
gaborgsomogyi authored and Gyula Fora committed Aug 6, 2021
1 parent 543959e commit 646ff2d
Show file tree
Hide file tree
Showing 40 changed files with 847 additions and 159 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -207,10 +207,7 @@ private RestClusterClient(
if (restClient != null) {
this.restClient = restClient;
} else {
this.restClient =
new RestClient(
restClusterClientConfiguration.getRestClientConfiguration(),
executorService);
this.restClient = new RestClient(configuration, executorService);
}

this.waitStrategy = checkNotNull(waitStrategy);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,7 @@
import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
import org.apache.flink.runtime.rest.RestClient;
import org.apache.flink.runtime.rest.RestClientConfiguration;
import org.apache.flink.runtime.rest.RestServerEndpoint;
import org.apache.flink.runtime.rest.RestServerEndpointConfiguration;
import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerException;
Expand Down Expand Up @@ -94,8 +92,6 @@ public class RestClusterClientSavepointTriggerTest extends TestLogger {
private static final GatewayRetriever<DispatcherGateway> mockGatewayRetriever =
() -> CompletableFuture.completedFuture(mockRestfulGateway);

private static RestServerEndpointConfiguration restServerEndpointConfiguration;

private static ExecutorService executor;

private static final Configuration REST_CONFIG;
Expand All @@ -112,8 +108,6 @@ public class RestClusterClientSavepointTriggerTest extends TestLogger {

@BeforeClass
public static void setUp() throws ConfigurationException {
restServerEndpointConfiguration =
RestServerEndpointConfiguration.fromConfiguration(REST_CONFIG);
executor =
Executors.newSingleThreadExecutor(
new ExecutorThreadFactory(
Expand Down Expand Up @@ -273,7 +267,7 @@ private static RestServerEndpoint createRestServerEndpoint(
final FunctionWithException<TriggerId, SavepointInfo, RestHandlerException>
savepointHandlerLogic)
throws Exception {
return TestRestServerEndpoint.builder(restServerEndpointConfiguration)
return TestRestServerEndpoint.builder(REST_CONFIG)
.withHandler(new TestSavepointTriggerHandler(triggerHandlerLogic))
.withHandler(new TestSavepointHandler(savepointHandlerLogic))
.buildAndStart();
Expand Down Expand Up @@ -355,7 +349,7 @@ private RestClusterClient<StandaloneClusterId> createRestClusterClient(final int
clientConfig.setInteger(RestOptions.PORT, port);
return new RestClusterClient<>(
clientConfig,
new RestClient(RestClientConfiguration.fromConfiguration(REST_CONFIG), executor),
new RestClient(REST_CONFIG, executor),
StandaloneClusterId.getInstance(),
(attempt) -> 0);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,8 +45,6 @@
import org.apache.flink.runtime.rest.FileUpload;
import org.apache.flink.runtime.rest.HttpMethodWrapper;
import org.apache.flink.runtime.rest.RestClient;
import org.apache.flink.runtime.rest.RestClientConfiguration;
import org.apache.flink.runtime.rest.RestServerEndpointConfiguration;
import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerException;
Expand Down Expand Up @@ -154,8 +152,6 @@ public class RestClusterClientTest extends TestLogger {

private GatewayRetriever<DispatcherGateway> mockGatewayRetriever;

private RestServerEndpointConfiguration restServerEndpointConfiguration;

private volatile FailHttpRequestPredicate failHttpRequest = FailHttpRequestPredicate.never();

private ExecutorService executor;
Expand All @@ -177,8 +173,6 @@ public class RestClusterClientTest extends TestLogger {

@Before
public void setUp() throws Exception {
restServerEndpointConfiguration =
RestServerEndpointConfiguration.fromConfiguration(restConfig);
mockGatewayRetriever = () -> CompletableFuture.completedFuture(mockRestfulGateway);

executor =
Expand Down Expand Up @@ -209,7 +203,7 @@ private RestClusterClient<StandaloneClusterId> createRestClusterClient(final int

@Nonnull
private RestClient createRestClient() throws ConfigurationException {
return new RestClient(RestClientConfiguration.fromConfiguration(restConfig), executor) {
return new RestClient(restConfig, executor) {
@Override
public <
M extends MessageHeaders<R, P, U>,
Expand Down Expand Up @@ -1048,8 +1042,7 @@ private TestHandler(MessageHeaders<R, P, M> headers) {

private TestRestServerEndpoint createRestServerEndpoint(
final AbstractRestHandler<?, ?, ?, ?>... abstractRestHandlers) throws Exception {
TestRestServerEndpoint.Builder builder =
TestRestServerEndpoint.builder(restServerEndpointConfiguration);
TestRestServerEndpoint.Builder builder = TestRestServerEndpoint.builder(restConfig);
Arrays.stream(abstractRestHandlers).forEach(builder::withHandler);

return builder.buildAndStart();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
import org.apache.flink.runtime.rest.util.DocumentingRestEndpoint;
import org.apache.flink.runtime.rest.versioning.RestAPIVersion;
import org.apache.flink.util.ConfigurationException;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.SerializableString;
Expand Down Expand Up @@ -99,7 +100,7 @@ public class RestAPIDocGenerator {
* @param args args[0] contains the directory into which the generated files are placed
* @throws IOException if any file operation failed
*/
public static void main(String[] args) throws IOException {
public static void main(String[] args) throws IOException, ConfigurationException {
String outputDirectory = args[0];

for (final RestAPIVersion apiVersion : RestAPIVersion.values()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.rest.RestClient;
import org.apache.flink.runtime.rest.RestClientConfiguration;
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders;
Expand Down Expand Up @@ -133,9 +132,7 @@ public ClusterController startCluster(int numTaskManagers) throws IOException {
distribution.startFlinkCluster();

try (final RestClient restClient =
new RestClient(
RestClientConfiguration.fromConfiguration(new Configuration()),
Executors.directExecutor())) {
new RestClient(new Configuration(), Executors.directExecutor())) {
for (int retryAttempt = 0; retryAttempt < 30; retryAttempt++) {
final CompletableFuture<TaskManagersInfo> localhost =
restClient.sendRequest(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.rest.RestClient;
import org.apache.flink.runtime.rest.RestClientConfiguration;
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.job.metrics.JobManagerMetricsHeaders;
Expand Down Expand Up @@ -94,9 +93,7 @@ public static void shutdownExecutor() {
public void testReporter() throws Exception {
try (ClusterController ignored = dist.startCluster(1)) {
final RestClient restClient =
new RestClient(
RestClientConfiguration.fromConfiguration(new Configuration()),
scheduledExecutorService);
new RestClient(new Configuration(), scheduledExecutorService);

checkJobManagerMetricAvailability(restClient);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,18 +18,22 @@

package org.apache.flink.runtime.webmonitor.utils;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.runtime.io.network.netty.InboundChannelHandlerFactory;
import org.apache.flink.runtime.io.network.netty.SSLHandlerFactory;
import org.apache.flink.runtime.rest.handler.router.Router;
import org.apache.flink.runtime.rest.handler.router.RouterHandler;
import org.apache.flink.runtime.webmonitor.HttpRequestHandler;
import org.apache.flink.runtime.webmonitor.PipelineErrorHandler;
import org.apache.flink.util.ConfigurationException;
import org.apache.flink.util.Preconditions;

import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
Expand All @@ -45,7 +49,14 @@
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.ServiceLoader;

/** This classes encapsulates the boot-strapping of netty for the web-frontend. */
public class WebFrontendBootstrap {
Expand All @@ -55,6 +66,8 @@ public class WebFrontendBootstrap {
private final ServerBootstrap bootstrap;
private final Channel serverChannel;
private final String restAddress;
private final Map<String, String> responseHeaders;
@VisibleForTesting List<InboundChannelHandlerFactory> inboundChannelHandlerFactories;

public WebFrontendBootstrap(
Router router,
Expand All @@ -69,15 +82,34 @@ public WebFrontendBootstrap(
this.router = Preconditions.checkNotNull(router);
this.log = Preconditions.checkNotNull(log);
this.uploadDir = directory;
this.responseHeaders = new HashMap<>();
inboundChannelHandlerFactories = new ArrayList<>();
ServiceLoader<InboundChannelHandlerFactory> loader =
ServiceLoader.load(InboundChannelHandlerFactory.class);
final Iterator<InboundChannelHandlerFactory> factories = loader.iterator();
while (factories.hasNext()) {
try {
final InboundChannelHandlerFactory factory = factories.next();
if (factory != null) {
inboundChannelHandlerFactories.add(factory);
log.info("Loaded channel inbound factory: {}", factory);
}
} catch (Throwable e) {
log.error("Could not load channel inbound factory.", e);
throw e;
}
}
inboundChannelHandlerFactories.sort(
Comparator.comparingInt(InboundChannelHandlerFactory::priority).reversed());

ChannelInitializer<SocketChannel> initializer =
new ChannelInitializer<SocketChannel>() {

@Override
protected void initChannel(SocketChannel ch) {
protected void initChannel(SocketChannel ch) throws ConfigurationException {
RouterHandler handler =
new RouterHandler(
WebFrontendBootstrap.this.router, new HashMap<>());
WebFrontendBootstrap.this.router, responseHeaders);

// SSL should be the first handler in the pipeline
if (serverSSLFactory != null) {
Expand All @@ -87,8 +119,18 @@ protected void initChannel(SocketChannel ch) {
serverSSLFactory.createNettySSLHandler(ch.alloc()));
}

ch.pipeline().addLast(new HttpServerCodec());

for (InboundChannelHandlerFactory factory :
inboundChannelHandlerFactories) {
Optional<ChannelHandler> channelHandler =
factory.createHandler(config, responseHeaders);
if (channelHandler.isPresent()) {
ch.pipeline().addLast(channelHandler.get());
}
}

ch.pipeline()
.addLast(new HttpServerCodec())
.addLast(new ChunkedWriteHandler())
.addLast(new HttpRequestHandler(uploadDir))
.addLast(handler.getName(), handler)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.runtime.rest.util.DocumentingDispatcherRestEndpoint;
import org.apache.flink.runtime.rest.util.DocumentingRestEndpoint;
import org.apache.flink.runtime.rest.versioning.RestAPIVersion;
import org.apache.flink.util.ConfigurationException;
import org.apache.flink.util.TestLogger;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
Expand Down Expand Up @@ -70,7 +71,7 @@ public RestAPIStabilityTest(final RestAPIVersion apiVersion) {
}

@Test
public void testDispatcherRestAPIStability() throws IOException {
public void testDispatcherRestAPIStability() throws IOException, ConfigurationException {
final String versionedSnapshotFileName =
String.format(SNAPSHOT_RESOURCE_PATTERN, apiVersion.getURLVersionPrefix());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -424,7 +424,7 @@ private static void runJob() throws Exception {
env.execute();
}

static Tuple2<Integer, String> getFromHTTP(String url) throws Exception {
public static Tuple2<Integer, String> getFromHTTP(String url) throws Exception {
URL u = new URL(url);
HttpURLConnection connection = (HttpURLConnection) u.openConnection();
connection.setConnectTimeout(100000);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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:https://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.webmonitor.utils;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.network.netty.Prio0InboundChannelHandlerFactory;
import org.apache.flink.runtime.io.network.netty.Prio1InboundChannelHandlerFactory;
import org.apache.flink.runtime.rest.handler.router.Router;
import org.apache.flink.runtime.webmonitor.history.HistoryServerStaticFileServerHandler;
import org.apache.flink.runtime.webmonitor.history.HistoryServerTest;

import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.slf4j.LoggerFactory;

import java.io.File;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;

/** Tests for the WebFrontendBootstrap. */
public class WebFrontendBootstrapTest {

@Rule public TemporaryFolder tmp = new TemporaryFolder();

@Test
public void testHandlersMustBeLoaded() throws Exception {
File webDir = tmp.newFolder("webDir");
Configuration configuration = new Configuration();
configuration.setString(
Prio0InboundChannelHandlerFactory.REDIRECT_FROM_URL, "/nonExisting");
configuration.setString(Prio0InboundChannelHandlerFactory.REDIRECT_TO_URL, "/index.html");
Router router =
new Router().addGet("/:*", new HistoryServerStaticFileServerHandler(webDir));
WebFrontendBootstrap webUI =
new WebFrontendBootstrap(
router,
LoggerFactory.getLogger(WebFrontendBootstrapTest.class),
tmp.newFolder("uploadDir"),
null,
"localhost",
0,
configuration);

assertEquals(webUI.inboundChannelHandlerFactories.size(), 2);
assertTrue(
webUI.inboundChannelHandlerFactories.get(0)
instanceof Prio1InboundChannelHandlerFactory);
assertTrue(
webUI.inboundChannelHandlerFactories.get(1)
instanceof Prio0InboundChannelHandlerFactory);

int port = webUI.getServerPort();
try {
Tuple2<Integer, String> index =
HistoryServerTest.getFromHTTP("http:https://localhost:" + port + "/index.html");
Assert.assertEquals(index.f0.intValue(), 200);
Assert.assertTrue(index.f1.contains("Apache Flink Web Dashboard"));

Tuple2<Integer, String> index2 =
HistoryServerTest.getFromHTTP("http:https://localhost:" + port + "/nonExisting");
Assert.assertEquals(index2.f0.intValue(), 200);
Assert.assertEquals(index, index2);
} finally {
webUI.shutdown();
}
}
}
Loading

0 comments on commit 646ff2d

Please sign in to comment.