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; 021 022import org.apache.hadoop.conf.Configuration; 023import org.apache.hadoop.fs.FileSystem; 024import org.apache.hadoop.fs.Path; 025import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter; 026import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory; 027import org.apache.hadoop.hbase.regionserver.HStore; 028import org.apache.hadoop.hbase.regionserver.InternalScanner; 029import org.apache.hadoop.hbase.regionserver.StoreFileWriter; 030import org.apache.hadoop.hbase.regionserver.StoreScanner; 031import org.apache.yetus.audience.InterfaceAudience; 032import org.slf4j.Logger; 033import org.slf4j.LoggerFactory; 034 035/** 036 * Base class for implementing a Compactor which will generate multiple output files after 037 * compaction. 038 */ 039@InterfaceAudience.Private 040public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWriter> 041 extends Compactor<T> { 042 043 private static final Logger LOG = LoggerFactory.getLogger(AbstractMultiOutputCompactor.class); 044 045 public AbstractMultiOutputCompactor(Configuration conf, HStore store) { 046 super(conf, store); 047 } 048 049 protected void initMultiWriter(AbstractMultiFileWriter writer, InternalScanner scanner, 050 final FileDetails fd, final boolean shouldDropBehind) { 051 WriterFactory writerFactory = new WriterFactory() { 052 @Override 053 public StoreFileWriter createWriter() throws IOException { 054 return createTmpWriter(fd, shouldDropBehind); 055 } 056 057 @Override 058 public StoreFileWriter createWriterWithStoragePolicy(String fileStoragePolicy) 059 throws IOException { 060 return createTmpWriter(fd, shouldDropBehind, fileStoragePolicy); 061 } 062 }; 063 // Prepare multi-writer, and perform the compaction using scanner and writer. 064 // It is ok here if storeScanner is null. 065 StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner) scanner : null; 066 writer.init(storeScanner, writerFactory); 067 } 068 069 @Override 070 protected void abortWriter(T writer) throws IOException { 071 FileSystem fs = store.getFileSystem(); 072 for (Path leftoverFile : writer.abortWriters()) { 073 try { 074 fs.delete(leftoverFile, false); 075 } catch (IOException e) { 076 LOG.warn( 077 "Failed to delete the leftover file " + leftoverFile + " after an unfinished compaction.", 078 e); 079 } 080 } 081 } 082}