Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-2358] [dashboard] First part dashboard server backend
- Adds a separate Maven project for easier maintenance. Also allows users to refer to runtime without web libraries. - Simple HTTP server based on Netty HTTP (slim dependency, since we use netty anyways) - REST URL parsing via Netty Router - Abstract stubs for handlers that deal with errors and request/response - First set of URL request handlers that produce JSON responses This closes #677 This closes #623 This closes #297
- Loading branch information
1 parent
75d1639
commit 44ee1c1
Showing
48 changed files
with
3,201 additions
and
136 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,108 @@ | |||
<?xml version="1.0" encoding="UTF-8"?> | |||
<!-- | |||
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. | |||
--> | |||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" | |||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> | |||
|
|||
<modelVersion>4.0.0</modelVersion> | |||
|
|||
<parent> | |||
<groupId>org.apache.flink</groupId> | |||
<artifactId>flink-parent</artifactId> | |||
<version>0.10-SNAPSHOT</version> | |||
<relativePath>..</relativePath> | |||
</parent> | |||
|
|||
<artifactId>flink-runtime-web</artifactId> | |||
<name>flink-runtime-web</name> | |||
|
|||
<packaging>jar</packaging> | |||
|
|||
<dependencies> | |||
|
|||
<!-- =================================================== | |||
Flink Dependencies | |||
=================================================== --> | |||
|
|||
<dependency> | |||
<groupId>org.apache.flink</groupId> | |||
<artifactId>flink-runtime</artifactId> | |||
<version>${project.version}</version> | |||
</dependency> | |||
|
|||
<dependency> | |||
<groupId>org.apache.flink</groupId> | |||
<artifactId>flink-test-utils</artifactId> | |||
<version>${project.version}</version> | |||
</dependency> | |||
|
|||
<dependency> | |||
<groupId>org.apache.flink</groupId> | |||
<artifactId>flink-java-examples</artifactId> | |||
<version>${project.version}</version> | |||
</dependency> | |||
|
|||
<!-- =================================================== | |||
Dependencies for the Web Server | |||
=================================================== --> | |||
|
|||
<dependency> | |||
<groupId>tv.cntt</groupId> | |||
<artifactId>netty-router</artifactId> | |||
<version>1.10</version> | |||
</dependency> | |||
|
|||
<dependency> | |||
<groupId>org.javassist</groupId> | |||
<artifactId>javassist</artifactId> | |||
<version>3.18.2-GA</version> | |||
</dependency> | |||
|
|||
<!-- =================================================== | |||
Dependencies for Actor Calls | |||
=================================================== --> | |||
|
|||
<dependency> | |||
<groupId>org.scala-lang</groupId> | |||
<artifactId>scala-library</artifactId> | |||
</dependency> | |||
|
|||
<dependency> | |||
<groupId>com.typesafe.akka</groupId> | |||
<artifactId>akka-actor_${scala.binary.version}</artifactId> | |||
</dependency> | |||
|
|||
<dependency> | |||
<groupId>com.typesafe.akka</groupId> | |||
<artifactId>akka-remote_${scala.binary.version}</artifactId> | |||
</dependency> | |||
|
|||
<!-- =================================================== | |||
Utilities | |||
=================================================== --> | |||
|
|||
<dependency> | |||
<groupId>com.google.guava</groupId> | |||
<artifactId>guava</artifactId> | |||
<version>${guava.version}</version> | |||
</dependency> | |||
|
|||
</dependencies> | |||
|
|||
</project> |
91 changes: 91 additions & 0 deletions
91
...k-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,91 @@ | |||
/* | |||
* 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; | |||
|
|||
import akka.actor.ActorRef; | |||
import akka.pattern.Patterns; | |||
import akka.util.Timeout; | |||
|
|||
import org.apache.flink.api.common.JobID; | |||
import org.apache.flink.runtime.executiongraph.ExecutionGraph; | |||
import org.apache.flink.runtime.messages.JobManagerMessages; | |||
|
|||
import scala.concurrent.Await; | |||
import scala.concurrent.Future; | |||
import scala.concurrent.duration.FiniteDuration; | |||
|
|||
import java.util.WeakHashMap; | |||
|
|||
/** | |||
* Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archiver. | |||
* <p> | |||
* The holder will cache the ExecutionGraph behind a weak reference, which will be cleared | |||
* at some point once no one else is pointing to the ExecutionGraph. | |||
* Note that while the holder runs in the same JVM as the JobManager or Archive, the reference should | |||
* stay valid. | |||
*/ | |||
public class ExecutionGraphHolder { | |||
|
|||
private final ActorRef source; | |||
|
|||
private final FiniteDuration timeout; | |||
|
|||
private final WeakHashMap<JobID, ExecutionGraph> cache = new WeakHashMap<JobID, ExecutionGraph>(); | |||
|
|||
|
|||
public ExecutionGraphHolder(ActorRef source) { | |||
this(source, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT); | |||
} | |||
|
|||
public ExecutionGraphHolder(ActorRef source, FiniteDuration timeout) { | |||
if (source == null || timeout == null) { | |||
throw new NullPointerException(); | |||
} | |||
this.source = source; | |||
this.timeout = timeout; | |||
} | |||
|
|||
|
|||
public ExecutionGraph getExecutionGraph(JobID jid) { | |||
ExecutionGraph cached = cache.get(jid); | |||
if (cached != null) { | |||
return cached; | |||
} | |||
|
|||
try { | |||
Timeout to = new Timeout(timeout); | |||
Future<Object> future = Patterns.ask(source, new JobManagerMessages.RequestJob(jid), to); | |||
Object result = Await.result(future, timeout); | |||
if (result instanceof JobManagerMessages.JobNotFound) { | |||
return null; | |||
} | |||
else if (result instanceof JobManagerMessages.JobFound) { | |||
ExecutionGraph eg = ((JobManagerMessages.JobFound) result).executionGraph(); | |||
cache.put(jid, eg); | |||
return eg; | |||
} | |||
else { | |||
throw new RuntimeException("Unknown response from JobManager / Archive: " + result); | |||
} | |||
} | |||
catch (Exception e) { | |||
throw new RuntimeException("Error requesting execution graph", e); | |||
} | |||
} | |||
} |
Oops, something went wrong.