| <!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.zookeeper, class: ZKNodeTracker"> |
| <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.zookeeper;</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 org.apache.hadoop.hbase.Abortable;</span> |
| <span class="source-line-no">021</span><span id="line-21">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span> |
| <span class="source-line-no">022</span><span id="line-22">import org.apache.yetus.audience.InterfaceAudience;</span> |
| <span class="source-line-no">023</span><span id="line-23">import org.apache.zookeeper.KeeperException;</span> |
| <span class="source-line-no">024</span><span id="line-24">import org.slf4j.Logger;</span> |
| <span class="source-line-no">025</span><span id="line-25">import org.slf4j.LoggerFactory;</span> |
| <span class="source-line-no">026</span><span id="line-26"></span> |
| <span class="source-line-no">027</span><span id="line-27">/**</span> |
| <span class="source-line-no">028</span><span id="line-28"> * Tracks the availability and value of a single ZooKeeper node.</span> |
| <span class="source-line-no">029</span><span id="line-29"> * <p></span> |
| <span class="source-line-no">030</span><span id="line-30"> * Utilizes the {@link ZKListener} interface to get the necessary ZooKeeper events related to the</span> |
| <span class="source-line-no">031</span><span id="line-31"> * node.</span> |
| <span class="source-line-no">032</span><span id="line-32"> * <p></span> |
| <span class="source-line-no">033</span><span id="line-33"> * This is the base class used by trackers in both the Master and RegionServers.</span> |
| <span class="source-line-no">034</span><span id="line-34"> */</span> |
| <span class="source-line-no">035</span><span id="line-35">@InterfaceAudience.Private</span> |
| <span class="source-line-no">036</span><span id="line-36">public abstract class ZKNodeTracker extends ZKListener {</span> |
| <span class="source-line-no">037</span><span id="line-37"> // LOG is being used in subclasses, hence keeping it protected</span> |
| <span class="source-line-no">038</span><span id="line-38"> protected static final Logger LOG = LoggerFactory.getLogger(ZKNodeTracker.class);</span> |
| <span class="source-line-no">039</span><span id="line-39"> /** Path of node being tracked */</span> |
| <span class="source-line-no">040</span><span id="line-40"> protected final String node;</span> |
| <span class="source-line-no">041</span><span id="line-41"></span> |
| <span class="source-line-no">042</span><span id="line-42"> /** Data of the node being tracked */</span> |
| <span class="source-line-no">043</span><span id="line-43"> private byte[] data;</span> |
| <span class="source-line-no">044</span><span id="line-44"></span> |
| <span class="source-line-no">045</span><span id="line-45"> /** Used to abort if a fatal error occurs */</span> |
| <span class="source-line-no">046</span><span id="line-46"> protected final Abortable abortable;</span> |
| <span class="source-line-no">047</span><span id="line-47"></span> |
| <span class="source-line-no">048</span><span id="line-48"> private boolean stopped = false;</span> |
| <span class="source-line-no">049</span><span id="line-49"></span> |
| <span class="source-line-no">050</span><span id="line-50"> /**</span> |
| <span class="source-line-no">051</span><span id="line-51"> * Constructs a new ZK node tracker.</span> |
| <span class="source-line-no">052</span><span id="line-52"> * <p/></span> |
| <span class="source-line-no">053</span><span id="line-53"> * After construction, use {@link #start} to kick off tracking.</span> |
| <span class="source-line-no">054</span><span id="line-54"> * @param watcher reference to the {@link ZKWatcher} which also contains configuration and</span> |
| <span class="source-line-no">055</span><span id="line-55"> * constants</span> |
| <span class="source-line-no">056</span><span id="line-56"> * @param node path of the node being tracked</span> |
| <span class="source-line-no">057</span><span id="line-57"> * @param abortable used to abort if a fatal error occurs</span> |
| <span class="source-line-no">058</span><span id="line-58"> */</span> |
| <span class="source-line-no">059</span><span id="line-59"> public ZKNodeTracker(ZKWatcher watcher, String node, Abortable abortable) {</span> |
| <span class="source-line-no">060</span><span id="line-60"> super(watcher);</span> |
| <span class="source-line-no">061</span><span id="line-61"> this.node = node;</span> |
| <span class="source-line-no">062</span><span id="line-62"> this.abortable = abortable;</span> |
| <span class="source-line-no">063</span><span id="line-63"> this.data = null;</span> |
| <span class="source-line-no">064</span><span id="line-64"> }</span> |
| <span class="source-line-no">065</span><span id="line-65"></span> |
| <span class="source-line-no">066</span><span id="line-66"> /**</span> |
| <span class="source-line-no">067</span><span id="line-67"> * Starts the tracking of the node in ZooKeeper.</span> |
| <span class="source-line-no">068</span><span id="line-68"> * <p/></span> |
| <span class="source-line-no">069</span><span id="line-69"> * Use {@link #blockUntilAvailable()} to block until the node is available or</span> |
| <span class="source-line-no">070</span><span id="line-70"> * {@link #getData(boolean)} to get the data of the node if it is available.</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 synchronized void start() {</span> |
| <span class="source-line-no">073</span><span id="line-73"> this.watcher.registerListener(this);</span> |
| <span class="source-line-no">074</span><span id="line-74"> try {</span> |
| <span class="source-line-no">075</span><span id="line-75"> if (ZKUtil.watchAndCheckExists(watcher, node)) {</span> |
| <span class="source-line-no">076</span><span id="line-76"> byte[] data = ZKUtil.getDataAndWatch(watcher, node);</span> |
| <span class="source-line-no">077</span><span id="line-77"> if (data != null) {</span> |
| <span class="source-line-no">078</span><span id="line-78"> this.data = data;</span> |
| <span class="source-line-no">079</span><span id="line-79"> } else {</span> |
| <span class="source-line-no">080</span><span id="line-80"> // It existed but now does not, try again to ensure a watch is set</span> |
| <span class="source-line-no">081</span><span id="line-81"> LOG.debug("Try starting again because there is no data from {}", node);</span> |
| <span class="source-line-no">082</span><span id="line-82"> start();</span> |
| <span class="source-line-no">083</span><span id="line-83"> }</span> |
| <span class="source-line-no">084</span><span id="line-84"> }</span> |
| <span class="source-line-no">085</span><span id="line-85"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">086</span><span id="line-86"> abortable.abort("Unexpected exception during initialization, aborting", e);</span> |
| <span class="source-line-no">087</span><span id="line-87"> }</span> |
| <span class="source-line-no">088</span><span id="line-88"> postStart();</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"> /**</span> |
| <span class="source-line-no">092</span><span id="line-92"> * Called after start is called. Sub classes could implement this method to load more data on zk.</span> |
| <span class="source-line-no">093</span><span id="line-93"> */</span> |
| <span class="source-line-no">094</span><span id="line-94"> protected void postStart() {</span> |
| <span class="source-line-no">095</span><span id="line-95"> }</span> |
| <span class="source-line-no">096</span><span id="line-96"></span> |
| <span class="source-line-no">097</span><span id="line-97"> public synchronized void stop() {</span> |
| <span class="source-line-no">098</span><span id="line-98"> this.stopped = true;</span> |
| <span class="source-line-no">099</span><span id="line-99"> notifyAll();</span> |
| <span class="source-line-no">100</span><span id="line-100"> }</span> |
| <span class="source-line-no">101</span><span id="line-101"></span> |
| <span class="source-line-no">102</span><span id="line-102"> /**</span> |
| <span class="source-line-no">103</span><span id="line-103"> * Gets the data of the node, blocking until the node is available.</span> |
| <span class="source-line-no">104</span><span id="line-104"> * @return data of the node</span> |
| <span class="source-line-no">105</span><span id="line-105"> * @throws InterruptedException if the waiting thread is interrupted</span> |
| <span class="source-line-no">106</span><span id="line-106"> */</span> |
| <span class="source-line-no">107</span><span id="line-107"> public synchronized byte[] blockUntilAvailable() throws InterruptedException {</span> |
| <span class="source-line-no">108</span><span id="line-108"> return blockUntilAvailable(0, false);</span> |
| <span class="source-line-no">109</span><span id="line-109"> }</span> |
| <span class="source-line-no">110</span><span id="line-110"></span> |
| <span class="source-line-no">111</span><span id="line-111"> /**</span> |
| <span class="source-line-no">112</span><span id="line-112"> * Gets the data of the node, blocking until the node is available or the specified timeout has</span> |
| <span class="source-line-no">113</span><span id="line-113"> * elapsed.</span> |
| <span class="source-line-no">114</span><span id="line-114"> * @param timeout maximum time to wait for the node data to be available, n milliseconds. Pass 0</span> |
| <span class="source-line-no">115</span><span id="line-115"> * for no timeout.</span> |
| <span class="source-line-no">116</span><span id="line-116"> * @return data of the node</span> |
| <span class="source-line-no">117</span><span id="line-117"> * @throws InterruptedException if the waiting thread is interrupted</span> |
| <span class="source-line-no">118</span><span id="line-118"> */</span> |
| <span class="source-line-no">119</span><span id="line-119"> public synchronized byte[] blockUntilAvailable(long timeout, boolean refresh)</span> |
| <span class="source-line-no">120</span><span id="line-120"> throws InterruptedException {</span> |
| <span class="source-line-no">121</span><span id="line-121"> if (timeout < 0) {</span> |
| <span class="source-line-no">122</span><span id="line-122"> throw new IllegalArgumentException();</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"> boolean notimeout = timeout == 0;</span> |
| <span class="source-line-no">126</span><span id="line-126"> long startTime = EnvironmentEdgeManager.currentTime();</span> |
| <span class="source-line-no">127</span><span id="line-127"> long remaining = timeout;</span> |
| <span class="source-line-no">128</span><span id="line-128"> if (refresh) {</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"> // This does not create a watch if the node does not exists</span> |
| <span class="source-line-no">131</span><span id="line-131"> this.data = ZKUtil.getDataAndWatch(watcher, node);</span> |
| <span class="source-line-no">132</span><span id="line-132"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">133</span><span id="line-133"> // We use to abort here, but in some cases the abort is ignored (</span> |
| <span class="source-line-no">134</span><span id="line-134"> // (empty Abortable), so it's better to log...</span> |
| <span class="source-line-no">135</span><span id="line-135"> LOG.warn("Unexpected exception handling blockUntilAvailable", e);</span> |
| <span class="source-line-no">136</span><span id="line-136"> abortable.abort("Unexpected exception handling blockUntilAvailable", e);</span> |
| <span class="source-line-no">137</span><span id="line-137"> }</span> |
| <span class="source-line-no">138</span><span id="line-138"> }</span> |
| <span class="source-line-no">139</span><span id="line-139"> boolean nodeExistsChecked = (!refresh || data != null);</span> |
| <span class="source-line-no">140</span><span id="line-140"> while (!this.stopped && (notimeout || remaining > 0) && this.data == null) {</span> |
| <span class="source-line-no">141</span><span id="line-141"> if (!nodeExistsChecked) {</span> |
| <span class="source-line-no">142</span><span id="line-142"> try {</span> |
| <span class="source-line-no">143</span><span id="line-143"> nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1);</span> |
| <span class="source-line-no">144</span><span id="line-144"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">145</span><span id="line-145"> LOG.warn("Got exception while trying to check existence in ZooKeeper" + " of the node: "</span> |
| <span class="source-line-no">146</span><span id="line-146"> + node + ", retrying if timeout not reached", e);</span> |
| <span class="source-line-no">147</span><span id="line-147"> }</span> |
| <span class="source-line-no">148</span><span id="line-148"></span> |
| <span class="source-line-no">149</span><span id="line-149"> // It did not exists, and now it does.</span> |
| <span class="source-line-no">150</span><span id="line-150"> if (nodeExistsChecked) {</span> |
| <span class="source-line-no">151</span><span id="line-151"> LOG.debug("Node {} now exists, resetting a watcher", node);</span> |
| <span class="source-line-no">152</span><span id="line-152"> try {</span> |
| <span class="source-line-no">153</span><span id="line-153"> // This does not create a watch if the node does not exists</span> |
| <span class="source-line-no">154</span><span id="line-154"> this.data = ZKUtil.getDataAndWatch(watcher, node);</span> |
| <span class="source-line-no">155</span><span id="line-155"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">156</span><span id="line-156"> LOG.warn("Unexpected exception handling blockUntilAvailable", e);</span> |
| <span class="source-line-no">157</span><span id="line-157"> abortable.abort("Unexpected exception handling blockUntilAvailable", e);</span> |
| <span class="source-line-no">158</span><span id="line-158"> }</span> |
| <span class="source-line-no">159</span><span id="line-159"> }</span> |
| <span class="source-line-no">160</span><span id="line-160"> }</span> |
| <span class="source-line-no">161</span><span id="line-161"> // We expect a notification; but we wait with a</span> |
| <span class="source-line-no">162</span><span id="line-162"> // a timeout to lower the impact of a race condition if any</span> |
| <span class="source-line-no">163</span><span id="line-163"> wait(100);</span> |
| <span class="source-line-no">164</span><span id="line-164"> remaining = timeout - (EnvironmentEdgeManager.currentTime() - startTime);</span> |
| <span class="source-line-no">165</span><span id="line-165"> }</span> |
| <span class="source-line-no">166</span><span id="line-166"> return this.data;</span> |
| <span class="source-line-no">167</span><span id="line-167"> }</span> |
| <span class="source-line-no">168</span><span id="line-168"></span> |
| <span class="source-line-no">169</span><span id="line-169"> /**</span> |
| <span class="source-line-no">170</span><span id="line-170"> * Gets the data of the node.</span> |
| <span class="source-line-no">171</span><span id="line-171"> * <p></span> |
| <span class="source-line-no">172</span><span id="line-172"> * If the node is currently available, the most up-to-date known version of the data is returned.</span> |
| <span class="source-line-no">173</span><span id="line-173"> * If the node is not currently available, null is returned.</span> |
| <span class="source-line-no">174</span><span id="line-174"> * @param refresh whether to refresh the data by calling ZK directly.</span> |
| <span class="source-line-no">175</span><span id="line-175"> * @return data of the node, null if unavailable</span> |
| <span class="source-line-no">176</span><span id="line-176"> */</span> |
| <span class="source-line-no">177</span><span id="line-177"> public synchronized byte[] getData(boolean refresh) {</span> |
| <span class="source-line-no">178</span><span id="line-178"> if (refresh) {</span> |
| <span class="source-line-no">179</span><span id="line-179"> try {</span> |
| <span class="source-line-no">180</span><span id="line-180"> this.data = ZKUtil.getDataAndWatch(watcher, node);</span> |
| <span class="source-line-no">181</span><span id="line-181"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">182</span><span id="line-182"> abortable.abort("Unexpected exception handling getData", e);</span> |
| <span class="source-line-no">183</span><span id="line-183"> }</span> |
| <span class="source-line-no">184</span><span id="line-184"> }</span> |
| <span class="source-line-no">185</span><span id="line-185"> return this.data;</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"> public String getNode() {</span> |
| <span class="source-line-no">189</span><span id="line-189"> return this.node;</span> |
| <span class="source-line-no">190</span><span id="line-190"> }</span> |
| <span class="source-line-no">191</span><span id="line-191"></span> |
| <span class="source-line-no">192</span><span id="line-192"> @Override</span> |
| <span class="source-line-no">193</span><span id="line-193"> public synchronized void nodeCreated(String path) {</span> |
| <span class="source-line-no">194</span><span id="line-194"> if (!path.equals(node)) {</span> |
| <span class="source-line-no">195</span><span id="line-195"> return;</span> |
| <span class="source-line-no">196</span><span id="line-196"> }</span> |
| <span class="source-line-no">197</span><span id="line-197"></span> |
| <span class="source-line-no">198</span><span id="line-198"> try {</span> |
| <span class="source-line-no">199</span><span id="line-199"> byte[] data = ZKUtil.getDataAndWatch(watcher, node);</span> |
| <span class="source-line-no">200</span><span id="line-200"> if (data != null) {</span> |
| <span class="source-line-no">201</span><span id="line-201"> this.data = data;</span> |
| <span class="source-line-no">202</span><span id="line-202"> notifyAll();</span> |
| <span class="source-line-no">203</span><span id="line-203"> } else {</span> |
| <span class="source-line-no">204</span><span id="line-204"> nodeDeleted(path);</span> |
| <span class="source-line-no">205</span><span id="line-205"> }</span> |
| <span class="source-line-no">206</span><span id="line-206"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">207</span><span id="line-207"> abortable.abort("Unexpected exception handling nodeCreated event", e);</span> |
| <span class="source-line-no">208</span><span id="line-208"> }</span> |
| <span class="source-line-no">209</span><span id="line-209"> }</span> |
| <span class="source-line-no">210</span><span id="line-210"></span> |
| <span class="source-line-no">211</span><span id="line-211"> @Override</span> |
| <span class="source-line-no">212</span><span id="line-212"> public synchronized void nodeDeleted(String path) {</span> |
| <span class="source-line-no">213</span><span id="line-213"> if (path.equals(node)) {</span> |
| <span class="source-line-no">214</span><span id="line-214"> try {</span> |
| <span class="source-line-no">215</span><span id="line-215"> if (ZKUtil.watchAndCheckExists(watcher, node)) {</span> |
| <span class="source-line-no">216</span><span id="line-216"> nodeCreated(path);</span> |
| <span class="source-line-no">217</span><span id="line-217"> } else {</span> |
| <span class="source-line-no">218</span><span id="line-218"> this.data = null;</span> |
| <span class="source-line-no">219</span><span id="line-219"> }</span> |
| <span class="source-line-no">220</span><span id="line-220"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">221</span><span id="line-221"> abortable.abort("Unexpected exception handling nodeDeleted event", e);</span> |
| <span class="source-line-no">222</span><span id="line-222"> }</span> |
| <span class="source-line-no">223</span><span id="line-223"> }</span> |
| <span class="source-line-no">224</span><span id="line-224"> }</span> |
| <span class="source-line-no">225</span><span id="line-225"></span> |
| <span class="source-line-no">226</span><span id="line-226"> @Override</span> |
| <span class="source-line-no">227</span><span id="line-227"> public synchronized void nodeDataChanged(String path) {</span> |
| <span class="source-line-no">228</span><span id="line-228"> if (path.equals(node)) {</span> |
| <span class="source-line-no">229</span><span id="line-229"> nodeCreated(path);</span> |
| <span class="source-line-no">230</span><span id="line-230"> }</span> |
| <span class="source-line-no">231</span><span id="line-231"> }</span> |
| <span class="source-line-no">232</span><span id="line-232"></span> |
| <span class="source-line-no">233</span><span id="line-233"> /**</span> |
| <span class="source-line-no">234</span><span id="line-234"> * Checks if the baseznode set as per the property 'zookeeper.znode.parent' exists.</span> |
| <span class="source-line-no">235</span><span id="line-235"> * @return true if baseznode exists. false if doesnot exists.</span> |
| <span class="source-line-no">236</span><span id="line-236"> */</span> |
| <span class="source-line-no">237</span><span id="line-237"> public boolean checkIfBaseNodeAvailable() {</span> |
| <span class="source-line-no">238</span><span id="line-238"> try {</span> |
| <span class="source-line-no">239</span><span id="line-239"> if (ZKUtil.checkExists(watcher, watcher.getZNodePaths().baseZNode) == -1) {</span> |
| <span class="source-line-no">240</span><span id="line-240"> return false;</span> |
| <span class="source-line-no">241</span><span id="line-241"> }</span> |
| <span class="source-line-no">242</span><span id="line-242"> } catch (KeeperException e) {</span> |
| <span class="source-line-no">243</span><span id="line-243"> abortable.abort("Exception while checking if basenode (" + watcher.getZNodePaths().baseZNode</span> |
| <span class="source-line-no">244</span><span id="line-244"> + ") exists in ZooKeeper.", e);</span> |
| <span class="source-line-no">245</span><span id="line-245"> }</span> |
| <span class="source-line-no">246</span><span id="line-246"> return true;</span> |
| <span class="source-line-no">247</span><span id="line-247"> }</span> |
| <span class="source-line-no">248</span><span id="line-248"></span> |
| <span class="source-line-no">249</span><span id="line-249"> @Override</span> |
| <span class="source-line-no">250</span><span id="line-250"> public String toString() {</span> |
| <span class="source-line-no">251</span><span id="line-251"> return "ZKNodeTracker{" + "node='" + node + ", stopped=" + stopped + '}';</span> |
| <span class="source-line-no">252</span><span id="line-252"> }</span> |
| <span class="source-line-no">253</span><span id="line-253">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |