Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.flink.runtime.blob.BlobClient;
import org.apache.flink.runtime.client.ClientUtils;
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.util.concurrent.ExecutorThreadFactory;
import org.apache.flink.util.function.FunctionUtils;
Expand Down Expand Up @@ -184,7 +185,7 @@ private CompletableFuture<JobClient> submitAndGetJobClientFuture(
});
}

private static CompletableFuture<JobID> submitJob(
private CompletableFuture<JobID> submitJob(
final Configuration configuration,
final DispatcherGateway dispatcherGateway,
final StreamGraph streamGraph,
Expand All @@ -211,8 +212,15 @@ private static CompletableFuture<JobID> submitJob(
throw new CompletionException(e);
}

return dispatcherGateway.submitJob(streamGraph, rpcTimeout);
return internalSubmit(dispatcherGateway, streamGraph, rpcTimeout);
})
.thenApply(ack -> streamGraph.getJobID());
}

CompletableFuture<Acknowledge> internalSubmit(
final DispatcherGateway dispatcherGateway,
final StreamGraph streamGraph,
final Duration rpcTimeout) {
return dispatcherGateway.submitJob(streamGraph, rpcTimeout);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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.client.deployment.application.executors;

import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.PipelineExecutor;
import org.apache.flink.runtime.application.SingleJobApplication;
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.streaming.api.graph.StreamGraph;

import java.time.Duration;
import java.util.Collection;
import java.util.concurrent.CompletableFuture;

/** The {@link PipelineExecutor} to be used when executing a job from web submission. */
public class WebSubmissionExecutor extends EmbeddedExecutor {

public WebSubmissionExecutor(
Collection<JobID> submittedJobIds,
DispatcherGateway dispatcherGateway,
Configuration configuration,
EmbeddedJobClientCreator jobClientCreator) {
super(submittedJobIds, dispatcherGateway, configuration, jobClientCreator);
}

@Override
CompletableFuture<Acknowledge> internalSubmit(
final DispatcherGateway dispatcherGateway,
final StreamGraph streamGraph,
final Duration rpcTimeout) {
SingleJobApplication application = new SingleJobApplication(streamGraph);
return dispatcherGateway.submitApplication(application, rpcTimeout);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public WebSubmissionExecutorFactory(

@Override
public String getName() {
return EmbeddedExecutor.NAME;
return WebSubmissionExecutor.NAME;
}

@Override
Expand All @@ -72,7 +72,7 @@ public boolean isCompatibleWith(final Configuration configuration) {
public PipelineExecutor getExecutor(final Configuration configuration) {
checkNotNull(configuration);

return new EmbeddedExecutor(
return new WebSubmissionExecutor(
submittedJobIds,
dispatcherGateway,
configuration,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,6 @@ public PipelineExecutorFactory getExecutorFactory(final Configuration configurat

@Override
public Stream<String> getExecutorNames() {
return Stream.<String>builder().add(EmbeddedExecutor.NAME).build();
return Stream.<String>builder().add(WebSubmissionExecutor.NAME).build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.core.testutils.AllCallbackWrapper;
import org.apache.flink.runtime.application.SingleJobApplication;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.HandlerRequestException;
Expand All @@ -32,6 +33,8 @@
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.runtime.webmonitor.testutils.ParameterProgram;
import org.apache.flink.streaming.api.graph.ExecutionPlan;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.concurrent.FutureUtils;

import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -99,10 +102,17 @@ static void init(File tmpDir) throws Exception {
TestingDispatcherGateway.newBuilder()
.setBlobServerPort(
blobServerExtension.getCustomExtension().getBlobServerPort())
.setSubmitFunction(
jobGraph -> {
LAST_SUBMITTED_EXECUTION_PLAN_REFERENCE.set(jobGraph);
return CompletableFuture.completedFuture(Acknowledge.get());
.setSubmitApplicationFunction(
application -> {
if (application instanceof SingleJobApplication) {
LAST_SUBMITTED_EXECUTION_PLAN_REFERENCE.set(
((SingleJobApplication) application)
.getExecutionPlan());
return CompletableFuture.completedFuture(Acknowledge.get());
}
return FutureUtils.completedExceptionally(
new FlinkRuntimeException(
"Unsupported application type"));
})
.build();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,8 +62,8 @@ void testJarSubmission(@TempDir File uploadDir, @TempDir File temporaryFolder)
TestingDispatcherGateway.newBuilder()
.setBlobServerPort(
blobServerExtension.getCustomExtension().getBlobServerPort())
.setSubmitFunction(
jobGraph -> CompletableFuture.completedFuture(Acknowledge.get()))
.setSubmitApplicationFunction(
application -> CompletableFuture.completedFuture(Acknowledge.get()))
.build();

final JarHandlers handlers =
Expand Down
162 changes: 162 additions & 0 deletions flink-runtime-web/src/test/resources/rest_api_v1.snapshot
Original file line number Diff line number Diff line change
@@ -1,5 +1,167 @@
{
"calls" : [ {
"url" : "/applications/overview",
"method" : "GET",
"status-code" : "200 OK",
"file-upload" : false,
"path-parameters" : {
"pathParameters" : [ ]
},
"query-parameters" : {
"queryParameters" : [ ]
},
"request" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyRequestBody"
},
"response" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:MultipleApplicationsDetails",
"properties" : {
"applications" : {
"type" : "array",
"items" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:ApplicationDetails",
"properties" : {
"id" : {
"type" : "any"
},
"name" : {
"type" : "string"
},
"start-time" : {
"type" : "integer"
},
"end-time" : {
"type" : "integer"
},
"duration" : {
"type" : "integer"
},
"status" : {
"type" : "string"
},
"jobs" : {
"type" : "object",
"additionalProperties" : {
"type" : "integer"
}
}
}
}
}
}
}
}, {
"url" : "/applications/:applicationid",
"method" : "GET",
"status-code" : "200 OK",
"file-upload" : false,
"path-parameters" : {
"pathParameters" : [ {
"key" : "applicationid"
} ]
},
"query-parameters" : {
"queryParameters" : [ ]
},
"request" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyRequestBody"
},
"response" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:ApplicationDetailsInfo",
"properties" : {
"id" : {
"type" : "any"
},
"name" : {
"type" : "string"
},
"status" : {
"type" : "string"
},
"start-time" : {
"type" : "integer"
},
"end-time" : {
"type" : "integer"
},
"duration" : {
"type" : "integer"
},
"timestamps" : {
"type" : "object",
"additionalProperties" : {
"type" : "integer"
}
},
"jobs" : {
"type" : "array",
"items" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:JobDetails",
"properties" : {
"jid" : {
"type" : "any"
},
"name" : {
"type" : "string"
},
"start-time" : {
"type" : "integer"
},
"end-time" : {
"type" : "integer"
},
"duration" : {
"type" : "integer"
},
"state" : {
"type" : "string",
"enum" : [ "INITIALIZING", "CREATED", "RUNNING", "FAILING", "FAILED", "CANCELLING", "CANCELED", "FINISHED", "RESTARTING", "SUSPENDED", "RECONCILING" ]
},
"last-modification" : {
"type" : "integer"
},
"tasks" : {
"type" : "object",
"additionalProperties" : {
"type" : "integer"
}
},
"pending-operators" : {
"type" : "integer"
}
}
}
}
}
}
}, {
"url" : "/applications/:applicationid/cancel",
"method" : "POST",
"status-code" : "202 Accepted",
"file-upload" : false,
"path-parameters" : {
"pathParameters" : [ {
"key" : "applicationid"
} ]
},
"query-parameters" : {
"queryParameters" : [ ]
},
"request" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyRequestBody"
},
"response" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyResponseBody"
}
}, {
"url" : "/cluster",
"method" : "DELETE",
"status-code" : "200 OK",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;

import static org.apache.flink.util.Preconditions.checkNotNull;

/** Base class for all applications. */
public abstract class AbstractApplication implements Serializable {

Expand All @@ -62,7 +64,7 @@ public abstract class AbstractApplication implements Serializable {
private final Set<JobID> jobs = new HashSet<>();

public AbstractApplication(ApplicationID applicationId) {
this.applicationId = applicationId;
this.applicationId = checkNotNull(applicationId);
this.statusTimestamps = new long[ApplicationState.values().length];
this.applicationState = ApplicationState.CREATED;
this.statusTimestamps[ApplicationState.CREATED.ordinal()] = System.currentTimeMillis();
Expand Down
Loading