Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-16303][rest] Enable retrieval of custom JobManager log files
- Loading branch information
Showing
31 changed files
with
927 additions
and
160 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
85 changes: 85 additions & 0 deletions
85
...ain/java/org/apache/flink/runtime/rest/handler/cluster/AbstractJobManagerFileHandler.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* 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.rest.handler.cluster; | ||
|
||
import org.apache.flink.api.common.time.Time; | ||
import org.apache.flink.runtime.rest.handler.AbstractHandler; | ||
import org.apache.flink.runtime.rest.handler.HandlerRequest; | ||
import org.apache.flink.runtime.rest.handler.RestHandlerException; | ||
import org.apache.flink.runtime.rest.handler.util.HandlerUtils; | ||
import org.apache.flink.runtime.rest.messages.EmptyRequestBody; | ||
import org.apache.flink.runtime.rest.messages.MessageParameters; | ||
import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; | ||
import org.apache.flink.runtime.webmonitor.RestfulGateway; | ||
import org.apache.flink.runtime.webmonitor.WebMonitorUtils; | ||
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; | ||
import org.apache.flink.util.FlinkException; | ||
import org.apache.flink.util.Preconditions; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; | ||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest; | ||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
import java.io.File; | ||
import java.util.Map; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.CompletionException; | ||
|
||
/** | ||
* Base class for serving files from the JobManager. | ||
*/ | ||
public abstract class AbstractJobManagerFileHandler<M extends MessageParameters> extends AbstractHandler<RestfulGateway, EmptyRequestBody, M> { | ||
|
||
protected final WebMonitorUtils.LogFileLocation logFileLocation; | ||
|
||
protected AbstractJobManagerFileHandler( | ||
GatewayRetriever<? extends RestfulGateway> leaderRetriever, | ||
Time timeout, | ||
Map<String, String> responseHeaders, | ||
UntypedResponseMessageHeaders<EmptyRequestBody, M> messageHeaders, | ||
WebMonitorUtils.LogFileLocation logFileLocation) { | ||
super(leaderRetriever, timeout, responseHeaders, messageHeaders); | ||
|
||
Preconditions.checkNotNull(logFileLocation); | ||
this.logFileLocation = logFileLocation; | ||
} | ||
|
||
@Override | ||
protected CompletableFuture<Void> respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest<EmptyRequestBody, M> handlerRequest, RestfulGateway gateway) { | ||
File file = getFile(handlerRequest); | ||
if (file != null && file.exists()) { | ||
return CompletableFuture.completedFuture(file).thenAcceptAsync(logFile -> { | ||
try { | ||
HandlerUtils.transferFile( | ||
ctx, | ||
logFile, | ||
httpRequest); | ||
} catch (FlinkException e) { | ||
throw new CompletionException(new FlinkException("Could not transfer file to client.", e)); | ||
} | ||
}); | ||
} else { | ||
throw new CompletionException(new RestHandlerException( | ||
"This file is not exist in JobManager log dir.", | ||
HttpResponseStatus.NOT_FOUND)); | ||
} | ||
} | ||
|
||
protected abstract File getFile(HandlerRequest<EmptyRequestBody, M> handlerRequest); | ||
} |
61 changes: 61 additions & 0 deletions
61
...c/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerCustomLogHandler.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,61 @@ | ||
/* | ||
* 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.rest.handler.cluster; | ||
|
||
import org.apache.flink.api.common.time.Time; | ||
import org.apache.flink.runtime.rest.handler.HandlerRequest; | ||
import org.apache.flink.runtime.rest.handler.RestHandlerException; | ||
import org.apache.flink.runtime.rest.messages.EmptyRequestBody; | ||
import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; | ||
import org.apache.flink.runtime.rest.messages.cluster.FileMessageParameters; | ||
import org.apache.flink.runtime.rest.messages.taskmanager.LogFileNamePathParameter; | ||
import org.apache.flink.runtime.webmonitor.RestfulGateway; | ||
import org.apache.flink.runtime.webmonitor.WebMonitorUtils; | ||
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; | ||
import org.apache.flink.util.StringUtils; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
import java.io.File; | ||
import java.util.Map; | ||
import java.util.concurrent.CompletionException; | ||
|
||
/** | ||
* Rest handler which serves the custom log file from JobManager. | ||
*/ | ||
public class JobManagerCustomLogHandler extends AbstractJobManagerFileHandler<FileMessageParameters> { | ||
|
||
public JobManagerCustomLogHandler( | ||
GatewayRetriever<? extends RestfulGateway> leaderRetriever, | ||
Time timeout, Map<String, String> responseHeaders, | ||
UntypedResponseMessageHeaders<EmptyRequestBody, FileMessageParameters> messageHeaders, | ||
WebMonitorUtils.LogFileLocation logFileLocation) { | ||
super(leaderRetriever, timeout, responseHeaders, messageHeaders, logFileLocation); | ||
} | ||
|
||
@Override | ||
protected File getFile(HandlerRequest<EmptyRequestBody, FileMessageParameters> handlerRequest) { | ||
final String logDir = logFileLocation.logFile.getParent(); | ||
if (logFileLocation.logFile == null || StringUtils.isNullOrWhitespaceOnly(logFileLocation.logFile.getParent())) { | ||
throw new CompletionException(new RestHandlerException("Can not get JobManager log dir.", HttpResponseStatus.NOT_FOUND)); | ||
} | ||
String filename = handlerRequest.getPathParameter(LogFileNamePathParameter.class); | ||
return new File(logDir, filename); | ||
} | ||
} |
50 changes: 50 additions & 0 deletions
50
...src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerLogFileHandler.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 | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
/* | ||
* 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.rest.handler.cluster; | ||
|
||
import org.apache.flink.api.common.time.Time; | ||
import org.apache.flink.runtime.rest.handler.HandlerRequest; | ||
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; | ||
import org.apache.flink.runtime.rest.messages.EmptyRequestBody; | ||
import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; | ||
import org.apache.flink.runtime.webmonitor.RestfulGateway; | ||
import org.apache.flink.runtime.webmonitor.WebMonitorUtils; | ||
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; | ||
|
||
import java.io.File; | ||
import java.util.Map; | ||
|
||
/** | ||
* Rest handler which serves the log files from JobManager. | ||
*/ | ||
public class JobManagerLogFileHandler extends AbstractJobManagerFileHandler<EmptyMessageParameters> { | ||
|
||
public JobManagerLogFileHandler( | ||
GatewayRetriever<? extends RestfulGateway> leaderRetriever, | ||
Time timeout, Map<String, String> responseHeaders, | ||
UntypedResponseMessageHeaders<EmptyRequestBody, EmptyMessageParameters> messageHeaders, | ||
WebMonitorUtils.LogFileLocation logFileLocation) { | ||
super(leaderRetriever, timeout, responseHeaders, messageHeaders, logFileLocation); | ||
} | ||
|
||
@Override | ||
protected File getFile(HandlerRequest<EmptyRequestBody, EmptyMessageParameters> handlerRequest) { | ||
return this.logFileLocation.logFile; | ||
} | ||
} |
Oops, something went wrong.