blob: e80845e50b0c5a8d9e39d9642ca564f172fa0443 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.replication, class: ZKReplicationPeerStorage">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.replication;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import java.io.IOException;</span>
<span class="source-line-no">021</span><span id="line-21">import java.util.Arrays;</span>
<span class="source-line-no">022</span><span id="line-22">import java.util.Collections;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.List;</span>
<span class="source-line-no">024</span><span id="line-24">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">025</span><span id="line-25">import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;</span>
<span class="source-line-no">026</span><span id="line-26">import org.apache.hadoop.hbase.exceptions.DeserializationException;</span>
<span class="source-line-no">027</span><span id="line-27">import org.apache.hadoop.hbase.zookeeper.ZKUtil;</span>
<span class="source-line-no">028</span><span id="line-28">import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.hadoop.hbase.zookeeper.ZKWatcher;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.hadoop.hbase.zookeeper.ZNodePaths;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.zookeeper.KeeperException;</span>
<span class="source-line-no">033</span><span id="line-33"></span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;</span>
<span class="source-line-no">035</span><span id="line-35"></span>
<span class="source-line-no">036</span><span id="line-36">/**</span>
<span class="source-line-no">037</span><span id="line-37"> * ZK based replication peer storage.</span>
<span class="source-line-no">038</span><span id="line-38"> */</span>
<span class="source-line-no">039</span><span id="line-39">@InterfaceAudience.Private</span>
<span class="source-line-no">040</span><span id="line-40">public class ZKReplicationPeerStorage extends ZKReplicationStorageBase</span>
<span class="source-line-no">041</span><span id="line-41"> implements ReplicationPeerStorage {</span>
<span class="source-line-no">042</span><span id="line-42"></span>
<span class="source-line-no">043</span><span id="line-43"> public static final String PEERS_ZNODE = "zookeeper.znode.replication.peers";</span>
<span class="source-line-no">044</span><span id="line-44"> public static final String PEERS_ZNODE_DEFAULT = "peers";</span>
<span class="source-line-no">045</span><span id="line-45"></span>
<span class="source-line-no">046</span><span id="line-46"> public static final String PEERS_STATE_ZNODE = "zookeeper.znode.replication.peers.state";</span>
<span class="source-line-no">047</span><span id="line-47"> public static final String PEERS_STATE_ZNODE_DEFAULT = "peer-state";</span>
<span class="source-line-no">048</span><span id="line-48"></span>
<span class="source-line-no">049</span><span id="line-49"> public static final byte[] ENABLED_ZNODE_BYTES =</span>
<span class="source-line-no">050</span><span id="line-50"> toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);</span>
<span class="source-line-no">051</span><span id="line-51"> public static final byte[] DISABLED_ZNODE_BYTES =</span>
<span class="source-line-no">052</span><span id="line-52"> toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);</span>
<span class="source-line-no">053</span><span id="line-53"></span>
<span class="source-line-no">054</span><span id="line-54"> public static final String SYNC_REPLICATION_STATE_ZNODE = "sync-rep-state";</span>
<span class="source-line-no">055</span><span id="line-55"></span>
<span class="source-line-no">056</span><span id="line-56"> public static final String NEW_SYNC_REPLICATION_STATE_ZNODE = "new-sync-rep-state";</span>
<span class="source-line-no">057</span><span id="line-57"></span>
<span class="source-line-no">058</span><span id="line-58"> public static final byte[] NONE_STATE_ZNODE_BYTES =</span>
<span class="source-line-no">059</span><span id="line-59"> SyncReplicationState.toByteArray(SyncReplicationState.NONE);</span>
<span class="source-line-no">060</span><span id="line-60"></span>
<span class="source-line-no">061</span><span id="line-61"> /**</span>
<span class="source-line-no">062</span><span id="line-62"> * The name of the znode that contains the replication status of a remote slave (i.e. peer)</span>
<span class="source-line-no">063</span><span id="line-63"> * cluster.</span>
<span class="source-line-no">064</span><span id="line-64"> */</span>
<span class="source-line-no">065</span><span id="line-65"> private final String peerStateNodeName;</span>
<span class="source-line-no">066</span><span id="line-66"></span>
<span class="source-line-no">067</span><span id="line-67"> /**</span>
<span class="source-line-no">068</span><span id="line-68"> * The name of the znode that contains a list of all remote slave (i.e. peer) clusters.</span>
<span class="source-line-no">069</span><span id="line-69"> */</span>
<span class="source-line-no">070</span><span id="line-70"> private final String peersZNode;</span>
<span class="source-line-no">071</span><span id="line-71"></span>
<span class="source-line-no">072</span><span id="line-72"> public ZKReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) {</span>
<span class="source-line-no">073</span><span id="line-73"> super(zookeeper, conf);</span>
<span class="source-line-no">074</span><span id="line-74"> this.peerStateNodeName = conf.get(PEERS_STATE_ZNODE, PEERS_STATE_ZNODE_DEFAULT);</span>
<span class="source-line-no">075</span><span id="line-75"> String peersZNodeName = conf.get(PEERS_ZNODE, PEERS_ZNODE_DEFAULT);</span>
<span class="source-line-no">076</span><span id="line-76"> this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);</span>
<span class="source-line-no">077</span><span id="line-77"> }</span>
<span class="source-line-no">078</span><span id="line-78"></span>
<span class="source-line-no">079</span><span id="line-79"> public String getPeerStateNode(String peerId) {</span>
<span class="source-line-no">080</span><span id="line-80"> return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName);</span>
<span class="source-line-no">081</span><span id="line-81"> }</span>
<span class="source-line-no">082</span><span id="line-82"></span>
<span class="source-line-no">083</span><span id="line-83"> public String getPeerNode(String peerId) {</span>
<span class="source-line-no">084</span><span id="line-84"> return ZNodePaths.joinZNode(peersZNode, peerId);</span>
<span class="source-line-no">085</span><span id="line-85"> }</span>
<span class="source-line-no">086</span><span id="line-86"></span>
<span class="source-line-no">087</span><span id="line-87"> public String getSyncReplicationStateNode(String peerId) {</span>
<span class="source-line-no">088</span><span id="line-88"> return ZNodePaths.joinZNode(getPeerNode(peerId), SYNC_REPLICATION_STATE_ZNODE);</span>
<span class="source-line-no">089</span><span id="line-89"> }</span>
<span class="source-line-no">090</span><span id="line-90"></span>
<span class="source-line-no">091</span><span id="line-91"> public String getNewSyncReplicationStateNode(String peerId) {</span>
<span class="source-line-no">092</span><span id="line-92"> return ZNodePaths.joinZNode(getPeerNode(peerId), NEW_SYNC_REPLICATION_STATE_ZNODE);</span>
<span class="source-line-no">093</span><span id="line-93"> }</span>
<span class="source-line-no">094</span><span id="line-94"></span>
<span class="source-line-no">095</span><span id="line-95"> @Override</span>
<span class="source-line-no">096</span><span id="line-96"> public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled,</span>
<span class="source-line-no">097</span><span id="line-97"> SyncReplicationState syncReplicationState) throws ReplicationException {</span>
<span class="source-line-no">098</span><span id="line-98"> List&lt;ZKUtilOp&gt; multiOps = Arrays.asList(</span>
<span class="source-line-no">099</span><span id="line-99"> ZKUtilOp.createAndFailSilent(getPeerNode(peerId),</span>
<span class="source-line-no">100</span><span id="line-100"> ReplicationPeerConfigUtil.toByteArray(peerConfig)),</span>
<span class="source-line-no">101</span><span id="line-101"> ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),</span>
<span class="source-line-no">102</span><span id="line-102"> enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES),</span>
<span class="source-line-no">103</span><span id="line-103"> ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId),</span>
<span class="source-line-no">104</span><span id="line-104"> SyncReplicationState.toByteArray(syncReplicationState)),</span>
<span class="source-line-no">105</span><span id="line-105"> ZKUtilOp.createAndFailSilent(getNewSyncReplicationStateNode(peerId), NONE_STATE_ZNODE_BYTES));</span>
<span class="source-line-no">106</span><span id="line-106"> try {</span>
<span class="source-line-no">107</span><span id="line-107"> ZKUtil.createWithParents(zookeeper, peersZNode);</span>
<span class="source-line-no">108</span><span id="line-108"> ZKUtil.multiOrSequential(zookeeper, multiOps, false);</span>
<span class="source-line-no">109</span><span id="line-109"> } catch (KeeperException e) {</span>
<span class="source-line-no">110</span><span id="line-110"> throw new ReplicationException(</span>
<span class="source-line-no">111</span><span id="line-111"> "Could not add peer with id=" + peerId + ", peerConfig=&gt;" + peerConfig + ", state="</span>
<span class="source-line-no">112</span><span id="line-112"> + (enabled ? "ENABLED" : "DISABLED") + ", syncReplicationState=" + syncReplicationState,</span>
<span class="source-line-no">113</span><span id="line-113"> e);</span>
<span class="source-line-no">114</span><span id="line-114"> }</span>
<span class="source-line-no">115</span><span id="line-115"> }</span>
<span class="source-line-no">116</span><span id="line-116"></span>
<span class="source-line-no">117</span><span id="line-117"> @Override</span>
<span class="source-line-no">118</span><span id="line-118"> public void removePeer(String peerId) throws ReplicationException {</span>
<span class="source-line-no">119</span><span id="line-119"> try {</span>
<span class="source-line-no">120</span><span id="line-120"> ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId));</span>
<span class="source-line-no">121</span><span id="line-121"> } catch (KeeperException e) {</span>
<span class="source-line-no">122</span><span id="line-122"> throw new ReplicationException("Could not remove peer with id=" + peerId, e);</span>
<span class="source-line-no">123</span><span id="line-123"> }</span>
<span class="source-line-no">124</span><span id="line-124"> }</span>
<span class="source-line-no">125</span><span id="line-125"></span>
<span class="source-line-no">126</span><span id="line-126"> @Override</span>
<span class="source-line-no">127</span><span id="line-127"> public void setPeerState(String peerId, boolean enabled) throws ReplicationException {</span>
<span class="source-line-no">128</span><span id="line-128"> byte[] stateBytes = enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES;</span>
<span class="source-line-no">129</span><span id="line-129"> try {</span>
<span class="source-line-no">130</span><span id="line-130"> ZKUtil.setData(zookeeper, getPeerStateNode(peerId), stateBytes);</span>
<span class="source-line-no">131</span><span id="line-131"> } catch (KeeperException e) {</span>
<span class="source-line-no">132</span><span id="line-132"> throw new ReplicationException("Unable to change state of the peer with id=" + peerId, e);</span>
<span class="source-line-no">133</span><span id="line-133"> }</span>
<span class="source-line-no">134</span><span id="line-134"> }</span>
<span class="source-line-no">135</span><span id="line-135"></span>
<span class="source-line-no">136</span><span id="line-136"> @Override</span>
<span class="source-line-no">137</span><span id="line-137"> public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)</span>
<span class="source-line-no">138</span><span id="line-138"> throws ReplicationException {</span>
<span class="source-line-no">139</span><span id="line-139"> try {</span>
<span class="source-line-no">140</span><span id="line-140"> ZKUtil.setData(this.zookeeper, getPeerNode(peerId),</span>
<span class="source-line-no">141</span><span id="line-141"> ReplicationPeerConfigUtil.toByteArray(peerConfig));</span>
<span class="source-line-no">142</span><span id="line-142"> } catch (KeeperException e) {</span>
<span class="source-line-no">143</span><span id="line-143"> throw new ReplicationException(</span>
<span class="source-line-no">144</span><span id="line-144"> "There was a problem trying to save changes to the " + "replication peer " + peerId, e);</span>
<span class="source-line-no">145</span><span id="line-145"> }</span>
<span class="source-line-no">146</span><span id="line-146"> }</span>
<span class="source-line-no">147</span><span id="line-147"></span>
<span class="source-line-no">148</span><span id="line-148"> @Override</span>
<span class="source-line-no">149</span><span id="line-149"> public List&lt;String&gt; listPeerIds() throws ReplicationException {</span>
<span class="source-line-no">150</span><span id="line-150"> try {</span>
<span class="source-line-no">151</span><span id="line-151"> List&lt;String&gt; children = ZKUtil.listChildrenNoWatch(zookeeper, peersZNode);</span>
<span class="source-line-no">152</span><span id="line-152"> return children != null ? children : Collections.emptyList();</span>
<span class="source-line-no">153</span><span id="line-153"> } catch (KeeperException e) {</span>
<span class="source-line-no">154</span><span id="line-154"> throw new ReplicationException("Cannot get the list of peers", e);</span>
<span class="source-line-no">155</span><span id="line-155"> }</span>
<span class="source-line-no">156</span><span id="line-156"> }</span>
<span class="source-line-no">157</span><span id="line-157"></span>
<span class="source-line-no">158</span><span id="line-158"> @Override</span>
<span class="source-line-no">159</span><span id="line-159"> public boolean isPeerEnabled(String peerId) throws ReplicationException {</span>
<span class="source-line-no">160</span><span id="line-160"> try {</span>
<span class="source-line-no">161</span><span id="line-161"> return Arrays.equals(ENABLED_ZNODE_BYTES,</span>
<span class="source-line-no">162</span><span id="line-162"> ZKUtil.getData(zookeeper, getPeerStateNode(peerId)));</span>
<span class="source-line-no">163</span><span id="line-163"> } catch (KeeperException | InterruptedException e) {</span>
<span class="source-line-no">164</span><span id="line-164"> throw new ReplicationException("Unable to get status of the peer with id=" + peerId, e);</span>
<span class="source-line-no">165</span><span id="line-165"> }</span>
<span class="source-line-no">166</span><span id="line-166"> }</span>
<span class="source-line-no">167</span><span id="line-167"></span>
<span class="source-line-no">168</span><span id="line-168"> @Override</span>
<span class="source-line-no">169</span><span id="line-169"> public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException {</span>
<span class="source-line-no">170</span><span id="line-170"> byte[] data;</span>
<span class="source-line-no">171</span><span id="line-171"> try {</span>
<span class="source-line-no">172</span><span id="line-172"> data = ZKUtil.getData(zookeeper, getPeerNode(peerId));</span>
<span class="source-line-no">173</span><span id="line-173"> } catch (KeeperException | InterruptedException e) {</span>
<span class="source-line-no">174</span><span id="line-174"> throw new ReplicationException("Error getting configuration for peer with id=" + peerId, e);</span>
<span class="source-line-no">175</span><span id="line-175"> }</span>
<span class="source-line-no">176</span><span id="line-176"> if (data == null || data.length == 0) {</span>
<span class="source-line-no">177</span><span id="line-177"> throw new ReplicationException(</span>
<span class="source-line-no">178</span><span id="line-178"> "Replication peer config data shouldn't be empty, peerId=" + peerId);</span>
<span class="source-line-no">179</span><span id="line-179"> }</span>
<span class="source-line-no">180</span><span id="line-180"> try {</span>
<span class="source-line-no">181</span><span id="line-181"> return ReplicationPeerConfigUtil.parsePeerFrom(data);</span>
<span class="source-line-no">182</span><span id="line-182"> } catch (DeserializationException e) {</span>
<span class="source-line-no">183</span><span id="line-183"> throw new ReplicationException(</span>
<span class="source-line-no">184</span><span id="line-184"> "Failed to parse replication peer config for peer with id=" + peerId, e);</span>
<span class="source-line-no">185</span><span id="line-185"> }</span>
<span class="source-line-no">186</span><span id="line-186"> }</span>
<span class="source-line-no">187</span><span id="line-187"></span>
<span class="source-line-no">188</span><span id="line-188"> @Override</span>
<span class="source-line-no">189</span><span id="line-189"> public void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state)</span>
<span class="source-line-no">190</span><span id="line-190"> throws ReplicationException {</span>
<span class="source-line-no">191</span><span id="line-191"> try {</span>
<span class="source-line-no">192</span><span id="line-192"> ZKUtil.createSetData(zookeeper, getNewSyncReplicationStateNode(peerId),</span>
<span class="source-line-no">193</span><span id="line-193"> SyncReplicationState.toByteArray(state));</span>
<span class="source-line-no">194</span><span id="line-194"> } catch (KeeperException e) {</span>
<span class="source-line-no">195</span><span id="line-195"> throw new ReplicationException(</span>
<span class="source-line-no">196</span><span id="line-196"> "Unable to set the new sync replication state for peer with id=" + peerId, e);</span>
<span class="source-line-no">197</span><span id="line-197"> }</span>
<span class="source-line-no">198</span><span id="line-198"> }</span>
<span class="source-line-no">199</span><span id="line-199"></span>
<span class="source-line-no">200</span><span id="line-200"> @Override</span>
<span class="source-line-no">201</span><span id="line-201"> public void transitPeerSyncReplicationState(String peerId) throws ReplicationException {</span>
<span class="source-line-no">202</span><span id="line-202"> String newStateNode = getNewSyncReplicationStateNode(peerId);</span>
<span class="source-line-no">203</span><span id="line-203"> try {</span>
<span class="source-line-no">204</span><span id="line-204"> byte[] data = ZKUtil.getData(zookeeper, newStateNode);</span>
<span class="source-line-no">205</span><span id="line-205"> ZKUtil.multiOrSequential(zookeeper,</span>
<span class="source-line-no">206</span><span id="line-206"> Arrays.asList(ZKUtilOp.setData(newStateNode, NONE_STATE_ZNODE_BYTES),</span>
<span class="source-line-no">207</span><span id="line-207"> ZKUtilOp.setData(getSyncReplicationStateNode(peerId), data)),</span>
<span class="source-line-no">208</span><span id="line-208"> false);</span>
<span class="source-line-no">209</span><span id="line-209"> } catch (KeeperException | InterruptedException e) {</span>
<span class="source-line-no">210</span><span id="line-210"> throw new ReplicationException(</span>
<span class="source-line-no">211</span><span id="line-211"> "Error transiting sync replication state for peer with id=" + peerId, e);</span>
<span class="source-line-no">212</span><span id="line-212"> }</span>
<span class="source-line-no">213</span><span id="line-213"> }</span>
<span class="source-line-no">214</span><span id="line-214"></span>
<span class="source-line-no">215</span><span id="line-215"> private SyncReplicationState getSyncReplicationState(String peerId, String path)</span>
<span class="source-line-no">216</span><span id="line-216"> throws ReplicationException {</span>
<span class="source-line-no">217</span><span id="line-217"> try {</span>
<span class="source-line-no">218</span><span id="line-218"> byte[] data = ZKUtil.getData(zookeeper, path);</span>
<span class="source-line-no">219</span><span id="line-219"> if (data == null || data.length == 0) {</span>
<span class="source-line-no">220</span><span id="line-220"> if (ZKUtil.checkExists(zookeeper, getPeerNode(peerId)) != -1) {</span>
<span class="source-line-no">221</span><span id="line-221"> // should be a peer from previous version, set the sync replication state for it.</span>
<span class="source-line-no">222</span><span id="line-222"> ZKUtil.createSetData(zookeeper, path, NONE_STATE_ZNODE_BYTES);</span>
<span class="source-line-no">223</span><span id="line-223"> return SyncReplicationState.NONE;</span>
<span class="source-line-no">224</span><span id="line-224"> } else {</span>
<span class="source-line-no">225</span><span id="line-225"> throw new ReplicationException(</span>
<span class="source-line-no">226</span><span id="line-226"> "Replication peer sync state shouldn't be empty, peerId=" + peerId);</span>
<span class="source-line-no">227</span><span id="line-227"> }</span>
<span class="source-line-no">228</span><span id="line-228"> }</span>
<span class="source-line-no">229</span><span id="line-229"> return SyncReplicationState.parseFrom(data);</span>
<span class="source-line-no">230</span><span id="line-230"> } catch (KeeperException | InterruptedException | IOException e) {</span>
<span class="source-line-no">231</span><span id="line-231"> throw new ReplicationException(</span>
<span class="source-line-no">232</span><span id="line-232"> "Error getting sync replication state of path " + path + " for peer with id=" + peerId, e);</span>
<span class="source-line-no">233</span><span id="line-233"> }</span>
<span class="source-line-no">234</span><span id="line-234"> }</span>
<span class="source-line-no">235</span><span id="line-235"></span>
<span class="source-line-no">236</span><span id="line-236"> @Override</span>
<span class="source-line-no">237</span><span id="line-237"> public SyncReplicationState getPeerNewSyncReplicationState(String peerId)</span>
<span class="source-line-no">238</span><span id="line-238"> throws ReplicationException {</span>
<span class="source-line-no">239</span><span id="line-239"> return getSyncReplicationState(peerId, getNewSyncReplicationStateNode(peerId));</span>
<span class="source-line-no">240</span><span id="line-240"> }</span>
<span class="source-line-no">241</span><span id="line-241"></span>
<span class="source-line-no">242</span><span id="line-242"> @Override</span>
<span class="source-line-no">243</span><span id="line-243"> public SyncReplicationState getPeerSyncReplicationState(String peerId)</span>
<span class="source-line-no">244</span><span id="line-244"> throws ReplicationException {</span>
<span class="source-line-no">245</span><span id="line-245"> return getSyncReplicationState(peerId, getSyncReplicationStateNode(peerId));</span>
<span class="source-line-no">246</span><span id="line-246"> }</span>
<span class="source-line-no">247</span><span id="line-247">}</span>
</pre>
</div>
</main>
</body>
</html>