From e7d389e6522ffb011afc497db430f37b2acc7098 Mon Sep 17 00:00:00 2001 From: gyao Date: Wed, 14 Feb 2018 20:47:11 +0100 Subject: [PATCH 01/15] [FLINK-7711][flip6] Implement JarListHandler This closes #5209. This closes #5455. --- .../flink/client/program/PackagedProgram.java | 3 + .../webmonitor/handlers/JarListHandler.java | 162 ++++++++++++++++ .../webmonitor/handlers/JarListHeaders.java | 65 +++++++ .../webmonitor/handlers/JarListInfo.java | 176 ++++++++++++++++++ .../handlers/ng/JarListInfoTest.java | 52 ++++++ .../runtime/webmonitor/WebMonitorUtils.java | 78 +++++--- 6 files changed, 512 insertions(+), 24 deletions(-) create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ng/JarListInfoTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java index 35bb04f875357..2d8cb1dacb95a 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java @@ -27,6 +27,8 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.InstantiationUtil; +import javax.annotation.Nullable; + import java.io.BufferedInputStream; import java.io.File; import java.io.FileOutputStream; @@ -380,6 +382,7 @@ else if (isUsingInteractiveMode()) { * This invocation is thrown if the Program can't be properly loaded. Causes * may be a missing / wrong class or manifest files. */ + @Nullable public String getDescription() throws ProgramInvocationException { if (ProgramDescription.class.isAssignableFrom(this.mainClass)) { diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java new file mode 100644 index 0000000000000..7848478dd9bcc --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java @@ -0,0 +1,162 @@ +/* + * 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.api.common.time.Time; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.rest.handler.AbstractRestHandler; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.MessageHeaders; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.FlinkException; + +import javax.annotation.Nonnull; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.jar.JarFile; +import java.util.jar.Manifest; + +import static java.util.Objects.requireNonNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Handle request for listing uploaded jars. + */ +public class JarListHandler extends AbstractRestHandler { + + private static final File[] EMPTY_FILES_ARRAY = new File[0]; + + private final File jarDir; + + private final Executor executor; + + public JarListHandler( + CompletableFuture localRestAddress, + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + MessageHeaders messageHeaders, + File jarDir, + Executor executor) { + super(localRestAddress, leaderRetriever, timeout, responseHeaders, messageHeaders); + + this.jarDir = requireNonNull(jarDir); + this.executor = requireNonNull(executor); + } + + @Override + protected CompletableFuture handleRequest(@Nonnull HandlerRequest request, @Nonnull RestfulGateway gateway) throws RestHandlerException { + final String localAddress; + checkState(localAddressFuture.isDone()); + + try { + localAddress = localAddressFuture.get(); + } catch (Exception e) { + return FutureUtils.completedExceptionally(e); + } + + return CompletableFuture.supplyAsync(() -> { + try { + final File[] list = getJarFiles(); + final List jarFileList = new ArrayList<>(list.length); + for (File f : list) { + // separate the uuid and the name parts. + String id = f.getName(); + + int startIndex = id.indexOf("_"); + if (startIndex < 0) { + continue; + } + String name = id.substring(startIndex + 1); + if (name.length() < 5 || !name.endsWith(".jar")) { + continue; + } + + List jarEntryList = new ArrayList<>(); + String[] classes = new String[0]; + try { + JarFile jar = new JarFile(f); + Manifest manifest = jar.getManifest(); + String assemblerClass = null; + + if (manifest != null) { + assemblerClass = manifest.getMainAttributes().getValue(PackagedProgram.MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS); + if (assemblerClass == null) { + assemblerClass = manifest.getMainAttributes().getValue(PackagedProgram.MANIFEST_ATTRIBUTE_MAIN_CLASS); + } + } + if (assemblerClass != null) { + classes = assemblerClass.split(","); + } + } catch (IOException ignored) { + // we simply show no entries here + } + + // show every entry class that can be loaded later on. + for (String clazz : classes) { + clazz = clazz.trim(); + + PackagedProgram program = null; + try { + program = new PackagedProgram(f, clazz, new String[0]); + } catch (Exception ignored) { + // ignore jar files which throw an error upon creating a PackagedProgram + } + if (program != null) { + JarListInfo.JarEntryInfo jarEntryInfo = new JarListInfo.JarEntryInfo(clazz, program.getDescription()); + jarEntryList.add(jarEntryInfo); + } + } + + jarFileList.add(new JarListInfo.JarFileInfo(id, name, f.lastModified(), jarEntryList)); + } + + return new JarListInfo(localAddress, jarFileList); + } catch (Exception e) { + throw new CompletionException(new FlinkException("Failed to fetch jar list.", e)); + } + }, executor); + } + + private File[] getJarFiles() { + final File[] list = jarDir.listFiles((dir, name) -> name.endsWith(".jar")); + if (list == null) { + log.warn("Jar upload dir {} does not exist, or had been deleted externally. " + + "Previously uploaded jars are no longer available.", jarDir); + return EMPTY_FILES_ARRAY; + } else { + // last modified ascending order + Arrays.sort(list, (f1, f2) -> Long.compare(f2.lastModified(), f1.lastModified())); + return list; + } + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java new file mode 100644 index 0000000000000..e771faf234fbc --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java @@ -0,0 +1,65 @@ +/* + * 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.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.MessageHeaders; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * Message headers for the {@link JarListHandler}. + */ +public class JarListHeaders implements MessageHeaders { + + public static final String URL = "/jars"; + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public Class getResponseClass() { + return JarListInfo.class; + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public EmptyMessageParameters getUnresolvedMessageParameters() { + return EmptyMessageParameters.getInstance(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java new file mode 100644 index 0000000000000..91686865b1397 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListInfo.java @@ -0,0 +1,176 @@ +/* + * 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.rest.messages.ResponseBody; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Response type of the {@link JarListHandler}. + */ +public class JarListInfo implements ResponseBody { + public static final String JAR_LIST_FIELD_ADDRESS = "address"; + public static final String JAR_LIST_FIELD_FILES = "files"; + + @JsonProperty(JAR_LIST_FIELD_ADDRESS) + private String address; + + @JsonProperty(JAR_LIST_FIELD_FILES) + private List jarFileList; + + @JsonCreator + public JarListInfo( + @JsonProperty(JAR_LIST_FIELD_ADDRESS) String address, + @JsonProperty(JAR_LIST_FIELD_FILES) List jarFileList) { + this.address = checkNotNull(address); + this.jarFileList = checkNotNull(jarFileList); + } + + @Override + public int hashCode() { + return Objects.hash(address, jarFileList); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (null == o || this.getClass() != o.getClass()) { + return false; + } + + JarListInfo that = (JarListInfo) o; + return Objects.equals(address, that.address) && + Objects.equals(jarFileList, that.jarFileList); + } + + //--------------------------------------------------------------------------------- + // Static helper classes + //--------------------------------------------------------------------------------- + + /** + * Nested class to encapsulate the jar file info. + */ + public static class JarFileInfo { + public static final String JAR_FILE_FIELD_ID = "id"; + public static final String JAR_FILE_FIELD_NAME = "name"; + public static final String JAR_FILE_FIELD_UPLOADED = "uploaded"; + public static final String JAR_FILE_FIELD_ENTRY = "entry"; + + @JsonProperty(JAR_FILE_FIELD_ID) + private String id; + + @JsonProperty(JAR_FILE_FIELD_NAME) + private String name; + + @JsonProperty(JAR_FILE_FIELD_UPLOADED) + private long uploaded; + + @JsonProperty(JAR_FILE_FIELD_ENTRY) + private List jarEntryList; + + @JsonCreator + public JarFileInfo( + @JsonProperty(JAR_FILE_FIELD_ID) String id, + @JsonProperty(JAR_FILE_FIELD_NAME) String name, + @JsonProperty(JAR_FILE_FIELD_UPLOADED) long uploaded, + @JsonProperty(JAR_FILE_FIELD_ENTRY) List jarEntryList) { + this.id = checkNotNull(id); + this.name = checkNotNull(name); + this.uploaded = uploaded; + this.jarEntryList = checkNotNull(jarEntryList); + } + + @Override + public int hashCode() { + return Objects.hash(id, name, uploaded, jarEntryList); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (null == o || this.getClass() != o.getClass()) { + return false; + } + + JarFileInfo that = (JarFileInfo) o; + return Objects.equals(id, that.id) && + Objects.equals(name, that.name) && + uploaded == that.uploaded && + Objects.equals(jarEntryList, that.jarEntryList); + } + } + + /** + * Nested class to encapsulate the jar entry info. + */ + public static class JarEntryInfo { + public static final String JAR_ENTRY_FIELD_NAME = "name"; + public static final String JAR_ENTRY_FIELD_DESC = "description"; + + @JsonProperty(JAR_ENTRY_FIELD_NAME) + private String name; + + @JsonProperty(JAR_ENTRY_FIELD_DESC) + @Nullable + private String description; + + @JsonCreator + public JarEntryInfo( + @JsonProperty(JAR_ENTRY_FIELD_NAME) String name, + @JsonProperty(JAR_ENTRY_FIELD_DESC) @Nullable String description) { + this.name = checkNotNull(name); + this.description = description; + } + + @Override + public int hashCode() { + return Objects.hash(name, description); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (null == o || this.getClass() != o.getClass()) { + return false; + } + + JarEntryInfo that = (JarEntryInfo) o; + return Objects.equals(name, that.name) && + Objects.equals(description, that.description); + } + } +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ng/JarListInfoTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ng/JarListInfoTest.java new file mode 100644 index 0000000000000..a32610f5f9d51 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ng/JarListInfoTest.java @@ -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.runtime.webmonitor.handlers.ng; + +import org.apache.flink.runtime.rest.messages.RestResponseMarshallingTestBase; +import org.apache.flink.runtime.webmonitor.handlers.JarListInfo; + +import java.util.ArrayList; +import java.util.List; + +/** + * Tests that the {@link JarListInfo} can be marshalled and unmarshalled. + */ +public class JarListInfoTest extends RestResponseMarshallingTestBase { + @Override + protected Class getTestResponseClass() { + return JarListInfo.class; + } + + @Override + protected JarListInfo getTestResponseInstance() throws Exception { + List jarEntryList1 = new ArrayList<>(); + jarEntryList1.add(new JarListInfo.JarEntryInfo("name1", "desc1")); + jarEntryList1.add(new JarListInfo.JarEntryInfo("name2", "desc2")); + + List jarEntryList2 = new ArrayList<>(); + jarEntryList2.add(new JarListInfo.JarEntryInfo("name3", "desc3")); + jarEntryList2.add(new JarListInfo.JarEntryInfo("name4", "desc4")); + + List jarFileList = new ArrayList<>(); + jarFileList.add(new JarListInfo.JarFileInfo("fileId1", "fileName1", System.currentTimeMillis(), jarEntryList1)); + jarFileList.add(new JarListInfo.JarFileInfo("fileId2", "fileName2", System.currentTimeMillis(), jarEntryList2)); + + return new JarListInfo("local", jarFileList); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java index 3b4da84e8b871..832b2e519cf96 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java @@ -224,31 +224,61 @@ public static Collection clazz = Class.forName(classname); - final Constructor constructor = clazz.getConstructor( - CompletableFuture.class, - GatewayRetriever.class, - Time.class, - Map.class, - MessageHeaders.class, - java.nio.file.Path.class, - Executor.class); - - final MessageHeaders jarUploadMessageHeaders = - (MessageHeaders) Class - .forName("org.apache.flink.runtime.webmonitor.handlers.JarUploadMessageHeaders") - .newInstance(); - - return Arrays.asList(Tuple2.of(jarUploadMessageHeaders, (ChannelInboundHandler) constructor.newInstance( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - jarUploadMessageHeaders, - uploadDir, - executor))); + final Constructor jarUploadHandlerConstrutor = Class + .forName(jarHandlerPackageName + "JarUploadHandler") + .getConstructor( + CompletableFuture.class, + GatewayRetriever.class, + Time.class, + Map.class, + MessageHeaders.class, + java.nio.file.Path.class, + Executor.class); + + final MessageHeaders jarUploadMessageHeaders = (MessageHeaders) Class + .forName(jarHandlerPackageName + "JarUploadMessageHeaders") + .newInstance(); + + final ChannelInboundHandler jarUploadHandler = (ChannelInboundHandler) jarUploadHandlerConstrutor + .newInstance( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + jarUploadMessageHeaders, + uploadDir, + executor); + + final Constructor jarListHandlerConstructor = Class + .forName(jarHandlerPackageName + "JarListHandler") + .getConstructor( + CompletableFuture.class, + GatewayRetriever.class, + Time.class, + Map.class, + MessageHeaders.class, + File.class, + Executor.class); + + final MessageHeaders jarListHeaders = (MessageHeaders) Class + .forName(jarHandlerPackageName + "JarListHeaders") + .newInstance(); + + final ChannelInboundHandler jarListHandler = (ChannelInboundHandler) jarListHandlerConstructor + .newInstance( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + jarListHeaders, + uploadDir.toFile(), + executor); + + return Arrays.asList( + Tuple2.of(jarUploadMessageHeaders, jarUploadHandler), + Tuple2.of(jarListHeaders, jarListHandler)); } catch (ClassNotFoundException | InvocationTargetException | InstantiationException | NoSuchMethodException | IllegalAccessException e) { throw new RuntimeException(e); } From 62ad9be6fe3995d8c23614960307b045d361962c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 13 Feb 2018 15:33:11 +0100 Subject: [PATCH 02/15] [FLINK-8647] [flip6] Introduce JobMasterConfiguration This commit introduces a JobMasterConfiguration which contains JobMaster specific configuration settings. This closes #5478. --- .../runtime/executiongraph/Execution.java | 2 +- .../runtime/jobmaster/JobManagerRunner.java | 12 +-- .../jobmaster/JobManagerSharedServices.java | 29 ++----- .../flink/runtime/jobmaster/JobMaster.java | 20 ++--- .../jobmaster/JobMasterConfiguration.java | 87 +++++++++++++++++++ .../apache/flink/runtime/akka/AkkaUtils.scala | 2 +- .../runtime/jobmaster/JobMasterTest.java | 14 ++- ...estingJobManagerSharedServicesBuilder.java | 12 +-- 8 files changed, 122 insertions(+), 56 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterConfiguration.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 0275004b3f2c6..14d88c353c750 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -707,7 +707,7 @@ else if (numConsumers == 0) { executor); // double check to resolve race conditions - if (consumerVertex.getExecutionState() == RUNNING){ + if (consumerVertex.getExecutionState() == RUNNING) { consumerVertex.sendPartitionInfos(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 8468f28fd3065..12bdc59480b32 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -90,7 +90,7 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F private final JobManagerMetricGroup jobManagerMetricGroup; - private final Time timeout; + private final Time rpcTimeout; /** flag marking the runner as shut down. */ private volatile boolean shutdown; @@ -154,14 +154,16 @@ public JobManagerRunner( this.runningJobsRegistry = haServices.getRunningJobsRegistry(); this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID()); - this.timeout = jobManagerSharedServices.getTimeout(); + final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); + + this.rpcTimeout = jobMasterConfiguration.getRpcTimeout(); // now start the JobManager this.jobManager = new JobMaster( rpcService, + jobMasterConfiguration, resourceId, jobGraph, - configuration, haServices, jobManagerSharedServices, heartbeatServices, @@ -367,7 +369,7 @@ public void grantLeadership(final UUID leaderSessionID) { runningJobsRegistry.setJobRunning(jobGraph.getJobID()); } - CompletableFuture startingFuture = jobManager.start(new JobMasterId(leaderSessionID), timeout); + CompletableFuture startingFuture = jobManager.start(new JobMasterId(leaderSessionID), rpcTimeout); startingFuture.whenCompleteAsync( (Acknowledge ack, Throwable throwable) -> { @@ -394,7 +396,7 @@ public void revokeLeadership() { log.info("JobManager for job {} ({}) was revoked leadership at {}.", jobGraph.getName(), jobGraph.getJobID(), getAddress()); - CompletableFuture suspendFuture = jobManager.suspend(new Exception("JobManager is no longer the leader."), timeout); + CompletableFuture suspendFuture = jobManager.suspend(new Exception("JobManager is no longer the leader."), rpcTimeout); suspendFuture.whenCompleteAsync( (Acknowledge ack, Throwable throwable) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java index d028f8841bd19..34b338e897375 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java @@ -60,29 +60,18 @@ public class JobManagerSharedServices { private final BackPressureStatsTracker backPressureStatsTracker; - private final Time timeout; - public JobManagerSharedServices( ScheduledExecutorService scheduledExecutorService, LibraryCacheManager libraryCacheManager, RestartStrategyFactory restartStrategyFactory, StackTraceSampleCoordinator stackTraceSampleCoordinator, - BackPressureStatsTracker backPressureStatsTracker, - Time backPressureStatsTrackerCleanupInterval, - Time timeout) { + BackPressureStatsTracker backPressureStatsTracker) { this.scheduledExecutorService = checkNotNull(scheduledExecutorService); this.libraryCacheManager = checkNotNull(libraryCacheManager); this.restartStrategyFactory = checkNotNull(restartStrategyFactory); this.stackTraceSampleCoordinator = checkNotNull(stackTraceSampleCoordinator); this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker); - this.timeout = checkNotNull(timeout); - - scheduledExecutorService.scheduleWithFixedDelay( - backPressureStatsTracker::cleanUpOperatorStatsCache, - backPressureStatsTrackerCleanupInterval.toMilliseconds(), - backPressureStatsTrackerCleanupInterval.toMilliseconds(), - TimeUnit.MILLISECONDS); } public ScheduledExecutorService getScheduledExecutorService() { @@ -101,10 +90,6 @@ public BackPressureStatsTracker getBackPressureStatsTracker() { return backPressureStatsTracker; } - public Time getTimeout() { - return timeout; - } - /** * Shutdown the {@link JobMaster} services. * @@ -160,7 +145,7 @@ public static JobManagerSharedServices fromConfiguration( try { timeout = AkkaUtils.getTimeout(config); } catch (NumberFormatException e) { - throw new IllegalConfigurationException(AkkaUtils.formatDurationParingErrorMessage()); + throw new IllegalConfigurationException(AkkaUtils.formatDurationParsingErrorMessage()); } final ScheduledExecutorService futureExecutor = Executors.newScheduledThreadPool( @@ -177,13 +162,17 @@ public static JobManagerSharedServices fromConfiguration( config.getInteger(WebOptions.BACKPRESSURE_REFRESH_INTERVAL), Time.milliseconds(config.getInteger(WebOptions.BACKPRESSURE_DELAY))); + futureExecutor.scheduleWithFixedDelay( + backPressureStatsTracker::cleanUpOperatorStatsCache, + cleanUpInterval, + cleanUpInterval, + TimeUnit.MILLISECONDS); + return new JobManagerSharedServices( futureExecutor, libraryCacheManager, RestartStrategyFactory.createRestartStrategyFactory(config), stackTraceSampleCoordinator, - backPressureStatsTracker, - Time.milliseconds(cleanUpInterval), - Time.milliseconds(timeout.toMillis())); + backPressureStatsTracker); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 8aa2617865a77..b7d533dcbed78 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -22,8 +22,6 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.metrics.MetricGroup; @@ -148,6 +146,8 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast // ------------------------------------------------------------------------ + private final JobMasterConfiguration jobMasterConfiguration; + private final ResourceID resourceId; /** Logical representation of the job. */ @@ -210,9 +210,9 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast public JobMaster( RpcService rpcService, + JobMasterConfiguration jobMasterConfiguration, ResourceID resourceId, JobGraph jobGraph, - Configuration configuration, HighAvailabilityServices highAvailabilityService, JobManagerSharedServices jobManagerSharedServices, HeartbeatServices heartbeatServices, @@ -228,9 +228,10 @@ public JobMaster( final JobMasterGateway selfGateway = getSelfGateway(JobMasterGateway.class); + this.jobMasterConfiguration = checkNotNull(jobMasterConfiguration); this.resourceId = checkNotNull(resourceId); this.jobGraph = checkNotNull(jobGraph); - this.rpcTimeout = jobManagerSharedServices.getTimeout(); + this.rpcTimeout = jobMasterConfiguration.getRpcTimeout(); this.highAvailabilityServices = checkNotNull(highAvailabilityService); this.blobServer = checkNotNull(blobServer); this.scheduledExecutorService = jobManagerSharedServices.getScheduledExecutorService(); @@ -280,17 +281,16 @@ public JobMaster( rpcService, jobGraph.getJobID(), SystemClock.getInstance(), - rpcTimeout, Time.milliseconds(configuration.getLong(JobManagerOptions.SLOT_REQUEST_TIMEOUT)), - Time.milliseconds(configuration.getLong(JobManagerOptions.SLOT_IDLE_TIMEOUT))); + rpcTimeout, + jobMasterConfiguration.getSlotRequestTimeout(), + jobMasterConfiguration.getSlotIdleTimeout()); this.slotPoolGateway = slotPool.getSelfGateway(SlotPoolGateway.class); - final Time allocationTimeout = Time.milliseconds(configuration.getLong(JobManagerOptions.SLOT_REQUEST_TIMEOUT)); - this.executionGraph = ExecutionGraphBuilder.buildGraph( null, jobGraph, - configuration, + jobMasterConfiguration.getConfiguration(), scheduledExecutorService, scheduledExecutorService, slotPool.getSlotProvider(), @@ -301,7 +301,7 @@ public JobMaster( jobMetricGroup, -1, blobServer, - allocationTimeout, + jobMasterConfiguration.getSlotRequestTimeout(), log); // register self as job status change listener diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterConfiguration.java new file mode 100644 index 0000000000000..15a30e26a2b37 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterConfiguration.java @@ -0,0 +1,87 @@ +/* + * 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.jobmaster; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.util.Preconditions; + +/** + * Configuration for the {@link JobMaster}. + */ +public class JobMasterConfiguration { + + private final Time rpcTimeout; + + private final Time slotRequestTimeout; + + private final Time slotIdleTimeout; + + private final Configuration configuration; + + public JobMasterConfiguration( + Time rpcTimeout, + Time slotRequestTimeout, + Time slotIdleTimeout, + Configuration configuration) { + this.rpcTimeout = Preconditions.checkNotNull(rpcTimeout); + this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout); + this.slotIdleTimeout = Preconditions.checkNotNull(slotIdleTimeout); + this.configuration = Preconditions.checkNotNull(configuration); + } + + public Time getRpcTimeout() { + return rpcTimeout; + } + + public Time getSlotRequestTimeout() { + return slotRequestTimeout; + } + + public Time getSlotIdleTimeout() { + return slotIdleTimeout; + } + + public Configuration getConfiguration() { + return configuration; + } + + public static JobMasterConfiguration fromConfiguration(Configuration configuration) { + + final Time rpcTimeout; + + try { + rpcTimeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis()); + } catch (NumberFormatException e) { + throw new IllegalConfigurationException(AkkaUtils.formatDurationParsingErrorMessage()); + } + + final Time slotRequestTimeout = Time.milliseconds(configuration.getLong(JobManagerOptions.SLOT_REQUEST_TIMEOUT)); + final Time slotIdleTimeout = Time.milliseconds(configuration.getLong(JobManagerOptions.SLOT_IDLE_TIMEOUT)); + + return new JobMasterConfiguration( + rpcTimeout, + slotRequestTimeout, + slotIdleTimeout, + configuration); + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 0c6be5db6eace..da700c00c51ad 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -723,7 +723,7 @@ object AkkaUtils { AddressFromURIString(akkaURL) } - def formatDurationParingErrorMessage: String = { + def formatDurationParsingErrorMessage: String = { "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " + "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|" + "(µs|micro|microsecond)|(ns|nano|nanosecond)" diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 29f8b38d6d1bf..e8f0bc4bd2f28 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -102,18 +102,17 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { Configuration configuration = new Configuration(); - final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder() - .setTimeout(testingTimeout) - .build(); + final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); + final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); try (BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore())) { blobServer.start(); final JobMaster jobMaster = new JobMaster( rpc, + jobMasterConfiguration, jmResourceId, jobGraph, - configuration, haServices, jobManagerSharedServices, heartbeatServices, @@ -204,18 +203,17 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { Configuration configuration = new Configuration(); - final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder() - .setTimeout(testingTimeout) - .build(); + final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); + final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); try (BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore())) { blobServer.start(); final JobMaster jobMaster = new JobMaster( rpc, + jobMasterConfiguration, jmResourceId, jobGraph, - configuration, haServices, jobManagerSharedServices, heartbeatServices, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java index 74b97a4d31fb4..f0b232a4d7ec4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.jobmaster; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; @@ -46,8 +45,6 @@ public class TestingJobManagerSharedServicesBuilder { private BackPressureStatsTracker backPressureStatsTracker; - private Time timeout; - public TestingJobManagerSharedServicesBuilder() { scheduledExecutorService = TestingUtils.defaultExecutor(); libraryCacheManager = mock(LibraryCacheManager.class); @@ -83,19 +80,12 @@ public TestingJobManagerSharedServicesBuilder setBackPressureStatsTracker(BackPr } - public TestingJobManagerSharedServicesBuilder setTimeout(Time timeout) { - this.timeout = timeout; - return this; - } - public JobManagerSharedServices build() { return new JobManagerSharedServices( scheduledExecutorService, libraryCacheManager, restartStrategyFactory, stackTraceSampleCoordinator, - backPressureStatsTracker, - timeout, - timeout); + backPressureStatsTracker); } } From a049b2387ca4934503bb77566407c06945a0d670 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 11:05:35 +0100 Subject: [PATCH 03/15] [FLINK-8610] [flip6] Remove RestfulGateway from JobMasterGateway The JobMaster no longer needs to implement the RestfulGateway. Therefore, it is removed by this commit. This closes #5433. --- .../flink/runtime/dispatcher/Dispatcher.java | 7 +- .../flink/runtime/jobmaster/JobMaster.java | 96 +------------------ .../runtime/jobmaster/JobMasterGateway.java | 37 ++++--- .../utils/TestingJobMasterGateway.java | 29 ++++-- 4 files changed, 54 insertions(+), 115 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 92ccc9b9fd9ac..0e8470852137b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -380,8 +380,7 @@ public CompletableFuture requestOperatorBackP if (jobManagerRunner == null) { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); } else { - return jobManagerRunner.getJobManagerGateway() - .requestOperatorBackPressureStats(jobId, jobVertexId); + return jobManagerRunner.getJobManagerGateway().requestOperatorBackPressureStats(jobVertexId); } } @@ -399,7 +398,7 @@ public CompletableFuture requestJob(JobID jobId, Time ti return CompletableFuture.completedFuture(serializableExecutionGraph); } } else { - return jobManagerRunner.getJobManagerGateway().requestJob(jobId, timeout); + return jobManagerRunner.getJobManagerGateway().requestJob(timeout); } } @@ -432,7 +431,7 @@ public CompletableFuture triggerSavepoint( if (jobManagerRunners.containsKey(jobId)) { return jobManagerRunners.get(jobId) .getJobManagerGateway() - .triggerSavepoint(jobId, targetDirectory, timeout); + .triggerSavepoint(targetDirectory, timeout); } else { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index b7d533dcbed78..aeac2df60737b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -68,10 +68,7 @@ import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; -import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; import org.apache.flink.runtime.messages.webmonitor.JobDetails; -import org.apache.flink.runtime.messages.webmonitor.JobsOverview; -import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateLocation; @@ -82,7 +79,6 @@ import org.apache.flink.runtime.registration.RetryingRegistration; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; -import org.apache.flink.runtime.resourcemanager.ResourceOverview; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats; import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; @@ -110,7 +106,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -124,7 +119,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -819,94 +813,15 @@ public CompletableFuture requestJobStatus(Time timeout) { return CompletableFuture.completedFuture(executionGraph.getState()); } - //---------------------------------------------------------------------------------------------- - // RestfulGateway RPC methods - //---------------------------------------------------------------------------------------------- - @Override - public CompletableFuture cancelJob(JobID jobId, Time timeout) { - if (jobGraph.getJobID().equals(jobId)) { - return cancel(timeout); - } else { - return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); - } - } - - @Override - public CompletableFuture stopJob(JobID jobId, Time timeout) { - if (jobGraph.getJobID().equals(jobId)) { - return stop(timeout); - } else { - return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); - } - } - - @Override - public CompletableFuture requestRestAddress(Time timeout) { - return restAddressFuture; - } - - @Override - public CompletableFuture requestJob(JobID jobId, Time timeout) { - if (jobGraph.getJobID().equals(jobId)) { - return CompletableFuture.completedFuture(ArchivedExecutionGraph.createFrom(executionGraph)); - } else { - return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); - } - } - - @Override - public CompletableFuture requestMultipleJobDetails(Time timeout) { - return requestJobDetails(timeout) - .thenApply( - jobDetails -> new MultipleJobsDetails(Collections.singleton(jobDetails))); - } - - @Override - public CompletableFuture requestClusterOverview(Time timeout) { - final CompletableFuture resourceOverviewFuture; - if (resourceManagerConnection != null) { - resourceOverviewFuture = resourceManagerConnection.getTargetGateway().requestResourceOverview(timeout); - } else { - resourceOverviewFuture = CompletableFuture.completedFuture(ResourceOverview.empty()); - } - - Collection jobStatuses = Collections.singleton(executionGraph.getState()); - - return resourceOverviewFuture.thenApply( - (ResourceOverview resourceOverview) -> new ClusterOverview( - resourceOverview, - JobsOverview.create(jobStatuses))); - } - - @Override - public CompletableFuture> requestMetricQueryServicePaths(Time timeout) { - if (metricQueryServicePath != null) { - return CompletableFuture.completedFuture(Collections.singleton(metricQueryServicePath)); - } else { - return CompletableFuture.completedFuture(Collections.emptyList()); - } - } - - @Override - public CompletableFuture>> requestTaskManagerMetricQueryServicePaths(Time timeout) { - if (resourceManagerConnection != null) { - return resourceManagerConnection.getTargetGateway().requestTaskManagerMetricQueryServicePaths(timeout); - } else { - return CompletableFuture.completedFuture(Collections.emptyList()); - } + public CompletableFuture requestJob(Time timeout) { + return CompletableFuture.completedFuture(ArchivedExecutionGraph.createFrom(executionGraph)); } @Override public CompletableFuture triggerSavepoint( - final JobID jobId, - final String targetDirectory, - final Time timeout) { - checkArgument( - jobGraph.getJobID().equals(jobId), - "Expected job id %s, was %s", - jobGraph.getJobID(), - jobId); + final String targetDirectory, + final Time timeout) { try { return executionGraph.getCheckpointCoordinator() .triggerSavepoint(System.currentTimeMillis(), targetDirectory) @@ -917,8 +832,7 @@ public CompletableFuture triggerSavepoint( } @Override - public CompletableFuture requestOperatorBackPressureStats( - final JobID jobId, final JobVertexID jobVertexId) { + public CompletableFuture requestOperatorBackPressureStats(final JobVertexID jobVertexId) { final ExecutionJobVertex jobVertex = executionGraph.getJobVertex(jobVertexId); if (jobVertex == null) { return FutureUtils.completedExceptionally(new FlinkException("JobVertexID not found " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 9b9f3e34cff86..0dcf3fb4b687a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.jobmaster; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -35,16 +34,15 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmaster.message.ClassloadingProps; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.runtime.webmonitor.RestfulGateway; import java.util.Collection; import java.util.concurrent.CompletableFuture; @@ -55,7 +53,6 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway, FencedRpcGateway, - RestfulGateway, KvStateLocationOracle, KvStateRegistryGateway { @@ -219,16 +216,30 @@ CompletableFuture registerTaskManager( CompletableFuture requestJobStatus(@RpcTimeout Time timeout); /** - * Requests the {@link ArchivedExecutionGraph} for the given jobId. If there is no such graph, then - * the future is completed with a {@link FlinkJobNotFoundException}. + * Requests the {@link ArchivedExecutionGraph} of the executed job. * - *

