View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.regionserver;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.List;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
30  import org.apache.hadoop.hbase.KeyValue.KVComparator;
31  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
32  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
33  import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
34  import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
35  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
36  
37  import com.google.common.base.Preconditions;
38  
39  /**
40   * The storage engine that implements the stripe-based store/compaction scheme.
41   */
42  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
43  public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
44    StripeCompactionPolicy, StripeCompactor, StripeStoreFileManager> {
45    static final Log LOG = LogFactory.getLog(StripeStoreEngine.class);
46    private StripeStoreConfig config;
47  
48    @Override
49    public boolean needsCompaction(List<StoreFile> filesCompacting) {
50      return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting);
51    }
52  
53    @Override
54    public CompactionContext createCompaction() {
55      return new StripeCompaction();
56    }
57  
58    @Override
59    protected void createComponents(
60        Configuration conf, Store store, KVComparator comparator) throws IOException {
61      this.config = new StripeStoreConfig(conf, store);
62      this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
63      this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
64      this.storeFlusher = new StripeStoreFlusher(
65        conf, store, this.compactionPolicy, this.storeFileManager);
66      this.compactor = new StripeCompactor(conf, store);
67    }
68  
69    /**
70     * Represents one instance of stripe compaction, with the necessary context and flow.
71     */
72    private class StripeCompaction extends CompactionContext {
73      private StripeCompactionPolicy.StripeCompactionRequest stripeRequest = null;
74  
75      @Override
76      public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
77        return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting);
78      }
79  
80      @Override
81      public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
82          boolean mayUseOffPeak, boolean forceMajor) throws IOException {
83        this.stripeRequest = compactionPolicy.selectCompaction(
84            storeFileManager, filesCompacting, mayUseOffPeak);
85        this.request = (this.stripeRequest == null)
86            ? new CompactionRequest(new ArrayList<StoreFile>()) : this.stripeRequest.getRequest();
87        return this.stripeRequest != null;
88      }
89  
90      @Override
91      public void forceSelect(CompactionRequest request) {
92        super.forceSelect(request);
93        if (this.stripeRequest != null) {
94          this.stripeRequest.setRequest(this.request);
95        } else {
96          LOG.warn("Stripe store is forced to take an arbitrary file list and compact it.");
97          this.stripeRequest = compactionPolicy.createEmptyRequest(storeFileManager, this.request);
98        }
99      }
100 
101     @Override
102     public List<Path> compact(CompactionThroughputController throughputController)
103         throws IOException {
104       Preconditions.checkArgument(this.stripeRequest != null, "Cannot compact without selection");
105       return this.stripeRequest.execute(compactor, throughputController);
106     }
107   }
108 }