You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2020/03/04 14:52:10 UTC
[flink] branch release-1.9 updated: [FLINK-15386][coordination]
Correct JobID comparison
This is an automated email from the ASF dual-hosted git repository.
chesnay pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-1.9 by this push:
new bd6d330 [FLINK-15386][coordination] Correct JobID comparison
bd6d330 is described below
commit bd6d330902920116c43c5794797cf4560ec14319
Author: Meng Li (Ethan) <et...@gmail.com>
AuthorDate: Wed Mar 4 08:51:51 2020 -0600
[FLINK-15386][coordination] Correct JobID comparison
---
.../SingleJobSubmittedJobGraphStore.java | 2 +-
.../SingleJobSubmittedJobGraphStoreTest.java | 84 ++++++++++++++++++++++
2 files changed, 85 insertions(+), 1 deletion(-)
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java
index 16d90a6..e1cdff0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java
@@ -60,7 +60,7 @@ public class SingleJobSubmittedJobGraphStore implements SubmittedJobGraphStore {
@Override
public void putJobGraph(SubmittedJobGraph jobGraph) throws Exception {
- if (!jobGraph.getJobId().equals(jobGraph.getJobId())) {
+ if (!this.jobGraph.getJobID().equals(jobGraph.getJobId())) {
throw new FlinkException("Cannot put additional jobs into this submitted job graph store.");
}
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStoreTest.java
new file mode 100644
index 0000000..b7b8054
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStoreTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.dispatcher;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraph;
+import org.apache.flink.util.FlinkException;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test {@link SingleJobSubmittedJobGraphStore} functionality.
+ */
+public class SingleJobSubmittedJobGraphStoreTest {
+
+ @Rule public ExpectedException expectedException = ExpectedException.none();
+
+ private static SingleJobSubmittedJobGraphStore singleJobSubmittedJobGraphStore;
+ private static JobID thisJobID;
+ private static JobGraph thisJobGraph;
+ private static JobID otherJobID;
+
+ @BeforeClass
+ public static void init() {
+ thisJobID = new JobID(100, 100);
+ otherJobID = new JobID(999, 999);
+
+ thisJobGraph = new JobGraph(thisJobID, "thisJobGraph");
+ singleJobSubmittedJobGraphStore = new SingleJobSubmittedJobGraphStore(thisJobGraph);
+ }
+
+ @Test
+ public void testPutJobGraph() throws Exception {
+ assertEquals(1, singleJobSubmittedJobGraphStore.getJobIds().size());
+
+ SubmittedJobGraph otherSubmittedJobGraph = new SubmittedJobGraph(new JobGraph(otherJobID, "otherJobGraph"));
+ //no-op
+ singleJobSubmittedJobGraphStore.putJobGraph(new SubmittedJobGraph(thisJobGraph));
+
+ expectedException.expect(FlinkException.class);
+ singleJobSubmittedJobGraphStore.putJobGraph(otherSubmittedJobGraph);
+ }
+
+ @Test
+ public void testRecoverJobGraph() throws Exception {
+ SubmittedJobGraph submittedJobGraph = singleJobSubmittedJobGraphStore.recoverJobGraph(thisJobID);
+
+ assertEquals(1, singleJobSubmittedJobGraphStore.getJobIds().size());
+ assertEquals(submittedJobGraph.getJobId(), thisJobID);
+
+ expectedException.expect(FlinkException.class);
+ singleJobSubmittedJobGraphStore.recoverJobGraph(otherJobID);
+ }
+
+ @Test
+ public void testRemoveJobGraph() {
+ assertEquals(1, singleJobSubmittedJobGraphStore.getJobIds().size());
+ //no-op
+ singleJobSubmittedJobGraphStore.removeJobGraph(thisJobID);
+ assertEquals(1, singleJobSubmittedJobGraphStore.getJobIds().size());
+ }
+}