Note: We enforce that the returned future contains a {@link ArchivedExecutionGraph} unlike - * the super interface. + * @param timeout for the rpc call + * @return Future which is completed with the {@link ArchivedExecutionGraph} of the executed job + */ + CompletableFuture requestJob(@RpcTimeout Time timeout); + + /** + * Triggers taking a savepoint of the executed job. + * + * @param targetDirectory to which to write the savepoint data + * @param timeout for the rpc call + * @return Future which is completed with the savepoint path once completed + */ + CompletableFuture triggerSavepoint( + final String targetDirectory, + final Time timeout); + + /** + * Requests the statistics on operator back pressure. * - * @param jobId identifying the job whose AccessExecutionGraph is requested - * @param timeout for the asynchronous operation - * @return Future containing the AccessExecutionGraph for the given jobId, otherwise {@link FlinkJobNotFoundException} + * @param jobVertexId JobVertex for which the stats are requested. + * @return A Future to the {@link OperatorBackPressureStatsResponse} or {@code null} if the stats are + * not available (yet). */ - @Override - CompletableFuture requestJob(JobID jobId, @RpcTimeout Time timeout); + CompletableFuture requestOperatorBackPressureStats(JobVertexID jobVertexId); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java index ae7a4f3e9662f..168b32bf10736 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java @@ -41,11 +41,11 @@ import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.runtime.webmonitor.TestingRestfulGateway; import java.net.InetSocketAddress; import java.util.Collection; @@ -54,7 +54,7 @@ /** * {@link JobMasterGateway} implementation for testing purposes. */ -public class TestingJobMasterGateway extends TestingRestfulGateway implements JobMasterGateway { +public class TestingJobMasterGateway implements JobMasterGateway { @Override public CompletableFuture cancel(Time timeout) { @@ -137,27 +137,32 @@ public CompletableFuture requestJobStatus(Time timeout) { } @Override - public void acknowledgeCheckpoint(JobID jobID, ExecutionAttemptID executionAttemptID, long checkpointId, CheckpointMetrics checkpointMetrics, TaskStateSnapshot subtaskState) { + public CompletableFuture requestJob(Time timeout) { throw new UnsupportedOperationException(); } @Override - public void declineCheckpoint(JobID jobID, ExecutionAttemptID executionAttemptID, long checkpointId, Throwable cause) { + public CompletableFuture triggerSavepoint(String targetDirectory, Time timeout) { throw new UnsupportedOperationException(); } @Override - public JobMasterId getFencingToken() { + public CompletableFuture requestOperatorBackPressureStats(JobVertexID jobVertexId) { + throw new UnsupportedOperationException(); + } + + @Override + public void acknowledgeCheckpoint(JobID jobID, ExecutionAttemptID executionAttemptID, long checkpointId, CheckpointMetrics checkpointMetrics, TaskStateSnapshot subtaskState) { throw new UnsupportedOperationException(); } @Override - public CompletableFuture triggerSavepoint(JobID jobId, String targetDirectory, Time timeout) { + public void declineCheckpoint(JobID jobID, ExecutionAttemptID executionAttemptID, long checkpointId, Throwable cause) { throw new UnsupportedOperationException(); } @Override - public CompletableFuture requestJob(JobID jobId, Time timeout) { + public JobMasterId getFencingToken() { throw new UnsupportedOperationException(); } @@ -175,4 +180,14 @@ public CompletableFuture notifyKvStateRegistered(JobID jobId, JobVe public CompletableFuture notifyKvStateUnregistered(JobID jobId, JobVertexID jobVertexId, KeyGroupRange keyGroupRange, String registrationName) { throw new UnsupportedOperationException(); } + + @Override + public String getAddress() { + return null; + } + + @Override + public String getHostname() { + return null; + } } From abc95cb6a24a5cd631cc6bad86d11788cd09a58b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 7 Feb 2018 14:12:54 +0100 Subject: [PATCH 04/15] [hotfix] Fix checkstyle violations in JobExecutionResult --- .../org/apache/flink/api/common/JobExecutionResult.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java index 0c87cd0f3fb5f..a200d123713c2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java @@ -18,8 +18,8 @@ package org.apache.flink.api.common; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; import java.util.Collections; import java.util.Map; @@ -97,7 +97,7 @@ public T getAccumulatorResult(String accumulatorName) { public Map getAllAccumulatorResults() { return this.accumulatorResults; } - + /** * Gets the accumulator with the given name as an integer. * @@ -120,9 +120,8 @@ public Integer getIntCounterResult(String accumulatorName) { return (Integer) result; } - /** - * Returns a dummy object for wrapping a JobSubmissionResult + * Returns a dummy object for wrapping a JobSubmissionResult. * @param result The SubmissionResult * @return a JobExecutionResult * @deprecated Will be removed in future versions. From 9cfbe5798459ad7b43224e35b0ed57ae93613efe Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 11:08:04 +0100 Subject: [PATCH 05/15] [FLINK-8611] [flip6] Add result future to JobManagerRunner This commit adds a CompletableFuture to the JobManagerRunner. This future will be completed once the job has reached a globally terminal state. This closes #5434. --- .../flink/runtime/jobmaster/JobManagerRunner.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 12bdc59480b32..5e476bf2e3f63 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -92,6 +92,8 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F private final Time rpcTimeout; + private final CompletableFuture resultFuture; + /** flag marking the runner as shut down. */ private volatile boolean shutdown; @@ -174,6 +176,8 @@ public JobManagerRunner( userCodeLoader, restAddress, metricRegistry.getMetricQueryServicePath()); + + this.resultFuture = new CompletableFuture<>(); } catch (Throwable t) { // clean up everything @@ -197,6 +201,10 @@ public JobGraph getJobGraph() { return jobGraph; } + public CompletableFuture getResultFuture() { + return resultFuture; + } + //---------------------------------------------------------------------------------------------- // Lifecycle management //---------------------------------------------------------------------------------------------- @@ -241,6 +249,9 @@ private CompletableFuture shutdownInternally() { if (exception != null) { throw new CompletionException(new FlinkException("Could not properly shut down the JobManagerRunner.", exception)); } + + // cancel the result future if not already completed + resultFuture.cancel(false); }); } } @@ -254,6 +265,9 @@ private CompletableFuture shutdownInternally() { */ @Override public void jobReachedGloballyTerminalState(ArchivedExecutionGraph executionGraph) { + // complete the result future with the terminal execution graph + resultFuture.complete(executionGraph); + try { unregisterJobFromHighAvailability(); shutdownInternally(); From c08acd6fa6484e2aec4a74dd104c7cda7eda05c9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 11:16:12 +0100 Subject: [PATCH 06/15] [FLINK-8612] [flip6] Enable non-detached job mode The non-detached job mode waits until has served the JobResult of a completed job at least once before it terminates. This closes #5435. --- .../runtime/akka/AkkaJobManagerGateway.java | 6 + .../flink/runtime/dispatcher/Dispatcher.java | 19 ++ .../runtime/dispatcher/MiniDispatcher.java | 13 ++ .../job/JobExecutionResultHandler.java | 9 +- .../runtime/webmonitor/RestfulGateway.java | 10 ++ .../dispatcher/MiniDispatcherTest.java | 169 ++++++++++++------ .../job/JobExecutionResultHandlerTest.java | 8 +- .../webmonitor/TestingRestfulGateway.java | 20 +++ 8 files changed, 194 insertions(+), 60 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java index 37a27c724ef4f..8a8eda6ce9116 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobManagerGateway; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.JobManagerMessages; @@ -247,6 +248,11 @@ public CompletableFuture requestJob(JobID jobId, Time time }); } + @Override + public CompletableFuture requestJobResult(JobID jobId, Time timeout) { + return requestJob(jobId, timeout).thenApply(JobResult::createFrom); + } + @Override public CompletableFuture requestClusterOverview(Time timeout) { return FutureUtils.toJava( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 0e8470852137b..6456194103c55 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; @@ -402,6 +403,24 @@ public CompletableFuture requestJob(JobID jobId, Time ti } } + @Override + public CompletableFuture requestJobResult(JobID jobId, Time timeout) { + final JobManagerRunner jobManagerRunner = jobManagerRunners.get(jobId); + + if (jobManagerRunner == null) { + final ArchivedExecutionGraph archivedExecutionGraph = archivedExecutionGraphStore.get(jobId); + + if (archivedExecutionGraph == null) { + return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); + } else { + return CompletableFuture.completedFuture(JobResult.createFrom(archivedExecutionGraph)); + } + } else { + return jobManagerRunner.getResultFuture().thenApply( + (ArchivedExecutionGraph archivedExecutionGraph) -> JobResult.createFrom(archivedExecutionGraph)); + } + } + @Override public CompletableFuture> requestMetricQueryServicePaths(Time timeout) { final String metricQueryServicePath = metricRegistry.getMetricQueryServicePath(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index 26d357e7ba52b..b617aec75ba61 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; @@ -100,6 +101,18 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) return acknowledgeCompletableFuture; } + @Override + public CompletableFuture requestJobResult(JobID jobId, Time timeout) { + final CompletableFuture jobResultFuture = super.requestJobResult(jobId, timeout); + + if (executionMode == ClusterEntrypoint.ExecutionMode.NORMAL) { + // terminate the MiniDispatcher once we served the first JobResult successfully + jobResultFuture.thenRun(this::shutDown); + } + + return jobResultFuture; + } + @Override protected void jobReachedGloballyTerminalState(ArchivedExecutionGraph archivedExecutionGraph) { super.jobReachedGloballyTerminalState(archivedExecutionGraph); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandler.java index 9d2f953c63aa9..318a7d22900c1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandler.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.rest.handler.AbstractRestHandler; import org.apache.flink.runtime.rest.handler.HandlerRequest; @@ -75,12 +74,8 @@ protected CompletableFuture handleRequest( jobStatus -> { if (jobStatus.isGloballyTerminalState()) { return gateway - .requestJob(jobId, timeout) - .thenApply( - executionGraph -> { - final JobResult jobResult = JobResult.createFrom(executionGraph); - return JobExecutionResultResponseBody.created(jobResult); - }); + .requestJobResult(jobId, timeout) + .thenApply(JobExecutionResultResponseBody::created); } else { return CompletableFuture.completedFuture( JobExecutionResultResponseBody.inProgress()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java index 6cad0fb39a45c..65a46649a6b70 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; @@ -83,6 +84,15 @@ public interface RestfulGateway extends RpcGateway { */ CompletableFuture requestJob(JobID jobId, @RpcTimeout Time timeout); + /** + * Requests the {@link JobResult} of a job specified by the given jobId. + * + * @param jobId identifying the job for which to retrieve the {@link JobResult}. + * @param timeout for the asynchronous operation + * @return Future which is completed with the job's {@link JobResult} once the job has finished + */ + CompletableFuture requestJobResult(JobID jobId, @RpcTimeout Time timeout); + /** * Requests job details currently being executed on the Flink cluster. * diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index bc2478d3e6de4..1040eee2ea2cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.NoOpMetricRegistry; @@ -56,14 +57,12 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -77,6 +76,11 @@ public class MiniDispatcherTest extends TestLogger { private static final JobGraph jobGraph = new JobGraph(); + private static final ArchivedExecutionGraph archivedExecutionGraph = new ArchivedExecutionGraphBuilder() + .setJobID(jobGraph.getJobID()) + .setState(JobStatus.FINISHED) + .build(); + private static final Time timeout = Time.seconds(10L); @ClassRule @@ -88,12 +92,22 @@ public class MiniDispatcherTest extends TestLogger { private static BlobServer blobServer; - private MiniDispatcher miniDispatcher; + private final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); + + private final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); + + private final ArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); private CompletableFuture jobGraphFuture; private TestingLeaderElectionService dispatcherLeaderElectionService; + private TestingHighAvailabilityServices highAvailabilityServices; + + private TestingFatalErrorHandler testingFatalErrorHandler; + + private TestingJobManagerRunnerFactory testingJobManagerRunnerFactory; + @BeforeClass public static void setupClass() throws IOException { rpcService = new TestingRpcService(); @@ -107,48 +121,30 @@ public static void setupClass() throws IOException { @Before public void setup() throws Exception { dispatcherLeaderElectionService = new TestingLeaderElectionService(); - final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); - final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); - final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); - final ArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); - final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + highAvailabilityServices = new TestingHighAvailabilityServices(); + testingFatalErrorHandler = new TestingFatalErrorHandler(); highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); jobGraphFuture = new CompletableFuture<>(); - final TestingJobManagerRunnerFactory testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(jobGraphFuture); - miniDispatcher = new MiniDispatcher( - rpcService, - UUID.randomUUID().toString(), - configuration, - highAvailabilityServices, - resourceManagerGateway, - blobServer, - heartbeatServices, - NoOpMetricRegistry.INSTANCE, - archivedExecutionGraphStore, - testingJobManagerRunnerFactory, - testingFatalErrorHandler, - null, - jobGraph, - ClusterEntrypoint.ExecutionMode.DETACHED); - - miniDispatcher.start(); + testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(jobGraphFuture); } @After - public void teardown() throws InterruptedException, ExecutionException, TimeoutException { - if (miniDispatcher != null) { - RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); - miniDispatcher = null; - } + public void teardown() throws Exception { + testingFatalErrorHandler.rethrowError(); } @AfterClass public static void teardownClass() throws IOException { - blobServer.close(); - rpcService.stopService(); + if (blobServer != null) { + blobServer.close(); + } + + if (rpcService != null) { + rpcService.stopService(); + } } /** @@ -157,12 +153,20 @@ public static void teardownClass() throws IOException { */ @Test public void testSingleJobRecovery() throws Exception { - // wait until the Dispatcher is the leader - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + final MiniDispatcher miniDispatcher = createMiniDispatcher(ClusterEntrypoint.ExecutionMode.DETACHED); - final JobGraph actualJobGraph = jobGraphFuture.get(); + miniDispatcher.start(); - assertThat(actualJobGraph.getJobID(), is(jobGraph.getJobID())); + try { + // wait until the Dispatcher is the leader + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + + final JobGraph actualJobGraph = jobGraphFuture.get(); + + assertThat(actualJobGraph.getJobID(), is(jobGraph.getJobID())); + } finally { + RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); + } } /** @@ -171,25 +175,88 @@ public void testSingleJobRecovery() throws Exception { */ @Test public void testTerminationAfterJobCompletion() throws Exception { - final Dispatcher.DispatcherOnCompleteActions completeActions = miniDispatcher.new DispatcherOnCompleteActions(jobGraph.getJobID()); + final MiniDispatcher miniDispatcher = createMiniDispatcher(ClusterEntrypoint.ExecutionMode.DETACHED); + + miniDispatcher.start(); + + try { + final Dispatcher.DispatcherOnCompleteActions completeActions = miniDispatcher.new DispatcherOnCompleteActions(jobGraph.getJobID()); + + // wait until the Dispatcher is the leader + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final ArchivedExecutionGraph archivedExecutionGraph = new ArchivedExecutionGraphBuilder() - .setJobID(jobGraph.getJobID()) - .setState(JobStatus.FINISHED) - .build(); + // wait until we have submitted the job + jobGraphFuture.get(); - // wait until the Dispatcher is the leader - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + completeActions.jobReachedGloballyTerminalState(archivedExecutionGraph); - // wait until we have submitted the job - jobGraphFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + // wait until we terminate + miniDispatcher.getTerminationFuture().get(); + } finally { + RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); + } + } + + /** + * Tests that the {@link MiniDispatcher} only terminates in {@link ClusterEntrypoint.ExecutionMode#NORMAL} + * after it has served the {@link org.apache.flink.runtime.jobmaster.JobResult} once. + */ + @Test + public void testJobResultRetrieval() throws Exception { + final MiniDispatcher miniDispatcher = createMiniDispatcher(ClusterEntrypoint.ExecutionMode.NORMAL); + + miniDispatcher.start(); - completeActions.jobReachedGloballyTerminalState(archivedExecutionGraph); + try { + final Dispatcher.DispatcherOnCompleteActions completeActions = miniDispatcher.new DispatcherOnCompleteActions(jobGraph.getJobID()); - final CompletableFuture terminationFuture = miniDispatcher.getTerminationFuture(); + // wait until the Dispatcher is the leader + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - // wait until we terminate - terminationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + // wait until we have submitted the job + jobGraphFuture.get(); + + completeActions.jobReachedGloballyTerminalState(archivedExecutionGraph); + + final CompletableFuture terminationFuture = miniDispatcher.getTerminationFuture(); + + assertThat(terminationFuture.isDone(), is(false)); + + final DispatcherGateway dispatcherGateway = miniDispatcher.getSelfGateway(DispatcherGateway.class); + + final CompletableFuture jobResultFuture = dispatcherGateway.requestJobResult(jobGraph.getJobID(), timeout); + + final JobResult jobResult = jobResultFuture.get(); + + assertThat(jobResult.getJobId(), is(jobGraph.getJobID())); + + terminationFuture.get(); + } finally { + RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); + } + } + + // -------------------------------------------------------- + // Utilities + // -------------------------------------------------------- + + @Nonnull + private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode executionMode) throws Exception { + return new MiniDispatcher( + rpcService, + UUID.randomUUID().toString(), + configuration, + highAvailabilityServices, + resourceManagerGateway, + blobServer, + heartbeatServices, + NoOpMetricRegistry.INSTANCE, + archivedExecutionGraphStore, + testingJobManagerRunnerFactory, + testingFatalErrorHandler, + null, + jobGraph, + executionMode); } private static final class TestingJobManagerRunnerFactory implements Dispatcher.JobManagerRunnerFactory { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java index 0861089c6f3ba..d69c5d5de1acb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; @@ -32,6 +33,7 @@ import org.apache.flink.runtime.rest.messages.job.JobExecutionResultResponseBody; import org.apache.flink.runtime.rest.messages.queue.QueueStatus; import org.apache.flink.runtime.webmonitor.TestingRestfulGateway; +import org.apache.flink.testutils.category.Flip6; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -39,6 +41,7 @@ import org.junit.Before; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.Collections; import java.util.concurrent.CompletableFuture; @@ -54,6 +57,7 @@ /** * Tests for {@link JobExecutionResultHandler}. */ +@Category(Flip6.class) public class JobExecutionResultHandlerTest extends TestLogger { private static final JobID TEST_JOB_ID = new JobID(); @@ -109,10 +113,10 @@ public void testCompletedResult() throws Exception { assertThat(jobId, equalTo(TEST_JOB_ID)); return CompletableFuture.completedFuture(jobStatus); }) - .setRequestJobFunction( + .setRequestJobResultFunction( jobId -> { assertThat(jobId, equalTo(TEST_JOB_ID)); - return CompletableFuture.completedFuture(executionGraph); + return CompletableFuture.completedFuture(JobResult.createFrom(executionGraph)); } ) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java index dc88f7350b2d2..b06bc2000b045 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; @@ -45,6 +46,7 @@ public class TestingRestfulGateway implements RestfulGateway { static final Function> DEFAULT_CANCEL_JOB_FUNCTION = jobId -> CompletableFuture.completedFuture(Acknowledge.get()); static final Function> DEFAULT_STOP_JOB_FUNCTION = jobId -> CompletableFuture.completedFuture(Acknowledge.get()); + static final Function> DEFAULT_REQUEST_JOB_RESULT_FUNCTION = jobId -> FutureUtils.completedExceptionally(new UnsupportedOperationException()); static final Function> DEFAULT_REQUEST_JOB_FUNCTION = jobId -> FutureUtils.completedExceptionally(new UnsupportedOperationException()); static final Function> DEFAULT_REQUEST_JOB_STATUS_FUNCTION = jobId -> FutureUtils.completedExceptionally(new UnsupportedOperationException()); static final Supplier> DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER = () -> CompletableFuture.completedFuture(new MultipleJobsDetails(Collections.emptyList())); @@ -66,6 +68,8 @@ public class TestingRestfulGateway implements RestfulGateway { protected Function> requestJobFunction; + protected Function> requestJobResultFunction; + protected Function> requestJobStatusFunction; protected Supplier> requestMultipleJobDetailsSupplier; @@ -86,6 +90,7 @@ public TestingRestfulGateway() { DEFAULT_CANCEL_JOB_FUNCTION, DEFAULT_STOP_JOB_FUNCTION, DEFAULT_REQUEST_JOB_FUNCTION, + DEFAULT_REQUEST_JOB_RESULT_FUNCTION, DEFAULT_REQUEST_JOB_STATUS_FUNCTION, DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER, DEFAULT_REQUEST_CLUSTER_OVERVIEW_SUPPLIER, @@ -101,6 +106,7 @@ public TestingRestfulGateway( Function> cancelJobFunction, Function> stopJobFunction, Function> requestJobFunction, + Function> requestJobResultFunction, Function> requestJobStatusFunction, Supplier> requestMultipleJobDetailsSupplier, Supplier> requestClusterOverviewSupplier, @@ -113,6 +119,7 @@ public TestingRestfulGateway( this.cancelJobFunction = cancelJobFunction; this.stopJobFunction = stopJobFunction; this.requestJobFunction = requestJobFunction; + this.requestJobResultFunction = requestJobResultFunction; this.requestJobStatusFunction = requestJobStatusFunction; this.requestMultipleJobDetailsSupplier = requestMultipleJobDetailsSupplier; this.requestClusterOverviewSupplier = requestClusterOverviewSupplier; @@ -141,6 +148,11 @@ public CompletableFuture requestJob(JobID jobId, return requestJobFunction.apply(jobId); } + @Override + public CompletableFuture requestJobResult(JobID jobId, Time timeout) { + return requestJobResultFunction.apply(jobId); + } + @Override public CompletableFuture requestJobStatus(JobID jobId, Time timeout) { return requestJobStatusFunction.apply(jobId); @@ -195,6 +207,7 @@ public static final class Builder { private Function> cancelJobFunction; private Function> stopJobFunction; private Function> requestJobFunction; + private Function> requestJobResultFunction; private Function> requestJobStatusFunction; private Supplier> requestMultipleJobDetailsSupplier; private Supplier> requestClusterOverviewSupplier; @@ -206,6 +219,7 @@ public Builder() { cancelJobFunction = DEFAULT_CANCEL_JOB_FUNCTION; stopJobFunction = DEFAULT_STOP_JOB_FUNCTION; requestJobFunction = DEFAULT_REQUEST_JOB_FUNCTION; + requestJobResultFunction = DEFAULT_REQUEST_JOB_RESULT_FUNCTION; requestJobStatusFunction = DEFAULT_REQUEST_JOB_STATUS_FUNCTION; requestMultipleJobDetailsSupplier = DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER; requestClusterOverviewSupplier = DEFAULT_REQUEST_CLUSTER_OVERVIEW_SUPPLIER; @@ -234,6 +248,11 @@ public Builder setRequestJobFunction(Function> requestJobResultFunction) { + this.requestJobResultFunction = requestJobResultFunction; + return this; + } + public Builder setRequestJobStatusFunction(Function> requestJobStatusFunction) { this.requestJobStatusFunction = requestJobStatusFunction; return this; @@ -282,6 +301,7 @@ public TestingRestfulGateway build() { cancelJobFunction, stopJobFunction, requestJobFunction, + requestJobResultFunction, requestJobStatusFunction, requestMultipleJobDetailsSupplier, requestClusterOverviewSupplier, From d0ee3936279653eff9d06fe7a7746be1e5cc6bc3 Mon Sep 17 00:00:00 2001 From: zjureel Date: Tue, 19 Dec 2017 17:07:56 +0800 Subject: [PATCH 07/15] [FLINK-7857][flip6] Port JobVertexDetailsHandler to REST endpoint --- .../handler/job/JobVertexDetailsHandler.java | 132 ++++++++++++ .../messages/JobVertexDetailsHeaders.java | 68 +++++++ .../rest/messages/JobVertexDetailsInfo.java | 188 ++++++++++++++++++ .../webmonitor/WebMonitorEndpoint.java | 13 ++ .../messages/JobVertexDetailsInfoTest.java | 86 ++++++++ 5 files changed, 487 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java new file mode 100644 index 0000000000000..668a543100f92 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java @@ -0,0 +1,132 @@ +/* + * 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.job; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rest.NotFoundException; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; +import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; +import org.apache.flink.runtime.rest.handler.util.MutableIOMetrics; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobIDPathParameter; +import org.apache.flink.runtime.rest.messages.JobVertexDetailsInfo; +import org.apache.flink.runtime.rest.messages.JobVertexIdPathParameter; +import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters; +import org.apache.flink.runtime.rest.messages.MessageHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** + * Request handler for the job vertex details. + */ +public class JobVertexDetailsHandler extends AbstractExecutionGraphHandler { + private final MetricFetcher metricFetcher; + + public JobVertexDetailsHandler( + CompletableFuture localRestAddress, + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + MessageHeaders messageHeaders, + ExecutionGraphCache executionGraphCache, + Executor executor, + MetricFetcher metricFetcher) { + super( + localRestAddress, + leaderRetriever, + timeout, + responseHeaders, + messageHeaders, + executionGraphCache, + executor); + this.metricFetcher = metricFetcher; + } + + @Override + protected JobVertexDetailsInfo handleRequest(HandlerRequest request, AccessExecutionGraph executionGraph) { + JobID jobID = request.getPathParameter(JobIDPathParameter.class); + JobVertexID jobVertexID = request.getPathParameter(JobVertexIdPathParameter.class); + AccessExecutionJobVertex jobVertex = executionGraph.getJobVertex(jobVertexID); + + List subtasks = new ArrayList<>(); + final long now = System.currentTimeMillis(); + int num = 0; + for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) { + final ExecutionState status = vertex.getExecutionState(); + + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); + String locationString = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort(); + + long startTime = vertex.getStateTimestamp(ExecutionState.DEPLOYING); + if (startTime == 0) { + startTime = -1; + } + long endTime = status.isTerminal() ? vertex.getStateTimestamp(status) : -1; + long duration = startTime > 0 ? ((endTime > 0 ? endTime : now) - startTime) : -1; + + MutableIOMetrics counts = new MutableIOMetrics(); + counts.addIOMetrics( + vertex.getCurrentExecutionAttempt(), + metricFetcher, + jobID.toString(), + jobVertex.getJobVertexId().toString()); + subtasks.add(new JobVertexDetailsInfo.VertexTaskDetail( + num, + status, + vertex.getCurrentExecutionAttempt().getAttemptNumber(), + locationString, + startTime, + endTime, + duration, + new IOMetricsInfo( + counts.getNumBytesInLocal() + counts.getNumBytesInRemote(), + counts.isNumBytesInLocalComplete() && counts.isNumBytesInRemoteComplete(), + counts.getNumBytesOut(), + counts.isNumBytesOutComplete(), + counts.getNumRecordsIn(), + counts.isNumRecordsInComplete(), + counts.getNumRecordsOut(), + counts.isNumRecordsOutComplete()))); + + num++; + } + + return new JobVertexDetailsInfo( + jobVertex.getJobVertexId(), + jobVertex.getName(), + jobVertex.getParallelism(), + now, + subtasks); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java new file mode 100644 index 0000000000000..ee64333b90b8f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java @@ -0,0 +1,68 @@ +/* + * 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.messages; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.job.JobVertexDetailsHandler; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * Message headers for the {@link JobVertexDetailsHandler}. + */ +public class JobVertexDetailsHeaders implements MessageHeaders { + + private static final JobVertexDetailsHeaders INSTANCE = new JobVertexDetailsHeaders(); + + public static final String URL = "/jobs/:" + JobIDPathParameter.KEY + "/vertices/:" + JobVertexIdPathParameter.KEY; + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public Class getResponseClass() { + return JobVertexDetailsInfo.class; + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public JobVertexMessageParameters getUnresolvedMessageParameters() { + return new JobVertexMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + + public static JobVertexDetailsHeaders getInstance() { + return INSTANCE; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java new file mode 100644 index 0000000000000..df007f45c0759 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java @@ -0,0 +1,188 @@ +/* + * 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.messages; + +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rest.handler.job.JobVertexDetailsHandler; +import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; +import org.apache.flink.runtime.rest.messages.json.JobVertexIDDeserializer; +import org.apache.flink.runtime.rest.messages.json.JobVertexIDSerializer; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; + +import java.util.List; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Response type of the {@link JobVertexDetailsHandler}. + */ +public class JobVertexDetailsInfo implements ResponseBody { + public static final String FIELD_NAME_VERTEX_ID = "id"; + public static final String FIELD_NAME_VERTEX_NAME = "name"; + public static final String FIELD_NAME_PARALLELISM = "parallelism"; + public static final String FIELD_NAME_NOW = "now"; + public static final String FIELD_NAME_SUBTASKS = "subtasks"; + + @JsonProperty(FIELD_NAME_VERTEX_ID) + @JsonSerialize(using = JobVertexIDSerializer.class) + private final JobVertexID id; + + @JsonProperty(FIELD_NAME_VERTEX_NAME) + private final String name; + + @JsonProperty(FIELD_NAME_PARALLELISM) + private final int parallelism; + + @JsonProperty(FIELD_NAME_NOW) + private final long now; + + @JsonProperty(FIELD_NAME_SUBTASKS) + private final List subtasks; + + @JsonCreator + public JobVertexDetailsInfo( + @JsonDeserialize(using = JobVertexIDDeserializer.class) @JsonProperty(FIELD_NAME_VERTEX_ID) JobVertexID id, + @JsonProperty(FIELD_NAME_VERTEX_NAME) String name, + @JsonProperty(FIELD_NAME_PARALLELISM) int parallelism, + @JsonProperty(FIELD_NAME_NOW) long now, + @JsonProperty(FIELD_NAME_SUBTASKS) List subtasks) { + this.id = checkNotNull(id); + this.name = checkNotNull(name); + this.parallelism = parallelism; + this.now = now; + this.subtasks = checkNotNull(subtasks); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (null == o || this.getClass() != o.getClass()) { + return false; + } + + JobVertexDetailsInfo that = (JobVertexDetailsInfo) o; + return Objects.equals(id, that.id) && + Objects.equals(name, that.name) && + parallelism == that.parallelism && + now == that.now && + Objects.equals(subtasks, that.subtasks); + } + + @Override + public int hashCode() { + return Objects.hash(id, name, parallelism, now, subtasks); + } + + //--------------------------------------------------------------------------------- + // Static helper classes + //--------------------------------------------------------------------------------- + + /** + * Vertex task detail class. + */ + public static final class VertexTaskDetail { + public static final String FIELD_NAME_SUBTASK = "subtask"; + public static final String FIELD_NAME_STATUS = "status"; + public static final String FIELD_NAME_ATTEMPT = "attempt"; + public static final String FIELD_NAME_HOST = "host"; + public static final String FIELD_NAME_START_TIME = "start_time"; + public static final String FIELD_NAME_END_TIME = "end-time"; + public static final String FIELD_NAME_DURATION = "duration"; + public static final String FIELD_NAME_METRICS = "metrics"; + + @JsonProperty(FIELD_NAME_SUBTASK) + private final int subtask; + + @JsonProperty(FIELD_NAME_STATUS) + private final ExecutionState status; + + @JsonProperty(FIELD_NAME_ATTEMPT) + private final int attempt; + + @JsonProperty(FIELD_NAME_HOST) + private final String host; + + @JsonProperty(FIELD_NAME_START_TIME) + private final long startTime; + + @JsonProperty(FIELD_NAME_END_TIME) + private final long endTime; + + @JsonProperty(FIELD_NAME_DURATION) + private final long duration; + + @JsonProperty(FIELD_NAME_METRICS) + private final IOMetricsInfo metrics; + + @JsonCreator + public VertexTaskDetail( + @JsonProperty(FIELD_NAME_SUBTASK) int subtask, + @JsonProperty(FIELD_NAME_STATUS) ExecutionState status, + @JsonProperty(FIELD_NAME_ATTEMPT) int attempt, + @JsonProperty(FIELD_NAME_HOST) String host, + @JsonProperty(FIELD_NAME_START_TIME) long startTime, + @JsonProperty(FIELD_NAME_END_TIME) long endTime, + @JsonProperty(FIELD_NAME_DURATION) long duration, + @JsonProperty(FIELD_NAME_METRICS) IOMetricsInfo metrics) { + this.subtask = subtask; + this.status = checkNotNull(status); + this.attempt = attempt; + this.host = checkNotNull(host); + this.startTime = startTime; + this.endTime = endTime; + this.duration = duration; + this.metrics = checkNotNull(metrics); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (null == o || this.getClass() != o.getClass()) { + return false; + } + + VertexTaskDetail that = (VertexTaskDetail) o; + return subtask == that.subtask && + Objects.equals(status, that.status) && + attempt == that.attempt && + Objects.equals(host, that.host) && + startTime == that.startTime && + endTime == that.endTime && + duration == that.duration && + Objects.equals(metrics, that.metrics); + } + + @Override + public int hashCode() { + return Objects.hash(subtask, status, attempt, host, startTime, endTime, duration, metrics); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index b148d36340e13..5e4c72b791bd5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.rest.handler.job.JobTerminationHandler; import org.apache.flink.runtime.rest.handler.job.JobVertexAccumulatorsHandler; import org.apache.flink.runtime.rest.handler.job.JobVertexBackPressureHandler; +import org.apache.flink.runtime.rest.handler.job.JobVertexDetailsHandler; import org.apache.flink.runtime.rest.handler.job.JobVertexTaskManagersHandler; import org.apache.flink.runtime.rest.handler.job.JobsOverviewHandler; import org.apache.flink.runtime.rest.handler.job.SubtaskCurrentAttemptDetailsHandler; @@ -82,6 +83,7 @@ import org.apache.flink.runtime.rest.messages.JobTerminationHeaders; import org.apache.flink.runtime.rest.messages.JobVertexAccumulatorsHeaders; import org.apache.flink.runtime.rest.messages.JobVertexBackPressureHeaders; +import org.apache.flink.runtime.rest.messages.JobVertexDetailsHeaders; import org.apache.flink.runtime.rest.messages.JobVertexTaskManagersHeaders; import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders; import org.apache.flink.runtime.rest.messages.SubtasksTimesHeaders; @@ -470,6 +472,16 @@ protected List> initiali JobTerminationHeaders.getInstance(), TerminationModeQueryParameter.TerminationMode.STOP); + final JobVertexDetailsHandler jobVertexDetailsHandler = new JobVertexDetailsHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobVertexDetailsHeaders.getInstance(), + executionGraphCache, + executor, + metricFetcher); + final File tmpDir = restConfiguration.getTmpDir(); Optional> optWebContent; @@ -517,6 +529,7 @@ protected List> initiali handlers.add(Tuple2.of(JobVertexTaskManagersHeaders.getInstance(), jobVertexTaskManagersHandler)); handlers.add(Tuple2.of(JobVertexBackPressureHeaders.getInstance(), jobVertexBackPressureHandler)); handlers.add(Tuple2.of(JobTerminationHeaders.getInstance(), jobCancelTerminationHandler)); + handlers.add(Tuple2.of(JobVertexDetailsHeaders.getInstance(), jobVertexDetailsHandler)); // TODO: Remove once the Yarn proxy can forward all REST verbs handlers.add(Tuple2.of(YarnCancelJobTerminationHeaders.getInstance(), jobCancelTerminationHandler)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java new file mode 100644 index 0000000000000..904996ad954f2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfoTest.java @@ -0,0 +1,86 @@ +/* + * 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.messages; + +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +/** + * Tests that the {@link JobVertexDetailsInfo} can be marshalled and unmarshalled. + */ +public class JobVertexDetailsInfoTest extends RestResponseMarshallingTestBase { + @Override + protected Class getTestResponseClass() { + return JobVertexDetailsInfo.class; + } + + @Override + protected JobVertexDetailsInfo getTestResponseInstance() throws Exception { + final Random random = new Random(); + final IOMetricsInfo jobVertexMetrics = new IOMetricsInfo( + random.nextLong(), + random.nextBoolean(), + random.nextLong(), + random.nextBoolean(), + random.nextLong(), + random.nextBoolean(), + random.nextLong(), + random.nextBoolean()); + List vertexTaskDetailList = new ArrayList<>(); + vertexTaskDetailList.add(new JobVertexDetailsInfo.VertexTaskDetail( + 0, + ExecutionState.CREATED, + random.nextInt(), + "local1", + System.currentTimeMillis(), + System.currentTimeMillis(), + 1L, + jobVertexMetrics)); + vertexTaskDetailList.add(new JobVertexDetailsInfo.VertexTaskDetail( + 1, + ExecutionState.FAILED, + random.nextInt(), + "local2", + System.currentTimeMillis(), + System.currentTimeMillis(), + 1L, + jobVertexMetrics)); + vertexTaskDetailList.add(new JobVertexDetailsInfo.VertexTaskDetail( + 2, + ExecutionState.FINISHED, + random.nextInt(), + "local3", + System.currentTimeMillis(), + System.currentTimeMillis(), + 1L, + jobVertexMetrics)); + + return new JobVertexDetailsInfo( + new JobVertexID(), + "jobVertex" + random.nextLong(), + random.nextInt(), + System.currentTimeMillis(), + vertexTaskDetailList); + } +} From 8559d496c09a4ba050fb86155332f72b35df6935 Mon Sep 17 00:00:00 2001 From: gyao Date: Thu, 15 Feb 2018 11:32:15 +0100 Subject: [PATCH 08/15] [FLINK-7857][flip6] Return status 404 if JobVertex is unknown This closes #5493. This closes #5035. --- .../runtime/rest/handler/job/JobVertexDetailsHandler.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java index 668a543100f92..b4693a5530d25 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexDetailsHandler.java @@ -74,11 +74,17 @@ public JobVertexDetailsHandler( } @Override - protected JobVertexDetailsInfo handleRequest(HandlerRequest request, AccessExecutionGraph executionGraph) { + protected JobVertexDetailsInfo handleRequest( + HandlerRequest request, + AccessExecutionGraph executionGraph) throws NotFoundException { JobID jobID = request.getPathParameter(JobIDPathParameter.class); JobVertexID jobVertexID = request.getPathParameter(JobVertexIdPathParameter.class); AccessExecutionJobVertex jobVertex = executionGraph.getJobVertex(jobVertexID); + if (jobVertex == null) { + throw new NotFoundException(String.format("JobVertex %s not found", jobVertexID)); + } + List subtasks = new ArrayList<>(); final long now = System.currentTimeMillis(); int num = 0; From d2344c98759d6616455833cb3c1083fec12fc56a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Feb 2018 11:37:58 +0100 Subject: [PATCH 09/15] [FLINK-8662] [tests] Harden FutureUtilsTest#testRetryWithDelay This commit moves the start of the time measurement before the triggering of the retry with delay operation. This closes #5494. --- .../flink/runtime/concurrent/FutureUtils.java | 5 ++- .../runtime/concurrent/FutureUtilsTest.java | 36 ++++++++----------- .../ManuallyTriggeredScheduledExecutor.java | 6 ++++ 3 files changed, 22 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index 17381a90f16ef..e9310c0115063 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -187,7 +187,7 @@ private static void retryOperationWithDelay( if (!resultFuture.isDone()) { final CompletableFuture operationResultFuture = operation.get(); - operationResultFuture.whenCompleteAsync( + operationResultFuture.whenComplete( (t, throwable) -> { if (throwable != null) { if (throwable instanceof CancellationException) { @@ -213,8 +213,7 @@ private static void retryOperationWithDelay( } else { resultFuture.complete(t); } - }, - scheduledExecutor); + }); resultFuture.whenComplete( (t, throwable) -> operationResultFuture.cancel(false)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java index 7d14ff28a1c05..cbc8a9abdb3ce 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java @@ -26,8 +26,8 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; +import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; @@ -45,13 +45,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; /** * Tests for the utility methods in {@link FutureUtils}. @@ -175,9 +168,11 @@ public void testRetryWithDelayFailure() throws Throwable { @Test public void testRetryWithDelay() throws Exception { final int retries = 4; - final Time delay = Time.milliseconds(50L); + final Time delay = Time.milliseconds(5L); final AtomicInteger countDown = new AtomicInteger(retries); + long start = System.currentTimeMillis(); + CompletableFuture retryFuture = FutureUtils.retryWithDelay( () -> { if (countDown.getAndDecrement() == 0) { @@ -190,8 +185,6 @@ public void testRetryWithDelay() throws Exception { delay, TestingUtils.defaultScheduledExecutor()); - long start = System.currentTimeMillis(); - Boolean result = retryFuture.get(); long completionTime = System.currentTimeMillis() - start; @@ -205,29 +198,28 @@ public void testRetryWithDelay() throws Exception { */ @Test public void testRetryWithDelayCancellation() { - ScheduledFuture scheduledFutureMock = mock(ScheduledFuture.class); - ScheduledExecutor scheduledExecutorMock = mock(ScheduledExecutor.class); - doReturn(scheduledFutureMock).when(scheduledExecutorMock).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); - doAnswer( - (InvocationOnMock invocation) -> { - invocation.getArgumentAt(0, Runnable.class).run(); - return null; - }).when(scheduledExecutorMock).execute(any(Runnable.class)); + final ManuallyTriggeredScheduledExecutor scheduledExecutor = new ManuallyTriggeredScheduledExecutor(); CompletableFuture retryFuture = FutureUtils.retryWithDelay( () -> FutureUtils.completedExceptionally(new FlinkException("Test exception")), 1, TestingUtils.infiniteTime(), - scheduledExecutorMock); + scheduledExecutor); assertFalse(retryFuture.isDone()); - verify(scheduledExecutorMock).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + final Collection> scheduledTasks = scheduledExecutor.getScheduledTasks(); + + assertFalse(scheduledTasks.isEmpty()); + + final ScheduledFuture scheduledFuture = scheduledTasks.iterator().next(); + + assertFalse(scheduledFuture.isDone()); retryFuture.cancel(false); assertTrue(retryFuture.isCancelled()); - verify(scheduledFutureMock).cancel(anyBoolean()); + assertTrue(scheduledFuture.isCancelled()); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java index 1fc7705ac42e6..8a4658701f144 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java @@ -21,6 +21,8 @@ import org.apache.flink.core.testutils.ManuallyTriggeredDirectExecutor; import org.apache.flink.util.Preconditions; +import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -62,6 +64,10 @@ public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialD return insertRunnable(command, true); } + Collection> getScheduledTasks() { + return new ArrayList<>(scheduledTasks); + } + /** * Triggers all registered tasks. */ From 9a00b3f20a904b5e93b9c48ea11950f715a530f4 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 7 Feb 2018 18:21:06 +0100 Subject: [PATCH 10/15] [hotfix] [yarn] Write number of slots to configuration --- .../apache/flink/yarn/AbstractYarnClusterDescriptor.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 583ee54748553..d3fce4c8f05fd 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -482,7 +482,7 @@ protected ClusterClient deployInternal( flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); ApplicationReport report = startAppMaster( - flinkConfiguration, + new Configuration(flinkConfiguration), yarnClusterEntrypoint, jobGraph, yarnClient, @@ -794,6 +794,10 @@ public ApplicationReport startAppMaster( homeDir, ""); + configuration.setInteger( + ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, + clusterSpecification.getSlotsPerTaskManager()); + // Upload the flink configuration // write out configuration file File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); From cc91b6ac0f67093beb9c17c6fa1beb16a0380b3b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 7 Feb 2018 18:58:32 +0100 Subject: [PATCH 11/15] [hotfix] [yarn] Remove unnecessary TaskManager configuration generation --- .../flink/yarn/YarnResourceManager.java | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 305a2f575f75c..d07ac5a66a5ab 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -20,9 +20,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -60,9 +58,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; - -import scala.concurrent.duration.FiniteDuration; /** * The yarn implementation of the resource manager. Used when the system is started @@ -443,17 +438,18 @@ private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource taskManagerParameters.taskManagerTotalMemoryMB(), taskManagerParameters.taskManagerHeapSizeMB(), taskManagerParameters.taskManagerDirectMemoryLimitMB()); - int timeout = flinkConfig.getInteger(TaskManagerOptions.MAX_REGISTRATION_DURATION.key(), - DEFAULT_TASK_MANAGER_REGISTRATION_DURATION); - FiniteDuration teRegistrationTimeout = new FiniteDuration(timeout, TimeUnit.SECONDS); - final Configuration taskManagerConfig = BootstrapTools.generateTaskManagerConfiguration( - flinkConfig, "", 0, 1, teRegistrationTimeout); - log.debug("TaskManager configuration: {}", taskManagerConfig); + + log.debug("TaskManager configuration: {}", flinkConfig); ContainerLaunchContext taskExecutorLaunchContext = Utils.createTaskExecutorContext( - flinkConfig, yarnConfig, env, - taskManagerParameters, taskManagerConfig, - currDir, YarnTaskExecutorRunner.class, log); + flinkConfig, + yarnConfig, + env, + taskManagerParameters, + flinkConfig, + currDir, + YarnTaskExecutorRunner.class, + log); // set a special environment variable to uniquely identify this container taskExecutorLaunchContext.getEnvironment() From 9c820e91730b517ca660275c80370714f4729e36 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 8 Feb 2018 10:27:27 +0100 Subject: [PATCH 12/15] [hotfix] Only log retrying exception on debug in RetryingRegistration --- .../registration/RetryingRegistration.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java index 279714b559f32..69597292ffcb1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java @@ -207,7 +207,22 @@ public void startRegistration() { resourceManagerAcceptFuture.whenCompleteAsync( (Void v, Throwable failure) -> { if (failure != null && !canceled) { - log.warn("Could not resolve {} address {}, retrying in {} ms", targetName, targetAddress, delayOnError, failure); + final Throwable strippedFailure = ExceptionUtils.stripCompletionException(failure); + if (log.isDebugEnabled()) { + log.debug( + "Could not resolve {} address {}, retrying in {} ms.", + targetName, + targetAddress, + delayOnError, + strippedFailure); + } else { + log.info( + "Could not resolve {} address {}, retrying in {} ms: {}.", + targetName, + targetAddress, + delayOnError, + strippedFailure.getMessage()); + } startRegistrationLater(delayOnError); } From 633c78699083a21ca0578d5da34042f8c3368292 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 30 Jan 2018 09:22:03 +0100 Subject: [PATCH 13/15] [FLINK-8614] [flip6] Activate Flip-6 mode per default This commit enables the Flip-6 mode per default. Additionally, it disables some of the Yarn tests which no longer apply to Flip-6 (tests which wait for a number of started TM container without a job submission). This closes #5437. --- .../flink/configuration/CoreOptions.java | 12 ++++----- .../configuration/JobManagerOptions.java | 2 +- flink-dist/src/main/flink-bin/bin/config.sh | 2 +- .../runtime/jobmaster/JobManagerRunner.java | 2 +- .../flink/runtime/jobmaster/JobMaster.java | 27 ++++++++++--------- .../apache/flink/runtime/rest/RestClient.java | 2 +- .../yarn/YARNHighAvailabilityITCase.java | 3 +++ .../YARNSessionCapacitySchedulerITCase.java | 12 ++++++--- .../flink/yarn/YARNSessionFIFOITCase.java | 10 ++++--- .../org/apache/flink/yarn/YarnTestBase.java | 4 +++ .../flink/yarn/YarnResourceManager.java | 2 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 2 +- 12 files changed, 47 insertions(+), 33 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index c44219f41a0d8..9bd7fab0ee492 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -229,16 +229,16 @@ public static ConfigOption fileSystemConnectionLimitStreamInactivityTimeou // Distributed architecture // ------------------------------------------------------------------------ + /** + * Constant value for the Flip-6 execution mode. + */ + public static final String FLIP6_MODE = "flip6"; + /** * Switch to select the execution mode. Possible values are 'flip6' and 'old'. */ public static final ConfigOption MODE = ConfigOptions .key("mode") - .defaultValue("old") + .defaultValue(FLIP6_MODE) .withDescription("Switch to select the execution mode. Possible values are 'flip6' and 'old'."); - - /** - * Constant value for the Flip-6 execution mode. - */ - public static final String FLIP6_MODE = "flip6"; } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index 74ec5b7c1ba83..ade3958f131ed 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -140,7 +140,7 @@ public class JobManagerOptions { public static final ConfigOption SLOT_IDLE_TIMEOUT = key("slot.idle.timeout") - .defaultValue(20L * 1000L) + .defaultValue(10L * 1000L) .withDescription("The timeout in milliseconds for a idle slot in Slot Pool."); // --------------------------------------------------------------------------------------------- diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index 3e4e2c1370de4..09ec6da85f0df 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -264,7 +264,7 @@ fi # Define FLIP if it is not already set if [ -z "${FLINK_MODE}" ]; then - FLINK_MODE=$(readFromConfig ${KEY_FLINK_MODE} "old" "${YAML_CONF}") + FLINK_MODE=$(readFromConfig ${KEY_FLINK_MODE} "flip6" "${YAML_CONF}") fi diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 5e476bf2e3f63..5264093228aad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -410,7 +410,7 @@ public void revokeLeadership() { log.info("JobManager for job {} ({}) was revoked leadership at {}.", jobGraph.getName(), jobGraph.getJobID(), getAddress()); - CompletableFuture suspendFuture = jobManager.suspend(new Exception("JobManager is no longer the leader."), rpcTimeout); + CompletableFuture suspendFuture = jobManager.suspend(new FlinkException("JobManager is no longer the leader."), rpcTimeout); suspendFuture.whenCompleteAsync( (Acknowledge ack, Throwable throwable) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index aeac2df60737b..139c0538e8a25 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -347,7 +347,7 @@ public CompletableFuture start(final JobMasterId newJobMasterId, fi * @param timeout for this operation * @return Future acknowledge indicating that the job has been suspended. Otherwise the future contains an exception */ - public CompletableFuture suspend(final Throwable cause, final Time timeout) { + public CompletableFuture suspend(final Exception cause, final Time timeout) { CompletableFuture suspendFuture = callAsyncWithoutFencing(() -> suspendExecution(cause), timeout); stop(); @@ -375,7 +375,7 @@ public void postStop() throws Exception { resourceManagerHeartbeatManager.stop(); // make sure there is a graceful exit - suspendExecution(new Exception("JobManager is shutting down.")); + suspendExecution(new FlinkException("JobManager is shutting down.")); // shut down will internally release all registered slots slotPool.shutDown(); @@ -595,14 +595,11 @@ public void declineCheckpoint( final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); if (checkpointCoordinator != null) { - getRpcService().execute(new Runnable() { - @Override - public void run() { - try { - checkpointCoordinator.receiveDeclineMessage(decline); - } catch (Exception e) { - log.error("Error in CheckpointCoordinator while processing {}", decline, e); - } + getRpcService().execute(() -> { + try { + checkpointCoordinator.receiveDeclineMessage(decline); + } catch (Exception e) { + log.error("Error in CheckpointCoordinator while processing {}", decline, e); } }); } else { @@ -915,7 +912,7 @@ private Acknowledge startJobExecution(JobMasterId newJobMasterId) throws Excepti * * @param cause The reason of why this job been suspended. */ - private Acknowledge suspendExecution(final Throwable cause) { + private Acknowledge suspendExecution(final Exception cause) { validateRunsInMainThread(); if (getFencingToken() == null) { @@ -939,7 +936,7 @@ private Acknowledge suspendExecution(final Throwable cause) { slotPoolGateway.suspend(); // disconnect from resource manager: - closeResourceManagerConnection(new Exception("Execution was suspended.", cause)); + closeResourceManagerConnection(cause); return Acknowledge.get(); } @@ -1037,7 +1034,11 @@ public void requestHeartbeat(ResourceID resourceID, Void payload) { private void closeResourceManagerConnection(Exception cause) { if (resourceManagerConnection != null) { - log.info("Close ResourceManager connection {}.", resourceManagerConnection.getResourceManagerResourceID(), cause); + if (log.isDebugEnabled()) { + log.debug("Close ResourceManager connection {}.", resourceManagerConnection.getResourceManagerResourceID(), cause); + } else { + log.info("Close ResourceManager connection {}: {}.", resourceManagerConnection.getResourceManagerResourceID(), cause.getMessage()); + } resourceManagerHeartbeatManager.unmonitorTarget(resourceManagerConnection.getResourceManagerResourceID()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java index 43a99818401bd..42612eaa34b86 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java @@ -151,7 +151,7 @@ public , U extends MessageParameters, R extend String targetUrl = MessageParameters.resolveUrl(messageHeaders.getTargetRestEndpointURL(), messageParameters); - LOG.debug("Sending request of class {} to {}", request.getClass(), targetUrl); + LOG.debug("Sending request of class {} to {}:{}{}", request.getClass(), targetAddress, targetPort, targetUrl); // serialize payload StringWriter sw = new StringWriter(); objectMapper.writeValue(sw, request); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index 5a5fb742b95b9..05be03a175561 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -55,6 +55,8 @@ import scala.concurrent.duration.FiniteDuration; +import static org.junit.Assume.assumeTrue; + /** * Tests that verify correct HA behavior. */ @@ -104,6 +106,7 @@ public static void teardown() throws Exception { */ @Test public void testMultipleAMKill() throws Exception { + assumeTrue("This test only works with the old actor based code.", !flip6); final int numberKillingAttempts = numberApplicationAttempts - 1; String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); final Configuration configuration = GlobalConfiguration.loadConfiguration(); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index c806c5e82b0cb..3a674ad082989 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -78,6 +78,7 @@ import static junit.framework.TestCase.assertTrue; import static org.apache.flink.yarn.UtilsTest.addTestAppender; import static org.apache.flink.yarn.UtilsTest.checkForLogString; +import static org.junit.Assume.assumeTrue; /** * This test starts a MiniYARNCluster with a CapacityScheduler. @@ -101,6 +102,7 @@ public static void setup() { */ @Test public void testClientStartup() throws IOException { + assumeTrue("Flip-6 does not start TMs upfront.", !flip6); LOG.info("Starting testClientStartup()"); runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), "-n", "1", @@ -130,7 +132,7 @@ public void perJobYarnCluster() throws IOException { "-yjm", "768", "-ytm", "1024", exampleJarLocation.getAbsolutePath()}, /* test succeeded after this string */ - "Job execution complete", + "Program execution finished", /* prohibited strings: (to verify the parallelism) */ // (we should see "DataSink (...) (1/2)" and "DataSink (...) (2/2)" instead) new String[]{"DataSink \\(.*\\) \\(1/1\\) switched to FINISHED"}, @@ -177,7 +179,7 @@ public void perJobYarnClusterOffHeap() throws IOException { "-yD", "taskmanager.memory.size=" + offHeapMemory, "-yD", "taskmanager.memory.preallocate=true", exampleJarLocation.getAbsolutePath()}, /* test succeeded after this string */ - "Job execution complete", + "Program execution finished", /* prohibited strings: (to verify the parallelism) */ // (we should see "DataSink (...) (1/2)" and "DataSink (...) (2/2)" instead) new String[]{"DataSink \\(.*\\) \\(1/1\\) switched to FINISHED"}, @@ -190,6 +192,7 @@ public void perJobYarnClusterOffHeap() throws IOException { */ @Test(timeout = 100000) // timeout after 100 seconds public void testTaskManagerFailure() throws Exception { + assumeTrue("Flip-6 does not start TMs upfront.", !flip6); LOG.info("Starting testTaskManagerFailure()"); Runner runner = startWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), "-n", "1", @@ -402,7 +405,7 @@ public void perJobYarnClusterWithParallelism() throws IOException { "-yjm", "768", "-ytm", "1024", exampleJarLocation.getAbsolutePath()}, /* test succeeded after this string */ - "Job execution complete", + "Program execution finished", /* prohibited strings: (we want to see "DataSink (...) (2/2) switched to FINISHED") */ new String[]{"DataSink \\(.*\\) \\(1/1\\) switched to FINISHED"}, RunTypes.CLI_FRONTEND, 0, true); @@ -477,7 +480,8 @@ private void testDetachedPerJobYarnClusterInternal(String job) throws IOExceptio "-yD", "yarn.tags=test-tag", "-ytm", "1024", "-ys", "2", // test requesting slots from YARN. - "--yarndetached", job, + "-p", "2", + "--detached", job, "--input", tmpInFile.getAbsoluteFile().toString(), "--output", tmpOutFolder.getAbsoluteFile().toString()}, "Job has been submitted with JobID", diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index 975dd28e1d5aa..e54518793526d 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -98,10 +98,12 @@ public void testDetachedMode() throws InterruptedException, IOException { runner.join(); checkForLogString("The Flink YARN client has been started in detached mode"); - LOG.info("Waiting until two containers are running"); - // wait until two containers are running - while (getRunningContainers() < 2) { - sleep(500); + if (!flip6) { + LOG.info("Waiting until two containers are running"); + // wait until two containers are running + while (getRunningContainers() < 2) { + sleep(500); + } } //additional sleep for the JM/TM to start and establish connection diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index eeda32d9d899c..c863e145f05e9 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -20,6 +20,7 @@ import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.Preconditions; @@ -187,6 +188,7 @@ public void sleep() { private YarnClient yarnClient = null; protected org.apache.flink.configuration.Configuration flinkConfiguration; + protected boolean flip6; @Before public void checkClusterEmpty() throws IOException, YarnException { @@ -207,6 +209,8 @@ public void checkClusterEmpty() throws IOException, YarnException { } flinkConfiguration = new org.apache.flink.configuration.Configuration(); + + flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE)); } /** diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index d07ac5a66a5ab..87324cba91a3b 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -432,7 +432,7 @@ private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource final ContaineredTaskManagerParameters taskManagerParameters = ContaineredTaskManagerParameters.create(flinkConfig, resource.getMemory(), 1); - log.info("TaskExecutor{} will be started with container size {} MB, JVM heap size {} MB, " + + log.info("TaskExecutor {} will be started with container size {} MB, JVM heap size {} MB, " + "JVM direct memory limit {} MB", containerId, taskManagerParameters.taskManagerTotalMemoryMB(), diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index a289b66cbf84a..8882236452913 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -365,7 +365,7 @@ private AbstractYarnClusterDescriptor createDescriptor( } private ClusterSpecification createClusterSpecification(Configuration configuration, CommandLine cmd) { - if (!cmd.hasOption(container.getOpt())) { // number of containers is required option! + if (!flip6 && !cmd.hasOption(container.getOpt())) { // number of containers is required option! LOG.error("Missing required argument {}", container.getOpt()); printUsage(); throw new IllegalArgumentException("Missing required argument " + container.getOpt()); From 457ffa80eeef5b360644b6e97b4cb32dc1975471 Mon Sep 17 00:00:00 2001 From: gyao Date: Fri, 16 Feb 2018 15:00:23 +0100 Subject: [PATCH 14/15] [hotfix] Remove unused code in JarActionHandler --- .../handlers/legacy/JarActionHandler.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/legacy/JarActionHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/legacy/JarActionHandler.java index de48e366f71ba..b63188a3ad38f 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/legacy/JarActionHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/legacy/JarActionHandler.java @@ -36,15 +36,10 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.rest.handler.legacy.AbstractJsonRequestHandler; -import org.apache.flink.runtime.rest.handler.legacy.JsonFactory; import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor; -import org.apache.flink.util.ExceptionUtils; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import java.io.File; import java.io.IOException; -import java.io.StringWriter; import java.net.URISyntaxException; import java.net.URL; import java.util.ArrayList; @@ -109,18 +104,6 @@ protected Tuple2 getJobGraphAndClassLoader(JarActionHandl return Tuple2.of(graph, classLoader); } - protected String sendError(Exception e) throws Exception { - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - - gen.writeStartObject(); - gen.writeStringField("error", ExceptionUtils.stringifyException(e)); - gen.writeEndObject(); - gen.close(); - - return writer.toString(); - } - /** * Wrapper for all configuration that is parsed from query and path args. */ From 274bdc2803511289093f9a2310367727a94479ee Mon Sep 17 00:00:00 2001 From: gyao Date: Fri, 16 Feb 2018 15:15:19 +0100 Subject: [PATCH 15/15] [FLINK-7715][flip6] Implement JarRunHandler --- .../apache/flink/client/cli/CliFrontend.java | 2 +- .../flink/client/program/PackagedProgram.java | 4 +- .../AllowNonRestoredStateQueryParameter.java | 45 ++++ .../handlers/EntryClassQueryParameter.java | 31 +++ .../handlers/JarIdPathParameter.java | 44 ++++ .../webmonitor/handlers/JarRunHandler.java | 229 ++++++++++++++++++ .../webmonitor/handlers/JarRunHeaders.java | 60 +++++ .../handlers/JarRunMessageParameters.java | 60 +++++ .../handlers/JarRunResponseBody.java | 54 +++++ .../handlers/ParallelismQueryParameter.java | 44 ++++ .../handlers/ProgramArgsQueryParameter.java | 33 +++ .../handlers/SavepointPathQueryParameter.java | 31 +++ .../handlers/StringQueryParameter.java | 42 ++++ .../webmonitor/WebMonitorUtilsTest.java | 7 +- ...lowNonRestoredStateQueryParameterTest.java | 46 ++++ .../handlers/JarRunHandlerTest.java | 99 ++++++++ .../handlers/JarRunResponseBodyTest.java | 47 ++++ .../ParallelismQueryParameterTest.java | 42 ++++ .../scripts/modules/submit/submit.ctrl.coffee | 26 +- .../web-dashboard/web/js/hs/index.js | 4 +- .../web-dashboard/web/js/index.js | 4 +- .../dispatcher/DispatcherRestEndpoint.java | 3 +- .../runtime/webmonitor/WebMonitorUtils.java | 33 ++- 23 files changed, 972 insertions(+), 18 deletions(-) create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarIdPathParameter.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunMessageParameters.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBody.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ProgramArgsQueryParameter.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SavepointPathQueryParameter.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBodyTest.java create mode 100644 flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index 23e82bcec8713..2d984d674fe88 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -1126,7 +1126,7 @@ public static List> loadCustomCommandLines(Configuration co * @return JobGraph extracted from the PackagedProgram * @throws ProgramInvocationException if the JobGraph generation failed */ - private static JobGraph createJobGraph(Configuration configuration, PackagedProgram packagedProgram, int defaultParallelism) throws ProgramInvocationException { + public static JobGraph createJobGraph(Configuration configuration, PackagedProgram packagedProgram, int defaultParallelism) throws ProgramInvocationException { Thread.currentThread().setContextClassLoader(packagedProgram.getUserCodeClassLoader()); final Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration); final FlinkPlan flinkPlan; diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java index 2d8cb1dacb95a..d81cacbb4f44d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java @@ -145,7 +145,7 @@ public PackagedProgram(File jarFile, List classpaths, String... args) throw * This invocation is thrown if the Program can't be properly loaded. Causes * may be a missing / wrong class or manifest files. */ - public PackagedProgram(File jarFile, String entryPointClassName, String... args) throws ProgramInvocationException { + public PackagedProgram(File jarFile, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { this(jarFile, Collections.emptyList(), entryPointClassName, args); } @@ -168,7 +168,7 @@ public PackagedProgram(File jarFile, String entryPointClassName, String... args) * This invocation is thrown if the Program can't be properly loaded. Causes * may be a missing / wrong class or manifest files. */ - public PackagedProgram(File jarFile, List classpaths, String entryPointClassName, String... args) throws ProgramInvocationException { + public PackagedProgram(File jarFile, List classpaths, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { if (jarFile == null) { throw new IllegalArgumentException("The jar file must not be null."); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java new file mode 100644 index 0000000000000..7ad014ea685f8 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java @@ -0,0 +1,45 @@ +/* + * 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.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; + +/** + * Query parameter that specifies whether non restored state is allowed if the savepoint + * contains state for an operator that is not part of the job. + * + * @see SavepointRestoreSettings#allowNonRestoredState() + */ +public class AllowNonRestoredStateQueryParameter extends MessageQueryParameter { + + protected AllowNonRestoredStateQueryParameter() { + super("allowNonRestoredState", MessageParameterRequisiteness.OPTIONAL); + } + + @Override + public Boolean convertValueFromString(final String value) { + return Boolean.valueOf(value); + } + + @Override + public String convertStringToValue(final Boolean value) { + return value.toString(); + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java new file mode 100644 index 0000000000000..05298a6b01af9 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java @@ -0,0 +1,31 @@ +/* + * 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 java.io.File; + +/** + * Query parameter specifying the name of the entry point class. + * @see org.apache.flink.client.program.PackagedProgram#PackagedProgram(File, String, String...) + */ +public class EntryClassQueryParameter extends StringQueryParameter { + public EntryClassQueryParameter() { + super("entry-class", MessageParameterRequisiteness.OPTIONAL); + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarIdPathParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarIdPathParameter.java new file mode 100644 index 0000000000000..d4bd602f485c9 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarIdPathParameter.java @@ -0,0 +1,44 @@ +/* + * 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.rest.messages.ConversionException; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; + +/** + * Path parameter to identify uploaded jar files. + */ +public class JarIdPathParameter extends MessagePathParameter { + + public static final String KEY = "jarid"; + + protected JarIdPathParameter() { + super(KEY); + } + + @Override + protected String convertFromString(final String value) throws ConversionException { + return value; + } + + @Override + protected String convertToString(final String value) { + return value; + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java new file mode 100644 index 0000000000000..6c4ebeade9ab7 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java @@ -0,0 +1,229 @@ +/* + * 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.annotation.VisibleForTesting; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.rest.handler.AbstractRestHandler; +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.MessageHeaders; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static java.util.Objects.requireNonNull; +import static org.apache.flink.shaded.guava18.com.google.common.base.Strings.emptyToNull; + +/** + * Handler to submit jobs uploaded via the Web UI. + */ +public class JarRunHandler extends + AbstractRestHandler { + + private static final Pattern ARGUMENTS_TOKENIZE_PATTERN = Pattern.compile("([^\"\']\\S*|\".+?\"|\'.+?\')\\s*"); + + private final Path jarDir; + + private final Configuration configuration; + + private final Executor executor; + + private final RestClusterClient restClusterClient; + + public JarRunHandler( + final CompletableFuture localRestAddress, + final GatewayRetriever leaderRetriever, + final Time timeout, + final Map responseHeaders, + final MessageHeaders messageHeaders, + final Path jarDir, + final Configuration configuration, + final Executor executor) { + super(localRestAddress, leaderRetriever, timeout, responseHeaders, messageHeaders); + + this.jarDir = requireNonNull(jarDir); + this.configuration = requireNonNull(configuration); + this.executor = requireNonNull(executor); + try { + this.restClusterClient = new RestClusterClient<>(configuration, ""); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + protected CompletableFuture handleRequest( + @Nonnull final HandlerRequest request, + @Nonnull final DispatcherGateway gateway) throws RestHandlerException { + + final String pathParameter = request.getPathParameter(JarIdPathParameter.class); + final Path jarFile = jarDir.resolve(pathParameter); + + final String entryClass = emptyToNull(getQueryParameter(request, EntryClassQueryParameter.class)); + final List programArgs = tokenizeArguments(getQueryParameter(request, ProgramArgsQueryParameter.class)); + final int parallelism = getQueryParameter(request, ParallelismQueryParameter.class, -1); + final SavepointRestoreSettings savepointRestoreSettings = getSavepointRestoreSettings(request); + + final CompletableFuture jobGraphFuture = getJobGraphAsync( + jarFile, + entryClass, + programArgs, + savepointRestoreSettings, + parallelism); + + return jobGraphFuture.thenCompose(jobGraph -> restClusterClient + .submitJob(jobGraph) + .thenApply((jobSubmitResponseBody -> new JarRunResponseBody(jobGraph.getJobID())))) + .exceptionally(throwable -> { + throw new CompletionException(new RestHandlerException( + throwable.getMessage(), + HttpResponseStatus.INTERNAL_SERVER_ERROR, + throwable)); + }); + } + + private static SavepointRestoreSettings getSavepointRestoreSettings( + final @Nonnull HandlerRequest request) + throws RestHandlerException { + + final boolean allowNonRestoredState = getQueryParameter(request, AllowNonRestoredStateQueryParameter.class, false); + final String savepointPath = getQueryParameter(request, SavepointPathQueryParameter.class); + final SavepointRestoreSettings savepointRestoreSettings; + if (savepointPath != null) { + savepointRestoreSettings = SavepointRestoreSettings.forPath( + savepointPath, + allowNonRestoredState); + } else { + savepointRestoreSettings = SavepointRestoreSettings.none(); + } + return savepointRestoreSettings; + } + + private CompletableFuture getJobGraphAsync( + final Path jarFile, + @Nullable final String entryClass, + final List programArgs, + final SavepointRestoreSettings savepointRestoreSettings, + final int parallelism) { + + return CompletableFuture.supplyAsync(() -> { + if (!Files.exists(jarFile)) { + throw new CompletionException(new RestHandlerException( + String.format("Jar file %s does not exist", jarFile), HttpResponseStatus.BAD_REQUEST)); + } + + final JobGraph jobGraph; + try { + final PackagedProgram packagedProgram = new PackagedProgram( + jarFile.toFile(), + entryClass, + programArgs.toArray(new String[programArgs.size()])); + jobGraph = CliFrontend.createJobGraph(configuration, packagedProgram, parallelism); + } catch (final ProgramInvocationException e) { + throw new CompletionException(e); + } + jobGraph.setSavepointRestoreSettings(savepointRestoreSettings); + return jobGraph; + }, executor); + } + + /** + * Takes program arguments as a single string, and splits them into a list of string. + * + *

+	 * tokenizeArguments("--foo bar")            = ["--foo" "bar"]
+	 * tokenizeArguments("--foo \"bar baz\"")    = ["--foo" "bar baz"]
+	 * tokenizeArguments("--foo 'bar baz'")      = ["--foo" "bar baz"]
+	 * 
+ * + * WARNING: This method does not respect escaped quotes. + */ + @VisibleForTesting + static List tokenizeArguments(@Nullable final String args) { + if (args == null) { + return Collections.emptyList(); + } + final Matcher matcher = ARGUMENTS_TOKENIZE_PATTERN.matcher(args); + final List tokens = new ArrayList<>(); + while (matcher.find()) { + tokens.add(matcher.group() + .trim() + .replace("\"", "") + .replace("\'", "")); + } + return tokens; + } + + /** + * Returns the value of a query parameter, or {@code null} if the query parameter is not set. + * @throws RestHandlerException If the query parameter is repeated. + */ + @VisibleForTesting + static > X getQueryParameter( + final HandlerRequest request, + final Class

queryParameterClass) throws RestHandlerException { + return getQueryParameter(request, queryParameterClass, null); + } + + @VisibleForTesting + static > X getQueryParameter( + final HandlerRequest request, + final Class

queryParameterClass, + final X defaultValue) throws RestHandlerException { + + final List values = request.getQueryParameter(queryParameterClass); + final X value; + if (values.size() > 1) { + throw new RestHandlerException( + String.format("Expected only one value %s.", values), + HttpResponseStatus.BAD_REQUEST); + } else if (values.size() == 1) { + value = values.get(0); + } else { + value = defaultValue; + } + return value; + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java new file mode 100644 index 0000000000000..a6d7ecbce55b4 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java @@ -0,0 +1,60 @@ +/* + * 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.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.MessageHeaders; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * {@link MessageHeaders} for {@link JarRunHandler}. + */ +public class JarRunHeaders implements MessageHeaders { + @Override + public Class getResponseClass() { + return JarRunResponseBody.class; + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public JarRunMessageParameters getUnresolvedMessageParameters() { + return new JarRunMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.POST; + } + + @Override + public String getTargetRestEndpointURL() { + return "/jars/:" + JarIdPathParameter.KEY + "/run"; + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunMessageParameters.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunMessageParameters.java new file mode 100644 index 0000000000000..2d9428cee8cdd --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunMessageParameters.java @@ -0,0 +1,60 @@ +/* + * 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.rest.messages.MessageParameters; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +/** + * {@link MessageParameters} for {@link JarRunHandler}. + */ +public class JarRunMessageParameters extends MessageParameters { + + private final JarIdPathParameter jarIdPathParameter = new JarIdPathParameter(); + + private final ProgramArgsQueryParameter programArgsQueryParameter = new ProgramArgsQueryParameter(); + + private final EntryClassQueryParameter entryClassQueryParameter = new EntryClassQueryParameter(); + + private final ParallelismQueryParameter parallelismQueryParameter = new ParallelismQueryParameter(); + + private final AllowNonRestoredStateQueryParameter allowNonRestoredStateQueryParameter = new AllowNonRestoredStateQueryParameter(); + + private final SavepointPathQueryParameter savepointPathQueryParameter = new SavepointPathQueryParameter(); + + @Override + public Collection> getPathParameters() { + return Collections.singletonList(jarIdPathParameter); + } + + @Override + public Collection> getQueryParameters() { + return Collections.unmodifiableCollection(Arrays.asList( + programArgsQueryParameter, + entryClassQueryParameter, + parallelismQueryParameter, + allowNonRestoredStateQueryParameter, + savepointPathQueryParameter)); + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBody.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBody.java new file mode 100644 index 0000000000000..1195249a888a6 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBody.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.api.common.JobID; +import org.apache.flink.runtime.rest.messages.ResponseBody; +import org.apache.flink.runtime.rest.messages.json.JobIDDeserializer; +import org.apache.flink.runtime.rest.messages.json.JobIDSerializer; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; + +import static java.util.Objects.requireNonNull; + +/** + * Response for {@link JarRunHandler}. + */ +public class JarRunResponseBody implements ResponseBody { + + @JsonProperty("jobid") + @JsonDeserialize(using = JobIDDeserializer.class) + @JsonSerialize(using = JobIDSerializer.class) + private final JobID jobId; + + @JsonCreator + public JarRunResponseBody( + @JsonProperty("jobid") + @JsonDeserialize(using = JobIDDeserializer.class) final JobID jobId) { + this.jobId = requireNonNull(jobId); + } + + public JobID getJobId() { + return jobId; + } + +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java new file mode 100644 index 0000000000000..26cb16c7b0e33 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java @@ -0,0 +1,44 @@ +/* + * 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.rest.messages.MessageQueryParameter; + +/** + * Query parameter specifying the parallelism of the job. + * @see JarRunHandler + */ +public class ParallelismQueryParameter extends MessageQueryParameter { + + private static final String KEY = "parallelism"; + + public ParallelismQueryParameter() { + super(KEY, MessageParameterRequisiteness.OPTIONAL); + } + + @Override + public Integer convertValueFromString(final String value) { + return Integer.valueOf(value); + } + + @Override + public String convertStringToValue(final Integer value) { + return value.toString(); + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ProgramArgsQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ProgramArgsQueryParameter.java new file mode 100644 index 0000000000000..8f546ab1d4d92 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ProgramArgsQueryParameter.java @@ -0,0 +1,33 @@ +/* + * 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 java.io.File; + +/** + * Query parameter specifying the arguments for the program. + * @see org.apache.flink.client.program.PackagedProgram#PackagedProgram(File, String, String...) + */ +public class ProgramArgsQueryParameter extends StringQueryParameter { + + public ProgramArgsQueryParameter() { + super("program-args", MessageParameterRequisiteness.OPTIONAL); + } + +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SavepointPathQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SavepointPathQueryParameter.java new file mode 100644 index 0000000000000..c1c6d2b55cd7f --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SavepointPathQueryParameter.java @@ -0,0 +1,31 @@ +/* + * 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; + +/** + * Query parameter that specifies the savepoint to restore from. + */ +public class SavepointPathQueryParameter extends StringQueryParameter { + + public static final String KEY = "savepointPath"; + + public SavepointPathQueryParameter() { + super(KEY, MessageParameterRequisiteness.OPTIONAL); + } +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java new file mode 100644 index 0000000000000..226c5929c5693 --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java @@ -0,0 +1,42 @@ +/* + * 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.rest.messages.MessageQueryParameter; + +/** + * Abstract super class for {@link String} query parameters. + */ +public abstract class StringQueryParameter extends MessageQueryParameter { + + public StringQueryParameter(final String key, final MessageParameterRequisiteness requisiteness) { + super(key, requisiteness); + } + + @Override + public final String convertValueFromString(final String value) { + return value; + } + + @Override + public final String convertStringToValue(final String value) { + return value; + } + +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorUtilsTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorUtilsTest.java index 772584a822221..106fa7ccac8f5 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorUtilsTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorUtilsTest.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.webmonitor; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.webmonitor.handlers.JarUploadHandler; import org.apache.flink.runtime.webmonitor.history.JsonArchivist; @@ -55,12 +57,15 @@ public void testGetArchivers() { */ @Test public void testTryLoadJarHandlers() { + final Configuration configuration = new Configuration(); + configuration.setString(JobManagerOptions.ADDRESS, "localhost"); assertThat(WebMonitorUtils.tryLoadJarHandlers( CompletableFuture::new, CompletableFuture.completedFuture("localhost:12345"), Time.seconds(10), Collections.emptyMap(), Paths.get("/tmp"), - Executors.directExecutor()), not(empty())); + Executors.directExecutor(), + configuration), not(empty())); } } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java new file mode 100644 index 0000000000000..80fba4b903cc8 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java @@ -0,0 +1,46 @@ +/* + * 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.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link AllowNonRestoredStateQueryParameter}. + */ +public class AllowNonRestoredStateQueryParameterTest { + + private final AllowNonRestoredStateQueryParameter allowNonRestoredStateQueryParameter = + new AllowNonRestoredStateQueryParameter(); + + @Test + public void testConvertStringToValue() { + assertEquals("false", allowNonRestoredStateQueryParameter.convertStringToValue(false)); + assertEquals("true", allowNonRestoredStateQueryParameter.convertStringToValue(true)); + } + + @Test + public void testConvertValueFromString() { + assertEquals(false, allowNonRestoredStateQueryParameter.convertValueFromString("false")); + assertEquals(true, allowNonRestoredStateQueryParameter.convertValueFromString("true")); + assertEquals(true, allowNonRestoredStateQueryParameter.convertValueFromString("TRUE")); + } + +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java new file mode 100644 index 0000000000000..71c830f07cf73 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java @@ -0,0 +1,99 @@ +/* + * 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.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link JarRunHandler}. + */ +public class JarRunHandlerTest { + + @Test + public void testTokenizeNonQuoted() { + final List arguments = JarRunHandler.tokenizeArguments("--foo bar"); + assertThat(arguments.get(0), equalTo("--foo")); + assertThat(arguments.get(1), equalTo("bar")); + } + + @Test + public void testTokenizeSingleQuoted() { + final List arguments = JarRunHandler.tokenizeArguments("--foo 'bar baz '"); + assertThat(arguments.get(0), equalTo("--foo")); + assertThat(arguments.get(1), equalTo("bar baz ")); + } + + @Test + public void testTokenizeDoubleQuoted() { + final List arguments = JarRunHandler.tokenizeArguments("--name \"K. Bote \""); + assertThat(arguments.get(0), equalTo("--name")); + assertThat(arguments.get(1), equalTo("K. Bote ")); + } + + @Test + public void testGetQueryParameter() throws Exception { + final Boolean queryParameter = JarRunHandler.getQueryParameter( + new HandlerRequest<>( + EmptyRequestBody.getInstance(), + new JarRunMessageParameters(), + Collections.emptyMap(), + Collections.singletonMap("allowNonRestoredState", Collections.singletonList("true"))), + AllowNonRestoredStateQueryParameter.class); + assertThat(queryParameter, equalTo(true)); + } + + @Test + public void testGetQueryParameterRepeated() throws Exception { + try { + JarRunHandler.getQueryParameter( + new HandlerRequest<>( + EmptyRequestBody.getInstance(), + new JarRunMessageParameters(), + Collections.emptyMap(), + Collections.singletonMap("allowNonRestoredState", Arrays.asList("true", "false"))), + AllowNonRestoredStateQueryParameter.class); + } catch (final RestHandlerException e) { + assertThat(e.getMessage(), containsString("Expected only one value")); + } + } + + @Test + public void testGetQueryParameterDefaultValue() throws Exception { + final Boolean allowNonRestoredState = JarRunHandler.getQueryParameter( + new HandlerRequest<>( + EmptyRequestBody.getInstance(), + new JarRunMessageParameters(), + Collections.emptyMap(), + Collections.singletonMap("allowNonRestoredState", Collections.emptyList())), + AllowNonRestoredStateQueryParameter.class, true); + assertThat(allowNonRestoredState, equalTo(true)); + } +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBodyTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBodyTest.java new file mode 100644 index 0000000000000..92497c64d09e2 --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunResponseBodyTest.java @@ -0,0 +1,47 @@ +/* + * 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.api.common.JobID; +import org.apache.flink.runtime.rest.messages.RestResponseMarshallingTestBase; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link JarRunResponseBody}. + */ +public class JarRunResponseBodyTest extends RestResponseMarshallingTestBase { + + @Override + protected Class getTestResponseClass() { + return JarRunResponseBody.class; + } + + @Override + protected JarRunResponseBody getTestResponseInstance() throws Exception { + return new JarRunResponseBody(new JobID()); + } + + @Override + protected void assertOriginalEqualsToUnmarshalled( + final JarRunResponseBody expected, + final JarRunResponseBody actual) { + assertEquals(expected.getJobId(), actual.getJobId()); + } +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java new file mode 100644 index 0000000000000..e073fb6e38ecf --- /dev/null +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java @@ -0,0 +1,42 @@ +/* + * 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.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link ParallelismQueryParameter}. + */ +public class ParallelismQueryParameterTest { + + private final ParallelismQueryParameter parallelismQueryParameter = new ParallelismQueryParameter(); + + @Test + public void testConvertStringToValue() { + assertEquals("42", parallelismQueryParameter.convertStringToValue(42)); + } + + @Test + public void testConvertValueFromString() { + assertEquals(42, (int) parallelismQueryParameter.convertValueFromString("42")); + } + +} diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee index bfd39b9ff4d34..4e14a6508f586 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/modules/submit/submit.ctrl.coffee @@ -98,14 +98,26 @@ angular.module('flinkApp') $scope.state['submit-button'] = "Submitting" $scope.state['plan-button'] = "Show Plan" $scope.error = null + + queryParameters = {} + + if $scope.state['entry-class'] + queryParameters['entry-class'] = $scope.state['entry-class'] + + if $scope.state.parallelism + queryParameters['parallelism'] = $scope.state['parallelism'] + + if $scope.state['program-args'] + queryParameters['program-args'] = $scope.state['program-args'] + + if $scope.state['savepointPath'] + queryParameters['savepointPath'] = $scope.state['savepointPath'] + + if $scope.state['allowNonRestoredState'] + queryParameters['allowNonRestoredState'] = $scope.state['allowNonRestoredState'] + JobSubmitService.runJob( - $scope.state.selected, { - 'entry-class': $scope.state['entry-class'], - parallelism: $scope.state.parallelism, - 'program-args': $scope.state['program-args'], - savepointPath: $scope.state['savepointPath'], - allowNonRestoredState: $scope.state['allowNonRestoredState'] - } + $scope.state.selected, queryParameters ).then (data) -> if action == $scope.state['action-time'] $scope.state['submit-button'] = "Submit" diff --git a/flink-runtime-web/web-dashboard/web/js/hs/index.js b/flink-runtime-web/web-dashboard/web/js/hs/index.js index 7cd45e19e3366..7260097f772a7 100644 --- a/flink-runtime-web/web-dashboard/web/js/hs/index.js +++ b/flink-runtime-web/web-dashboard/web/js/hs/index.js @@ -1,2 +1,2 @@ -angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).value("watermarksConfig",{noWatermark:-0x8000000000000000}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,i){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsController"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.watermarks",{url:"/watermarks",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.watermarks.html"}}}).state("single-job.plan.taskmanagers",{url:"/taskmanagers",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.taskmanagers.html",controller:"JobPlanTaskManagersController"}}}).state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.history.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).state("single-job.exceptions",{url:"/exceptions",views:{details:{templateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}),t.otherwise("/completed-jobs")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,s,a;return"undefined"==typeof e||null===e?"":(o=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),i=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===i?0===s?o+"ms":s+"s ":i+"m "+s+"s":t?n+"h "+i+"m":n+"h "+i+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,s).then(function(e){var t,n,i,o,s,a,l;i=NaN,l={},o=e.values;for(t in o)a=o[t],s=t.replace(".currentLowWatermark",""),l[s]=a,(isNaN(i)||au.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(s[i]=e,t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,s,a){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,s,a;s=r.children()[0],o=r.width(),angular.element(s).attr("width",o),a=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(s).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(s).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"

",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=J(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(h(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+J(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),s=0,u=v.length;s-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,a,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),s=i,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,s;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){ -return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,s){if(o.id===r.id&&(i.metrics[e][t].splice(s,1),s",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,s,a;return"undefined"==typeof e||null===e?"":(o=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),i=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===i?0===s?o+"ms":s+"s ":i+"m "+s+"s":t?n+"h "+i+"m":n+"h "+i+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,s).then(function(e){var t,n,i,o,s,a,l;i=NaN,l={},o=e.values;for(t in o)a=o[t],s=t.replace(".currentLowWatermark",""),l[s]=a,(isNaN(i)||au.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(s[i]=e,t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,s,a){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,s,a;s=r.children()[0],o=r.width(),angular.element(s).attr("width",o),a=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(s).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(s).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=J(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(h(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+J(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),s=0,u=v.length;s-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,a,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),s=i,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,s;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues() +}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,s){if(o.id===r.id&&(i.metrics[e][t].splice(s,1),s",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,a,s;return"undefined"==typeof e||null===e?"":(o=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),i=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===i?0===a?o+"ms":a+"s ":i+"m "+a+"s":t?n+"h "+i+"m":n+"h "+i+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+a+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,a).then(function(e){var t,n,i,o,a,s,l;i=NaN,l={},o=e.values;for(t in o)s=o[t],a=t.replace(".currentLowWatermark",""),l[a]=s,(isNaN(i)||su.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),a={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(a[i]=e,t>=n-1)return r.resolve(a)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,a,s){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,a,s;a=r.children()[0],o=r.width(),angular.element(a).attr("width",o),s=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(a).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:s(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(a).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,M,y,J;p=null,C=d3.behavior.zoom(),J=[],h=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},g=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":g(t)?"node-iteration":"node-normal"},a=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=y(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(g(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+y(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},y=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},s=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:a(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),a=0,u=v.length;a-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return a=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,s,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),a=i,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,a;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,a){if(o.id===r.id&&(i.metrics[e][t].splice(a,1),a",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,a,s;return"undefined"==typeof e||null===e?"":(o=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),i=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===i?0===a?o+"ms":a+"s ":i+"m "+a+"s":t?n+"h "+i+"m":n+"h "+i+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+a+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,a).then(function(e){var t,n,i,o,a,s,l;i=NaN,l={},o=e.values;for(t in o)s=o[t],a=t.replace(".currentLowWatermark",""),l[a]=s,(isNaN(i)||su.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),a={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(a[i]=e,t>=n-1)return r.resolve(a)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,a,s){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,a,s;a=r.children()[0],o=r.width(),angular.element(a).attr("width",o),s=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(a).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:s(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(a).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,y,M,J;p=null,C=d3.behavior.zoom(),J=[],h=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},g=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":g(t)?"node-iteration":"node-normal"},a=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=M(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(g(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+M(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},M=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},s=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:a(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),a=0,u=v.length;a-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return a=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,s,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),a=i,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,a;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,a){if(o.id===r.id&&(i.metrics[e][t].splice(a,1),a> initiali timeout, responseHeaders, uploadDir, - executor)); + executor, + clusterConfiguration)); } handlers.add(Tuple2.of(blobServerPortHandler.getMessageHeaders(), blobServerPortHandler)); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java index 832b2e519cf96..ff245338045cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java @@ -218,7 +218,8 @@ public static Collection responseHeaders, java.nio.file.Path uploadDir, - Executor executor) { + Executor executor, + Configuration configuration) { if (!isFlinkRuntimeWebInClassPath()) { return Collections.emptyList(); @@ -276,9 +277,37 @@ public static Collection jarRunHandlerConstructor = Class + .forName(jarHandlerPackageName + "JarRunHandler") + .getConstructor( + CompletableFuture.class, + GatewayRetriever.class, + Time.class, + Map.class, + MessageHeaders.class, + java.nio.file.Path.class, + Configuration.class, + Executor.class); + + final MessageHeaders jarRunHandlerHeaders = (MessageHeaders) Class + .forName(jarHandlerPackageName + "JarRunHeaders") + .newInstance(); + + final ChannelInboundHandler jarRunHandler = (ChannelInboundHandler) jarRunHandlerConstructor + .newInstance( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + jarRunHandlerHeaders, + uploadDir, + configuration, + executor); + return Arrays.asList( Tuple2.of(jarUploadMessageHeaders, jarUploadHandler), - Tuple2.of(jarListHeaders, jarListHandler)); + Tuple2.of(jarListHeaders, jarListHandler), + Tuple2.of(jarRunHandlerHeaders, jarRunHandler)); } catch (ClassNotFoundException | InvocationTargetException | InstantiationException | NoSuchMethodException | IllegalAccessException e) { throw new RuntimeException(e); }