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 034/** 035 * Compact passed set of files. Create an instance and then call 036 * {@link #compact(CompactionRequestImpl, ThroughputController, User)} 037 */ 038@InterfaceAudience.Private 039public class DefaultCompactor extends Compactor<StoreFileWriter> { 040 private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class); 041 042 public DefaultCompactor(Configuration conf, HStore store) { 043 super(conf, store); 044 } 045 046 private final CellSinkFactory<StoreFileWriter> writerFactory = 047 new CellSinkFactory<StoreFileWriter>() { 048 @Override 049 public StoreFileWriter createWriter(InternalScanner scanner, FileDetails fd, 050 boolean shouldDropBehind, boolean major, Consumer<Path> writerCreationTracker) 051 throws IOException { 052 return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major, 053 writerCreationTracker); 054 } 055 }; 056 057 /** 058 * Do a minor/major compaction on an explicit set of storefiles from a Store. 059 */ 060 public List<Path> compact(final CompactionRequestImpl request, 061 ThroughputController throughputController, User user) throws IOException { 062 return compact(request, defaultScannerFactory, writerFactory, throughputController, user); 063 } 064 065 @Override 066 protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd, 067 CompactionRequestImpl request) throws IOException { 068 List<Path> newFiles = writer.getPaths(); 069 writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); 070 writer.close(); 071 return newFiles; 072 } 073 074 @Override 075 protected final void abortWriter(StoreFileWriter writer) throws IOException { 076 List<Path> leftoverFiles = writer.getPaths(); 077 try { 078 writer.close(); 079 } catch (IOException e) { 080 LOG.warn("Failed to close the writer after an unfinished compaction.", e); 081 } 082 try { 083 for (Path path : leftoverFiles) { 084 store.getFileSystem().delete(path, false); 085 } 086 } catch (IOException e) { 087 LOG.warn("Failed to delete the leftover file {} after an unfinished compaction.", 088 leftoverFiles, e); 089 } 090 } 091 092}