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.ArrayList;
022import java.util.List;
023
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.CellComparator;
027import org.apache.hadoop.hbase.HBaseInterfaceAudience;
028import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
029import org.apache.yetus.audience.InterfaceAudience;
030import org.slf4j.Logger;
031import org.slf4j.LoggerFactory;
032import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
033import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
034import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
035import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
036import org.apache.hadoop.hbase.security.User;
037
038import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
039
040/**
041 * The storage engine that implements the stripe-based store/compaction scheme.
042 */
043@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
044public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
045  StripeCompactionPolicy, StripeCompactor, StripeStoreFileManager> {
046  private static final Logger LOG = LoggerFactory.getLogger(StripeStoreEngine.class);
047  private StripeStoreConfig config;
048
049  @Override
050  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
051    return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting);
052  }
053
054  @Override
055  public CompactionContext createCompaction() {
056    return new StripeCompaction();
057  }
058
059  @Override
060  protected void createComponents(
061      Configuration conf, HStore store, CellComparator comparator) throws IOException {
062    this.config = new StripeStoreConfig(conf, store);
063    this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
064    this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
065    this.storeFlusher = new StripeStoreFlusher(
066      conf, store, this.compactionPolicy, this.storeFileManager);
067    this.compactor = new StripeCompactor(conf, store);
068  }
069
070  /**
071   * Represents one instance of stripe compaction, with the necessary context and flow.
072   */
073  private class StripeCompaction extends CompactionContext {
074    private StripeCompactionPolicy.StripeCompactionRequest stripeRequest = null;
075
076    @Override
077    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
078      return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting);
079    }
080
081    @Override
082    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
083        boolean mayUseOffPeak, boolean forceMajor) throws IOException {
084      this.stripeRequest = compactionPolicy.selectCompaction(
085          storeFileManager, filesCompacting, mayUseOffPeak);
086      this.request = (this.stripeRequest == null)
087          ? new CompactionRequestImpl(new ArrayList<>()) : this.stripeRequest.getRequest();
088      return this.stripeRequest != null;
089    }
090
091    @Override
092    public void forceSelect(CompactionRequestImpl request) {
093      super.forceSelect(request);
094      if (this.stripeRequest != null) {
095        this.stripeRequest.setRequest(this.request);
096      } else {
097        LOG.warn("Stripe store is forced to take an arbitrary file list and compact it.");
098        this.stripeRequest = compactionPolicy.createEmptyRequest(storeFileManager, this.request);
099      }
100    }
101
102    @Override
103    public List<Path> compact(ThroughputController throughputController, User user)
104        throws IOException {
105      Preconditions.checkArgument(this.stripeRequest != null, "Cannot compact without selection");
106      return this.stripeRequest.execute(compactor, throughputController, user);
107    }
108  }
109}