001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver;
019
020import java.io.IOException;
021import java.util.List;
022import org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.fs.Path;
024import org.apache.hadoop.hbase.CellComparator;
025import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
026import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
027import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
028import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest;
029import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor;
030import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
031import org.apache.hadoop.hbase.security.User;
032import org.apache.yetus.audience.InterfaceAudience;
033
034/**
035 * HBASE-15400 This store engine allows us to store data in date tiered layout with exponential
036 * sizing so that the more recent data has more granularity. Time-range scan will perform the best
037 * with most recent data. When data reach maxAge, they are compacted in fixed-size time windows for
038 * TTL and archiving. Please refer to design spec for more details.
039 * https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8/edit#heading=h.uk6y5pd3oqgx
040 */
041@InterfaceAudience.Private
042public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
043  DateTieredCompactionPolicy, DateTieredCompactor, DefaultStoreFileManager> {
044  @Override
045  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
046    return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(), filesCompacting);
047  }
048
049  @Override
050  public CompactionContext createCompaction() throws IOException {
051    return new DateTieredCompactionContext();
052  }
053
054  @Override
055  protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator)
056    throws IOException {
057    this.compactionPolicy = new DateTieredCompactionPolicy(conf, store);
058    this.storeFileManager = new DefaultStoreFileManager(kvComparator,
059      StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf, compactionPolicy.getConf());
060    this.storeFlusher = new DefaultStoreFlusher(conf, store);
061    this.compactor = new DateTieredCompactor(conf, store);
062  }
063
064  private final class DateTieredCompactionContext extends CompactionContext {
065
066    @Override
067    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
068      return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(),
069        filesCompacting);
070    }
071
072    @Override
073    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
074      boolean mayUseOffPeak, boolean forceMajor) throws IOException {
075      request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting,
076        isUserCompaction, mayUseOffPeak, forceMajor);
077      return request != null;
078    }
079
080    @Override
081    public void forceSelect(CompactionRequestImpl request) {
082      if (!(request instanceof DateTieredCompactionRequest)) {
083        throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: "
084          + request.getClass().getCanonicalName());
085      }
086      super.forceSelect(request);
087    }
088
089    @Override
090    public List<Path> compact(ThroughputController throughputController, User user)
091      throws IOException {
092      if (request instanceof DateTieredCompactionRequest) {
093        DateTieredCompactionRequest compactionRequest = (DateTieredCompactionRequest) request;
094        return compactor.compact(request, compactionRequest.getBoundaries(),
095          compactionRequest.getBoundariesPolicies(), throughputController, user);
096      } else {
097        throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: "
098          + request.getClass().getCanonicalName());
099      }
100    }
101  }
102}