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.List;
22  
23  import org.apache.hadoop.conf.Configuration;
24  import org.apache.hadoop.fs.Path;
25  import org.apache.hadoop.hbase.KeyValue.KVComparator;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
28  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
29  import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
30  import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest;
31  import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor;
32  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
33  import org.apache.hadoop.hbase.security.User;
34  
35  /**
36   * HBASE-15400 This store engine allows us to store data in date tiered layout with exponential
37   * sizing so that the more recent data has more granularity. Time-range scan will perform the
38   * best with most recent data. When data reach maxAge, they are compacted in fixed-size time
39   * windows for TTL and archiving. Please refer to design spec for more details.
40   * https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8/edit#heading=h.uk6y5pd3oqgx
41   */
42  @InterfaceAudience.Private
43  public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
44    DateTieredCompactionPolicy, DateTieredCompactor, DefaultStoreFileManager> {
45    @Override
46    public boolean needsCompaction(List<StoreFile> filesCompacting) {
47      return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(),
48        filesCompacting);
49    }
50  
51    @Override
52    public CompactionContext createCompaction() throws IOException {
53      return new DateTieredCompactionContext();
54    }
55  
56    @Override
57    protected void createComponents(Configuration conf, Store store, KVComparator kvComparator)
58        throws IOException {
59      this.compactionPolicy = new DateTieredCompactionPolicy(conf, store);
60      this.storeFileManager =
61          new DefaultStoreFileManager(kvComparator, StoreFile.Comparators.SEQ_ID_MAX_TIMESTAMP, conf,
62              compactionPolicy.getConf());
63      this.storeFlusher = new DefaultStoreFlusher(conf, store);
64      this.compactor = new DateTieredCompactor(conf, store);
65    }
66  
67    private final class DateTieredCompactionContext extends CompactionContext {
68  
69      @Override
70      public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
71        return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(),
72          filesCompacting);
73      }
74  
75      @Override
76      public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
77          boolean mayUseOffPeak, boolean forceMajor) throws IOException {
78        request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting,
79          isUserCompaction, mayUseOffPeak, forceMajor);
80        return request != null;
81      }
82  
83      @Override
84      public void forceSelect(CompactionRequest request) {
85        if (!(request instanceof DateTieredCompactionRequest)) {
86          throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: "
87              + request.getClass().getCanonicalName());
88        }
89        super.forceSelect(request);
90      }
91  
92      public List<Path> compact(CompactionThroughputController throughputController)
93          throws IOException {
94        return compact(throughputController, null);
95      }
96  
97      public List<Path> compact(CompactionThroughputController throughputController, User user)
98          throws IOException {
99        if (request instanceof DateTieredCompactionRequest) {
100         return compactor.compact(request, ((DateTieredCompactionRequest) request).getBoundaries(),
101           throughputController, user);
102       } else {
103         throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: "
104           + request.getClass().getCanonicalName());
105       }
106     }
107   }
108 }