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