Class DefaultCompactionPlanner

java.lang.Object
org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner
All Implemented Interfaces:
CompactionPlanner

public class DefaultCompactionPlanner extends Object implements CompactionPlanner
Finds the largest continuous set of small files that meet the compaction ratio and do not prevent future compactions.

The following configuration options are supported. Replace <service> with the name of the compaction service you are configuring.

  • tserver.compaction.major.service.<service>.opts.executors This is a json array of objects where each object has the fields:
    Default Compaction Planner Executor options
    Field Name Description
    name name or alias of the executor (required)
    type valid values 'internal' or 'external' (required)
    maxSize threshold sum of the input files (required for all but one of the configs)
    numThreads number of threads for this executor configuration (required for 'internal', cannot be specified for 'external')
    queue name of the external compaction queue (required for 'external', cannot be specified for 'internal')

    The maxSize field determines the maximum size of compaction that will run on an executor. The maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and represents the sum of the input files for a given compaction. One executor can have no max size and it will run everything that is too large for the other executors. If all executors have a max size, then system compactions will only run for compactions smaller than the largest max size. User, chop, and selector compactions will always run, even if there is no executor for their size. These compactions will run on the executor with the largest max size. The following example value for this property will create 3 threads to run compactions of files whose file size sum is less than 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and run all other compactions on Compactors configured to run compactions for Queue1:
     
     [{"name":"small", "type": "internal", "maxSize":"100M","numThreads":3},
      {"name":"medium", "type": "internal", "maxSize":"500M","numThreads":3},
      {"name: "large", "type": "external", "queue", "Queue1"}
     ]
     
    Note that the use of 'external' requires that the CompactionCoordinator and at least one Compactor for Queue1 is running.
  • tserver.compaction.major.service.<service>.opts.maxOpen This determines the maximum number of files that will be included in a single compaction.
Since:
2.1.0
See Also:
  • Constructor Details

    • DefaultCompactionPlanner

      public DefaultCompactionPlanner()
  • Method Details

    • init

      public void init(CompactionPlanner.InitParameters params)
      Specified by:
      init in interface CompactionPlanner
    • makePlan

      Description copied from interface: CompactionPlanner

      Plan what work a compaction service should do. The kind of compaction returned by CompactionPlanner.PlanningParameters.getKind() determines what must be done with the files returned by CompactionPlanner.PlanningParameters.getCandidates(). The following are the expectations for the candidates for each kind.

      • CompactionKind.SYSTEM The planner is not required to do anything with the candidates and can choose to compact zero or more of them. The candidates may represent a subset of all the files in the case where a user compaction is in progress or other compactions are running.
      • CompactionKind.USER and CompactionKind.SELECTED. The planner is required to eventually compact all candidates. Its ok to return a compaction plan that compacts a subset. When the planner compacts a subset, it will eventually be called again later. When it is called later the candidates will contain the files it did not compact and the results of any previous compactions it scheduled. The planner must eventually compact all of the files in the candidate set down to a single file. The compaction service will keep calling the planner until it does.
      • CompactionKind.CHOP. The planner is required to eventually compact all candidates. One major difference with USER compactions is this kind is not required to compact all files to a single file. It is ok to return a compaction plan that compacts a subset of the candidates. When the planner compacts a subset, it will eventually be called later. When it is called later the candidates will contain the files it did not compact.

      For a chop compaction assume the following happens.

      1. The candidate set passed to makePlan contains the files [F1,F2,F3,F4] and kind is CHOP
      2. The planner returns a job to compact files [F1,F2] on executor E1
      3. The compaction runs compacting [F1,F2] into file [F5]

      For the case above, eventually the planner will called again with a candidate set of [F3,F4] and it must eventually compact those two files.

      For a user and selector compaction assume the same thing happens, it will result in a slightly different outcome.

      1. The candidate set passed to makePlan contains the files [F1,F2,F3,F4] and kind is USER
      2. The planner returns a job to compact files [F1,F2] on executor E1
      3. The compaction runs compacting [F1,F2] into file [F5]

      For the case above, eventually the planner will called again with a candidate set of [F3,F4,F5] and it must eventually compact those three files to one. The difference with CHOP compactions is that the result of intermediate compactions are included in the candidate set.

      When a planner returns a compactions plan, task will be queued on executors. Previously queued task that do not match the latest plan are removed. The planner is called periodically, whenever a new file is added, and whenever a compaction finishes.

      Use CompactionPlanner.PlanningParameters.createPlanBuilder() to build the plan this function returns.

      Specified by:
      makePlan in interface CompactionPlanner
      See Also: