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.regionserver;
019
020import java.io.IOException;
021import java.util.List;
022import org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.fs.Path;
024import org.apache.hadoop.hbase.CellComparator;
025import org.apache.hadoop.hbase.HBaseInterfaceAudience;
026import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
027import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
028import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
029import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
030import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
031import org.apache.hadoop.hbase.security.User;
032import org.apache.hadoop.hbase.util.ReflectionUtils;
033import org.apache.yetus.audience.InterfaceAudience;
034
035/**
036 * Default StoreEngine creates the default compactor, policy, and store file manager, or their
037 * derivatives.
038 */
039@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
040public class DefaultStoreEngine extends StoreEngine<DefaultStoreFlusher, RatioBasedCompactionPolicy,
041  DefaultCompactor, DefaultStoreFileManager> {
042
043  public static final String DEFAULT_STORE_FLUSHER_CLASS_KEY =
044    "hbase.hstore.defaultengine.storeflusher.class";
045  public static final String DEFAULT_COMPACTOR_CLASS_KEY =
046    "hbase.hstore.defaultengine.compactor.class";
047  public static final String DEFAULT_COMPACTION_POLICY_CLASS_KEY =
048    "hbase.hstore.defaultengine.compactionpolicy.class";
049
050  private static final Class<? extends DefaultStoreFlusher> DEFAULT_STORE_FLUSHER_CLASS =
051    DefaultStoreFlusher.class;
052  private static final Class<? extends DefaultCompactor> DEFAULT_COMPACTOR_CLASS =
053    DefaultCompactor.class;
054  private static final Class<? extends RatioBasedCompactionPolicy> DEFAULT_COMPACTION_POLICY_CLASS =
055    ExploringCompactionPolicy.class;
056
057  @Override
058  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
059    return compactionPolicy.needsCompaction(this.storeFileManager.getStorefiles(), filesCompacting);
060  }
061
062  @Override
063  protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator)
064    throws IOException {
065    createCompactor(conf, store);
066    createCompactionPolicy(conf, store);
067    createStoreFlusher(conf, store);
068    storeFileManager = new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID, conf,
069      compactionPolicy.getConf());
070  }
071
072  protected void createCompactor(Configuration conf, HStore store) throws IOException {
073    String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
074    try {
075      compactor = ReflectionUtils.instantiateWithCustomCtor(className,
076        new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
077    } catch (Exception e) {
078      throw new IOException("Unable to load configured compactor '" + className + "'", e);
079    }
080  }
081
082  protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException {
083    String className =
084      conf.get(DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
085    try {
086      compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
087        new Class[] { Configuration.class, StoreConfigInformation.class },
088        new Object[] { conf, store });
089    } catch (Exception e) {
090      throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
091    }
092  }
093
094  protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
095    String className =
096      conf.get(DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName());
097    try {
098      storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className,
099        new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
100    } catch (Exception e) {
101      throw new IOException("Unable to load configured store flusher '" + className + "'", e);
102    }
103  }
104
105  @Override
106  public CompactionContext createCompaction() {
107    return new DefaultCompactionContext();
108  }
109
110  private class DefaultCompactionContext extends CompactionContext {
111    @Override
112    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
113      boolean mayUseOffPeak, boolean forceMajor) throws IOException {
114      request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting,
115        isUserCompaction, mayUseOffPeak, forceMajor);
116      return request != null;
117    }
118
119    @Override
120    public List<Path> compact(ThroughputController throughputController, User user)
121      throws IOException {
122      return compactor.compact(request, throughputController, user);
123    }
124
125    @Override
126    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
127      return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(),
128        filesCompacting);
129    }
130  }
131
132}