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.HBaseInterfaceAudience; 026import org.apache.hadoop.hbase.conf.ConfigKey; 027import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; 028import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy; 029import org.apache.hadoop.hbase.regionserver.compactions.Compactor; 030import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; 031import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy; 032import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; 033import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; 034import org.apache.hadoop.hbase.security.User; 035import org.apache.hadoop.hbase.util.ReflectionUtils; 036import org.apache.yetus.audience.InterfaceAudience; 037 038/** 039 * Default StoreEngine creates the default compactor, policy, and store file manager, or their 040 * derivatives. 041 */ 042@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) 043public class DefaultStoreEngine extends StoreEngine<DefaultStoreFlusher, RatioBasedCompactionPolicy, 044 DefaultCompactor, DefaultStoreFileManager> { 045 046 public static final String DEFAULT_STORE_FLUSHER_CLASS_KEY = 047 ConfigKey.CLASS("hbase.hstore.defaultengine.storeflusher.class", StoreFlusher.class); 048 public static final String DEFAULT_COMPACTOR_CLASS_KEY = 049 ConfigKey.CLASS("hbase.hstore.defaultengine.compactor.class", Compactor.class); 050 public static final String DEFAULT_COMPACTION_POLICY_CLASS_KEY = 051 ConfigKey.CLASS("hbase.hstore.defaultengine.compactionpolicy.class", CompactionPolicy.class); 052 053 private static final Class<? extends DefaultStoreFlusher> DEFAULT_STORE_FLUSHER_CLASS = 054 DefaultStoreFlusher.class; 055 private static final Class<? extends DefaultCompactor> DEFAULT_COMPACTOR_CLASS = 056 DefaultCompactor.class; 057 private static final Class<? extends RatioBasedCompactionPolicy> DEFAULT_COMPACTION_POLICY_CLASS = 058 ExploringCompactionPolicy.class; 059 060 @Override 061 public boolean needsCompaction(List<HStoreFile> filesCompacting) { 062 return compactionPolicy.needsCompaction(this.storeFileManager.getStoreFiles(), filesCompacting); 063 } 064 065 @Override 066 protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator) 067 throws IOException { 068 createCompactor(conf, store); 069 createCompactionPolicy(conf, store); 070 createStoreFlusher(conf, store); 071 storeFileManager = new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID, conf, 072 compactionPolicy.getConf()); 073 } 074 075 protected void createCompactor(Configuration conf, HStore store) throws IOException { 076 String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName()); 077 try { 078 compactor = ReflectionUtils.instantiateWithCustomCtor(className, 079 new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); 080 } catch (Exception e) { 081 throw new IOException("Unable to load configured compactor '" + className + "'", e); 082 } 083 } 084 085 protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException { 086 String className = 087 conf.get(DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName()); 088 try { 089 compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className, 090 new Class[] { Configuration.class, StoreConfigInformation.class }, 091 new Object[] { conf, store }); 092 } catch (Exception e) { 093 throw new IOException("Unable to load configured compaction policy '" + className + "'", e); 094 } 095 } 096 097 protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { 098 String className = 099 conf.get(DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName()); 100 try { 101 storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className, 102 new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); 103 } catch (Exception e) { 104 throw new IOException("Unable to load configured store flusher '" + className + "'", e); 105 } 106 } 107 108 @Override 109 public CompactionContext createCompaction() { 110 return new DefaultCompactionContext(); 111 } 112 113 private class DefaultCompactionContext extends CompactionContext { 114 @Override 115 public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction, 116 boolean mayUseOffPeak, boolean forceMajor) throws IOException { 117 request = compactionPolicy.selectCompaction(storeFileManager.getStoreFiles(), filesCompacting, 118 isUserCompaction, mayUseOffPeak, forceMajor); 119 return request != null; 120 } 121 122 @Override 123 public List<Path> compact(ThroughputController throughputController, User user) 124 throws IOException { 125 return compactor.compact(request, throughputController, user); 126 } 127 128 @Override 129 public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) { 130 return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStoreFiles(), 131 filesCompacting); 132 } 133 } 134 135}