forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-10329] Fail ZooKeeperSubmittedJobGraphStore#removeJobGraph if …
…job cannot be removed Fail properly with an exception if we cannot remove the JobGraph in ZooKeeperSubmittedJobGraphStore# removeJobGraph. This is necessary in order to notify callers about the unsuccessful attempt.
- Loading branch information
1 parent
9c4c138
commit 60ed037
Showing
4 changed files
with
164 additions
and
39 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
63 changes: 63 additions & 0 deletions
63
...c/test/java/org/apache/flink/runtime/checkpoint/TestingRetrievableStateStorageHelper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* 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.checkpoint; | ||
|
||
import org.apache.flink.runtime.state.RetrievableStateHandle; | ||
import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper; | ||
|
||
import java.io.Serializable; | ||
|
||
/** | ||
* {@link RetrievableStateStorageHelper} implementation for testing purposes. | ||
* | ||
* @param <T> type of the element to store | ||
*/ | ||
public final class TestingRetrievableStateStorageHelper<T extends Serializable> implements RetrievableStateStorageHelper<T> { | ||
|
||
@Override | ||
public RetrievableStateHandle<T> store(T state) { | ||
return new TestingRetrievableStateHandle<>(state); | ||
} | ||
|
||
private static final class TestingRetrievableStateHandle<T extends Serializable> implements RetrievableStateHandle<T> { | ||
|
||
private static final long serialVersionUID = 137053380713794300L; | ||
|
||
private final T state; | ||
|
||
private TestingRetrievableStateHandle(T state) { | ||
this.state = state; | ||
} | ||
|
||
@Override | ||
public T retrieveState() { | ||
return state; | ||
} | ||
|
||
@Override | ||
public void discardState() { | ||
// no op | ||
} | ||
|
||
@Override | ||
public long getStateSize() { | ||
return 0; | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
96 changes: 96 additions & 0 deletions
96
...rc/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStoreTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
/* | ||
* 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.jobmanager; | ||
|
||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.HighAvailabilityOptions; | ||
import org.apache.flink.runtime.checkpoint.TestingRetrievableStateStorageHelper; | ||
import org.apache.flink.runtime.jobgraph.JobGraph; | ||
import org.apache.flink.runtime.util.ZooKeeperUtils; | ||
import org.apache.flink.runtime.zookeeper.ZooKeeperResource; | ||
import org.apache.flink.util.TestLogger; | ||
|
||
import org.apache.curator.framework.CuratorFramework; | ||
import org.junit.Before; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
|
||
import javax.annotation.Nonnull; | ||
|
||
import static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.notNullValue; | ||
import static org.junit.Assert.assertThat; | ||
import static org.junit.Assert.fail; | ||
|
||
/** | ||
* Tests for the {@link ZooKeeperSubmittedJobGraphStore}. | ||
*/ | ||
public class ZooKeeperSubmittedJobGraphStoreTest extends TestLogger { | ||
|
||
@Rule | ||
public ZooKeeperResource zooKeeperResource = new ZooKeeperResource(); | ||
|
||
private Configuration configuration; | ||
|
||
@Before | ||
public void setup() { | ||
configuration = new Configuration(); | ||
configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString()); | ||
} | ||
|
||
/** | ||
* Tests that we fail with an exception if the job cannot be removed from the | ||
* ZooKeeperSubmittedJobGraphStore. | ||
*/ | ||
@Test | ||
public void testJobGraphRemovalFailure() throws Exception { | ||
try (final CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration)) { | ||
final TestingRetrievableStateStorageHelper<SubmittedJobGraph> stateStorage = new TestingRetrievableStateStorageHelper<>(); | ||
final ZooKeeperSubmittedJobGraphStore submittedJobGraphStore = createSubmittedJobGraphStore(client, stateStorage); | ||
submittedJobGraphStore.start(null); | ||
final ZooKeeperSubmittedJobGraphStore otherSubmittedJobGraphStore = createSubmittedJobGraphStore(client, stateStorage); | ||
otherSubmittedJobGraphStore.start(null); | ||
|
||
final SubmittedJobGraph jobGraph = new SubmittedJobGraph(new JobGraph(), null); | ||
submittedJobGraphStore.putJobGraph(jobGraph); | ||
|
||
final SubmittedJobGraph recoveredJobGraph = otherSubmittedJobGraphStore.recoverJobGraph(jobGraph.getJobId()); | ||
|
||
assertThat(recoveredJobGraph, is(notNullValue())); | ||
|
||
try { | ||
otherSubmittedJobGraphStore.removeJobGraph(recoveredJobGraph.getJobId()); | ||
fail("It should not be possible to remove the JobGraph since the first store still has a lock on it."); | ||
} catch (Exception ignored) { | ||
// expected | ||
} | ||
|
||
otherSubmittedJobGraphStore.stop(); | ||
} | ||
} | ||
|
||
@Nonnull | ||
public ZooKeeperSubmittedJobGraphStore createSubmittedJobGraphStore(CuratorFramework client, TestingRetrievableStateStorageHelper<SubmittedJobGraph> stateStorage) throws Exception { | ||
return new ZooKeeperSubmittedJobGraphStore( | ||
client, | ||
"/foobar", | ||
stateStorage); | ||
} | ||
|
||
} |