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.util.List;
23  
24  import org.apache.hadoop.hbase.classification.InterfaceAudience;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.fs.Path;
27  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
28  import org.apache.hadoop.hbase.KeyValue.KVComparator;
29  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
30  import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
31  import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
32  import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
33  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
34  import org.apache.hadoop.hbase.util.ReflectionUtils;
35  
36  /**
37   * Default StoreEngine creates the default compactor, policy, and store file manager, or
38   * their derivatives.
39   */
40  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
41  public class DefaultStoreEngine extends StoreEngine<
42    DefaultStoreFlusher, RatioBasedCompactionPolicy, DefaultCompactor, DefaultStoreFileManager> {
43  
44    public static final String DEFAULT_STORE_FLUSHER_CLASS_KEY =
45        "hbase.hstore.defaultengine.storeflusher.class";
46    public static final String DEFAULT_COMPACTOR_CLASS_KEY =
47        "hbase.hstore.defaultengine.compactor.class";
48    public static final String DEFAULT_COMPACTION_POLICY_CLASS_KEY =
49        "hbase.hstore.defaultengine.compactionpolicy.class";
50  
51    private static final Class<? extends DefaultStoreFlusher>
52      DEFAULT_STORE_FLUSHER_CLASS = DefaultStoreFlusher.class;
53    private static final Class<? extends DefaultCompactor>
54      DEFAULT_COMPACTOR_CLASS = DefaultCompactor.class;
55    private static final Class<? extends RatioBasedCompactionPolicy>
56      DEFAULT_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class;
57  
58    @Override
59    public boolean needsCompaction(List<StoreFile> filesCompacting) {
60      return compactionPolicy.needsCompaction(
61          this.storeFileManager.getStorefiles(), filesCompacting);
62    }
63  
64    @Override
65    protected void createComponents(
66        Configuration conf, Store store, KVComparator kvComparator) throws IOException {
67      String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
68      try {
69        compactor = ReflectionUtils.instantiateWithCustomCtor(className,
70            new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
71      } catch (Exception e) {
72        throw new IOException("Unable to load configured compactor '" + className + "'", e);
73      }
74      className = conf.get(
75          DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
76      try {
77        compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
78            new Class[] { Configuration.class, StoreConfigInformation.class },
79            new Object[] { conf, store });
80      } catch (Exception e) {
81        throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
82      }
83      storeFileManager = new DefaultStoreFileManager(kvComparator, conf, compactionPolicy.getConf());
84      className = conf.get(
85          DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName());
86      try {
87        storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className,
88            new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
89      } catch (Exception e) {
90        throw new IOException("Unable to load configured store flusher '" + className + "'", e);
91      }
92    }
93  
94  
95    @Override
96    public CompactionContext createCompaction() {
97      return new DefaultCompactionContext();
98    }
99  
100   private class DefaultCompactionContext extends CompactionContext {
101     @Override
102     public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
103         boolean mayUseOffPeak, boolean forceMajor) throws IOException {
104       request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(),
105           filesCompacting, isUserCompaction, mayUseOffPeak, forceMajor);
106       return request != null;
107     }
108 
109     @Override
110     public List<Path> compact(CompactionThroughputController throughputController)
111         throws IOException {
112       return compactor.compact(request, throughputController);
113     }
114 
115     @Override
116     public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
117       return compactionPolicy.preSelectCompactionForCoprocessor(
118           storeFileManager.getStorefiles(), filesCompacting);
119     }
120   }
121 
122 }