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 java.util.Optional;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.conf.Configured;
025import org.apache.hadoop.hbase.HBaseInterfaceAudience;
026import org.apache.hadoop.hbase.HConstants;
027import org.apache.hadoop.hbase.client.TableDescriptor;
028import org.apache.hadoop.util.ReflectionUtils;
029import org.apache.yetus.audience.InterfaceAudience;
030
031import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
032
033/**
034 * A split policy determines when a Region should be split.
035 * @see SteppingSplitPolicy Default split policy since 2.0.0
036 * @see IncreasingToUpperBoundRegionSplitPolicy Default split policy since 0.94.0
037 * @see ConstantSizeRegionSplitPolicy Default split policy before 0.94.0
038 */
039@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
040public abstract class RegionSplitPolicy extends Configured {
041  private static final Class<? extends RegionSplitPolicy> DEFAULT_SPLIT_POLICY_CLASS =
042    SteppingSplitPolicy.class;
043
044  /**
045   * The region configured for this split policy. As of hbase-2.0.0, RegionSplitPolicy can be
046   * instantiated on the Master-side so the Phoenix local-indexer can block default hbase behavior.
047   * This is an exotic usage. Should not trouble any other users of RegionSplitPolicy.
048   */
049  protected HRegion region;
050
051  /**
052   * Upon construction, this method will be called with the region to be governed. It will be called
053   * once and only once.
054   */
055  protected void configureForRegion(HRegion region) {
056    Preconditions.checkState(this.region == null, "Policy already configured for region {}",
057      this.region);
058
059    this.region = region;
060  }
061
062  /** Returns true if the specified region should be split. */
063  protected abstract boolean shouldSplit();
064
065  /** Returns {@code true} if the specified region can be split. */
066  protected boolean canSplit() {
067    return !region.getRegionInfo().isMetaRegion() && region.isAvailable()
068      && region.getStores().stream().allMatch(HStore::canSplit);
069  }
070
071  /**
072   * @return the key at which the region should be split, or null if it cannot be split. This will
073   *         only be called if shouldSplit previously returned true.
074   */
075  protected byte[] getSplitPoint() {
076    List<HStore> stores = region.getStores();
077
078    byte[] splitPointFromLargestStore = null;
079    long largestStoreSize = 0;
080    for (HStore s : stores) {
081      Optional<byte[]> splitPoint = s.getSplitPoint();
082      // Store also returns null if it has references as way of indicating it is not splittable
083      long storeSize = s.getSize();
084      if (splitPoint.isPresent() && largestStoreSize < storeSize) {
085        splitPointFromLargestStore = splitPoint.get();
086        largestStoreSize = storeSize;
087      }
088    }
089
090    return splitPointFromLargestStore;
091  }
092
093  /**
094   * Create the RegionSplitPolicy configured for the given table.
095   * @return a RegionSplitPolicy
096   */
097  public static RegionSplitPolicy create(HRegion region, Configuration conf) throws IOException {
098    Preconditions.checkNotNull(region, "Region should not be null.");
099    Class<? extends RegionSplitPolicy> clazz =
100      getSplitPolicyClass(region.getTableDescriptor(), conf);
101    RegionSplitPolicy policy = ReflectionUtils.newInstance(clazz, conf);
102    policy.configureForRegion(region);
103    return policy;
104  }
105
106  public static Class<? extends RegionSplitPolicy> getSplitPolicyClass(TableDescriptor htd,
107    Configuration conf) throws IOException {
108    String className = htd.getRegionSplitPolicyClassName();
109    if (className == null) {
110      className =
111        conf.get(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, DEFAULT_SPLIT_POLICY_CLASS.getName());
112    }
113
114    try {
115      Class<? extends RegionSplitPolicy> clazz =
116        Class.forName(className).asSubclass(RegionSplitPolicy.class);
117      return clazz;
118    } catch (Exception e) {
119      throw new IOException("Unable to load configured region split policy '" + className
120        + "' for table '" + htd.getTableName() + "'", e);
121    }
122  }
123
124  /**
125   * In
126   * {@link HRegionFileSystem#splitStoreFile(org.apache.hadoop.hbase.client.RegionInfo, String, HStoreFile, byte[], boolean, RegionSplitPolicy)}
127   * we are not creating the split reference if split row does not lie inside the StoreFile range.
128   * But in some use cases we may need to create the split reference even when the split row does
129   * not lie inside the StoreFile range. This method can be used to decide, whether to skip the the
130   * StoreFile range check or not.
131   * <p>
132   * This method is not for general use. It is a mechanism put in place by Phoenix local indexing to
133   * defeat standard hbase behaviors. Phoenix local indices are very likely the only folks who would
134   * make use of this method. On the Master-side, we will instantiate a RegionSplitPolicy instance
135   * and run this method ONLY... none of the others make sense on the Master-side.
136   * </p>
137   * TODO: Shutdown this phoenix specialization or do it via some other means.
138   * @return whether to skip the StoreFile range check or not
139   */
140  protected boolean skipStoreFileRangeCheck(String familyName) {
141    return false;
142  }
143}