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-5074] [runtime] Add a ZooKeeper-based RunningJobRegistry
This closes apache#2903
- Loading branch information
1 parent
8780cb6
commit 544f534
Showing
4 changed files
with
183 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
94 changes: 94 additions & 0 deletions
94
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.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,94 @@ | ||
/* | ||
* 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:https://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.highavailability; | ||
|
||
import org.apache.curator.framework.CuratorFramework; | ||
import org.apache.flink.api.common.JobID; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.HighAvailabilityOptions; | ||
import org.apache.zookeeper.data.Stat; | ||
|
||
import java.io.IOException; | ||
|
||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
|
||
/** | ||
* A zookeeper based registry for running jobs, highly available. | ||
*/ | ||
public class ZookeeperRegistry implements RunningJobsRegistry { | ||
|
||
private static final String DEFAULT_HA_JOB_REGISTRY_PATH = "/running_job_registry/"; | ||
|
||
/** The ZooKeeper client to use */ | ||
private final CuratorFramework client; | ||
|
||
private final String runningJobPath; | ||
|
||
private static final String HA_JOB_REGISTRY_PATH = "high-availability.zookeeper.job.registry"; | ||
|
||
public ZookeeperRegistry(final CuratorFramework client, final Configuration configuration) { | ||
this.client = client; | ||
runningJobPath = configuration.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_ROOT) + | ||
configuration.getString(HA_JOB_REGISTRY_PATH, DEFAULT_HA_JOB_REGISTRY_PATH); | ||
} | ||
|
||
@Override | ||
public void setJobRunning(JobID jobID) throws IOException { | ||
checkNotNull(jobID); | ||
|
||
try { | ||
String zkPath = runningJobPath + jobID.toString(); | ||
this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient()); | ||
this.client.setData().forPath(zkPath); | ||
} | ||
catch (Exception e) { | ||
throw new IOException("Set running state to zk fail for job " + jobID.toString(), e); | ||
} | ||
} | ||
|
||
@Override | ||
public void setJobFinished(JobID jobID) throws IOException { | ||
checkNotNull(jobID); | ||
|
||
try { | ||
String zkPath = runningJobPath + jobID.toString(); | ||
this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient()); | ||
this.client.delete().forPath(zkPath); | ||
} | ||
catch (Exception e) { | ||
throw new IOException("Set finished state to zk fail for job " + jobID.toString(), e); | ||
} | ||
} | ||
|
||
@Override | ||
public boolean isJobRunning(JobID jobID) throws IOException { | ||
checkNotNull(jobID); | ||
|
||
try { | ||
Stat stat = client.checkExists().forPath(runningJobPath + jobID.toString()); | ||
if (stat != null) { | ||
return true; | ||
} | ||
return false; | ||
} | ||
catch (Exception e) { | ||
throw new IOException("Get running state from zk fail for job " + jobID.toString(), e); | ||
} | ||
} | ||
} |
78 changes: 78 additions & 0 deletions
78
...untime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.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,78 @@ | ||
/* | ||
* 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:https://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.highavailability; | ||
|
||
import org.apache.curator.test.TestingServer; | ||
import org.apache.flink.api.common.JobID; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.HighAvailabilityOptions; | ||
import org.apache.flink.util.TestLogger; | ||
import org.junit.After; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import static org.junit.Assert.*; | ||
|
||
public class ZooKeeperRegistryTest extends TestLogger { | ||
private TestingServer testingServer; | ||
|
||
private static Logger LOG = LoggerFactory.getLogger(ZooKeeperRegistryTest.class); | ||
|
||
@Before | ||
public void before() throws Exception { | ||
testingServer = new TestingServer(); | ||
} | ||
|
||
@After | ||
public void after() throws Exception { | ||
testingServer.stop(); | ||
testingServer = null; | ||
} | ||
|
||
/** | ||
* Tests that the function of ZookeeperRegistry, setJobRunning(), setJobFinished(), isJobRunning() | ||
*/ | ||
@Test | ||
public void testZooKeeperRegistry() throws Exception { | ||
Configuration configuration = new Configuration(); | ||
configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString()); | ||
configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); | ||
|
||
HighAvailabilityServices zkHaService = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(configuration); | ||
RunningJobsRegistry zkRegistry = zkHaService.getRunningJobsRegistry(); | ||
|
||
try { | ||
JobID jobID = JobID.generate(); | ||
assertTrue(!zkRegistry.isJobRunning(jobID)); | ||
|
||
zkRegistry.setJobRunning(jobID); | ||
assertTrue(zkRegistry.isJobRunning(jobID)); | ||
|
||
zkRegistry.setJobFinished(jobID); | ||
assertTrue(!zkRegistry.isJobRunning(jobID)); | ||
|
||
} finally { | ||
if (zkHaService != null) { | ||
zkHaService.close(); | ||
} | ||
} | ||
} | ||
} |