| <!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> |
| <html lang="en"> |
| <head> |
| <title>Source code</title> |
| <link rel="stylesheet" type="text/css" href="../../../../../../../stylesheet.css" title="Style"> |
| </head> |
| <body> |
| <div class="sourceContainer"> |
| <pre><span class="sourceLineNo">001</span>/*<a name="line.1"></a> |
| <span class="sourceLineNo">002</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.2"></a> |
| <span class="sourceLineNo">003</span> * or more contributor license agreements. See the NOTICE file<a name="line.3"></a> |
| <span class="sourceLineNo">004</span> * distributed with this work for additional information<a name="line.4"></a> |
| <span class="sourceLineNo">005</span> * regarding copyright ownership. The ASF licenses this file<a name="line.5"></a> |
| <span class="sourceLineNo">006</span> * to you under the Apache License, Version 2.0 (the<a name="line.6"></a> |
| <span class="sourceLineNo">007</span> * "License"); you may not use this file except in compliance<a name="line.7"></a> |
| <span class="sourceLineNo">008</span> * with the License. You may obtain a copy of the License at<a name="line.8"></a> |
| <span class="sourceLineNo">009</span> *<a name="line.9"></a> |
| <span class="sourceLineNo">010</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.10"></a> |
| <span class="sourceLineNo">011</span> *<a name="line.11"></a> |
| <span class="sourceLineNo">012</span> * Unless required by applicable law or agreed to in writing, software<a name="line.12"></a> |
| <span class="sourceLineNo">013</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.13"></a> |
| <span class="sourceLineNo">014</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.14"></a> |
| <span class="sourceLineNo">015</span> * See the License for the specific language governing permissions and<a name="line.15"></a> |
| <span class="sourceLineNo">016</span> * limitations under the License.<a name="line.16"></a> |
| <span class="sourceLineNo">017</span> */<a name="line.17"></a> |
| <span class="sourceLineNo">018</span>package org.apache.hadoop.hbase.master.replication;<a name="line.18"></a> |
| <span class="sourceLineNo">019</span><a name="line.19"></a> |
| <span class="sourceLineNo">020</span>import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerRemoteWALDir;<a name="line.20"></a> |
| <span class="sourceLineNo">021</span>import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerReplayWALDir;<a name="line.21"></a> |
| <span class="sourceLineNo">022</span>import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerSnapshotWALDir;<a name="line.22"></a> |
| <span class="sourceLineNo">023</span><a name="line.23"></a> |
| <span class="sourceLineNo">024</span>import java.io.IOException;<a name="line.24"></a> |
| <span class="sourceLineNo">025</span>import java.util.ArrayList;<a name="line.25"></a> |
| <span class="sourceLineNo">026</span>import java.util.HashSet;<a name="line.26"></a> |
| <span class="sourceLineNo">027</span>import java.util.List;<a name="line.27"></a> |
| <span class="sourceLineNo">028</span>import java.util.Optional;<a name="line.28"></a> |
| <span class="sourceLineNo">029</span>import java.util.Set;<a name="line.29"></a> |
| <span class="sourceLineNo">030</span>import java.util.concurrent.ConcurrentHashMap;<a name="line.30"></a> |
| <span class="sourceLineNo">031</span>import java.util.concurrent.ConcurrentMap;<a name="line.31"></a> |
| <span class="sourceLineNo">032</span>import org.apache.hadoop.fs.FileStatus;<a name="line.32"></a> |
| <span class="sourceLineNo">033</span>import org.apache.hadoop.fs.FileSystem;<a name="line.33"></a> |
| <span class="sourceLineNo">034</span>import org.apache.hadoop.fs.Path;<a name="line.34"></a> |
| <span class="sourceLineNo">035</span>import org.apache.hadoop.hbase.ServerName;<a name="line.35"></a> |
| <span class="sourceLineNo">036</span>import org.apache.hadoop.hbase.master.MasterServices;<a name="line.36"></a> |
| <span class="sourceLineNo">037</span>import org.apache.hadoop.hbase.master.ServerListener;<a name="line.37"></a> |
| <span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.master.ServerManager;<a name="line.38"></a> |
| <span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;<a name="line.39"></a> |
| <span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.procedure2.Procedure;<a name="line.40"></a> |
| <span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.procedure2.ProcedureEvent;<a name="line.41"></a> |
| <span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;<a name="line.42"></a> |
| <span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.replication.ReplicationException;<a name="line.43"></a> |
| <span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.replication.ReplicationUtils;<a name="line.44"></a> |
| <span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.util.FSUtils;<a name="line.45"></a> |
| <span class="sourceLineNo">046</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.46"></a> |
| <span class="sourceLineNo">047</span>import org.slf4j.Logger;<a name="line.47"></a> |
| <span class="sourceLineNo">048</span>import org.slf4j.LoggerFactory;<a name="line.48"></a> |
| <span class="sourceLineNo">049</span><a name="line.49"></a> |
| <span class="sourceLineNo">050</span>/**<a name="line.50"></a> |
| <span class="sourceLineNo">051</span> * The manager for replaying remote wal.<a name="line.51"></a> |
| <span class="sourceLineNo">052</span> * <p/><a name="line.52"></a> |
| <span class="sourceLineNo">053</span> * First, it will be used to balance the replay work across all the region servers. We will record<a name="line.53"></a> |
| <span class="sourceLineNo">054</span> * the region servers which have already been used for replaying wal, and prevent sending new replay<a name="line.54"></a> |
| <span class="sourceLineNo">055</span> * work to it, until the previous replay work has been done, where we will remove the region server<a name="line.55"></a> |
| <span class="sourceLineNo">056</span> * from the used worker set. See the comment for {@code UsedReplayWorkersForPeer} for more details.<a name="line.56"></a> |
| <span class="sourceLineNo">057</span> * <p/><a name="line.57"></a> |
| <span class="sourceLineNo">058</span> * Second, the logic for managing the remote wal directory is kept here. Before replaying the wals,<a name="line.58"></a> |
| <span class="sourceLineNo">059</span> * we will rename the remote wal directory, the new name is called 'replay' directory, see<a name="line.59"></a> |
| <span class="sourceLineNo">060</span> * {@link #renameToPeerReplayWALDir(String)}. This is used to prevent further writing of remote<a name="line.60"></a> |
| <span class="sourceLineNo">061</span> * wals, which is very important for keeping consistency. And then we will start replaying all the<a name="line.61"></a> |
| <span class="sourceLineNo">062</span> * wals, once a wal has been replayed, we will truncate the file, so that if there are crashes<a name="line.62"></a> |
| <span class="sourceLineNo">063</span> * happen, we do not need to replay all the wals again, see {@link #finishReplayWAL(String)} and<a name="line.63"></a> |
| <span class="sourceLineNo">064</span> * {@link #isReplayWALFinished(String)}. After replaying all the wals, we will rename the 'replay'<a name="line.64"></a> |
| <span class="sourceLineNo">065</span> * directory, the new name is called 'snapshot' directory. In the directory, we will keep all the<a name="line.65"></a> |
| <span class="sourceLineNo">066</span> * names for the wals being replayed, since all the files should have been truncated. When we<a name="line.66"></a> |
| <span class="sourceLineNo">067</span> * transitting original the ACTIVE cluster to STANDBY later, and there are region server crashes, we<a name="line.67"></a> |
| <span class="sourceLineNo">068</span> * will see the wals in this directory to determine whether a wal should be split and replayed or<a name="line.68"></a> |
| <span class="sourceLineNo">069</span> * not. You can see the code in {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker} for more<a name="line.69"></a> |
| <span class="sourceLineNo">070</span> * details.<a name="line.70"></a> |
| <span class="sourceLineNo">071</span> */<a name="line.71"></a> |
| <span class="sourceLineNo">072</span>@InterfaceAudience.Private<a name="line.72"></a> |
| <span class="sourceLineNo">073</span>public class SyncReplicationReplayWALManager {<a name="line.73"></a> |
| <span class="sourceLineNo">074</span><a name="line.74"></a> |
| <span class="sourceLineNo">075</span> private static final Logger LOG = LoggerFactory.getLogger(SyncReplicationReplayWALManager.class);<a name="line.75"></a> |
| <span class="sourceLineNo">076</span><a name="line.76"></a> |
| <span class="sourceLineNo">077</span> private final ServerManager serverManager;<a name="line.77"></a> |
| <span class="sourceLineNo">078</span><a name="line.78"></a> |
| <span class="sourceLineNo">079</span> private final FileSystem fs;<a name="line.79"></a> |
| <span class="sourceLineNo">080</span><a name="line.80"></a> |
| <span class="sourceLineNo">081</span> private final Path walRootDir;<a name="line.81"></a> |
| <span class="sourceLineNo">082</span><a name="line.82"></a> |
| <span class="sourceLineNo">083</span> private final Path remoteWALDir;<a name="line.83"></a> |
| <span class="sourceLineNo">084</span><a name="line.84"></a> |
| <span class="sourceLineNo">085</span> /**<a name="line.85"></a> |
| <span class="sourceLineNo">086</span> * This class is used to record the used workers(region servers) for a replication peer. For<a name="line.86"></a> |
| <span class="sourceLineNo">087</span> * balancing the replaying remote wal job, we will only schedule one remote replay procedure each<a name="line.87"></a> |
| <span class="sourceLineNo">088</span> * time. So when acquiring a worker, we will first get all the region servers for this cluster,<a name="line.88"></a> |
| <span class="sourceLineNo">089</span> * and then filter out the used ones.<a name="line.89"></a> |
| <span class="sourceLineNo">090</span> * <p/><a name="line.90"></a> |
| <span class="sourceLineNo">091</span> * The {@link ProcedureEvent} is used for notifying procedures that there are available workers<a name="line.91"></a> |
| <span class="sourceLineNo">092</span> * now. We used to use sleeping and retrying before, but if the interval is too large, for<a name="line.92"></a> |
| <span class="sourceLineNo">093</span> * example, exponential backoff, then it is not effective, but if the interval is too small, then<a name="line.93"></a> |
| <span class="sourceLineNo">094</span> * we will flood the procedure wal.<a name="line.94"></a> |
| <span class="sourceLineNo">095</span> * <p/><a name="line.95"></a> |
| <span class="sourceLineNo">096</span> * The states are only kept in memory, so when restarting, we need to reconstruct these<a name="line.96"></a> |
| <span class="sourceLineNo">097</span> * information, using the information stored in related procedures. See the {@code afterReplay}<a name="line.97"></a> |
| <span class="sourceLineNo">098</span> * method in {@link RecoverStandbyProcedure} and {@link SyncReplicationReplayWALProcedure} for<a name="line.98"></a> |
| <span class="sourceLineNo">099</span> * more details.<a name="line.99"></a> |
| <span class="sourceLineNo">100</span> */<a name="line.100"></a> |
| <span class="sourceLineNo">101</span> private static final class UsedReplayWorkersForPeer {<a name="line.101"></a> |
| <span class="sourceLineNo">102</span><a name="line.102"></a> |
| <span class="sourceLineNo">103</span> private final Set<ServerName> usedWorkers = new HashSet<ServerName>();<a name="line.103"></a> |
| <span class="sourceLineNo">104</span><a name="line.104"></a> |
| <span class="sourceLineNo">105</span> private final ProcedureEvent<?> event;<a name="line.105"></a> |
| <span class="sourceLineNo">106</span><a name="line.106"></a> |
| <span class="sourceLineNo">107</span> public UsedReplayWorkersForPeer(String peerId) {<a name="line.107"></a> |
| <span class="sourceLineNo">108</span> this.event = new ProcedureEvent<>(peerId);<a name="line.108"></a> |
| <span class="sourceLineNo">109</span> }<a name="line.109"></a> |
| <span class="sourceLineNo">110</span><a name="line.110"></a> |
| <span class="sourceLineNo">111</span> public void used(ServerName worker) {<a name="line.111"></a> |
| <span class="sourceLineNo">112</span> usedWorkers.add(worker);<a name="line.112"></a> |
| <span class="sourceLineNo">113</span> }<a name="line.113"></a> |
| <span class="sourceLineNo">114</span><a name="line.114"></a> |
| <span class="sourceLineNo">115</span> public Optional<ServerName> acquire(ServerManager serverManager) {<a name="line.115"></a> |
| <span class="sourceLineNo">116</span> Optional<ServerName> worker = serverManager.getOnlineServers().keySet().stream()<a name="line.116"></a> |
| <span class="sourceLineNo">117</span> .filter(server -> !usedWorkers.contains(server)).findAny();<a name="line.117"></a> |
| <span class="sourceLineNo">118</span> worker.ifPresent(usedWorkers::add);<a name="line.118"></a> |
| <span class="sourceLineNo">119</span> return worker;<a name="line.119"></a> |
| <span class="sourceLineNo">120</span> }<a name="line.120"></a> |
| <span class="sourceLineNo">121</span><a name="line.121"></a> |
| <span class="sourceLineNo">122</span> public void release(ServerName worker) {<a name="line.122"></a> |
| <span class="sourceLineNo">123</span> usedWorkers.remove(worker);<a name="line.123"></a> |
| <span class="sourceLineNo">124</span> }<a name="line.124"></a> |
| <span class="sourceLineNo">125</span><a name="line.125"></a> |
| <span class="sourceLineNo">126</span> public void suspend(Procedure<?> proc) {<a name="line.126"></a> |
| <span class="sourceLineNo">127</span> event.suspend();<a name="line.127"></a> |
| <span class="sourceLineNo">128</span> event.suspendIfNotReady(proc);<a name="line.128"></a> |
| <span class="sourceLineNo">129</span> }<a name="line.129"></a> |
| <span class="sourceLineNo">130</span><a name="line.130"></a> |
| <span class="sourceLineNo">131</span> public void wake(MasterProcedureScheduler scheduler) {<a name="line.131"></a> |
| <span class="sourceLineNo">132</span> if (!event.isReady()) {<a name="line.132"></a> |
| <span class="sourceLineNo">133</span> event.wake(scheduler);<a name="line.133"></a> |
| <span class="sourceLineNo">134</span> }<a name="line.134"></a> |
| <span class="sourceLineNo">135</span> }<a name="line.135"></a> |
| <span class="sourceLineNo">136</span> }<a name="line.136"></a> |
| <span class="sourceLineNo">137</span><a name="line.137"></a> |
| <span class="sourceLineNo">138</span> private final ConcurrentMap<String, UsedReplayWorkersForPeer> usedWorkersByPeer =<a name="line.138"></a> |
| <span class="sourceLineNo">139</span> new ConcurrentHashMap<>();<a name="line.139"></a> |
| <span class="sourceLineNo">140</span><a name="line.140"></a> |
| <span class="sourceLineNo">141</span> public SyncReplicationReplayWALManager(MasterServices services)<a name="line.141"></a> |
| <span class="sourceLineNo">142</span> throws IOException, ReplicationException {<a name="line.142"></a> |
| <span class="sourceLineNo">143</span> this.serverManager = services.getServerManager();<a name="line.143"></a> |
| <span class="sourceLineNo">144</span> this.fs = services.getMasterFileSystem().getWALFileSystem();<a name="line.144"></a> |
| <span class="sourceLineNo">145</span> this.walRootDir = services.getMasterFileSystem().getWALRootDir();<a name="line.145"></a> |
| <span class="sourceLineNo">146</span> this.remoteWALDir = new Path(this.walRootDir, ReplicationUtils.REMOTE_WAL_DIR_NAME);<a name="line.146"></a> |
| <span class="sourceLineNo">147</span> serverManager.registerListener(new ServerListener() {<a name="line.147"></a> |
| <span class="sourceLineNo">148</span><a name="line.148"></a> |
| <span class="sourceLineNo">149</span> @Override<a name="line.149"></a> |
| <span class="sourceLineNo">150</span> public void serverAdded(ServerName serverName) {<a name="line.150"></a> |
| <span class="sourceLineNo">151</span> MasterProcedureScheduler scheduler =<a name="line.151"></a> |
| <span class="sourceLineNo">152</span> services.getMasterProcedureExecutor().getEnvironment().getProcedureScheduler();<a name="line.152"></a> |
| <span class="sourceLineNo">153</span> for (UsedReplayWorkersForPeer usedWorkers : usedWorkersByPeer.values()) {<a name="line.153"></a> |
| <span class="sourceLineNo">154</span> synchronized (usedWorkers) {<a name="line.154"></a> |
| <span class="sourceLineNo">155</span> usedWorkers.wake(scheduler);<a name="line.155"></a> |
| <span class="sourceLineNo">156</span> }<a name="line.156"></a> |
| <span class="sourceLineNo">157</span> }<a name="line.157"></a> |
| <span class="sourceLineNo">158</span> }<a name="line.158"></a> |
| <span class="sourceLineNo">159</span> });<a name="line.159"></a> |
| <span class="sourceLineNo">160</span> }<a name="line.160"></a> |
| <span class="sourceLineNo">161</span><a name="line.161"></a> |
| <span class="sourceLineNo">162</span> public void registerPeer(String peerId) {<a name="line.162"></a> |
| <span class="sourceLineNo">163</span> usedWorkersByPeer.put(peerId, new UsedReplayWorkersForPeer(peerId));<a name="line.163"></a> |
| <span class="sourceLineNo">164</span> }<a name="line.164"></a> |
| <span class="sourceLineNo">165</span><a name="line.165"></a> |
| <span class="sourceLineNo">166</span> public void unregisterPeer(String peerId) {<a name="line.166"></a> |
| <span class="sourceLineNo">167</span> usedWorkersByPeer.remove(peerId);<a name="line.167"></a> |
| <span class="sourceLineNo">168</span> }<a name="line.168"></a> |
| <span class="sourceLineNo">169</span><a name="line.169"></a> |
| <span class="sourceLineNo">170</span> /**<a name="line.170"></a> |
| <span class="sourceLineNo">171</span> * Get a worker for replaying remote wal for a give peer. If no worker available, i.e, all the<a name="line.171"></a> |
| <span class="sourceLineNo">172</span> * region servers have been used by others, a {@link ProcedureSuspendedException} will be thrown<a name="line.172"></a> |
| <span class="sourceLineNo">173</span> * to suspend the procedure. And it will be woken up later when there are available workers,<a name="line.173"></a> |
| <span class="sourceLineNo">174</span> * either by others release a worker, or there is a new region server joins the cluster.<a name="line.174"></a> |
| <span class="sourceLineNo">175</span> */<a name="line.175"></a> |
| <span class="sourceLineNo">176</span> public ServerName acquirePeerWorker(String peerId, Procedure<?> proc)<a name="line.176"></a> |
| <span class="sourceLineNo">177</span> throws ProcedureSuspendedException {<a name="line.177"></a> |
| <span class="sourceLineNo">178</span> UsedReplayWorkersForPeer usedWorkers = usedWorkersByPeer.get(peerId);<a name="line.178"></a> |
| <span class="sourceLineNo">179</span> synchronized (usedWorkers) {<a name="line.179"></a> |
| <span class="sourceLineNo">180</span> Optional<ServerName> worker = usedWorkers.acquire(serverManager);<a name="line.180"></a> |
| <span class="sourceLineNo">181</span> if (worker.isPresent()) {<a name="line.181"></a> |
| <span class="sourceLineNo">182</span> return worker.get();<a name="line.182"></a> |
| <span class="sourceLineNo">183</span> }<a name="line.183"></a> |
| <span class="sourceLineNo">184</span> // no worker available right now, suspend the procedure<a name="line.184"></a> |
| <span class="sourceLineNo">185</span> usedWorkers.suspend(proc);<a name="line.185"></a> |
| <span class="sourceLineNo">186</span> }<a name="line.186"></a> |
| <span class="sourceLineNo">187</span> throw new ProcedureSuspendedException();<a name="line.187"></a> |
| <span class="sourceLineNo">188</span> }<a name="line.188"></a> |
| <span class="sourceLineNo">189</span><a name="line.189"></a> |
| <span class="sourceLineNo">190</span> public void releasePeerWorker(String peerId, ServerName worker,<a name="line.190"></a> |
| <span class="sourceLineNo">191</span> MasterProcedureScheduler scheduler) {<a name="line.191"></a> |
| <span class="sourceLineNo">192</span> UsedReplayWorkersForPeer usedWorkers = usedWorkersByPeer.get(peerId);<a name="line.192"></a> |
| <span class="sourceLineNo">193</span> synchronized (usedWorkers) {<a name="line.193"></a> |
| <span class="sourceLineNo">194</span> usedWorkers.release(worker);<a name="line.194"></a> |
| <span class="sourceLineNo">195</span> usedWorkers.wake(scheduler);<a name="line.195"></a> |
| <span class="sourceLineNo">196</span> }<a name="line.196"></a> |
| <span class="sourceLineNo">197</span> }<a name="line.197"></a> |
| <span class="sourceLineNo">198</span><a name="line.198"></a> |
| <span class="sourceLineNo">199</span> /**<a name="line.199"></a> |
| <span class="sourceLineNo">200</span> * Will only be called when loading procedures, where we need to construct the used worker set for<a name="line.200"></a> |
| <span class="sourceLineNo">201</span> * each peer.<a name="line.201"></a> |
| <span class="sourceLineNo">202</span> */<a name="line.202"></a> |
| <span class="sourceLineNo">203</span> public void addUsedPeerWorker(String peerId, ServerName worker) {<a name="line.203"></a> |
| <span class="sourceLineNo">204</span> usedWorkersByPeer.get(peerId).used(worker);<a name="line.204"></a> |
| <span class="sourceLineNo">205</span> }<a name="line.205"></a> |
| <span class="sourceLineNo">206</span><a name="line.206"></a> |
| <span class="sourceLineNo">207</span> public void createPeerRemoteWALDir(String peerId) throws IOException {<a name="line.207"></a> |
| <span class="sourceLineNo">208</span> Path peerRemoteWALDir = getPeerRemoteWALDir(remoteWALDir, peerId);<a name="line.208"></a> |
| <span class="sourceLineNo">209</span> if (!fs.exists(peerRemoteWALDir) && !fs.mkdirs(peerRemoteWALDir)) {<a name="line.209"></a> |
| <span class="sourceLineNo">210</span> throw new IOException("Unable to mkdir " + peerRemoteWALDir);<a name="line.210"></a> |
| <span class="sourceLineNo">211</span> }<a name="line.211"></a> |
| <span class="sourceLineNo">212</span> }<a name="line.212"></a> |
| <span class="sourceLineNo">213</span><a name="line.213"></a> |
| <span class="sourceLineNo">214</span> private void rename(Path src, Path dst, String peerId) throws IOException {<a name="line.214"></a> |
| <span class="sourceLineNo">215</span> if (fs.exists(src)) {<a name="line.215"></a> |
| <span class="sourceLineNo">216</span> deleteDir(dst, peerId);<a name="line.216"></a> |
| <span class="sourceLineNo">217</span> if (!fs.rename(src, dst)) {<a name="line.217"></a> |
| <span class="sourceLineNo">218</span> throw new IOException(<a name="line.218"></a> |
| <span class="sourceLineNo">219</span> "Failed to rename dir from " + src + " to " + dst + " for peer id=" + peerId);<a name="line.219"></a> |
| <span class="sourceLineNo">220</span> }<a name="line.220"></a> |
| <span class="sourceLineNo">221</span> LOG.info("Renamed dir from {} to {} for peer id={}", src, dst, peerId);<a name="line.221"></a> |
| <span class="sourceLineNo">222</span> } else if (!fs.exists(dst)) {<a name="line.222"></a> |
| <span class="sourceLineNo">223</span> throw new IOException(<a name="line.223"></a> |
| <span class="sourceLineNo">224</span> "Want to rename from " + src + " to " + dst + ", but they both do not exist");<a name="line.224"></a> |
| <span class="sourceLineNo">225</span> }<a name="line.225"></a> |
| <span class="sourceLineNo">226</span> }<a name="line.226"></a> |
| <span class="sourceLineNo">227</span><a name="line.227"></a> |
| <span class="sourceLineNo">228</span> public void renameToPeerReplayWALDir(String peerId) throws IOException {<a name="line.228"></a> |
| <span class="sourceLineNo">229</span> rename(getPeerRemoteWALDir(remoteWALDir, peerId), getPeerReplayWALDir(remoteWALDir, peerId),<a name="line.229"></a> |
| <span class="sourceLineNo">230</span> peerId);<a name="line.230"></a> |
| <span class="sourceLineNo">231</span> }<a name="line.231"></a> |
| <span class="sourceLineNo">232</span><a name="line.232"></a> |
| <span class="sourceLineNo">233</span> public void renameToPeerSnapshotWALDir(String peerId) throws IOException {<a name="line.233"></a> |
| <span class="sourceLineNo">234</span> rename(getPeerReplayWALDir(remoteWALDir, peerId), getPeerSnapshotWALDir(remoteWALDir, peerId),<a name="line.234"></a> |
| <span class="sourceLineNo">235</span> peerId);<a name="line.235"></a> |
| <span class="sourceLineNo">236</span> }<a name="line.236"></a> |
| <span class="sourceLineNo">237</span><a name="line.237"></a> |
| <span class="sourceLineNo">238</span> public List<Path> getReplayWALsAndCleanUpUnusedFiles(String peerId) throws IOException {<a name="line.238"></a> |
| <span class="sourceLineNo">239</span> Path peerReplayWALDir = getPeerReplayWALDir(remoteWALDir, peerId);<a name="line.239"></a> |
| <span class="sourceLineNo">240</span> for (FileStatus status : fs.listStatus(peerReplayWALDir,<a name="line.240"></a> |
| <span class="sourceLineNo">241</span> p -> p.getName().endsWith(ReplicationUtils.RENAME_WAL_SUFFIX))) {<a name="line.241"></a> |
| <span class="sourceLineNo">242</span> Path src = status.getPath();<a name="line.242"></a> |
| <span class="sourceLineNo">243</span> String srcName = src.getName();<a name="line.243"></a> |
| <span class="sourceLineNo">244</span> String dstName =<a name="line.244"></a> |
| <span class="sourceLineNo">245</span> srcName.substring(0, srcName.length() - ReplicationUtils.RENAME_WAL_SUFFIX.length());<a name="line.245"></a> |
| <span class="sourceLineNo">246</span> FSUtils.renameFile(fs, src, new Path(src.getParent(), dstName));<a name="line.246"></a> |
| <span class="sourceLineNo">247</span> }<a name="line.247"></a> |
| <span class="sourceLineNo">248</span> List<Path> wals = new ArrayList<>();<a name="line.248"></a> |
| <span class="sourceLineNo">249</span> for (FileStatus status : fs.listStatus(peerReplayWALDir)) {<a name="line.249"></a> |
| <span class="sourceLineNo">250</span> Path path = status.getPath();<a name="line.250"></a> |
| <span class="sourceLineNo">251</span> if (path.getName().endsWith(ReplicationUtils.SYNC_WAL_SUFFIX)) {<a name="line.251"></a> |
| <span class="sourceLineNo">252</span> wals.add(path);<a name="line.252"></a> |
| <span class="sourceLineNo">253</span> } else {<a name="line.253"></a> |
| <span class="sourceLineNo">254</span> if (!fs.delete(path, true)) {<a name="line.254"></a> |
| <span class="sourceLineNo">255</span> LOG.warn("Can not delete unused file: " + path);<a name="line.255"></a> |
| <span class="sourceLineNo">256</span> }<a name="line.256"></a> |
| <span class="sourceLineNo">257</span> }<a name="line.257"></a> |
| <span class="sourceLineNo">258</span> }<a name="line.258"></a> |
| <span class="sourceLineNo">259</span> return wals;<a name="line.259"></a> |
| <span class="sourceLineNo">260</span> }<a name="line.260"></a> |
| <span class="sourceLineNo">261</span><a name="line.261"></a> |
| <span class="sourceLineNo">262</span> private void deleteDir(Path dir, String peerId) throws IOException {<a name="line.262"></a> |
| <span class="sourceLineNo">263</span> if (!fs.delete(dir, true) && fs.exists(dir)) {<a name="line.263"></a> |
| <span class="sourceLineNo">264</span> throw new IOException("Failed to remove dir " + dir + " for peer id=" + peerId);<a name="line.264"></a> |
| <span class="sourceLineNo">265</span> }<a name="line.265"></a> |
| <span class="sourceLineNo">266</span> }<a name="line.266"></a> |
| <span class="sourceLineNo">267</span><a name="line.267"></a> |
| <span class="sourceLineNo">268</span> public void removePeerRemoteWALs(String peerId) throws IOException {<a name="line.268"></a> |
| <span class="sourceLineNo">269</span> deleteDir(getPeerRemoteWALDir(remoteWALDir, peerId), peerId);<a name="line.269"></a> |
| <span class="sourceLineNo">270</span> deleteDir(getPeerReplayWALDir(remoteWALDir, peerId), peerId);<a name="line.270"></a> |
| <span class="sourceLineNo">271</span> deleteDir(getPeerSnapshotWALDir(remoteWALDir, peerId), peerId);<a name="line.271"></a> |
| <span class="sourceLineNo">272</span> }<a name="line.272"></a> |
| <span class="sourceLineNo">273</span><a name="line.273"></a> |
| <span class="sourceLineNo">274</span> public String removeWALRootPath(Path path) {<a name="line.274"></a> |
| <span class="sourceLineNo">275</span> String pathStr = path.toString();<a name="line.275"></a> |
| <span class="sourceLineNo">276</span> // remove the "/" too.<a name="line.276"></a> |
| <span class="sourceLineNo">277</span> return pathStr.substring(walRootDir.toString().length() + 1);<a name="line.277"></a> |
| <span class="sourceLineNo">278</span> }<a name="line.278"></a> |
| <span class="sourceLineNo">279</span><a name="line.279"></a> |
| <span class="sourceLineNo">280</span> public void finishReplayWAL(String wal) throws IOException {<a name="line.280"></a> |
| <span class="sourceLineNo">281</span> Path walPath = new Path(walRootDir, wal);<a name="line.281"></a> |
| <span class="sourceLineNo">282</span> fs.truncate(walPath, 0);<a name="line.282"></a> |
| <span class="sourceLineNo">283</span> }<a name="line.283"></a> |
| <span class="sourceLineNo">284</span><a name="line.284"></a> |
| <span class="sourceLineNo">285</span> public boolean isReplayWALFinished(String wal) throws IOException {<a name="line.285"></a> |
| <span class="sourceLineNo">286</span> Path walPath = new Path(walRootDir, wal);<a name="line.286"></a> |
| <span class="sourceLineNo">287</span> return fs.getFileStatus(walPath).getLen() == 0;<a name="line.287"></a> |
| <span class="sourceLineNo">288</span> }<a name="line.288"></a> |
| <span class="sourceLineNo">289</span><a name="line.289"></a> |
| <span class="sourceLineNo">290</span> public Path getRemoteWALDir() {<a name="line.290"></a> |
| <span class="sourceLineNo">291</span> return remoteWALDir;<a name="line.291"></a> |
| <span class="sourceLineNo">292</span> }<a name="line.292"></a> |
| <span class="sourceLineNo">293</span>}<a name="line.293"></a> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </body> |
| </html> |