001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.procedure.flush;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Collection;
023import java.util.List;
024import java.util.concurrent.Callable;
025import java.util.concurrent.ExecutionException;
026import java.util.concurrent.ExecutorCompletionService;
027import java.util.concurrent.Future;
028import java.util.concurrent.ThreadPoolExecutor;
029import java.util.concurrent.TimeUnit;
030
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.hbase.Abortable;
033import org.apache.hadoop.hbase.DaemonThreadFactory;
034import org.apache.hadoop.hbase.DroppedSnapshotException;
035import org.apache.hadoop.hbase.HBaseInterfaceAudience;
036import org.apache.hadoop.hbase.TableName;
037import org.apache.hadoop.hbase.errorhandling.ForeignException;
038import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
039import org.apache.hadoop.hbase.procedure.ProcedureMember;
040import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
041import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
042import org.apache.hadoop.hbase.procedure.Subprocedure;
043import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
044import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
045import org.apache.hadoop.hbase.regionserver.HRegion;
046import org.apache.hadoop.hbase.regionserver.HRegionServer;
047import org.apache.hadoop.hbase.regionserver.RegionServerServices;
048import org.apache.hadoop.hbase.util.Threads;
049import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
050import org.apache.zookeeper.KeeperException;
051
052import org.apache.yetus.audience.InterfaceAudience;
053import org.slf4j.Logger;
054import org.slf4j.LoggerFactory;
055
056/**
057 * This manager class handles flushing of the regions for table on a {@link HRegionServer}.
058 */
059@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
060public class RegionServerFlushTableProcedureManager extends RegionServerProcedureManager {
061  private static final Logger LOG =
062      LoggerFactory.getLogger(RegionServerFlushTableProcedureManager.class);
063
064  private static final String CONCURENT_FLUSH_TASKS_KEY =
065      "hbase.flush.procedure.region.concurrentTasks";
066  private static final int DEFAULT_CONCURRENT_FLUSH_TASKS = 3;
067
068  public static final String FLUSH_REQUEST_THREADS_KEY =
069      "hbase.flush.procedure.region.pool.threads";
070  public static final int FLUSH_REQUEST_THREADS_DEFAULT = 10;
071
072  public static final String FLUSH_TIMEOUT_MILLIS_KEY =
073      "hbase.flush.procedure.region.timeout";
074  public static final long FLUSH_TIMEOUT_MILLIS_DEFAULT = 60000;
075
076  public static final String FLUSH_REQUEST_WAKE_MILLIS_KEY =
077      "hbase.flush.procedure.region.wakefrequency";
078  private static final long FLUSH_REQUEST_WAKE_MILLIS_DEFAULT = 500;
079
080  private RegionServerServices rss;
081  private ProcedureMemberRpcs memberRpcs;
082  private ProcedureMember member;
083
084  /**
085   * Exposed for testing.
086   * @param conf HBase configuration.
087   * @param server region server.
088   * @param memberRpc use specified memberRpc instance
089   * @param procMember use specified ProcedureMember
090   */
091   RegionServerFlushTableProcedureManager(Configuration conf, HRegionServer server,
092      ProcedureMemberRpcs memberRpc, ProcedureMember procMember) {
093    this.rss = server;
094    this.memberRpcs = memberRpc;
095    this.member = procMember;
096  }
097
098  public RegionServerFlushTableProcedureManager() {}
099
100  /**
101   * Start accepting flush table requests.
102   */
103  @Override
104  public void start() {
105    LOG.debug("Start region server flush procedure manager " + rss.getServerName().toString());
106    this.memberRpcs.start(rss.getServerName().toString(), member);
107  }
108
109  /**
110   * Close <tt>this</tt> and all running tasks
111   * @param force forcefully stop all running tasks
112   * @throws IOException
113   */
114  @Override
115  public void stop(boolean force) throws IOException {
116    String mode = force ? "abruptly" : "gracefully";
117    LOG.info("Stopping region server flush procedure manager " + mode + ".");
118
119    try {
120      this.member.close();
121    } finally {
122      this.memberRpcs.close();
123    }
124  }
125
126  /**
127   * If in a running state, creates the specified subprocedure to flush table regions.
128   *
129   * Because this gets the local list of regions to flush and not the set the master had,
130   * there is a possibility of a race where regions may be missed.
131   *
132   * @param table
133   * @return Subprocedure to submit to the ProcedureMemeber.
134   */
135  public Subprocedure buildSubprocedure(String table) {
136
137    // don't run the subprocedure if the parent is stop(ping)
138    if (rss.isStopping() || rss.isStopped()) {
139      throw new IllegalStateException("Can't start flush region subprocedure on RS: "
140          + rss.getServerName() + ", because stopping/stopped!");
141    }
142
143    // check to see if this server is hosting any regions for the table
144    List<HRegion> involvedRegions;
145    try {
146      involvedRegions = getRegionsToFlush(table);
147    } catch (IOException e1) {
148      throw new IllegalStateException("Failed to figure out if there is region to flush.", e1);
149    }
150
151    // We need to run the subprocedure even if we have no relevant regions.  The coordinator
152    // expects participation in the procedure and without sending message the master procedure
153    // will hang and fail.
154
155    LOG.debug("Launching subprocedure to flush regions for " + table);
156    ForeignExceptionDispatcher exnDispatcher = new ForeignExceptionDispatcher(table);
157    Configuration conf = rss.getConfiguration();
158    long timeoutMillis = conf.getLong(FLUSH_TIMEOUT_MILLIS_KEY,
159        FLUSH_TIMEOUT_MILLIS_DEFAULT);
160    long wakeMillis = conf.getLong(FLUSH_REQUEST_WAKE_MILLIS_KEY,
161        FLUSH_REQUEST_WAKE_MILLIS_DEFAULT);
162
163    FlushTableSubprocedurePool taskManager =
164        new FlushTableSubprocedurePool(rss.getServerName().toString(), conf, rss);
165    return new FlushTableSubprocedure(member, exnDispatcher, wakeMillis,
166      timeoutMillis, involvedRegions, table, taskManager);
167  }
168
169  /**
170   * Get the list of regions to flush for the table on this server
171   *
172   * It is possible that if a region moves somewhere between the calls
173   * we'll miss the region.
174   *
175   * @param table
176   * @return the list of online regions. Empty list is returned if no regions.
177   * @throws IOException
178   */
179  private List<HRegion> getRegionsToFlush(String table) throws IOException {
180    return (List<HRegion>) rss.getRegions(TableName.valueOf(table));
181  }
182
183  public class FlushTableSubprocedureBuilder implements SubprocedureFactory {
184
185    @Override
186    public Subprocedure buildSubprocedure(String name, byte[] data) {
187      // The name of the procedure instance from the master is the table name.
188      return RegionServerFlushTableProcedureManager.this.buildSubprocedure(name);
189    }
190
191  }
192
193  /**
194   * We use the FlushTableSubprocedurePool, a class specific thread pool instead of
195   * {@link org.apache.hadoop.hbase.executor.ExecutorService}.
196   *
197   * It uses a {@link java.util.concurrent.ExecutorCompletionService} which provides queuing of
198   * completed tasks which lets us efficiently cancel pending tasks upon the earliest operation
199   * failures.
200   */
201  static class FlushTableSubprocedurePool {
202    private final Abortable abortable;
203    private final ExecutorCompletionService<Void> taskPool;
204    private final ThreadPoolExecutor executor;
205    private volatile boolean stopped;
206    private final List<Future<Void>> futures = new ArrayList<>();
207    private final String name;
208
209    FlushTableSubprocedurePool(String name, Configuration conf, Abortable abortable) {
210      this.abortable = abortable;
211      // configure the executor service
212      long keepAlive = conf.getLong(
213        RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_KEY,
214        RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_DEFAULT);
215      int threads = conf.getInt(CONCURENT_FLUSH_TASKS_KEY, DEFAULT_CONCURRENT_FLUSH_TASKS);
216      this.name = name;
217      executor = Threads.getBoundedCachedThreadPool(threads, keepAlive, TimeUnit.MILLISECONDS,
218          new DaemonThreadFactory("rs(" + name + ")-flush-proc-pool-"));
219      taskPool = new ExecutorCompletionService<>(executor);
220    }
221
222    boolean hasTasks() {
223      return futures.size() != 0;
224    }
225
226    /**
227     * Submit a task to the pool.
228     *
229     * NOTE: all must be submitted before you can safely {@link #waitForOutstandingTasks()}.
230     */
231    void submitTask(final Callable<Void> task) {
232      Future<Void> f = this.taskPool.submit(task);
233      futures.add(f);
234    }
235
236    /**
237     * Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)}.
238     * This *must* be called after all tasks are submitted via submitTask.
239     *
240     * @return <tt>true</tt> on success, <tt>false</tt> otherwise
241     * @throws InterruptedException
242     */
243    boolean waitForOutstandingTasks() throws ForeignException, InterruptedException {
244      LOG.debug("Waiting for local region flush to finish.");
245
246      int sz = futures.size();
247      try {
248        // Using the completion service to process the futures.
249        for (int i = 0; i < sz; i++) {
250          Future<Void> f = taskPool.take();
251          f.get();
252          if (!futures.remove(f)) {
253            LOG.warn("unexpected future" + f);
254          }
255          LOG.debug("Completed " + (i+1) + "/" + sz +  " local region flush tasks.");
256        }
257        LOG.debug("Completed " + sz +  " local region flush tasks.");
258        return true;
259      } catch (InterruptedException e) {
260        LOG.warn("Got InterruptedException in FlushSubprocedurePool", e);
261        if (!stopped) {
262          Thread.currentThread().interrupt();
263          throw new ForeignException("FlushSubprocedurePool", e);
264        }
265        // we are stopped so we can just exit.
266      } catch (ExecutionException e) {
267        Throwable cause = e.getCause();
268        if (cause instanceof ForeignException) {
269          LOG.warn("Rethrowing ForeignException from FlushSubprocedurePool", e);
270          throw (ForeignException)e.getCause();
271        } else if (cause instanceof DroppedSnapshotException) {
272          // we have to abort the region server according to contract of flush
273          abortable.abort("Received DroppedSnapshotException, aborting", cause);
274        }
275        LOG.warn("Got Exception in FlushSubprocedurePool", e);
276        throw new ForeignException(name, e.getCause());
277      } finally {
278        cancelTasks();
279      }
280      return false;
281    }
282
283    /**
284     * This attempts to cancel out all pending and in progress tasks. Does not interrupt the running
285     * tasks itself. An ongoing HRegion.flush() should not be interrupted (see HBASE-13877).
286     * @throws InterruptedException
287     */
288    void cancelTasks() throws InterruptedException {
289      Collection<Future<Void>> tasks = futures;
290      LOG.debug("cancelling " + tasks.size() + " flush region tasks " + name);
291      for (Future<Void> f: tasks) {
292        f.cancel(false);
293      }
294
295      // evict remaining tasks and futures from taskPool.
296      futures.clear();
297      while (taskPool.poll() != null) {}
298      stop();
299    }
300
301    /**
302     * Gracefully shutdown the thread pool. An ongoing HRegion.flush() should not be
303     * interrupted (see HBASE-13877)
304     */
305    void stop() {
306      if (this.stopped) return;
307
308      this.stopped = true;
309      this.executor.shutdown();
310    }
311  }
312
313  /**
314   * Initialize this region server flush procedure manager
315   * Uses a zookeeper based member controller.
316   * @param rss region server
317   * @throws KeeperException if the zookeeper cannot be reached
318   */
319  @Override
320  public void initialize(RegionServerServices rss) throws KeeperException {
321    this.rss = rss;
322    ZKWatcher zkw = rss.getZooKeeper();
323    this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
324      MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE);
325
326    Configuration conf = rss.getConfiguration();
327    long keepAlive = conf.getLong(FLUSH_TIMEOUT_MILLIS_KEY, FLUSH_TIMEOUT_MILLIS_DEFAULT);
328    int opThreads = conf.getInt(FLUSH_REQUEST_THREADS_KEY, FLUSH_REQUEST_THREADS_DEFAULT);
329
330    // create the actual flush table procedure member
331    ThreadPoolExecutor pool = ProcedureMember.defaultPool(rss.getServerName().toString(),
332      opThreads, keepAlive);
333    this.member = new ProcedureMember(memberRpcs, pool, new FlushTableSubprocedureBuilder());
334  }
335
336  @Override
337  public String getProcedureSignature() {
338    return MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE;
339  }
340
341}