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.compactions; 019 020import java.io.IOException; 021import java.util.List; 022import java.util.function.Consumer; 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.fs.Path; 025import org.apache.hadoop.hbase.regionserver.HStore; 026import org.apache.hadoop.hbase.regionserver.InternalScanner; 027import org.apache.hadoop.hbase.regionserver.StoreFileWriter; 028import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; 029import org.apache.hadoop.hbase.security.User; 030import org.apache.yetus.audience.InterfaceAudience; 031import org.slf4j.Logger; 032import org.slf4j.LoggerFactory; 033 034import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 035 036/** 037 * Compact passed set of files. Create an instance and then call 038 * {@link #compact(CompactionRequestImpl, ThroughputController, User)} 039 */ 040@InterfaceAudience.Private 041public class DefaultCompactor extends Compactor<StoreFileWriter> { 042 private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class); 043 044 public DefaultCompactor(Configuration conf, HStore store) { 045 super(conf, store); 046 } 047 048 private final CellSinkFactory<StoreFileWriter> writerFactory = 049 new CellSinkFactory<StoreFileWriter>() { 050 @Override 051 public StoreFileWriter createWriter(InternalScanner scanner, FileDetails fd, 052 boolean shouldDropBehind, boolean major, Consumer<Path> writerCreationTracker) 053 throws IOException { 054 return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major, 055 writerCreationTracker); 056 } 057 }; 058 059 /** 060 * Do a minor/major compaction on an explicit set of storefiles from a Store. 061 */ 062 public List<Path> compact(final CompactionRequestImpl request, 063 ThroughputController throughputController, User user) throws IOException { 064 return compact(request, defaultScannerFactory, writerFactory, throughputController, user); 065 } 066 067 @Override 068 protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd, 069 CompactionRequestImpl request) throws IOException { 070 List<Path> newFiles = Lists.newArrayList(writer.getPath()); 071 writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); 072 writer.close(); 073 return newFiles; 074 } 075 076 @Override 077 protected final void abortWriter(StoreFileWriter writer) throws IOException { 078 Path leftoverFile = writer.getPath(); 079 try { 080 writer.close(); 081 } catch (IOException e) { 082 LOG.warn("Failed to close the writer after an unfinished compaction.", e); 083 } 084 try { 085 store.getFileSystem().delete(leftoverFile, false); 086 } catch (IOException e) { 087 LOG.warn("Failed to delete the leftover file {} after an unfinished compaction.", 088 leftoverFile, e); 089 } 090 } 091 092}