[
https://issues.apache.org/jira/browse/FLINK-3478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15159560#comment-15159560
]
ASF GitHub Bot commented on FLINK-3478:
---------------------------------------
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/1697#discussion_r53841086
--- Diff:
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
---
@@ -196,16 +198,22 @@ public void channelRead0(ChannelHandlerContext ctx,
Routed routed) throws Except
* Response when running with leading JobManager.
*/
private void respondAsLeader(ChannelHandlerContext ctx, HttpRequest
request, String requestPath)
- throws IOException, ParseException {
+ throws IOException, ParseException, URISyntaxException {
// convert to absolute path
final File file = new File(rootPath, requestPath);
- if(!file.exists()) {
+ if (!file.exists()) {
// file does not exist. Try to load it with the
classloader
ClassLoader cl =
StaticFileServerHandler.class.getClassLoader();
+
try(InputStream resourceStream =
cl.getResourceAsStream("web" + requestPath)) {
- if (resourceStream == null) {
+ // Check that we don't load anything from
outside of the
+ // expected resource.
+ URI root = cl.getResource("web").toURI();
+ URI req = cl.getResource("web" +
requestPath).toURI();
--- End diff --
if the resources doesn't exist this will throw an NPE.
> Flink serves arbitary files through the web interface
> -----------------------------------------------------
>
> Key: FLINK-3478
> URL: https://issues.apache.org/jira/browse/FLINK-3478
> Project: Flink
> Issue Type: Bug
> Components: Webfrontend
> Affects Versions: 0.10.0, 1.0.0, 0.10.1
> Reporter: Maximilian Michels
> Assignee: Ufuk Celebi
> Priority: Blocker
> Fix For: 1.0.0, 0.10.3
>
>
> Flink serves arbitrary files through the web server of the 8081 port, e.g.
> {{../../../../../../../../../../etc/passwd}}.
> The requested path needs to be validated before it is served.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)