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.ScanInfo; 028import org.apache.hadoop.hbase.regionserver.ScanType; 029import org.apache.hadoop.hbase.regionserver.StoreFileScanner; 030import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter; 031import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; 032import org.apache.hadoop.hbase.security.User; 033import org.apache.hadoop.hbase.util.Bytes; 034import org.apache.yetus.audience.InterfaceAudience; 035import org.slf4j.Logger; 036import org.slf4j.LoggerFactory; 037 038/** 039 * This is the placeholder for stripe compactor. The implementation, as well as the proper javadoc, 040 * will be added in HBASE-7967. 041 */ 042@InterfaceAudience.Private 043public class StripeCompactor extends AbstractMultiOutputCompactor<StripeMultiFileWriter> { 044 private static final Logger LOG = LoggerFactory.getLogger(StripeCompactor.class); 045 046 public StripeCompactor(Configuration conf, HStore store) { 047 super(conf, store); 048 } 049 050 private final class StripeInternalScannerFactory implements InternalScannerFactory { 051 052 private final byte[] majorRangeFromRow; 053 054 private final byte[] majorRangeToRow; 055 056 public StripeInternalScannerFactory(byte[] majorRangeFromRow, byte[] majorRangeToRow) { 057 this.majorRangeFromRow = majorRangeFromRow; 058 this.majorRangeToRow = majorRangeToRow; 059 } 060 061 @Override 062 public ScanType getScanType(CompactionRequestImpl request) { 063 // If majorRangeFromRow and majorRangeToRow are not null, then we will not use the return 064 // value to create InternalScanner. See the createScanner method below. The return value is 065 // also used when calling coprocessor hooks. 066 return ScanType.COMPACT_RETAIN_DELETES; 067 } 068 069 @Override 070 public InternalScanner createScanner(ScanInfo scanInfo, List<StoreFileScanner> scanners, 071 ScanType scanType, FileDetails fd, long smallestReadPoint) throws IOException { 072 return (majorRangeFromRow == null) 073 ? StripeCompactor.this.createScanner(store, scanInfo, scanners, scanType, smallestReadPoint, 074 fd.earliestPutTs) 075 : StripeCompactor.this.createScanner(store, scanInfo, scanners, smallestReadPoint, 076 fd.earliestPutTs, majorRangeFromRow, majorRangeToRow); 077 } 078 } 079 080 public List<Path> compact(CompactionRequestImpl request, final List<byte[]> targetBoundaries, 081 final byte[] majorRangeFromRow, final byte[] majorRangeToRow, 082 ThroughputController throughputController, User user) throws IOException { 083 if (LOG.isDebugEnabled()) { 084 StringBuilder sb = new StringBuilder(); 085 sb.append("Executing compaction with " + targetBoundaries.size() + " boundaries:"); 086 for (byte[] tb : targetBoundaries) { 087 sb.append(" [").append(Bytes.toString(tb)).append("]"); 088 } 089 LOG.debug(sb.toString()); 090 } 091 return compact(request, new StripeInternalScannerFactory(majorRangeFromRow, majorRangeToRow), 092 new CellSinkFactory<StripeMultiFileWriter>() { 093 094 @Override 095 public StripeMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd, 096 boolean shouldDropBehind, boolean major, Consumer<Path> writerCreationTracker) 097 throws IOException { 098 StripeMultiFileWriter writer = new StripeMultiFileWriter.BoundaryMultiWriter( 099 store.getComparator(), targetBoundaries, majorRangeFromRow, majorRangeToRow); 100 initMultiWriter(writer, scanner, fd, shouldDropBehind, major, writerCreationTracker); 101 return writer; 102 } 103 }, throughputController, user); 104 } 105 106 public List<Path> compact(CompactionRequestImpl request, final int targetCount, 107 final long targetSize, final byte[] left, final byte[] right, byte[] majorRangeFromRow, 108 byte[] majorRangeToRow, ThroughputController throughputController, User user) 109 throws IOException { 110 if (LOG.isDebugEnabled()) { 111 LOG.debug( 112 "Executing compaction with " + targetSize + " target file size, no more than " + targetCount 113 + " files, in [" + Bytes.toString(left) + "] [" + Bytes.toString(right) + "] range"); 114 } 115 return compact(request, new StripeInternalScannerFactory(majorRangeFromRow, majorRangeToRow), 116 new CellSinkFactory<StripeMultiFileWriter>() { 117 118 @Override 119 public StripeMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd, 120 boolean shouldDropBehind, boolean major, Consumer<Path> writerCreationTracker) 121 throws IOException { 122 StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter( 123 store.getComparator(), targetCount, targetSize, left, right); 124 initMultiWriter(writer, scanner, fd, shouldDropBehind, major, writerCreationTracker); 125 return writer; 126 } 127 }, throughputController, user); 128 } 129 130 @Override 131 protected List<Path> commitWriter(StripeMultiFileWriter writer, FileDetails fd, 132 CompactionRequestImpl request) throws IOException { 133 List<Path> newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor(), request.getFiles()); 134 assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata."; 135 return newFiles; 136 } 137 138}