View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.io.PrintWriter;
23  import java.io.StringWriter;
24  import java.util.ArrayList;
25  import java.util.Iterator;
26  import java.util.List;
27  import java.util.concurrent.BlockingQueue;
28  import java.util.concurrent.Executors;
29  import java.util.concurrent.PriorityBlockingQueue;
30  import java.util.concurrent.RejectedExecutionException;
31  import java.util.concurrent.RejectedExecutionHandler;
32  import java.util.concurrent.ThreadFactory;
33  import java.util.concurrent.ThreadPoolExecutor;
34  import java.util.concurrent.TimeUnit;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.hbase.classification.InterfaceAudience;
39  import org.apache.hadoop.hbase.conf.ConfigurationManager;
40  import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
43  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
44  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
45  import org.apache.hadoop.hbase.util.Pair;
46  import org.apache.hadoop.ipc.RemoteException;
47  import org.apache.hadoop.util.StringUtils;
48  
49  import com.google.common.base.Preconditions;
50  
51  /**
52   * Compact region on request and then run split if appropriate
53   */
54  @InterfaceAudience.Private
55  public class CompactSplitThread implements CompactionRequestor, PropagatingConfigurationObserver {
56    static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
57  
58    // Configuration key for the large compaction threads.
59    public final static String LARGE_COMPACTION_THREADS =
60        "hbase.regionserver.thread.compaction.large";
61    public final static int LARGE_COMPACTION_THREADS_DEFAULT = 1;
62    
63    // Configuration key for the small compaction threads.
64    public final static String SMALL_COMPACTION_THREADS =
65        "hbase.regionserver.thread.compaction.small";
66    public final static int SMALL_COMPACTION_THREADS_DEFAULT = 1;
67    
68    // Configuration key for split threads
69    public final static String SPLIT_THREADS = "hbase.regionserver.thread.split";
70    public final static int SPLIT_THREADS_DEFAULT = 1;
71    
72    // Configuration keys for merge threads
73    public final static String MERGE_THREADS = "hbase.regionserver.thread.merge";
74    public final static int MERGE_THREADS_DEFAULT = 1;
75    
76    private final HRegionServer server;
77    private final Configuration conf;
78  
79    private final ThreadPoolExecutor longCompactions;
80    private final ThreadPoolExecutor shortCompactions;
81    private final ThreadPoolExecutor splits;
82    private final ThreadPoolExecutor mergePool;
83  
84    /**
85     * Splitting should not take place if the total number of regions exceed this.
86     * This is not a hard limit to the number of regions but it is a guideline to
87     * stop splitting after number of online regions is greater than this.
88     */
89    private int regionSplitLimit;
90  
91    /** @param server */
92    CompactSplitThread(HRegionServer server) {
93      super();
94      this.server = server;
95      this.conf = server.getConfiguration();
96      this.regionSplitLimit = conf.getInt("hbase.regionserver.regionSplitLimit",
97          Integer.MAX_VALUE);
98  
99      int largeThreads = Math.max(1, conf.getInt(
100         LARGE_COMPACTION_THREADS, LARGE_COMPACTION_THREADS_DEFAULT));
101     int smallThreads = conf.getInt(
102         SMALL_COMPACTION_THREADS, SMALL_COMPACTION_THREADS_DEFAULT);
103 
104     int splitThreads = conf.getInt(SPLIT_THREADS, SPLIT_THREADS_DEFAULT);
105 
106     // if we have throttle threads, make sure the user also specified size
107     Preconditions.checkArgument(largeThreads > 0 && smallThreads > 0);
108 
109     final String n = Thread.currentThread().getName();
110 
111     this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
112         60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
113         new ThreadFactory() {
114           @Override
115           public Thread newThread(Runnable r) {
116             Thread t = new Thread(r);
117             t.setName(n + "-longCompactions-" + System.currentTimeMillis());
118             return t;
119           }
120       });
121     this.longCompactions.setRejectedExecutionHandler(new Rejection());
122     this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
123         60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
124         new ThreadFactory() {
125           @Override
126           public Thread newThread(Runnable r) {
127             Thread t = new Thread(r);
128             t.setName(n + "-shortCompactions-" + System.currentTimeMillis());
129             return t;
130           }
131       });
132     this.shortCompactions
133         .setRejectedExecutionHandler(new Rejection());
134     this.splits = (ThreadPoolExecutor)
135         Executors.newFixedThreadPool(splitThreads,
136             new ThreadFactory() {
137           @Override
138           public Thread newThread(Runnable r) {
139             Thread t = new Thread(r);
140             t.setName(n + "-splits-" + System.currentTimeMillis());
141             return t;
142           }
143       });
144     int mergeThreads = conf.getInt(MERGE_THREADS, MERGE_THREADS_DEFAULT);
145     this.mergePool = (ThreadPoolExecutor) Executors.newFixedThreadPool(
146         mergeThreads, new ThreadFactory() {
147           @Override
148           public Thread newThread(Runnable r) {
149             Thread t = new Thread(r);
150             t.setName(n + "-merges-" + System.currentTimeMillis());
151             return t;
152           }
153         });
154   }
155 
156   @Override
157   public String toString() {
158     return "compaction_queue=("
159         + longCompactions.getQueue().size() + ":"
160         + shortCompactions.getQueue().size() + ")"
161         + ", split_queue=" + splits.getQueue().size()
162         + ", merge_queue=" + mergePool.getQueue().size();
163   }
164   
165   public String dumpQueue() {
166     StringBuffer queueLists = new StringBuffer();
167     queueLists.append("Compaction/Split Queue dump:\n");
168     queueLists.append("  LargeCompation Queue:\n");
169     BlockingQueue<Runnable> lq = longCompactions.getQueue();
170     Iterator<Runnable> it = lq.iterator();
171     while(it.hasNext()){
172       queueLists.append("    "+it.next().toString());
173       queueLists.append("\n");
174     }
175 
176     if( shortCompactions != null ){
177       queueLists.append("\n");
178       queueLists.append("  SmallCompation Queue:\n");
179       lq = shortCompactions.getQueue();
180       it = lq.iterator();
181       while(it.hasNext()){
182         queueLists.append("    "+it.next().toString());
183         queueLists.append("\n");
184       }
185     }
186     
187     queueLists.append("\n");
188     queueLists.append("  Split Queue:\n");
189     lq = splits.getQueue();
190     it = lq.iterator();
191     while(it.hasNext()){
192       queueLists.append("    "+it.next().toString());
193       queueLists.append("\n");
194     }
195     
196     queueLists.append("\n");
197     queueLists.append("  Region Merge Queue:\n");
198     lq = mergePool.getQueue();
199     it = lq.iterator();
200     while (it.hasNext()) {
201       queueLists.append("    " + it.next().toString());
202       queueLists.append("\n");
203     }
204 
205     return queueLists.toString();
206   }
207 
208   public synchronized void requestRegionsMerge(final HRegion a,
209       final HRegion b, final boolean forcible) {
210     try {
211       mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible));
212       if (LOG.isDebugEnabled()) {
213         LOG.debug("Region merge requested for " + a + "," + b + ", forcible="
214             + forcible + ".  " + this);
215       }
216     } catch (RejectedExecutionException ree) {
217       LOG.warn("Could not execute merge for " + a + "," + b + ", forcible="
218           + forcible, ree);
219     }
220   }
221 
222   public synchronized boolean requestSplit(final HRegion r) {
223     // don't split regions that are blocking
224     if (shouldSplitRegion() && r.getCompactPriority() >= Store.PRIORITY_USER) {
225       byte[] midKey = r.checkSplit();
226       if (midKey != null) {
227         requestSplit(r, midKey);
228         return true;
229       }
230     }
231     return false;
232   }
233 
234   public synchronized void requestSplit(final HRegion r, byte[] midKey) {
235     if (midKey == null) {
236       LOG.debug("Region " + r.getRegionNameAsString() +
237         " not splittable because midkey=null");
238       if (r.shouldForceSplit()) {
239         r.clearSplit();
240       }
241       return;
242     }
243     try {
244       this.splits.execute(new SplitRequest(r, midKey, this.server));
245       if (LOG.isDebugEnabled()) {
246         LOG.debug("Split requested for " + r + ".  " + this);
247       }
248     } catch (RejectedExecutionException ree) {
249       LOG.info("Could not execute split for " + r, ree);
250     }
251   }
252 
253   @Override
254   public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why)
255       throws IOException {
256     return requestCompaction(r, why, null);
257   }
258 
259   @Override
260   public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
261       List<Pair<CompactionRequest, Store>> requests) throws IOException {
262     return requestCompaction(r, why, Store.NO_PRIORITY, requests);
263   }
264 
265   @Override
266   public synchronized CompactionRequest requestCompaction(final HRegion r, final Store s,
267       final String why, CompactionRequest request) throws IOException {
268     return requestCompaction(r, s, why, Store.NO_PRIORITY, request);
269   }
270 
271   @Override
272   public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
273       int p, List<Pair<CompactionRequest, Store>> requests) throws IOException {
274     return requestCompactionInternal(r, why, p, requests, true);
275   }
276 
277   private List<CompactionRequest> requestCompactionInternal(final HRegion r, final String why,
278       int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow) throws IOException {
279     // not a special compaction request, so make our own list
280     List<CompactionRequest> ret = null;
281     if (requests == null) {
282       ret = selectNow ? new ArrayList<CompactionRequest>(r.getStores().size()) : null;
283       for (Store s : r.getStores().values()) {
284         CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow);
285         if (selectNow) ret.add(cr);
286       }
287     } else {
288       Preconditions.checkArgument(selectNow); // only system requests have selectNow == false
289       ret = new ArrayList<CompactionRequest>(requests.size());
290       for (Pair<CompactionRequest, Store> pair : requests) {
291         ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst()));
292       }
293     }
294     return ret;
295   }
296 
297   public CompactionRequest requestCompaction(final HRegion r, final Store s,
298       final String why, int priority, CompactionRequest request) throws IOException {
299     return requestCompactionInternal(r, s, why, priority, request, true);
300   }
301 
302   public synchronized void requestSystemCompaction(
303       final HRegion r, final String why) throws IOException {
304     requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false);
305   }
306 
307   public void requestSystemCompaction(
308       final HRegion r, final Store s, final String why) throws IOException {
309     requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false);
310   }
311 
312   /**
313    * @param r HRegion store belongs to
314    * @param s Store to request compaction on
315    * @param why Why compaction requested -- used in debug messages
316    * @param priority override the default priority (NO_PRIORITY == decide)
317    * @param request custom compaction request. Can be <tt>null</tt> in which case a simple
318    *          compaction will be used.
319    */
320   private synchronized CompactionRequest requestCompactionInternal(final HRegion r, final Store s,
321       final String why, int priority, CompactionRequest request, boolean selectNow)
322           throws IOException {
323     if (this.server.isStopped()
324         || (r.getTableDesc() != null && !r.getTableDesc().isCompactionEnabled())) {
325       return null;
326     }
327 
328     CompactionContext compaction = null;
329     if (selectNow) {
330       compaction = selectCompaction(r, s, priority, request);
331       if (compaction == null) return null; // message logged inside
332     }
333 
334     // We assume that most compactions are small. So, put system compactions into small
335     // pool; we will do selection there, and move to large pool if necessary.
336     long size = selectNow ? compaction.getRequest().getSize() : 0;
337     ThreadPoolExecutor pool = (!selectNow && s.throttleCompaction(size))
338       ? longCompactions : shortCompactions;
339     pool.execute(new CompactionRunner(s, r, compaction, pool));
340     if (LOG.isDebugEnabled()) {
341       String type = (pool == shortCompactions) ? "Small " : "Large ";
342       LOG.debug(type + "Compaction requested: " + (selectNow ? compaction.toString() : "system")
343           + (why != null && !why.isEmpty() ? "; Because: " + why : "") + "; " + this);
344     }
345     return selectNow ? compaction.getRequest() : null;
346   }
347 
348   private CompactionContext selectCompaction(final HRegion r, final Store s,
349       int priority, CompactionRequest request) throws IOException {
350     CompactionContext compaction = s.requestCompaction(priority, request);
351     if (compaction == null) {
352       if(LOG.isDebugEnabled()) {
353         LOG.debug("Not compacting " + r.getRegionNameAsString() +
354             " because compaction request was cancelled");
355       }
356       return null;
357     }
358     assert compaction.hasSelection();
359     if (priority != Store.NO_PRIORITY) {
360       compaction.getRequest().setPriority(priority);
361     }
362     return compaction;
363   }
364 
365   /**
366    * Only interrupt once it's done with a run through the work loop.
367    */
368   void interruptIfNecessary() {
369     splits.shutdown();
370     mergePool.shutdown();
371     longCompactions.shutdown();
372     shortCompactions.shutdown();
373   }
374 
375   private void waitFor(ThreadPoolExecutor t, String name) {
376     boolean done = false;
377     while (!done) {
378       try {
379         done = t.awaitTermination(60, TimeUnit.SECONDS);
380         LOG.info("Waiting for " + name + " to finish...");
381         if (!done) {
382           t.shutdownNow();
383         }
384       } catch (InterruptedException ie) {
385         LOG.warn("Interrupted waiting for " + name + " to finish...");
386       }
387     }
388   }
389 
390   void join() {
391     waitFor(splits, "Split Thread");
392     waitFor(mergePool, "Merge Thread");
393     waitFor(longCompactions, "Large Compaction Thread");
394     waitFor(shortCompactions, "Small Compaction Thread");
395   }
396 
397   /**
398    * Returns the current size of the queue containing regions that are
399    * processed.
400    *
401    * @return The current size of the regions queue.
402    */
403   public int getCompactionQueueSize() {
404     return longCompactions.getQueue().size() + shortCompactions.getQueue().size();
405   }
406 
407   public int getLargeCompactionQueueSize() {
408     return longCompactions.getQueue().size();
409   }
410 
411 
412   public int getSmallCompactionQueueSize() {
413     return shortCompactions.getQueue().size();
414   }
415 
416 
417   private boolean shouldSplitRegion() {
418     return (regionSplitLimit > server.getNumberOfOnlineRegions());
419   }
420 
421   /**
422    * @return the regionSplitLimit
423    */
424   public int getRegionSplitLimit() {
425     return this.regionSplitLimit;
426   }
427 
428   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
429       justification="Contrived use of compareTo")
430   private class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
431     private final Store store;
432     private final HRegion region;
433     private CompactionContext compaction;
434     private int queuedPriority;
435     private ThreadPoolExecutor parent;
436 
437     public CompactionRunner(Store store, HRegion region,
438         CompactionContext compaction, ThreadPoolExecutor parent) {
439       super();
440       this.store = store;
441       this.region = region;
442       this.compaction = compaction;
443       this.queuedPriority = (this.compaction == null)
444           ? store.getCompactPriority() : compaction.getRequest().getPriority();
445       this.parent = parent;
446     }
447 
448     @Override
449     public String toString() {
450       return (this.compaction != null) ? ("Request = " + compaction.getRequest())
451           : ("Store = " + store.toString() + ", pri = " + queuedPriority);
452     }
453 
454     @Override
455     public void run() {
456       Preconditions.checkNotNull(server);
457       if (server.isStopped()
458           || (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
459         return;
460       }
461       // Common case - system compaction without a file selection. Select now.
462       if (this.compaction == null) {
463         int oldPriority = this.queuedPriority;
464         this.queuedPriority = this.store.getCompactPriority();
465         if (this.queuedPriority > oldPriority) {
466           // Store priority decreased while we were in queue (due to some other compaction?),
467           // requeue with new priority to avoid blocking potential higher priorities.
468           this.parent.execute(this);
469           return;
470         }
471         try {
472           this.compaction = selectCompaction(this.region, this.store, queuedPriority, null);
473         } catch (IOException ex) {
474           LOG.error("Compaction selection failed " + this, ex);
475           server.checkFileSystem();
476           return;
477         }
478         if (this.compaction == null) return; // nothing to do
479         // Now see if we are in correct pool for the size; if not, go to the correct one.
480         // We might end up waiting for a while, so cancel the selection.
481         assert this.compaction.hasSelection();
482         ThreadPoolExecutor pool = store.throttleCompaction(
483             compaction.getRequest().getSize()) ? longCompactions : shortCompactions;
484         if (this.parent != pool) {
485           this.store.cancelRequestedCompaction(this.compaction);
486           this.compaction = null;
487           this.parent = pool;
488           this.parent.execute(this);
489           return;
490         }
491       }
492       // Finally we can compact something.
493       assert this.compaction != null;
494 
495       this.compaction.getRequest().beforeExecute();
496       try {
497         // Note: please don't put single-compaction logic here;
498         //       put it into region/store/etc. This is CST logic.
499         long start = EnvironmentEdgeManager.currentTime();
500         boolean completed = region.compact(compaction, store);
501         long now = EnvironmentEdgeManager.currentTime();
502         LOG.info(((completed) ? "Completed" : "Aborted") + " compaction: " +
503               this + "; duration=" + StringUtils.formatTimeDiff(now, start));
504         if (completed) {
505           // degenerate case: blocked regions require recursive enqueues
506           if (store.getCompactPriority() <= 0) {
507             requestSystemCompaction(region, store, "Recursive enqueue");
508           } else {
509             // see if the compaction has caused us to exceed max region size
510             requestSplit(region);
511           }
512         }
513       } catch (IOException ex) {
514         IOException remoteEx =
515             ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;
516         LOG.error("Compaction failed " + this, remoteEx);
517         if (remoteEx != ex) {
518           LOG.info("Compaction failed at original callstack: " + formatStackTrace(ex));
519         }
520         server.checkFileSystem();
521       } catch (Exception ex) {
522         LOG.error("Compaction failed " + this, ex);
523         server.checkFileSystem();
524       } finally {
525         LOG.debug("CompactSplitThread Status: " + CompactSplitThread.this);
526       }
527       this.compaction.getRequest().afterExecute();
528     }
529 
530     private String formatStackTrace(Exception ex) {
531       StringWriter sw = new StringWriter();
532       PrintWriter pw = new PrintWriter(sw);
533       ex.printStackTrace(pw);
534       pw.flush();
535       return sw.toString();
536     }
537 
538     @Override
539     public int compareTo(CompactionRunner o) {
540       // Only compare the underlying request (if any), for queue sorting purposes.
541       int compareVal = queuedPriority - o.queuedPriority; // compare priority
542       if (compareVal != 0) return compareVal;
543       CompactionContext tc = this.compaction, oc = o.compaction;
544       // Sort pre-selected (user?) compactions before system ones with equal priority.
545       return (tc == null) ? ((oc == null) ? 0 : 1)
546           : ((oc == null) ? -1 : tc.getRequest().compareTo(oc.getRequest()));
547     }
548   }
549 
550   /**
551    * Cleanup class to use when rejecting a compaction request from the queue.
552    */
553   private static class Rejection implements RejectedExecutionHandler {
554     @Override
555     public void rejectedExecution(Runnable runnable, ThreadPoolExecutor pool) {
556       if (runnable instanceof CompactionRunner) {
557         CompactionRunner runner = (CompactionRunner)runnable;
558         LOG.debug("Compaction Rejected: " + runner);
559         runner.store.cancelRequestedCompaction(runner.compaction);
560       }
561     }
562   }
563 
564   /**
565    * {@inheritDoc}
566    */
567   @Override
568   public void onConfigurationChange(Configuration newConf) {
569     // Check if number of large / small compaction threads has changed, and then
570     // adjust the core pool size of the thread pools, by using the
571     // setCorePoolSize() method. According to the javadocs, it is safe to
572     // change the core pool size on-the-fly. We need to reset the maximum
573     // pool size, as well.
574     int largeThreads = Math.max(1, newConf.getInt(
575             LARGE_COMPACTION_THREADS,
576             LARGE_COMPACTION_THREADS_DEFAULT));
577     if (this.longCompactions.getCorePoolSize() != largeThreads) {
578       LOG.info("Changing the value of " + LARGE_COMPACTION_THREADS +
579               " from " + this.longCompactions.getCorePoolSize() + " to " +
580               largeThreads);
581       this.longCompactions.setMaximumPoolSize(largeThreads);
582       this.longCompactions.setCorePoolSize(largeThreads);
583     }
584 
585     int smallThreads = newConf.getInt(SMALL_COMPACTION_THREADS,
586             SMALL_COMPACTION_THREADS_DEFAULT);
587     if (this.shortCompactions.getCorePoolSize() != smallThreads) {
588       LOG.info("Changing the value of " + SMALL_COMPACTION_THREADS +
589                 " from " + this.shortCompactions.getCorePoolSize() + " to " +
590                 smallThreads);
591       this.shortCompactions.setMaximumPoolSize(smallThreads);
592       this.shortCompactions.setCorePoolSize(smallThreads);
593     }
594 
595     int splitThreads = newConf.getInt(SPLIT_THREADS,
596             SPLIT_THREADS_DEFAULT);
597     if (this.splits.getCorePoolSize() != splitThreads) {
598       LOG.info("Changing the value of " + SPLIT_THREADS +
599                 " from " + this.splits.getCorePoolSize() + " to " +
600                 splitThreads);
601       this.splits.setMaximumPoolSize(smallThreads);
602       this.splits.setCorePoolSize(smallThreads);
603     }
604 
605     int mergeThreads = newConf.getInt(MERGE_THREADS,
606             MERGE_THREADS_DEFAULT);
607     if (this.mergePool.getCorePoolSize() != mergeThreads) {
608       LOG.info("Changing the value of " + MERGE_THREADS +
609                 " from " + this.mergePool.getCorePoolSize() + " to " +
610                 mergeThreads);
611       this.mergePool.setMaximumPoolSize(smallThreads);
612       this.mergePool.setCorePoolSize(smallThreads);
613     }
614 
615     // We change this atomically here instead of reloading the config in order that upstream
616     // would be the only one with the flexibility to reload the config.
617     this.conf.reloadConfiguration();
618   }
619 
620   protected int getSmallCompactionThreadNum() {
621     return this.shortCompactions.getCorePoolSize();
622   }
623 
624   public int getLargeCompactionThreadNum() {
625     return this.longCompactions.getCorePoolSize();
626   }
627 
628   /**
629    * {@inheritDoc}
630    */
631   @Override
632   public void registerChildren(ConfigurationManager manager) {
633     // No children to register.
634   }
635 
636   /**
637    * {@inheritDoc}
638    */
639   @Override
640   public void deregisterChildren(ConfigurationManager manager) {
641     // No children to register
642   }
643 }