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    protected void createComponents(
58        Configuration conf, Store store, KVComparator kvComparator) throws IOException {
59      storeFileManager = new DefaultStoreFileManager(kvComparator, conf);
60      String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
61      try {
62        compactor = ReflectionUtils.instantiateWithCustomCtor(className,
63            new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
64      } catch (Exception e) {
65        throw new IOException("Unable to load configured compactor '" + className + "'", e);
66      }
67      className = conf.get(
68          DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
69      try {
70        compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
71            new Class[] { Configuration.class, StoreConfigInformation.class },
72            new Object[] { conf, store });
73      } catch (Exception e) {
74        throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
75      }
76      className = conf.get(
77          DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName());
78      try {
79        storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className,
80            new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
81      } catch (Exception e) {
82        throw new IOException("Unable to load configured store flusher '" + className + "'", e);
83      }
84    }
85  
86  
87    @Override
88    public CompactionContext createCompaction() {
89      return new DefaultCompactionContext();
90    }
91  
92    private class DefaultCompactionContext extends CompactionContext {
93      @Override
94      public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
95          boolean mayUseOffPeak, boolean forceMajor) throws IOException {
96        request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(),
97            filesCompacting, isUserCompaction, mayUseOffPeak, forceMajor);
98        return request != null;
99      }
100 
101     @Override
102     public List<Path> compact() throws IOException {
103       return compactor.compact(request);
104     }
105 
106     @Override
107     public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
108       return compactionPolicy.preSelectCompactionForCoprocessor(
109           storeFileManager.getStorefiles(), filesCompacting);
110     }
111   }
112 
113 }