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