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;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.hbase.Abortable;
032import org.apache.hadoop.hbase.DroppedSnapshotException;
033import org.apache.hadoop.hbase.HBaseInterfaceAudience;
034import org.apache.hadoop.hbase.TableName;
035import org.apache.hadoop.hbase.errorhandling.ForeignException;
036import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
037import org.apache.hadoop.hbase.procedure.ProcedureMember;
038import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
039import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
040import org.apache.hadoop.hbase.procedure.Subprocedure;
041import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
042import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
043import org.apache.hadoop.hbase.regionserver.HRegion;
044import org.apache.hadoop.hbase.regionserver.HRegionServer;
045import org.apache.hadoop.hbase.regionserver.RegionServerServices;
046import org.apache.hadoop.hbase.util.Strings;
047import org.apache.hadoop.hbase.util.Threads;
048import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
049import org.apache.yetus.audience.InterfaceAudience;
050import org.apache.zookeeper.KeeperException;
051import org.slf4j.Logger;
052import org.slf4j.LoggerFactory;
053
054import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
055import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
056
057import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
058
059/**
060 * This manager class handles flushing of the regions for table on a {@link HRegionServer}.
061 */
062@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
063public class RegionServerFlushTableProcedureManager extends RegionServerProcedureManager {
064  private static final Logger LOG =
065    LoggerFactory.getLogger(RegionServerFlushTableProcedureManager.class);
066
067  private static final String CONCURENT_FLUSH_TASKS_KEY =
068    "hbase.flush.procedure.region.concurrentTasks";
069  private static final int DEFAULT_CONCURRENT_FLUSH_TASKS = 3;
070
071  public static final String FLUSH_REQUEST_THREADS_KEY =
072    "hbase.flush.procedure.region.pool.threads";
073  public static final int FLUSH_REQUEST_THREADS_DEFAULT = 10;
074
075  public static final String FLUSH_TIMEOUT_MILLIS_KEY = "hbase.flush.procedure.region.timeout";
076  public static final long FLUSH_TIMEOUT_MILLIS_DEFAULT = 60000;
077
078  public static final String FLUSH_REQUEST_WAKE_MILLIS_KEY =
079    "hbase.flush.procedure.region.wakefrequency";
080  private static final long FLUSH_REQUEST_WAKE_MILLIS_DEFAULT = 500;
081
082  private RegionServerServices rss;
083  private ProcedureMemberRpcs memberRpcs;
084  private ProcedureMember member;
085
086  /**
087   * Exposed for testing.
088   * @param conf       HBase configuration.
089   * @param server     region server.
090   * @param memberRpc  use specified memberRpc instance
091   * @param procMember use specified ProcedureMember
092   */
093  RegionServerFlushTableProcedureManager(Configuration conf, HRegionServer server,
094    ProcedureMemberRpcs memberRpc, ProcedureMember procMember) {
095    this.rss = server;
096    this.memberRpcs = memberRpc;
097    this.member = procMember;
098  }
099
100  public RegionServerFlushTableProcedureManager() {
101  }
102
103  /**
104   * Start accepting flush table requests.
105   */
106  @Override
107  public void start() {
108    LOG.debug("Start region server flush procedure manager " + rss.getServerName().toString());
109    this.memberRpcs.start(rss.getServerName().toString(), member);
110  }
111
112  /**
113   * Close <tt>this</tt> and all running tasks
114   * @param force forcefully stop all running tasks
115   */
116  @Override
117  public void stop(boolean force) throws IOException {
118    String mode = force ? "abruptly" : "gracefully";
119    LOG.info("Stopping region server flush procedure manager " + mode + ".");
120
121    try {
122      this.member.close();
123    } finally {
124      this.memberRpcs.close();
125    }
126  }
127
128  /**
129   * If in a running state, creates the specified subprocedure to flush table regions. Because this
130   * gets the local list of regions to flush and not the set the master had, there is a possibility
131   * of a race where regions may be missed.
132   * @param table    table to flush
133   * @param families column families within a table
134   * @return Subprocedure to submit to the ProcedureMember.
135   */
136  public Subprocedure buildSubprocedure(String table, List<String> families) {
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, FLUSH_TIMEOUT_MILLIS_DEFAULT);
159    long wakeMillis =
160      conf.getLong(FLUSH_REQUEST_WAKE_MILLIS_KEY, FLUSH_REQUEST_WAKE_MILLIS_DEFAULT);
161
162    FlushTableSubprocedurePool taskManager =
163      new FlushTableSubprocedurePool(rss.getServerName().toString(), conf, rss);
164    return new FlushTableSubprocedure(member, exnDispatcher, wakeMillis, timeoutMillis,
165      involvedRegions, table, families, taskManager);
166  }
167
168  /**
169   * Get the list of regions to flush for the table on this server It is possible that if a region
170   * moves somewhere between the calls we'll miss the region.
171   * @return the list of online regions. Empty list is returned if no regions.
172   */
173  private List<HRegion> getRegionsToFlush(String table) throws IOException {
174    return (List<HRegion>) rss.getRegions(TableName.valueOf(table));
175  }
176
177  public class FlushTableSubprocedureBuilder implements SubprocedureFactory {
178
179    @Override
180    public Subprocedure buildSubprocedure(String name, byte[] data) {
181      List<String> families = null;
182      // Currently we do not put other data except families, so it is ok to
183      // judge by length that if families were specified
184      if (data.length > 0) {
185        try {
186          HBaseProtos.NameStringPair nsp = HBaseProtos.NameStringPair.parseFrom(data);
187          families = ImmutableList.copyOf(Strings.SPLITTER.split(nsp.getValue()));
188        } catch (Exception e) {
189          LOG.error("fail to get family by parsing from data", e);
190        }
191      }
192      // The name of the procedure instance from the master is the table name.
193      return RegionServerFlushTableProcedureManager.this.buildSubprocedure(name, families);
194    }
195
196  }
197
198  /**
199   * We use the FlushTableSubprocedurePool, a class specific thread pool instead of
200   * {@link org.apache.hadoop.hbase.executor.ExecutorService}. It uses a
201   * {@link java.util.concurrent.ExecutorCompletionService} which provides queuing of completed
202   * tasks which lets us efficiently cancel pending tasks upon the earliest operation failures.
203   */
204  static class FlushTableSubprocedurePool {
205    private final Abortable abortable;
206    private final ExecutorCompletionService<Void> taskPool;
207    private final ThreadPoolExecutor executor;
208    private volatile boolean stopped;
209    private final List<Future<Void>> futures = new ArrayList<>();
210    private final String name;
211
212    FlushTableSubprocedurePool(String name, Configuration conf, Abortable abortable) {
213      this.abortable = abortable;
214      // configure the executor service
215      long keepAlive = conf.getLong(RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_KEY,
216        RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_DEFAULT);
217      int threads = conf.getInt(CONCURENT_FLUSH_TASKS_KEY, DEFAULT_CONCURRENT_FLUSH_TASKS);
218      this.name = name;
219      executor = Threads.getBoundedCachedThreadPool(threads, keepAlive, TimeUnit.MILLISECONDS,
220        new ThreadFactoryBuilder().setNameFormat("rs(" + name + ")-flush-proc-pool-%d")
221          .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
222      taskPool = new ExecutorCompletionService<>(executor);
223    }
224
225    boolean hasTasks() {
226      return futures.size() != 0;
227    }
228
229    /**
230     * Submit a task to the pool. NOTE: all must be submitted before you can safely
231     * {@link #waitForOutstandingTasks()}.
232     */
233    void submitTask(final Callable<Void> task) {
234      Future<Void> f = this.taskPool.submit(task);
235      futures.add(f);
236    }
237
238    /**
239     * Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)}.
240     * This *must* be called after all tasks are submitted via submitTask.
241     * @return <tt>true</tt> on success, <tt>false</tt> otherwise
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     */
287    void cancelTasks() throws InterruptedException {
288      Collection<Future<Void>> tasks = futures;
289      LOG.debug("cancelling " + tasks.size() + " flush region tasks " + name);
290      for (Future<Void> f : tasks) {
291        f.cancel(false);
292      }
293
294      // evict remaining tasks and futures from taskPool.
295      futures.clear();
296      while (taskPool.poll() != null) {
297      }
298      stop();
299    }
300
301    /**
302     * Gracefully shutdown the thread pool. An ongoing HRegion.flush() should not be interrupted
303     * (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 Uses a zookeeper based member controller.
315   * @param rss region server
316   * @throws KeeperException if the zookeeper cannot be reached
317   */
318  @Override
319  public void initialize(RegionServerServices rss) throws KeeperException {
320    this.rss = rss;
321    ZKWatcher zkw = rss.getZooKeeper();
322    this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
323      MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE);
324
325    Configuration conf = rss.getConfiguration();
326    long keepAlive = conf.getLong(FLUSH_TIMEOUT_MILLIS_KEY, FLUSH_TIMEOUT_MILLIS_DEFAULT);
327    int opThreads = conf.getInt(FLUSH_REQUEST_THREADS_KEY, FLUSH_REQUEST_THREADS_DEFAULT);
328
329    // create the actual flush table procedure member
330    ThreadPoolExecutor pool =
331      ProcedureMember.defaultPool(rss.getServerName().toString(), opThreads, keepAlive);
332    this.member = new ProcedureMember(memberRpcs, pool, new FlushTableSubprocedureBuilder());
333  }
334
335  @Override
336  public String getProcedureSignature() {
337    return MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE;
338  }
339
340}