| <!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.replication.master;<a name="line.18"></a> |
| <span class="sourceLineNo">019</span><a name="line.19"></a> |
| <span class="sourceLineNo">020</span>import static org.hamcrest.MatcherAssert.assertThat;<a name="line.20"></a> |
| <span class="sourceLineNo">021</span>import static org.hamcrest.Matchers.emptyIterable;<a name="line.21"></a> |
| <span class="sourceLineNo">022</span>import static org.junit.Assert.assertFalse;<a name="line.22"></a> |
| <span class="sourceLineNo">023</span>import static org.junit.Assert.assertSame;<a name="line.23"></a> |
| <span class="sourceLineNo">024</span>import static org.junit.Assert.assertTrue;<a name="line.24"></a> |
| <span class="sourceLineNo">025</span>import static org.mockito.Mockito.mock;<a name="line.25"></a> |
| <span class="sourceLineNo">026</span>import static org.mockito.Mockito.when;<a name="line.26"></a> |
| <span class="sourceLineNo">027</span><a name="line.27"></a> |
| <span class="sourceLineNo">028</span>import java.util.ArrayList;<a name="line.28"></a> |
| <span class="sourceLineNo">029</span>import java.util.Arrays;<a name="line.29"></a> |
| <span class="sourceLineNo">030</span>import java.util.Collections;<a name="line.30"></a> |
| <span class="sourceLineNo">031</span>import java.util.Iterator;<a name="line.31"></a> |
| <span class="sourceLineNo">032</span>import java.util.List;<a name="line.32"></a> |
| <span class="sourceLineNo">033</span>import java.util.Map;<a name="line.33"></a> |
| <span class="sourceLineNo">034</span>import java.util.stream.Collectors;<a name="line.34"></a> |
| <span class="sourceLineNo">035</span>import java.util.stream.Stream;<a name="line.35"></a> |
| <span class="sourceLineNo">036</span>import org.apache.hadoop.conf.Configuration;<a name="line.36"></a> |
| <span class="sourceLineNo">037</span>import org.apache.hadoop.fs.FileStatus;<a name="line.37"></a> |
| <span class="sourceLineNo">038</span>import org.apache.hadoop.fs.Path;<a name="line.38"></a> |
| <span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.HBaseClassTestRule;<a name="line.39"></a> |
| <span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.HBaseConfiguration;<a name="line.40"></a> |
| <span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.ServerName;<a name="line.41"></a> |
| <span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.master.HMaster;<a name="line.42"></a> |
| <span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.master.MasterServices;<a name="line.43"></a> |
| <span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.master.ServerManager;<a name="line.44"></a> |
| <span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;<a name="line.45"></a> |
| <span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;<a name="line.46"></a> |
| <span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;<a name="line.47"></a> |
| <span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;<a name="line.48"></a> |
| <span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.replication.ReplicationException;<a name="line.49"></a> |
| <span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;<a name="line.50"></a> |
| <span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;<a name="line.51"></a> |
| <span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.replication.ReplicationQueueData;<a name="line.52"></a> |
| <span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.replication.ReplicationQueueId;<a name="line.53"></a> |
| <span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;<a name="line.54"></a> |
| <span class="sourceLineNo">055</span>import org.apache.hadoop.hbase.testclassification.MasterTests;<a name="line.55"></a> |
| <span class="sourceLineNo">056</span>import org.apache.hadoop.hbase.testclassification.SmallTests;<a name="line.56"></a> |
| <span class="sourceLineNo">057</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.57"></a> |
| <span class="sourceLineNo">058</span>import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;<a name="line.58"></a> |
| <span class="sourceLineNo">059</span>import org.junit.After;<a name="line.59"></a> |
| <span class="sourceLineNo">060</span>import org.junit.Before;<a name="line.60"></a> |
| <span class="sourceLineNo">061</span>import org.junit.ClassRule;<a name="line.61"></a> |
| <span class="sourceLineNo">062</span>import org.junit.Test;<a name="line.62"></a> |
| <span class="sourceLineNo">063</span>import org.junit.experimental.categories.Category;<a name="line.63"></a> |
| <span class="sourceLineNo">064</span><a name="line.64"></a> |
| <span class="sourceLineNo">065</span>import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;<a name="line.65"></a> |
| <span class="sourceLineNo">066</span><a name="line.66"></a> |
| <span class="sourceLineNo">067</span>@Category({ MasterTests.class, SmallTests.class })<a name="line.67"></a> |
| <span class="sourceLineNo">068</span>public class TestReplicationLogCleaner {<a name="line.68"></a> |
| <span class="sourceLineNo">069</span><a name="line.69"></a> |
| <span class="sourceLineNo">070</span> @ClassRule<a name="line.70"></a> |
| <span class="sourceLineNo">071</span> public static final HBaseClassTestRule CLASS_RULE =<a name="line.71"></a> |
| <span class="sourceLineNo">072</span> HBaseClassTestRule.forClass(TestReplicationLogCleaner.class);<a name="line.72"></a> |
| <span class="sourceLineNo">073</span><a name="line.73"></a> |
| <span class="sourceLineNo">074</span> private static final Configuration CONF = HBaseConfiguration.create();<a name="line.74"></a> |
| <span class="sourceLineNo">075</span><a name="line.75"></a> |
| <span class="sourceLineNo">076</span> private MasterServices services;<a name="line.76"></a> |
| <span class="sourceLineNo">077</span><a name="line.77"></a> |
| <span class="sourceLineNo">078</span> private ReplicationLogCleaner cleaner;<a name="line.78"></a> |
| <span class="sourceLineNo">079</span><a name="line.79"></a> |
| <span class="sourceLineNo">080</span> @Before<a name="line.80"></a> |
| <span class="sourceLineNo">081</span> public void setUp() throws ReplicationException {<a name="line.81"></a> |
| <span class="sourceLineNo">082</span> services = mock(MasterServices.class);<a name="line.82"></a> |
| <span class="sourceLineNo">083</span> when(services.getReplicationLogCleanerBarrier()).thenReturn(new ReplicationLogCleanerBarrier());<a name="line.83"></a> |
| <span class="sourceLineNo">084</span> ReplicationPeerManager rpm = mock(ReplicationPeerManager.class);<a name="line.84"></a> |
| <span class="sourceLineNo">085</span> when(services.getReplicationPeerManager()).thenReturn(rpm);<a name="line.85"></a> |
| <span class="sourceLineNo">086</span> when(rpm.listPeers(null)).thenReturn(new ArrayList<>());<a name="line.86"></a> |
| <span class="sourceLineNo">087</span> ReplicationQueueStorage rqs = mock(ReplicationQueueStorage.class);<a name="line.87"></a> |
| <span class="sourceLineNo">088</span> when(rpm.getQueueStorage()).thenReturn(rqs);<a name="line.88"></a> |
| <span class="sourceLineNo">089</span> when(rpm.getQueueStorage().hasData()).thenReturn(true);<a name="line.89"></a> |
| <span class="sourceLineNo">090</span> when(rqs.listAllQueues()).thenReturn(new ArrayList<>());<a name="line.90"></a> |
| <span class="sourceLineNo">091</span> ServerManager sm = mock(ServerManager.class);<a name="line.91"></a> |
| <span class="sourceLineNo">092</span> when(services.getServerManager()).thenReturn(sm);<a name="line.92"></a> |
| <span class="sourceLineNo">093</span> when(sm.getOnlineServersList()).thenReturn(new ArrayList<>());<a name="line.93"></a> |
| <span class="sourceLineNo">094</span> @SuppressWarnings("unchecked")<a name="line.94"></a> |
| <span class="sourceLineNo">095</span> ProcedureExecutor<MasterProcedureEnv> procExec = mock(ProcedureExecutor.class);<a name="line.95"></a> |
| <span class="sourceLineNo">096</span> when(services.getMasterProcedureExecutor()).thenReturn(procExec);<a name="line.96"></a> |
| <span class="sourceLineNo">097</span> when(procExec.getProcedures()).thenReturn(new ArrayList<>());<a name="line.97"></a> |
| <span class="sourceLineNo">098</span><a name="line.98"></a> |
| <span class="sourceLineNo">099</span> cleaner = new ReplicationLogCleaner();<a name="line.99"></a> |
| <span class="sourceLineNo">100</span> cleaner.setConf(CONF);<a name="line.100"></a> |
| <span class="sourceLineNo">101</span> Map<String, Object> params = ImmutableMap.of(HMaster.MASTER, services);<a name="line.101"></a> |
| <span class="sourceLineNo">102</span> cleaner.init(params);<a name="line.102"></a> |
| <span class="sourceLineNo">103</span> }<a name="line.103"></a> |
| <span class="sourceLineNo">104</span><a name="line.104"></a> |
| <span class="sourceLineNo">105</span> @After<a name="line.105"></a> |
| <span class="sourceLineNo">106</span> public void tearDown() {<a name="line.106"></a> |
| <span class="sourceLineNo">107</span> cleaner.postClean();<a name="line.107"></a> |
| <span class="sourceLineNo">108</span> }<a name="line.108"></a> |
| <span class="sourceLineNo">109</span><a name="line.109"></a> |
| <span class="sourceLineNo">110</span> private static Iterable<FileStatus> runCleaner(ReplicationLogCleaner cleaner,<a name="line.110"></a> |
| <span class="sourceLineNo">111</span> Iterable<FileStatus> files) {<a name="line.111"></a> |
| <span class="sourceLineNo">112</span> cleaner.preClean();<a name="line.112"></a> |
| <span class="sourceLineNo">113</span> return cleaner.getDeletableFiles(files);<a name="line.113"></a> |
| <span class="sourceLineNo">114</span> }<a name="line.114"></a> |
| <span class="sourceLineNo">115</span><a name="line.115"></a> |
| <span class="sourceLineNo">116</span> private static FileStatus createFileStatus(Path path) {<a name="line.116"></a> |
| <span class="sourceLineNo">117</span> return new FileStatus(100, false, 3, 256, EnvironmentEdgeManager.currentTime(), path);<a name="line.117"></a> |
| <span class="sourceLineNo">118</span> }<a name="line.118"></a> |
| <span class="sourceLineNo">119</span><a name="line.119"></a> |
| <span class="sourceLineNo">120</span> private static FileStatus createFileStatus(ServerName sn, int number) {<a name="line.120"></a> |
| <span class="sourceLineNo">121</span> Path path = new Path(sn.toString() + "." + number);<a name="line.121"></a> |
| <span class="sourceLineNo">122</span> return createFileStatus(path);<a name="line.122"></a> |
| <span class="sourceLineNo">123</span> }<a name="line.123"></a> |
| <span class="sourceLineNo">124</span><a name="line.124"></a> |
| <span class="sourceLineNo">125</span> private static ReplicationPeerDescription createPeer(String peerId) {<a name="line.125"></a> |
| <span class="sourceLineNo">126</span> return new ReplicationPeerDescription(peerId, true, null, null);<a name="line.126"></a> |
| <span class="sourceLineNo">127</span> }<a name="line.127"></a> |
| <span class="sourceLineNo">128</span><a name="line.128"></a> |
| <span class="sourceLineNo">129</span> private void addServer(ServerName serverName) {<a name="line.129"></a> |
| <span class="sourceLineNo">130</span> services.getServerManager().getOnlineServersList().add(serverName);<a name="line.130"></a> |
| <span class="sourceLineNo">131</span> }<a name="line.131"></a> |
| <span class="sourceLineNo">132</span><a name="line.132"></a> |
| <span class="sourceLineNo">133</span> private void addSCP(ServerName serverName, boolean finished) {<a name="line.133"></a> |
| <span class="sourceLineNo">134</span> ServerCrashProcedure scp = mock(ServerCrashProcedure.class);<a name="line.134"></a> |
| <span class="sourceLineNo">135</span> when(scp.getServerName()).thenReturn(serverName);<a name="line.135"></a> |
| <span class="sourceLineNo">136</span> when(scp.isFinished()).thenReturn(finished);<a name="line.136"></a> |
| <span class="sourceLineNo">137</span> services.getMasterProcedureExecutor().getProcedures().add(scp);<a name="line.137"></a> |
| <span class="sourceLineNo">138</span> }<a name="line.138"></a> |
| <span class="sourceLineNo">139</span><a name="line.139"></a> |
| <span class="sourceLineNo">140</span> private void addPeer(String... peerIds) {<a name="line.140"></a> |
| <span class="sourceLineNo">141</span> services.getReplicationPeerManager().listPeers(null).addAll(<a name="line.141"></a> |
| <span class="sourceLineNo">142</span> Stream.of(peerIds).map(TestReplicationLogCleaner::createPeer).collect(Collectors.toList()));<a name="line.142"></a> |
| <span class="sourceLineNo">143</span> }<a name="line.143"></a> |
| <span class="sourceLineNo">144</span><a name="line.144"></a> |
| <span class="sourceLineNo">145</span> private void addQueueData(ReplicationQueueData... datas) throws ReplicationException {<a name="line.145"></a> |
| <span class="sourceLineNo">146</span> services.getReplicationPeerManager().getQueueStorage().listAllQueues()<a name="line.146"></a> |
| <span class="sourceLineNo">147</span> .addAll(Arrays.asList(datas));<a name="line.147"></a> |
| <span class="sourceLineNo">148</span> }<a name="line.148"></a> |
| <span class="sourceLineNo">149</span><a name="line.149"></a> |
| <span class="sourceLineNo">150</span> @Test<a name="line.150"></a> |
| <span class="sourceLineNo">151</span> public void testNoConf() {<a name="line.151"></a> |
| <span class="sourceLineNo">152</span> ReplicationLogCleaner cleaner = new ReplicationLogCleaner();<a name="line.152"></a> |
| <span class="sourceLineNo">153</span> List<FileStatus> files = Arrays.asList(new FileStatus());<a name="line.153"></a> |
| <span class="sourceLineNo">154</span> assertSame(files, runCleaner(cleaner, files));<a name="line.154"></a> |
| <span class="sourceLineNo">155</span> cleaner.postClean();<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> @Test<a name="line.158"></a> |
| <span class="sourceLineNo">159</span> public void testCanNotFilter() {<a name="line.159"></a> |
| <span class="sourceLineNo">160</span> assertTrue(services.getReplicationLogCleanerBarrier().disable());<a name="line.160"></a> |
| <span class="sourceLineNo">161</span> List<FileStatus> files = Arrays.asList(new FileStatus());<a name="line.161"></a> |
| <span class="sourceLineNo">162</span> assertSame(Collections.emptyList(), runCleaner(cleaner, files));<a name="line.162"></a> |
| <span class="sourceLineNo">163</span> }<a name="line.163"></a> |
| <span class="sourceLineNo">164</span><a name="line.164"></a> |
| <span class="sourceLineNo">165</span> @Test<a name="line.165"></a> |
| <span class="sourceLineNo">166</span> public void testNoPeer() {<a name="line.166"></a> |
| <span class="sourceLineNo">167</span> Path path = new Path("/wal." + EnvironmentEdgeManager.currentTime());<a name="line.167"></a> |
| <span class="sourceLineNo">168</span> assertTrue(AbstractFSWALProvider.validateWALFilename(path.getName()));<a name="line.168"></a> |
| <span class="sourceLineNo">169</span> FileStatus file = createFileStatus(path);<a name="line.169"></a> |
| <span class="sourceLineNo">170</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.170"></a> |
| <span class="sourceLineNo">171</span> assertSame(file, iter.next());<a name="line.171"></a> |
| <span class="sourceLineNo">172</span> assertFalse(iter.hasNext());<a name="line.172"></a> |
| <span class="sourceLineNo">173</span> }<a name="line.173"></a> |
| <span class="sourceLineNo">174</span><a name="line.174"></a> |
| <span class="sourceLineNo">175</span> @Test<a name="line.175"></a> |
| <span class="sourceLineNo">176</span> public void testNotValidWalFile() {<a name="line.176"></a> |
| <span class="sourceLineNo">177</span> addPeer("1");<a name="line.177"></a> |
| <span class="sourceLineNo">178</span> Path path = new Path("/whatever");<a name="line.178"></a> |
| <span class="sourceLineNo">179</span> assertFalse(AbstractFSWALProvider.validateWALFilename(path.getName()));<a name="line.179"></a> |
| <span class="sourceLineNo">180</span> FileStatus file = createFileStatus(path);<a name="line.180"></a> |
| <span class="sourceLineNo">181</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.181"></a> |
| <span class="sourceLineNo">182</span> assertSame(file, iter.next());<a name="line.182"></a> |
| <span class="sourceLineNo">183</span> assertFalse(iter.hasNext());<a name="line.183"></a> |
| <span class="sourceLineNo">184</span> }<a name="line.184"></a> |
| <span class="sourceLineNo">185</span><a name="line.185"></a> |
| <span class="sourceLineNo">186</span> @Test<a name="line.186"></a> |
| <span class="sourceLineNo">187</span> public void testMetaWalFile() {<a name="line.187"></a> |
| <span class="sourceLineNo">188</span> addPeer("1");<a name="line.188"></a> |
| <span class="sourceLineNo">189</span> Path path = new Path(<a name="line.189"></a> |
| <span class="sourceLineNo">190</span> "/wal." + EnvironmentEdgeManager.currentTime() + AbstractFSWALProvider.META_WAL_PROVIDER_ID);<a name="line.190"></a> |
| <span class="sourceLineNo">191</span> assertTrue(AbstractFSWALProvider.validateWALFilename(path.getName()));<a name="line.191"></a> |
| <span class="sourceLineNo">192</span> assertTrue(AbstractFSWALProvider.isMetaFile(path));<a name="line.192"></a> |
| <span class="sourceLineNo">193</span> FileStatus file = createFileStatus(path);<a name="line.193"></a> |
| <span class="sourceLineNo">194</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.194"></a> |
| <span class="sourceLineNo">195</span> assertSame(file, iter.next());<a name="line.195"></a> |
| <span class="sourceLineNo">196</span> assertFalse(iter.hasNext());<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> @Test<a name="line.199"></a> |
| <span class="sourceLineNo">200</span> public void testLiveRegionServerNoQueues() {<a name="line.200"></a> |
| <span class="sourceLineNo">201</span> addPeer("1");<a name="line.201"></a> |
| <span class="sourceLineNo">202</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.202"></a> |
| <span class="sourceLineNo">203</span> addServer(sn);<a name="line.203"></a> |
| <span class="sourceLineNo">204</span> List<FileStatus> files = Arrays.asList(createFileStatus(sn, 1));<a name="line.204"></a> |
| <span class="sourceLineNo">205</span> assertThat(runCleaner(cleaner, files), emptyIterable());<a name="line.205"></a> |
| <span class="sourceLineNo">206</span> }<a name="line.206"></a> |
| <span class="sourceLineNo">207</span><a name="line.207"></a> |
| <span class="sourceLineNo">208</span> @Test<a name="line.208"></a> |
| <span class="sourceLineNo">209</span> public void testLiveRegionServerWithSCPNoQueues() {<a name="line.209"></a> |
| <span class="sourceLineNo">210</span> addPeer("1");<a name="line.210"></a> |
| <span class="sourceLineNo">211</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.211"></a> |
| <span class="sourceLineNo">212</span> addSCP(sn, false);<a name="line.212"></a> |
| <span class="sourceLineNo">213</span> List<FileStatus> files = Arrays.asList(createFileStatus(sn, 1));<a name="line.213"></a> |
| <span class="sourceLineNo">214</span> assertThat(runCleaner(cleaner, files), emptyIterable());<a name="line.214"></a> |
| <span class="sourceLineNo">215</span> }<a name="line.215"></a> |
| <span class="sourceLineNo">216</span><a name="line.216"></a> |
| <span class="sourceLineNo">217</span> @Test<a name="line.217"></a> |
| <span class="sourceLineNo">218</span> public void testDeadRegionServerNoQueues() {<a name="line.218"></a> |
| <span class="sourceLineNo">219</span> addPeer("1");<a name="line.219"></a> |
| <span class="sourceLineNo">220</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.220"></a> |
| <span class="sourceLineNo">221</span> FileStatus file = createFileStatus(sn, 1);<a name="line.221"></a> |
| <span class="sourceLineNo">222</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.222"></a> |
| <span class="sourceLineNo">223</span> assertSame(file, iter.next());<a name="line.223"></a> |
| <span class="sourceLineNo">224</span> assertFalse(iter.hasNext());<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> @Test<a name="line.227"></a> |
| <span class="sourceLineNo">228</span> public void testDeadRegionServerWithSCPNoQueues() {<a name="line.228"></a> |
| <span class="sourceLineNo">229</span> addPeer("1");<a name="line.229"></a> |
| <span class="sourceLineNo">230</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.230"></a> |
| <span class="sourceLineNo">231</span> addSCP(sn, true);<a name="line.231"></a> |
| <span class="sourceLineNo">232</span> FileStatus file = createFileStatus(sn, 1);<a name="line.232"></a> |
| <span class="sourceLineNo">233</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.233"></a> |
| <span class="sourceLineNo">234</span> assertSame(file, iter.next());<a name="line.234"></a> |
| <span class="sourceLineNo">235</span> assertFalse(iter.hasNext());<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> @Test<a name="line.238"></a> |
| <span class="sourceLineNo">239</span> public void testLiveRegionServerMissingQueue() throws ReplicationException {<a name="line.239"></a> |
| <span class="sourceLineNo">240</span> String peerId1 = "1";<a name="line.240"></a> |
| <span class="sourceLineNo">241</span> String peerId2 = "2";<a name="line.241"></a> |
| <span class="sourceLineNo">242</span> addPeer(peerId1, peerId2);<a name="line.242"></a> |
| <span class="sourceLineNo">243</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.243"></a> |
| <span class="sourceLineNo">244</span> addServer(sn);<a name="line.244"></a> |
| <span class="sourceLineNo">245</span> FileStatus file = createFileStatus(sn, 1);<a name="line.245"></a> |
| <span class="sourceLineNo">246</span> ReplicationQueueData data1 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId1),<a name="line.246"></a> |
| <span class="sourceLineNo">247</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.247"></a> |
| <span class="sourceLineNo">248</span> addQueueData(data1);<a name="line.248"></a> |
| <span class="sourceLineNo">249</span> assertThat(runCleaner(cleaner, Arrays.asList(file)), emptyIterable());<a name="line.249"></a> |
| <span class="sourceLineNo">250</span> }<a name="line.250"></a> |
| <span class="sourceLineNo">251</span><a name="line.251"></a> |
| <span class="sourceLineNo">252</span> @Test<a name="line.252"></a> |
| <span class="sourceLineNo">253</span> public void testLiveRegionServerShouldNotDelete() throws ReplicationException {<a name="line.253"></a> |
| <span class="sourceLineNo">254</span> String peerId = "1";<a name="line.254"></a> |
| <span class="sourceLineNo">255</span> addPeer(peerId);<a name="line.255"></a> |
| <span class="sourceLineNo">256</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.256"></a> |
| <span class="sourceLineNo">257</span> addServer(sn);<a name="line.257"></a> |
| <span class="sourceLineNo">258</span> FileStatus file = createFileStatus(sn, 1);<a name="line.258"></a> |
| <span class="sourceLineNo">259</span> ReplicationQueueData data = new ReplicationQueueData(new ReplicationQueueId(sn, peerId),<a name="line.259"></a> |
| <span class="sourceLineNo">260</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), 0)));<a name="line.260"></a> |
| <span class="sourceLineNo">261</span> addQueueData(data);<a name="line.261"></a> |
| <span class="sourceLineNo">262</span> assertThat(runCleaner(cleaner, Arrays.asList(file)), emptyIterable());<a name="line.262"></a> |
| <span class="sourceLineNo">263</span> }<a name="line.263"></a> |
| <span class="sourceLineNo">264</span><a name="line.264"></a> |
| <span class="sourceLineNo">265</span> @Test<a name="line.265"></a> |
| <span class="sourceLineNo">266</span> public void testLiveRegionServerShouldNotDeleteTwoPeers() throws ReplicationException {<a name="line.266"></a> |
| <span class="sourceLineNo">267</span> String peerId1 = "1";<a name="line.267"></a> |
| <span class="sourceLineNo">268</span> String peerId2 = "2";<a name="line.268"></a> |
| <span class="sourceLineNo">269</span> addPeer(peerId1, peerId2);<a name="line.269"></a> |
| <span class="sourceLineNo">270</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.270"></a> |
| <span class="sourceLineNo">271</span> addServer(sn);<a name="line.271"></a> |
| <span class="sourceLineNo">272</span> FileStatus file = createFileStatus(sn, 1);<a name="line.272"></a> |
| <span class="sourceLineNo">273</span> ReplicationQueueData data1 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId1),<a name="line.273"></a> |
| <span class="sourceLineNo">274</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.274"></a> |
| <span class="sourceLineNo">275</span> ReplicationQueueData data2 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId2),<a name="line.275"></a> |
| <span class="sourceLineNo">276</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), 0)));<a name="line.276"></a> |
| <span class="sourceLineNo">277</span> addQueueData(data1, data2);<a name="line.277"></a> |
| <span class="sourceLineNo">278</span> assertThat(runCleaner(cleaner, Arrays.asList(file)), emptyIterable());<a name="line.278"></a> |
| <span class="sourceLineNo">279</span> }<a name="line.279"></a> |
| <span class="sourceLineNo">280</span><a name="line.280"></a> |
| <span class="sourceLineNo">281</span> @Test<a name="line.281"></a> |
| <span class="sourceLineNo">282</span> public void testLiveRegionServerShouldDelete() throws ReplicationException {<a name="line.282"></a> |
| <span class="sourceLineNo">283</span> String peerId = "1";<a name="line.283"></a> |
| <span class="sourceLineNo">284</span> addPeer(peerId);<a name="line.284"></a> |
| <span class="sourceLineNo">285</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.285"></a> |
| <span class="sourceLineNo">286</span> addServer(sn);<a name="line.286"></a> |
| <span class="sourceLineNo">287</span> FileStatus file = createFileStatus(sn, 1);<a name="line.287"></a> |
| <span class="sourceLineNo">288</span> ReplicationQueueData data = new ReplicationQueueData(new ReplicationQueueId(sn, peerId),<a name="line.288"></a> |
| <span class="sourceLineNo">289</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.289"></a> |
| <span class="sourceLineNo">290</span> services.getReplicationPeerManager().getQueueStorage().listAllQueues().add(data);<a name="line.290"></a> |
| <span class="sourceLineNo">291</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.291"></a> |
| <span class="sourceLineNo">292</span> assertSame(file, iter.next());<a name="line.292"></a> |
| <span class="sourceLineNo">293</span> assertFalse(iter.hasNext());<a name="line.293"></a> |
| <span class="sourceLineNo">294</span> }<a name="line.294"></a> |
| <span class="sourceLineNo">295</span><a name="line.295"></a> |
| <span class="sourceLineNo">296</span> @Test<a name="line.296"></a> |
| <span class="sourceLineNo">297</span> public void testLiveRegionServerShouldDeleteTwoPeers() throws ReplicationException {<a name="line.297"></a> |
| <span class="sourceLineNo">298</span> String peerId1 = "1";<a name="line.298"></a> |
| <span class="sourceLineNo">299</span> String peerId2 = "2";<a name="line.299"></a> |
| <span class="sourceLineNo">300</span> addPeer(peerId1, peerId2);<a name="line.300"></a> |
| <span class="sourceLineNo">301</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.301"></a> |
| <span class="sourceLineNo">302</span> addServer(sn);<a name="line.302"></a> |
| <span class="sourceLineNo">303</span> FileStatus file = createFileStatus(sn, 1);<a name="line.303"></a> |
| <span class="sourceLineNo">304</span> ReplicationQueueData data1 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId1),<a name="line.304"></a> |
| <span class="sourceLineNo">305</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.305"></a> |
| <span class="sourceLineNo">306</span> ReplicationQueueData data2 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId2),<a name="line.306"></a> |
| <span class="sourceLineNo">307</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.307"></a> |
| <span class="sourceLineNo">308</span> addQueueData(data1, data2);<a name="line.308"></a> |
| <span class="sourceLineNo">309</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.309"></a> |
| <span class="sourceLineNo">310</span> assertSame(file, iter.next());<a name="line.310"></a> |
| <span class="sourceLineNo">311</span> assertFalse(iter.hasNext());<a name="line.311"></a> |
| <span class="sourceLineNo">312</span> }<a name="line.312"></a> |
| <span class="sourceLineNo">313</span><a name="line.313"></a> |
| <span class="sourceLineNo">314</span> @Test<a name="line.314"></a> |
| <span class="sourceLineNo">315</span> public void testDeadRegionServerMissingQueue() throws ReplicationException {<a name="line.315"></a> |
| <span class="sourceLineNo">316</span> String peerId1 = "1";<a name="line.316"></a> |
| <span class="sourceLineNo">317</span> String peerId2 = "2";<a name="line.317"></a> |
| <span class="sourceLineNo">318</span> addPeer(peerId1, peerId2);<a name="line.318"></a> |
| <span class="sourceLineNo">319</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.319"></a> |
| <span class="sourceLineNo">320</span> FileStatus file = createFileStatus(sn, 1);<a name="line.320"></a> |
| <span class="sourceLineNo">321</span> ReplicationQueueData data1 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId1),<a name="line.321"></a> |
| <span class="sourceLineNo">322</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.322"></a> |
| <span class="sourceLineNo">323</span> addQueueData(data1);<a name="line.323"></a> |
| <span class="sourceLineNo">324</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.324"></a> |
| <span class="sourceLineNo">325</span> assertSame(file, iter.next());<a name="line.325"></a> |
| <span class="sourceLineNo">326</span> assertFalse(iter.hasNext());<a name="line.326"></a> |
| <span class="sourceLineNo">327</span> }<a name="line.327"></a> |
| <span class="sourceLineNo">328</span><a name="line.328"></a> |
| <span class="sourceLineNo">329</span> @Test<a name="line.329"></a> |
| <span class="sourceLineNo">330</span> public void testDeadRegionServerShouldNotDelete() throws ReplicationException {<a name="line.330"></a> |
| <span class="sourceLineNo">331</span> String peerId = "1";<a name="line.331"></a> |
| <span class="sourceLineNo">332</span> addPeer(peerId);<a name="line.332"></a> |
| <span class="sourceLineNo">333</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.333"></a> |
| <span class="sourceLineNo">334</span> FileStatus file = createFileStatus(sn, 1);<a name="line.334"></a> |
| <span class="sourceLineNo">335</span> ReplicationQueueData data = new ReplicationQueueData(new ReplicationQueueId(sn, peerId),<a name="line.335"></a> |
| <span class="sourceLineNo">336</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), 0)));<a name="line.336"></a> |
| <span class="sourceLineNo">337</span> addQueueData(data);<a name="line.337"></a> |
| <span class="sourceLineNo">338</span> assertThat(runCleaner(cleaner, Arrays.asList(file)), emptyIterable());<a name="line.338"></a> |
| <span class="sourceLineNo">339</span> }<a name="line.339"></a> |
| <span class="sourceLineNo">340</span><a name="line.340"></a> |
| <span class="sourceLineNo">341</span> @Test<a name="line.341"></a> |
| <span class="sourceLineNo">342</span> public void testDeadRegionServerShouldNotDeleteTwoPeers() throws ReplicationException {<a name="line.342"></a> |
| <span class="sourceLineNo">343</span> String peerId1 = "1";<a name="line.343"></a> |
| <span class="sourceLineNo">344</span> String peerId2 = "2";<a name="line.344"></a> |
| <span class="sourceLineNo">345</span> addPeer(peerId1, peerId2);<a name="line.345"></a> |
| <span class="sourceLineNo">346</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.346"></a> |
| <span class="sourceLineNo">347</span> FileStatus file = createFileStatus(sn, 1);<a name="line.347"></a> |
| <span class="sourceLineNo">348</span> ReplicationQueueData data1 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId1),<a name="line.348"></a> |
| <span class="sourceLineNo">349</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.349"></a> |
| <span class="sourceLineNo">350</span> ReplicationQueueData data2 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId2),<a name="line.350"></a> |
| <span class="sourceLineNo">351</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), 0)));<a name="line.351"></a> |
| <span class="sourceLineNo">352</span> addQueueData(data1, data2);<a name="line.352"></a> |
| <span class="sourceLineNo">353</span> assertThat(runCleaner(cleaner, Arrays.asList(file)), emptyIterable());<a name="line.353"></a> |
| <span class="sourceLineNo">354</span> }<a name="line.354"></a> |
| <span class="sourceLineNo">355</span><a name="line.355"></a> |
| <span class="sourceLineNo">356</span> @Test<a name="line.356"></a> |
| <span class="sourceLineNo">357</span> public void testDeadRegionServerShouldDelete() throws ReplicationException {<a name="line.357"></a> |
| <span class="sourceLineNo">358</span> String peerId = "1";<a name="line.358"></a> |
| <span class="sourceLineNo">359</span> addPeer(peerId);<a name="line.359"></a> |
| <span class="sourceLineNo">360</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.360"></a> |
| <span class="sourceLineNo">361</span> FileStatus file = createFileStatus(sn, 1);<a name="line.361"></a> |
| <span class="sourceLineNo">362</span> ReplicationQueueData data = new ReplicationQueueData(new ReplicationQueueId(sn, peerId),<a name="line.362"></a> |
| <span class="sourceLineNo">363</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.363"></a> |
| <span class="sourceLineNo">364</span> services.getReplicationPeerManager().getQueueStorage().listAllQueues().add(data);<a name="line.364"></a> |
| <span class="sourceLineNo">365</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.365"></a> |
| <span class="sourceLineNo">366</span> assertSame(file, iter.next());<a name="line.366"></a> |
| <span class="sourceLineNo">367</span> assertFalse(iter.hasNext());<a name="line.367"></a> |
| <span class="sourceLineNo">368</span> }<a name="line.368"></a> |
| <span class="sourceLineNo">369</span><a name="line.369"></a> |
| <span class="sourceLineNo">370</span> @Test<a name="line.370"></a> |
| <span class="sourceLineNo">371</span> public void testDeadRegionServerShouldDeleteTwoPeers() throws ReplicationException {<a name="line.371"></a> |
| <span class="sourceLineNo">372</span> String peerId1 = "1";<a name="line.372"></a> |
| <span class="sourceLineNo">373</span> String peerId2 = "2";<a name="line.373"></a> |
| <span class="sourceLineNo">374</span> addPeer(peerId1, peerId2);<a name="line.374"></a> |
| <span class="sourceLineNo">375</span> ServerName sn = ServerName.valueOf("server,123," + EnvironmentEdgeManager.currentTime());<a name="line.375"></a> |
| <span class="sourceLineNo">376</span> FileStatus file = createFileStatus(sn, 1);<a name="line.376"></a> |
| <span class="sourceLineNo">377</span> ReplicationQueueData data1 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId1),<a name="line.377"></a> |
| <span class="sourceLineNo">378</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.378"></a> |
| <span class="sourceLineNo">379</span> ReplicationQueueData data2 = new ReplicationQueueData(new ReplicationQueueId(sn, peerId2),<a name="line.379"></a> |
| <span class="sourceLineNo">380</span> ImmutableMap.of(sn.toString(), new ReplicationGroupOffset(file.getPath().getName(), -1)));<a name="line.380"></a> |
| <span class="sourceLineNo">381</span> addQueueData(data1, data2);<a name="line.381"></a> |
| <span class="sourceLineNo">382</span> Iterator<FileStatus> iter = runCleaner(cleaner, Arrays.asList(file)).iterator();<a name="line.382"></a> |
| <span class="sourceLineNo">383</span> assertSame(file, iter.next());<a name="line.383"></a> |
| <span class="sourceLineNo">384</span> assertFalse(iter.hasNext());<a name="line.384"></a> |
| <span class="sourceLineNo">385</span> }<a name="line.385"></a> |
| <span class="sourceLineNo">386</span>}<a name="line.386"></a> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </body> |
| </html> |