You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2015/10/21 16:53:56 UTC

flink git commit: [FLINK-2878] [webmonitor] Fix unexpected leader address pattern

Repository: flink
Updated Branches:
  refs/heads/master 30c46ed13 -> 3cad56d28


[FLINK-2878] [webmonitor] Fix unexpected leader address pattern

The HandlerRedirectUtils.getRedirectAddress decides whether the retrieved leader address is equal to the local job manager address. The local job manager address is, however, in the form akka.tcp://flink@url/user/jobmanager whereas the leader address can be akka://flink/user/jobmanager if the local job manager is the current leader. Such a case produced a warning which is not correct. This PR checks for the local job manager address and signals that no redirection has to be done if it receives akka://flink/user/jobmanager.

Add test for HandlerRedirectUtils

This closes #1280.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3cad56d2
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3cad56d2
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3cad56d2

Branch: refs/heads/master
Commit: 3cad56d28d55025281873f53a28ec27ce1027992
Parents: 30c46ed
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Oct 21 15:35:02 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Wed Oct 21 16:53:28 2015 +0200

----------------------------------------------------------------------
 .../handlers/HandlerRedirectUtils.java          |  7 ++-
 .../handlers/HandlerRedirectUtilsTest.java      | 54 ++++++++++++++++++++
 .../runtime/instance/DummyActorGateway.java     | 13 ++++-
 3 files changed, 72 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3cad56d2/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
index 800c7c0..6c243ab 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
@@ -25,9 +25,11 @@ import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.HttpVersion;
 import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.webmonitor.files.MimeTypes;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.Option;
 import scala.Tuple2;
 
 import java.io.UnsupportedEncodingException;
@@ -58,7 +60,10 @@ public class HandlerRedirectUtils {
 		final String leaderAddress = leader._1().path();
 		final int webMonitorPort = leader._2();
 
-		if (!localJobManagerAddress.equals(leaderAddress)) {
+		final String jobManagerName = localJobManagerAddress.substring(localJobManagerAddress.lastIndexOf("/") + 1);
+
+		if (!localJobManagerAddress.equals(leaderAddress) &&
+			!leaderAddress.equals(JobManager.getLocalJobManagerAkkaURL(Option.apply(jobManagerName)))) {
 			// We are not the leader and need to redirect
 			Matcher matcher = LeaderAddressHostPattern.matcher(leaderAddress);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3cad56d2/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java
new file mode 100644
index 0000000..0f6ea96
--- /dev/null
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtilsTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.webmonitor.handlers;
+
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.DummyActorGateway;
+import org.junit.Test;
+import org.junit.Assert;
+import scala.Tuple2;
+
+public class HandlerRedirectUtilsTest {
+
+	static final String localJobManagerAddress = "akka.tcp://flink@127.0.0.1:1234/user/foobar";
+	static final String remoteURL = "127.0.0.2:1235";
+	static final String remotePath = "akka.tcp://flink@" + remoteURL + "/user/jobmanager";
+
+	@Test
+	public void testGetRedirectAddressWithLocalAkkaPath() throws Exception {
+		ActorGateway leaderGateway = new DummyActorGateway("akka://flink/user/foobar");
+
+		Tuple2<ActorGateway, Integer> leader = new Tuple2<>(leaderGateway, 1235);
+
+		String redirectingAddress =HandlerRedirectUtils.getRedirectAddress(localJobManagerAddress, leader);
+
+		Assert.assertNull(redirectingAddress);
+	}
+
+	@Test
+	public void testGetRedirectAddressWithRemoteAkkaPath() throws Exception {
+		ActorGateway leaderGateway = new DummyActorGateway(remotePath);
+
+		Tuple2<ActorGateway, Integer> leader = new Tuple2<>(leaderGateway, 1235);
+
+		String redirectingAddress =HandlerRedirectUtils.getRedirectAddress(localJobManagerAddress, leader);
+
+		Assert.assertEquals(remoteURL, redirectingAddress);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3cad56d2/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
index 3d27611..68e2aea 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
@@ -30,6 +30,17 @@ import java.util.UUID;
  */
 public class DummyActorGateway implements ActorGateway {
 	public static final DummyActorGateway INSTANCE = new DummyActorGateway();
+	private static final long serialVersionUID = -833861606769367952L;
+
+	private final String path;
+
+	public DummyActorGateway() {
+		this("DummyActorGateway");
+	}
+
+	public DummyActorGateway(String path) {
+		this.path = path;
+	}
 
 	@Override
 	public Future<Object> ask(Object message, FiniteDuration timeout) {
@@ -52,7 +63,7 @@ public class DummyActorGateway implements ActorGateway {
 
 	@Override
 	public String path() {
-		return "DummyInstanceGateway";
+		return path;
 	}
 
 	@Override