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.classification.InterfaceAudience;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.KeyValue.KVComparator;
30  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
31  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
32  import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
33  import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
34  
35  import com.google.common.base.Preconditions;
36  
37  /**
38   * The storage engine that implements the stripe-based store/compaction scheme.
39   */
40  @InterfaceAudience.Private
41  public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
42    StripeCompactionPolicy, StripeCompactor, StripeStoreFileManager> {
43    static final Log LOG = LogFactory.getLog(StripeStoreEngine.class);
44    private StripeStoreConfig config;
45  
46    @Override
47    public boolean needsCompaction(List<StoreFile> filesCompacting) {
48      return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting);
49    }
50  
51    @Override
52    public CompactionContext createCompaction() {
53      return new StripeCompaction();
54    }
55  
56    @Override
57    protected void createComponents(
58        Configuration conf, Store store, KVComparator comparator) throws IOException {
59      this.config = new StripeStoreConfig(conf, store);
60      this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
61      this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
62      this.storeFlusher = new StripeStoreFlusher(
63        conf, store, this.compactionPolicy, this.storeFileManager);
64      this.compactor = new StripeCompactor(conf, store);
65    }
66  
67    /**
68     * Represents one instance of stripe compaction, with the necessary context and flow.
69     */
70    private class StripeCompaction extends CompactionContext {
71      private StripeCompactionPolicy.StripeCompactionRequest stripeRequest = null;
72  
73      @Override
74      public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
75        return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting);
76      }
77  
78      @Override
79      public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
80          boolean mayUseOffPeak, boolean forceMajor) throws IOException {
81        this.stripeRequest = compactionPolicy.selectCompaction(
82            storeFileManager, filesCompacting, mayUseOffPeak);
83        this.request = (this.stripeRequest == null)
84            ? new CompactionRequest(new ArrayList<StoreFile>()) : this.stripeRequest.getRequest();
85        return this.stripeRequest != null;
86      }
87  
88      @Override
89      public void forceSelect(CompactionRequest request) {
90        super.forceSelect(request);
91        if (this.stripeRequest != null) {
92          this.stripeRequest.setRequest(this.request);
93        } else {
94          LOG.warn("Stripe store is forced to take an arbitrary file list and compact it.");
95          this.stripeRequest = compactionPolicy.createEmptyRequest(storeFileManager, this.request);
96        }
97      }
98  
99      @Override
100     public List<Path> compact() throws IOException {
101       Preconditions.checkArgument(this.stripeRequest != null, "Cannot compact without selection");
102       return this.stripeRequest.execute(compactor);
103     }
104   }
105 }