001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.regionserver;
020
021import java.io.IOException;
022import java.util.List;
023
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.CellComparator;
027import org.apache.hadoop.hbase.HBaseInterfaceAudience;
028import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
029import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
030import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
031import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
032import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
033import org.apache.hadoop.hbase.security.User;
034import org.apache.hadoop.hbase.util.ReflectionUtils;
035import org.apache.yetus.audience.InterfaceAudience;
036
037/**
038 * Default StoreEngine creates the default compactor, policy, and store file manager, or
039 * their derivatives.
040 */
041@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
042public class DefaultStoreEngine extends StoreEngine<
043  DefaultStoreFlusher, RatioBasedCompactionPolicy, DefaultCompactor, DefaultStoreFileManager> {
044
045  public static final String DEFAULT_STORE_FLUSHER_CLASS_KEY =
046      "hbase.hstore.defaultengine.storeflusher.class";
047  public static final String DEFAULT_COMPACTOR_CLASS_KEY =
048      "hbase.hstore.defaultengine.compactor.class";
049  public static final String DEFAULT_COMPACTION_POLICY_CLASS_KEY =
050      "hbase.hstore.defaultengine.compactionpolicy.class";
051
052  private static final Class<? extends DefaultStoreFlusher>
053    DEFAULT_STORE_FLUSHER_CLASS = DefaultStoreFlusher.class;
054  private static final Class<? extends DefaultCompactor>
055    DEFAULT_COMPACTOR_CLASS = DefaultCompactor.class;
056  private static final Class<? extends RatioBasedCompactionPolicy>
057    DEFAULT_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class;
058
059  @Override
060  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
061    return compactionPolicy.needsCompaction(
062        this.storeFileManager.getStorefiles(), filesCompacting);
063  }
064
065  @Override
066  protected void createComponents(
067      Configuration conf, HStore store, CellComparator kvComparator) throws IOException {
068    createCompactor(conf, store);
069    createCompactionPolicy(conf, store);
070    createStoreFlusher(conf, store);
071    storeFileManager =
072        new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID, conf,
073            compactionPolicy.getConf());
074  }
075
076  protected void createCompactor(Configuration conf, HStore store) throws IOException {
077    String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
078    try {
079      compactor = ReflectionUtils.instantiateWithCustomCtor(className,
080          new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
081    } catch (Exception e) {
082      throw new IOException("Unable to load configured compactor '" + className + "'", e);
083    }
084  }
085
086  protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException {
087    String className = conf.get(
088        DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
089    try {
090      compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
091          new Class[] { Configuration.class, StoreConfigInformation.class },
092          new Object[] { conf, store });
093    } catch (Exception e) {
094      throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
095    }
096  }
097
098  protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
099    String className = conf.get(
100        DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName());
101    try {
102      storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className,
103          new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
104    } catch (Exception e) {
105      throw new IOException("Unable to load configured store flusher '" + className + "'", e);
106    }
107  }
108
109  @Override
110  public CompactionContext createCompaction() {
111    return new DefaultCompactionContext();
112  }
113
114  private class DefaultCompactionContext extends CompactionContext {
115    @Override
116    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
117        boolean mayUseOffPeak, boolean forceMajor) throws IOException {
118      request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(),
119          filesCompacting, isUserCompaction, mayUseOffPeak, forceMajor);
120      return request != null;
121    }
122
123    @Override
124    public List<Path> compact(ThroughputController throughputController, User user)
125        throws IOException {
126      return compactor.compact(request, throughputController, user);
127    }
128
129    @Override
130    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
131      return compactionPolicy.preSelectCompactionForCoprocessor(
132          storeFileManager.getStorefiles(), filesCompacting);
133    }
134  }
135
136}