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;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.fs.Path;
025import org.apache.hadoop.hbase.CellComparator;
026import org.apache.hadoop.hbase.HBaseInterfaceAudience;
027import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
028import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
029import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
030import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
031import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
032import org.apache.hadoop.hbase.security.User;
033import org.apache.yetus.audience.InterfaceAudience;
034import org.slf4j.Logger;
035import org.slf4j.LoggerFactory;
036
037import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
038
039/**
040 * The storage engine that implements the stripe-based store/compaction scheme.
041 */
042@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
043public class StripeStoreEngine extends
044  StoreEngine<StripeStoreFlusher, StripeCompactionPolicy, StripeCompactor, StripeStoreFileManager> {
045  private static final Logger LOG = LoggerFactory.getLogger(StripeStoreEngine.class);
046  private StripeStoreConfig config;
047
048  @Override
049  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
050    return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting);
051  }
052
053  @Override
054  public CompactionContext createCompaction() {
055    return new StripeCompaction();
056  }
057
058  @Override
059  protected void createComponents(Configuration conf, HStore store, CellComparator comparator)
060    throws IOException {
061    this.config = new StripeStoreConfig(conf, store);
062    this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
063    this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
064    this.storeFlusher =
065      new StripeStoreFlusher(conf, store, this.compactionPolicy, this.storeFileManager);
066    this.compactor = new StripeCompactor(conf, store);
067  }
068
069  /**
070   * Represents one instance of stripe compaction, with the necessary context and flow.
071   */
072  private class StripeCompaction extends CompactionContext {
073    private StripeCompactionPolicy.StripeCompactionRequest stripeRequest = null;
074
075    @Override
076    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
077      return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting);
078    }
079
080    @Override
081    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
082      boolean mayUseOffPeak, boolean forceMajor) throws IOException {
083      this.stripeRequest =
084        compactionPolicy.selectCompaction(storeFileManager, filesCompacting, mayUseOffPeak);
085      this.request = (this.stripeRequest == null)
086        ? new CompactionRequestImpl(new ArrayList<>())
087        : 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}