blob: 64470e26f09b6504a30581ca1363c7752d79f6ba [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.procedure2, class: ProcedureFutureUtil">
<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.procedure2;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import java.io.IOException;</span>
<span class="source-line-no">021</span><span id="line-21">import java.util.concurrent.CompletableFuture;</span>
<span class="source-line-no">022</span><span id="line-22">import java.util.concurrent.ExecutorService;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.function.Consumer;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.function.Supplier;</span>
<span class="source-line-no">025</span><span id="line-25">import org.apache.commons.lang3.mutable.MutableBoolean;</span>
<span class="source-line-no">026</span><span id="line-26">import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;</span>
<span class="source-line-no">027</span><span id="line-27">import org.apache.hadoop.hbase.util.FutureUtils;</span>
<span class="source-line-no">028</span><span id="line-28">import org.apache.hadoop.hbase.util.IdLock;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">030</span><span id="line-30">import org.slf4j.Logger;</span>
<span class="source-line-no">031</span><span id="line-31">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">032</span><span id="line-32"></span>
<span class="source-line-no">033</span><span id="line-33">/**</span>
<span class="source-line-no">034</span><span id="line-34"> * A helper class for switching procedure out(yielding) while it is doing some time consuming</span>
<span class="source-line-no">035</span><span id="line-35"> * operation, such as updating meta, where we can get a {@link CompletableFuture} about the</span>
<span class="source-line-no">036</span><span id="line-36"> * operation.</span>
<span class="source-line-no">037</span><span id="line-37"> */</span>
<span class="source-line-no">038</span><span id="line-38">@InterfaceAudience.Private</span>
<span class="source-line-no">039</span><span id="line-39">public final class ProcedureFutureUtil {</span>
<span class="source-line-no">040</span><span id="line-40"></span>
<span class="source-line-no">041</span><span id="line-41"> private static final Logger LOG = LoggerFactory.getLogger(ProcedureFutureUtil.class);</span>
<span class="source-line-no">042</span><span id="line-42"></span>
<span class="source-line-no">043</span><span id="line-43"> private ProcedureFutureUtil() {</span>
<span class="source-line-no">044</span><span id="line-44"> }</span>
<span class="source-line-no">045</span><span id="line-45"></span>
<span class="source-line-no">046</span><span id="line-46"> public static boolean checkFuture(Procedure&lt;?&gt; proc, Supplier&lt;CompletableFuture&lt;Void&gt;&gt; getFuture,</span>
<span class="source-line-no">047</span><span id="line-47"> Consumer&lt;CompletableFuture&lt;Void&gt;&gt; setFuture, Runnable actionAfterDone) throws IOException {</span>
<span class="source-line-no">048</span><span id="line-48"> CompletableFuture&lt;Void&gt; future = getFuture.get();</span>
<span class="source-line-no">049</span><span id="line-49"> if (future == null) {</span>
<span class="source-line-no">050</span><span id="line-50"> return false;</span>
<span class="source-line-no">051</span><span id="line-51"> }</span>
<span class="source-line-no">052</span><span id="line-52"> // reset future</span>
<span class="source-line-no">053</span><span id="line-53"> setFuture.accept(null);</span>
<span class="source-line-no">054</span><span id="line-54"> FutureUtils.get(future);</span>
<span class="source-line-no">055</span><span id="line-55"> actionAfterDone.run();</span>
<span class="source-line-no">056</span><span id="line-56"> return true;</span>
<span class="source-line-no">057</span><span id="line-57"> }</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 static void suspendIfNecessary(Procedure&lt;?&gt; proc,</span>
<span class="source-line-no">060</span><span id="line-60"> Consumer&lt;CompletableFuture&lt;Void&gt;&gt; setFuture, CompletableFuture&lt;Void&gt; future,</span>
<span class="source-line-no">061</span><span id="line-61"> MasterProcedureEnv env, Runnable actionAfterDone)</span>
<span class="source-line-no">062</span><span id="line-62"> throws IOException, ProcedureSuspendedException {</span>
<span class="source-line-no">063</span><span id="line-63"> MutableBoolean completed = new MutableBoolean(false);</span>
<span class="source-line-no">064</span><span id="line-64"> Thread currentThread = Thread.currentThread();</span>
<span class="source-line-no">065</span><span id="line-65"> // This is for testing. In ProcedureTestingUtility, we will restart a ProcedureExecutor and</span>
<span class="source-line-no">066</span><span id="line-66"> // reuse it, for performance, so we need to make sure that all the procedure have been stopped.</span>
<span class="source-line-no">067</span><span id="line-67"> // But here, the callback of this future is not executed in a PEWorker, so in ProcedureExecutor</span>
<span class="source-line-no">068</span><span id="line-68"> // we have no way to stop it. So here, we will get the asyncTaskExecutor first, in the PEWorker</span>
<span class="source-line-no">069</span><span id="line-69"> // thread, where the ProcedureExecutor should have not been stopped yet, then when calling the</span>
<span class="source-line-no">070</span><span id="line-70"> // callback, if the ProcedureExecutor have already been stopped and restarted, the</span>
<span class="source-line-no">071</span><span id="line-71"> // asyncTaskExecutor will also be shutdown so we can not add anything back to the scheduler.</span>
<span class="source-line-no">072</span><span id="line-72"> ExecutorService asyncTaskExecutor = env.getAsyncTaskExecutor();</span>
<span class="source-line-no">073</span><span id="line-73"> FutureUtils.addListener(future, (r, e) -&gt; {</span>
<span class="source-line-no">074</span><span id="line-74"> if (Thread.currentThread() == currentThread) {</span>
<span class="source-line-no">075</span><span id="line-75"> LOG.debug("The future has completed while adding callback, give up suspending procedure {}",</span>
<span class="source-line-no">076</span><span id="line-76"> proc);</span>
<span class="source-line-no">077</span><span id="line-77"> // this means the future has already been completed, as we call the callback directly while</span>
<span class="source-line-no">078</span><span id="line-78"> // calling addListener, so here we just set completed to true without doing anything</span>
<span class="source-line-no">079</span><span id="line-79"> completed.setTrue();</span>
<span class="source-line-no">080</span><span id="line-80"> return;</span>
<span class="source-line-no">081</span><span id="line-81"> }</span>
<span class="source-line-no">082</span><span id="line-82"> LOG.debug("Going to wake up procedure {} because future has completed", proc);</span>
<span class="source-line-no">083</span><span id="line-83"> // This callback may be called inside netty's event loop, so we should not block it for a long</span>
<span class="source-line-no">084</span><span id="line-84"> // time. The worker executor will hold the execution lock while executing the procedure, and</span>
<span class="source-line-no">085</span><span id="line-85"> // we may persist the procedure state inside the lock, which is a time consuming operation.</span>
<span class="source-line-no">086</span><span id="line-86"> // And what makes things worse is that, we persist procedure state to master local region,</span>
<span class="source-line-no">087</span><span id="line-87"> // where the AsyncFSWAL implementation will use the same netty's event loop for dealing with</span>
<span class="source-line-no">088</span><span id="line-88"> // I/O, which could even cause dead lock.</span>
<span class="source-line-no">089</span><span id="line-89"> asyncTaskExecutor.execute(() -&gt; wakeUp(proc, env));</span>
<span class="source-line-no">090</span><span id="line-90"> });</span>
<span class="source-line-no">091</span><span id="line-91"> if (completed.getValue()) {</span>
<span class="source-line-no">092</span><span id="line-92"> FutureUtils.get(future);</span>
<span class="source-line-no">093</span><span id="line-93"> actionAfterDone.run();</span>
<span class="source-line-no">094</span><span id="line-94"> } else {</span>
<span class="source-line-no">095</span><span id="line-95"> // suspend the procedure</span>
<span class="source-line-no">096</span><span id="line-96"> setFuture.accept(future);</span>
<span class="source-line-no">097</span><span id="line-97"> proc.skipPersistence();</span>
<span class="source-line-no">098</span><span id="line-98"> suspend(proc);</span>
<span class="source-line-no">099</span><span id="line-99"> }</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"> public static void suspend(Procedure&lt;?&gt; proc) throws ProcedureSuspendedException {</span>
<span class="source-line-no">103</span><span id="line-103"> proc.skipPersistence();</span>
<span class="source-line-no">104</span><span id="line-104"> throw new ProcedureSuspendedException();</span>
<span class="source-line-no">105</span><span id="line-105"> }</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 static void wakeUp(Procedure&lt;?&gt; proc, MasterProcedureEnv env) {</span>
<span class="source-line-no">108</span><span id="line-108"> // should acquire procedure execution lock to make sure that the procedure executor has</span>
<span class="source-line-no">109</span><span id="line-109"> // finished putting this procedure to the WAITING_TIMEOUT state, otherwise there could be</span>
<span class="source-line-no">110</span><span id="line-110"> // race and cause unexpected result</span>
<span class="source-line-no">111</span><span id="line-111"> IdLock procLock = env.getMasterServices().getMasterProcedureExecutor().getProcExecutionLock();</span>
<span class="source-line-no">112</span><span id="line-112"> IdLock.Entry lockEntry;</span>
<span class="source-line-no">113</span><span id="line-113"> try {</span>
<span class="source-line-no">114</span><span id="line-114"> lockEntry = procLock.getLockEntry(proc.getProcId());</span>
<span class="source-line-no">115</span><span id="line-115"> } catch (IOException e) {</span>
<span class="source-line-no">116</span><span id="line-116"> LOG.error("Error while acquiring execution lock for procedure {}"</span>
<span class="source-line-no">117</span><span id="line-117"> + " when trying to wake it up, aborting...", proc, e);</span>
<span class="source-line-no">118</span><span id="line-118"> env.getMasterServices().abort("Can not acquire procedure execution lock", e);</span>
<span class="source-line-no">119</span><span id="line-119"> return;</span>
<span class="source-line-no">120</span><span id="line-120"> }</span>
<span class="source-line-no">121</span><span id="line-121"> try {</span>
<span class="source-line-no">122</span><span id="line-122"> env.getProcedureScheduler().addFront(proc);</span>
<span class="source-line-no">123</span><span id="line-123"> } finally {</span>
<span class="source-line-no">124</span><span id="line-124"> procLock.releaseLockEntry(lockEntry);</span>
<span class="source-line-no">125</span><span id="line-125"> }</span>
<span class="source-line-no">126</span><span id="line-126"> }</span>
<span class="source-line-no">127</span><span id="line-127">}</span>
</pre>
</div>
</main>
</body>
</html>