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; 022 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.fs.Path; 025import org.apache.hadoop.hbase.CellComparator; 026import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; 027import org.apache.yetus.audience.InterfaceAudience; 028import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; 029import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy; 030import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; 031import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor; 032import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; 033import org.apache.hadoop.hbase.security.User; 034 035/** 036 * HBASE-15400 This store engine allows us to store data in date tiered layout with exponential 037 * sizing so that the more recent data has more granularity. Time-range scan will perform the 038 * best with most recent data. When data reach maxAge, they are compacted in fixed-size time 039 * windows for TTL and archiving. Please refer to design spec for more details. 040 * https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8/edit#heading=h.uk6y5pd3oqgx 041 */ 042@InterfaceAudience.Private 043public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher, 044 DateTieredCompactionPolicy, DateTieredCompactor, DefaultStoreFileManager> { 045 @Override 046 public boolean needsCompaction(List<HStoreFile> filesCompacting) { 047 return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(), 048 filesCompacting); 049 } 050 051 @Override 052 public CompactionContext createCompaction() throws IOException { 053 return new DateTieredCompactionContext(); 054 } 055 056 @Override 057 protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator) 058 throws IOException { 059 this.compactionPolicy = new DateTieredCompactionPolicy(conf, store); 060 this.storeFileManager = 061 new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf, 062 compactionPolicy.getConf()); 063 this.storeFlusher = new DefaultStoreFlusher(conf, store); 064 this.compactor = new DateTieredCompactor(conf, store); 065 } 066 067 private final class DateTieredCompactionContext extends CompactionContext { 068 069 @Override 070 public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) { 071 return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(), 072 filesCompacting); 073 } 074 075 @Override 076 public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction, 077 boolean mayUseOffPeak, boolean forceMajor) throws IOException { 078 request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting, 079 isUserCompaction, mayUseOffPeak, forceMajor); 080 return request != null; 081 } 082 083 @Override 084 public void forceSelect(CompactionRequestImpl request) { 085 if (!(request instanceof DateTieredCompactionRequest)) { 086 throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: " 087 + request.getClass().getCanonicalName()); 088 } 089 super.forceSelect(request); 090 } 091 092 @Override 093 public List<Path> compact(ThroughputController throughputController, User user) 094 throws IOException { 095 if (request instanceof DateTieredCompactionRequest) { 096 return compactor.compact(request, ((DateTieredCompactionRequest) request).getBoundaries(), 097 throughputController, user); 098 } else { 099 throw new IllegalArgumentException("DateTieredCompactionRequest is expected. Actual: " 100 + request.getClass().getCanonicalName()); 101 } 102 } 103 } 104}