diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index bc97b013f10..06a13259587 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -35,6 +35,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.NavigableSet; import java.util.Optional; +import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -163,6 +164,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; @@ -3265,4 +3267,27 @@ public final class ProtobufUtil { .setTo(timeRange.getMax()) .build(); } + + public static byte[] toCompactionEventTrackerBytes(Set storeFiles) { + HFileProtos.CompactionEventTracker.Builder builder = + HFileProtos.CompactionEventTracker.newBuilder(); + storeFiles.forEach(sf -> builder.addCompactedStoreFile(ByteString.copyFromUtf8(sf))); + return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); + } + + public static Set toCompactedStoreFiles(byte[] bytes) throws IOException { + if (bytes != null && ProtobufUtil.isPBMagicPrefix(bytes)) { + int pbLen = ProtobufUtil.lengthOfPBMagic(); + HFileProtos.CompactionEventTracker.Builder builder = + HFileProtos.CompactionEventTracker.newBuilder(); + ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen); + HFileProtos.CompactionEventTracker compactionEventTracker = builder.build(); + List compactedStoreFiles = compactionEventTracker.getCompactedStoreFileList(); + if (compactedStoreFiles != null && compactedStoreFiles.size() != 0) { + return compactedStoreFiles.stream().map(ByteString::toStringUtf8) + .collect(Collectors.toSet()); + } + } + return Collections.emptySet(); + } } diff --git a/hbase-protocol-shaded/src/main/protobuf/HFile.proto b/hbase-protocol-shaded/src/main/protobuf/HFile.proto index 33f89a2832f..fd1b9b3680d 100644 --- a/hbase-protocol-shaded/src/main/protobuf/HFile.proto +++ b/hbase-protocol-shaded/src/main/protobuf/HFile.proto @@ -27,6 +27,10 @@ option optimize_for = SPEED; import "HBase.proto"; +message CompactionEventTracker { + repeated bytes compacted_store_file = 1; +} + // Map of name/values message FileInfoProto { repeated BytesBytesPair map_entry = 1; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java index 43d0ad8858a..f9cc40067d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import org.apache.hadoop.fs.Path; import org.apache.yetus.audience.InterfaceAudience; @@ -62,18 +63,24 @@ public abstract class AbstractMultiFileWriter implements CellSink, ShipperListen * comments in HBASE-15400 for more details. */ public List commitWriters(long maxSeqId, boolean majorCompaction) throws IOException { + return commitWriters(maxSeqId, majorCompaction, Collections.EMPTY_SET); + } + + public List commitWriters(long maxSeqId, boolean majorCompaction, + Collection storeFiles) throws IOException { preCommitWriters(); Collection writers = this.writers(); if (LOG.isDebugEnabled()) { - LOG.debug("Commit " + writers.size() + " writers, maxSeqId=" + maxSeqId - + ", majorCompaction=" + majorCompaction); + LOG.debug( + "Commit " + writers.size() + " writers, maxSeqId=" + maxSeqId + ", majorCompaction=" + + majorCompaction); } List paths = new ArrayList<>(); for (StoreFileWriter writer : writers) { if (writer == null) { continue; } - writer.appendMetadata(maxSeqId, majorCompaction); + writer.appendMetadata(maxSeqId, majorCompaction, storeFiles); preCloseWriter(writer); paths.add(writer.getPath()); writer.close(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index c656015252c..72b10bd38d8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -574,6 +574,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat totalValidStoreFile++; } + Set compactedStoreFiles = new HashSet<>(); ArrayList results = new ArrayList<>(files.size()); IOException ioe = null; try { @@ -583,6 +584,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat if (storeFile != null) { LOG.debug("loaded {}", storeFile); results.add(storeFile); + compactedStoreFiles.addAll(storeFile.getCompactedStoreFiles()); } } catch (InterruptedException e) { if (ioe == null) ioe = new InterruptedIOException(e.getMessage()); @@ -609,6 +611,21 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat throw ioe; } + // Remove the compacted files from result + List filesToRemove = new ArrayList<>(compactedStoreFiles.size()); + for (HStoreFile storeFile : results) { + if (compactedStoreFiles.contains(storeFile.getPath().getName())) { + LOG.warn("Clearing the compacted storefile {} from this store", storeFile); + storeFile.getReader().close(true); + filesToRemove.add(storeFile); + } + } + results.removeAll(filesToRemove); + if (!filesToRemove.isEmpty() && this.isPrimaryReplicaStore()) { + LOG.debug("Moving the files {} to archive", filesToRemove); + this.fs.removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(), filesToRemove); + } + return results; } @@ -930,7 +947,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat storeEngine.getStoreFileManager().clearCompactedFiles(); // clear the compacted files if (CollectionUtils.isNotEmpty(compactedfiles)) { - removeCompactedfiles(compactedfiles, true); + removeCompactedfiles(compactedfiles); } if (!result.isEmpty()) { // initialize the thread pool for closing store files in parallel. @@ -1115,7 +1132,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat .withMaxKeyCount(maxKeyCount) .withFavoredNodes(favoredNodes) .withFileContext(hFileContext) - .withShouldDropCacheBehind(shouldDropBehind); + .withShouldDropCacheBehind(shouldDropBehind) + .withCompactedFilesSupplier(this::getCompactedFiles); return builder.build(); } @@ -2528,11 +2546,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat * Closes and archives the compacted files under this store */ public synchronized void closeAndArchiveCompactedFiles() throws IOException { - closeAndArchiveCompactedFiles(false); - } - - @VisibleForTesting - public synchronized void closeAndArchiveCompactedFiles(boolean storeClosing) throws IOException { // ensure other threads do not attempt to archive the same files on close() archiveLock.lock(); try { @@ -2551,7 +2564,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat lock.readLock().unlock(); } if (CollectionUtils.isNotEmpty(copyCompactedfiles)) { - removeCompactedfiles(copyCompactedfiles, storeClosing); + removeCompactedfiles(copyCompactedfiles); } } finally { archiveLock.unlock(); @@ -2563,7 +2576,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat * @param compactedfiles The compacted files in this store that are not active in reads * @throws IOException */ - private void removeCompactedfiles(Collection compactedfiles, boolean storeClosing) + private void removeCompactedfiles(Collection compactedfiles) throws IOException { final List filesToRemove = new ArrayList<>(compactedfiles.size()); for (final HStoreFile file : compactedfiles) { @@ -2576,28 +2589,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat continue; } - //Compacted files in the list should always be marked compacted away. In the event - //they're contradicting in order to guarantee data consistency - //should we choose one and ignore the other? - if (storeClosing && !file.isCompactedAway()) { - String msg = - "Region closing but StoreFile is in compacted list but not compacted away: " + - file.getPath(); - throw new IllegalStateException(msg); - } - - //If store is closing we're ignoring any references to keep things consistent - //and remove compacted storefiles from the region directory - if (file.isCompactedAway() && (!file.isReferencedInReads() || storeClosing)) { - if (storeClosing && file.isReferencedInReads()) { - LOG.warn("Region closing but StoreFile still has references: file={}, refCount={}", - file.getPath(), r.getRefCount()); - } + if (file.isCompactedAway() && !file.isReferencedInReads()) { // Even if deleting fails we need not bother as any new scanners won't be // able to use the compacted file as the status is already compactedAway LOG.trace("Closing and archiving the file {}", file); r.close(true); - file.closeStreamReaders(true); // Just close and return filesToRemove.add(file); } else { @@ -2607,16 +2603,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat + ", refCount=" + r.getRefCount() + ", skipping for now."); } } catch (Exception e) { - String msg = "Exception while trying to close the compacted store file " + - file.getPath(); - if (storeClosing) { - msg = "Store is closing. " + msg; - } - LOG.error(msg, e); - //if we get an exception let caller know so it can abort the server - if (storeClosing) { - throw new IOException(msg, e); - } + LOG.error("Exception while trying to close the compacted store file {}", file.getPath(), + e); } } } @@ -2672,6 +2660,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat } } + @Override public int getCurrentParallelPutCount() { return currentParallelPutCount.get(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 9c94990f258..41ed3d71e40 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.Collections; +import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -40,14 +40,13 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.Bytes; - import org.apache.yetus.audience.InterfaceAudience; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** * A Store data file. Stores usually have one or more of these files. They @@ -63,7 +62,7 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; * writer and a reader is that we write once but read a lot more. */ @InterfaceAudience.Private -public class HStoreFile implements StoreFile, StoreFileReader.Listener { +public class HStoreFile implements StoreFile { private static final Logger LOG = LoggerFactory.getLogger(HStoreFile.class.getName()); @@ -83,6 +82,11 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY = Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION"); + /** + * Key for compaction event which contains the compacted storefiles in FileInfo + */ + public static final byte[] COMPACTION_EVENT_KEY = Bytes.toBytes("COMPACTION_EVENT_KEY"); + /** Bloom filter Type in FileInfo */ public static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE"); @@ -125,10 +129,6 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { // done. private final AtomicInteger refCount = new AtomicInteger(0); - // Set implementation must be of concurrent type - @VisibleForTesting - final Set streamReaders; - private final boolean noReadahead; private final boolean primaryReplica; @@ -183,6 +183,9 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { // It's set whenever you get a Reader. private boolean excludeFromMinorCompaction = false; + // This file was product of these compacted store files + private final Set compactedStoreFiles = new HashSet<>(); + /** * Map of the metadata entries in the corresponding HFile. Populated when Reader is opened * after which it is not modified again. @@ -232,7 +235,6 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { */ public HStoreFile(FileSystem fs, StoreFileInfo fileInfo, Configuration conf, CacheConfig cacheConf, BloomType cfBloomType, boolean primaryReplica) { - this.streamReaders = ConcurrentHashMap.newKeySet(); this.fs = fs; this.fileInfo = fileInfo; this.cacheConf = cacheConf; @@ -359,7 +361,6 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { /** * Opens reader on this store file. Called by Constructor. - * @throws IOException * @see #closeStoreFile(boolean) */ private void open() throws IOException { @@ -464,6 +465,14 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { "proceeding without", e); this.reader.timeRange = null; } + + try { + byte[] data = metadataMap.get(COMPACTION_EVENT_KEY); + this.compactedStoreFiles.addAll(ProtobufUtil.toCompactedStoreFiles(data)); + } catch (IOException e) { + LOG.error("Error reading compacted storefiles from meta data", e); + } + // initialize so we can reuse them after reader closed. firstKey = reader.getFirstKey(); lastKey = reader.getLastKey(); @@ -516,13 +525,9 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks, boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) throws IOException { - StoreFileReader reader = createStreamReader(canUseDropBehind); - reader.setListener(this); - StoreFileScanner sfScanner = reader.getStoreFileScanner(cacheBlocks, false, - isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn); - //Add reader once the scanner is created - streamReaders.add(reader); - return sfScanner; + return createStreamReader(canUseDropBehind) + .getStoreFileScanner(cacheBlocks, false, isCompaction, readPt, scannerOrder, + canOptimizeForNonNullColumn); } /** @@ -542,19 +547,6 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { this.reader.close(evictOnClose); this.reader = null; } - closeStreamReaders(evictOnClose); - } - - public void closeStreamReaders(boolean evictOnClose) throws IOException { - synchronized (this) { - for (StoreFileReader entry : streamReaders) { - //closing the reader will remove itself from streamReaders thanks to the Listener - entry.close(evictOnClose); - } - int size = streamReaders.size(); - Preconditions.checkState(size == 0, - "There are still streamReaders post close: " + size); - } } /** @@ -622,8 +614,7 @@ public class HStoreFile implements StoreFile, StoreFileReader.Listener { return tr != null ? OptionalLong.of(tr.getMax()) : OptionalLong.empty(); } - @Override - public void storeFileReaderClosed(StoreFileReader reader) { - streamReaders.remove(reader); + Set getCompactedStoreFiles() { + return Collections.unmodifiableSet(this.compactedStoreFiles); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java index 31276784a36..e1fc91844cd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java @@ -87,10 +87,6 @@ public class StoreFileReader { @VisibleForTesting final boolean shared; - private volatile Listener listener; - - private boolean closed = false; - private StoreFileReader(HFile.Reader reader, AtomicInteger refCount, boolean shared) { this.reader = reader; bloomFilterType = BloomType.NONE; @@ -184,9 +180,6 @@ public class StoreFileReader { if (!shared) { try { reader.close(false); - if (this.listener != null) { - this.listener.storeFileReaderClosed(this); - } } catch (IOException e) { LOG.warn("failed to close stream reader", e); } @@ -227,16 +220,7 @@ public class StoreFileReader { } public void close(boolean evictOnClose) throws IOException { - synchronized (this) { - if (closed) { - return; - } - reader.close(evictOnClose); - closed = true; - } - if (listener != null) { - listener.storeFileReaderClosed(this); - } + reader.close(evictOnClose); } /** @@ -709,14 +693,6 @@ public class StoreFileReader { this.skipResetSeqId = skipResetSeqId; } - public void setListener(Listener listener) { - this.listener = listener; - } - - public interface Listener { - void storeFileReaderClosed(StoreFileReader reader); - } - public int getPrefixLength() { return prefixLength; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 837a1e2e18b..7b4fb4a0c06 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_PARAM_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.COMPACTION_EVENT_KEY; import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT; import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS; import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; @@ -28,8 +29,14 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import java.util.UUID; +import java.util.function.Supplier; import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -55,8 +62,11 @@ import org.apache.hadoop.hbase.util.RowPrefixFixedLengthBloomContext; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + /** * A StoreFile writer. Use this to read/write HBase Store Files. It is package * local because it is an implementation detail of the HBase regionserver. @@ -74,30 +84,31 @@ public class StoreFileWriter implements CellSink, ShipperListener { private BloomContext bloomContext = null; private BloomContext deleteFamilyBloomContext = null; private final TimeRangeTracker timeRangeTracker; + private final Supplier> compactedFilesSupplier; protected HFile.Writer writer; - /** - * Creates an HFile.Writer that also write helpful meta data. - * @param fs file system to write to - * @param path file name to create - * @param conf user configuration - * @param comparator key comparator - * @param bloomType bloom filter setting - * @param maxKeys the expected maximum number of keys to be added. Was used - * for Bloom filter size in {@link HFile} format version 1. - * @param favoredNodes - * @param fileContext - The HFile context - * @param shouldDropCacheBehind Drop pages written to page cache after writing the store file. - * @throws IOException problem writing to FS - */ - private StoreFileWriter(FileSystem fs, Path path, - final Configuration conf, - CacheConfig cacheConf, - final CellComparator comparator, BloomType bloomType, long maxKeys, - InetSocketAddress[] favoredNodes, HFileContext fileContext, - boolean shouldDropCacheBehind) - throws IOException { + /** + * Creates an HFile.Writer that also write helpful meta data. + * + * @param fs file system to write to + * @param path file name to create + * @param conf user configuration + * @param comparator key comparator + * @param bloomType bloom filter setting + * @param maxKeys the expected maximum number of keys to be added. Was used + * for Bloom filter size in {@link HFile} format version 1. + * @param favoredNodes an array of favored nodes or possibly null + * @param fileContext The HFile context + * @param shouldDropCacheBehind Drop pages written to page cache after writing the store file. + * @param compactedFilesSupplier Returns the {@link HStore} compacted files which not archived + * @throws IOException problem writing to FS + */ + private StoreFileWriter(FileSystem fs, Path path, final Configuration conf, CacheConfig cacheConf, + final CellComparator comparator, BloomType bloomType, long maxKeys, + InetSocketAddress[] favoredNodes, HFileContext fileContext, boolean shouldDropCacheBehind, + Supplier> compactedFilesSupplier) throws IOException { + this.compactedFilesSupplier = compactedFilesSupplier; this.timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); // TODO : Change all writers to be specifically created for compaction context writer = HFile.getWriterFactory(conf, cacheConf) @@ -118,7 +129,7 @@ public class StoreFileWriter implements CellSink, ShipperListener { if (LOG.isTraceEnabled()) { LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", param: " + (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH? - Bytes.toInt(bloomParam):Bytes.toStringBinary(bloomParam)) + Bytes.toInt(bloomParam):Bytes.toStringBinary(bloomParam)) + ", " + generalBloomFilterWriter.getClass().getSimpleName()); } // init bloom context @@ -167,11 +178,54 @@ public class StoreFileWriter implements CellSink, ShipperListener { */ public void appendMetadata(final long maxSequenceId, final boolean majorCompaction) throws IOException { + appendMetadata(maxSequenceId, majorCompaction, Collections.emptySet()); + } + + /** + * Writes meta data. + * Call before {@link #close()} since its written as meta data to this file. + * @param maxSequenceId Maximum sequence id. + * @param majorCompaction True if this file is product of a major compaction + * @param storeFiles The compacted store files to generate this new file + * @throws IOException problem writing to FS + */ + public void appendMetadata(final long maxSequenceId, final boolean majorCompaction, + final Collection storeFiles) throws IOException { writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); + writer.appendFileInfo(COMPACTION_EVENT_KEY, toCompactionEventTrackerBytes(storeFiles)); appendTrackedTimestampsToMetadata(); } + /** + * Used when write {@link HStoreFile#COMPACTION_EVENT_KEY} to new file's file info. The compacted + * store files's name is needed. But if the compacted store file is a result of compaction, it's + * compacted files which still not archived is needed, too. And don't need to add compacted files + * recursively. If file A, B, C compacted to new file D, and file D compacted to new file E, will + * write A, B, C, D to file E's compacted files. So if file E compacted to new file F, will add E + * to F's compacted files first, then add E's compacted files: A, B, C, D to it. And no need to + * add D's compacted file, as D's compacted files has been in E's compacted files, too. + * See HBASE-20724 for more details. + * + * @param storeFiles The compacted store files to generate this new file + * @return bytes of CompactionEventTracker + */ + private byte[] toCompactionEventTrackerBytes(Collection storeFiles) { + Set notArchivedCompactedStoreFiles = + this.compactedFilesSupplier.get().stream().map(sf -> sf.getPath().getName()) + .collect(Collectors.toSet()); + Set compactedStoreFiles = new HashSet<>(); + for (HStoreFile storeFile : storeFiles) { + compactedStoreFiles.add(storeFile.getFileInfo().getPath().getName()); + for (String csf : storeFile.getCompactedStoreFiles()) { + if (notArchivedCompactedStoreFiles.contains(csf)) { + compactedStoreFiles.add(csf); + } + } + } + return ProtobufUtil.toCompactionEventTrackerBytes(compactedStoreFiles); + } + /** * Writes meta data. * Call before {@link #close()} since its written as meta data to this file. @@ -368,6 +422,7 @@ public class StoreFileWriter implements CellSink, ShipperListener { private InetSocketAddress[] favoredNodes; private HFileContext fileContext; private boolean shouldDropCacheBehind; + private Supplier> compactedFilesSupplier = () -> Collections.emptySet(); public Builder(Configuration conf, CacheConfig cacheConf, FileSystem fs) { @@ -449,6 +504,12 @@ public class StoreFileWriter implements CellSink, ShipperListener { return this; } + public Builder withCompactedFilesSupplier( + Supplier> compactedFilesSupplier) { + this.compactedFilesSupplier = compactedFilesSupplier; + return this; + } + /** * Create a store file writer. Client is responsible for closing file when * done. If metadata, add BEFORE closing using @@ -487,9 +548,9 @@ public class StoreFileWriter implements CellSink, ShipperListener { if (comparator == null) { comparator = CellComparator.getInstance(); } - return new StoreFileWriter(fs, filePath, - conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext, - shouldDropCacheBehind); + + return new StoreFileWriter(fs, filePath, conf, cacheConf, comparator, bloomType, maxKeyCount, + favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java index 21eaa941cd5..1bf52365e76 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java @@ -78,6 +78,6 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor commitWriter(DateTieredMultiFileWriter writer, FileDetails fd, CompactionRequestImpl request) throws IOException { - return writer.commitWriters(fd.maxSeqId, request.isAllFiles()); + return writer.commitWriters(fd.maxSeqId, request.isAllFiles(), request.getFiles()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java index 7a398eaed42..ac6a7dc0837 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java @@ -86,7 +86,7 @@ public class DefaultCompactor extends Compactor { protected List commitWriter(StoreFileWriter writer, FileDetails fd, CompactionRequestImpl request) throws IOException { List newFiles = Lists.newArrayList(writer.getPath()); - writer.appendMetadata(fd.maxSeqId, request.isAllFiles()); + writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); writer.close(); return newFiles; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java index 41e0a71b49d..fe07d9e888f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java @@ -127,7 +127,7 @@ public class StripeCompactor extends AbstractMultiOutputCompactor commitWriter(StripeMultiFileWriter writer, FileDetails fd, CompactionRequestImpl request) throws IOException { - List newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor()); + List newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor(), request.getFiles()); assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata."; return newFiles; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java index e1e84d7550b..fce4cf7642e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotasWithSnapshots.java @@ -405,10 +405,11 @@ public class TestSpaceQuotasWithSnapshots { // Compact the cloned table to force it to own its own files. TEST_UTIL.compact(tn2, true); // After the table is compacted, it should have its own files and be the same size as originally + // But The compaction result file has an additional compaction event tracker TEST_UTIL.waitFor(30_000, 1_000, new SpaceQuotaSnapshotPredicate(conn, tn2) { @Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception { - return snapshot.getUsage() == actualInitialSize; + return snapshot.getUsage() >= actualInitialSize; } }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileAfterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileAfterFailover.java new file mode 100644 index 00000000000..5b633c5b12c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileAfterFailover.java @@ -0,0 +1,197 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({LargeTests.class}) +public class TestCleanupCompactedFileAfterFailover { + + private static final Logger LOG = + LoggerFactory.getLogger(TestCleanupCompactedFileAfterFailover.class); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCleanupCompactedFileAfterFailover.class); + + private static HBaseTestingUtility TEST_UTIL; + private static Admin admin; + private static Table table; + + private static TableName TABLE_NAME = TableName.valueOf("TestCleanupCompactedFileAfterFailover"); + private static byte[] ROW = Bytes.toBytes("row"); + private static byte[] FAMILY = Bytes.toBytes("cf"); + private static byte[] QUALIFIER = Bytes.toBytes("cq"); + private static byte[] VALUE = Bytes.toBytes("value"); + private static final int RS_NUMBER = 5; + + @BeforeClass + public static void beforeClass() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + // Set the scanner lease to 20min, so the scanner can't be closed by RegionServer + TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 1200000); + TEST_UTIL.getConfiguration() + .setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 100); + TEST_UTIL.getConfiguration().set("dfs.blocksize", "64000"); + TEST_UTIL.getConfiguration().set("dfs.namenode.fs-limits.min-block-size", "1024"); + TEST_UTIL.getConfiguration().set(TimeToLiveHFileCleaner.TTL_CONF_KEY, "0"); + TEST_UTIL.startMiniCluster(RS_NUMBER); + admin = TEST_UTIL.getAdmin(); + } + + @AfterClass + public static void afterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Before + public void before() throws Exception { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME); + builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)); + admin.createTable(builder.build()); + TEST_UTIL.waitTableAvailable(TABLE_NAME); + table = TEST_UTIL.getConnection().getTable(TABLE_NAME); + } + + @After + public void after() throws Exception { + admin.disableTable(TABLE_NAME); + admin.deleteTable(TABLE_NAME); + } + + @Test + public void testCleanupAfterFailoverWithCompactOnce() throws Exception { + testCleanupAfterFailover(1); + } + + @Test + public void testCleanupAfterFailoverWithCompactTwice() throws Exception { + testCleanupAfterFailover(2); + } + + @Test + public void testCleanupAfterFailoverWithCompactThreeTimes() throws Exception { + testCleanupAfterFailover(3); + } + + private void testCleanupAfterFailover(int compactNum) throws Exception { + HRegionServer rsServedTable = null; + List regions = new ArrayList<>(); + for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster() + .getLiveRegionServerThreads()) { + HRegionServer rs = rsThread.getRegionServer(); + if (rs.getOnlineTables().contains(TABLE_NAME)) { + regions.addAll(rs.getRegions(TABLE_NAME)); + rsServedTable = rs; + } + } + assertNotNull(rsServedTable); + assertEquals("Table should only have one region", 1, regions.size()); + HRegion region = regions.get(0); + HStore store = region.getStore(FAMILY); + + writeDataAndFlush(3, region); + assertEquals(3, store.getStorefilesCount()); + + // Open a scanner and not close, then the storefile will be referenced + store.getScanner(new Scan(), null, 0); + + region.compact(true); + assertEquals(1, store.getStorefilesCount()); + // The compacted file should not be archived as there are references by user scanner + assertEquals(3, store.getStoreEngine().getStoreFileManager().getCompactedfiles().size()); + + for (int i = 1; i < compactNum; i++) { + // Compact again + region.compact(true); + assertEquals(1, store.getStorefilesCount()); + store.closeAndArchiveCompactedFiles(); + // Compacted storefiles still be 3 as the new compacted storefile was archived + assertEquals(3, store.getStoreEngine().getStoreFileManager().getCompactedfiles().size()); + } + + int walNum = rsServedTable.getWALs().size(); + // Roll WAL + rsServedTable.walRoller.requestRollAll(); + // Flush again + region.flush(true); + // The WAL which contains compaction event marker should be archived + assertEquals("The old WAL should be archived", walNum, rsServedTable.getWALs().size()); + + rsServedTable.kill(); + // Sleep to wait failover + Thread.sleep(3000); + TEST_UTIL.waitTableAvailable(TABLE_NAME); + + regions.clear(); + for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster() + .getLiveRegionServerThreads()) { + HRegionServer rs = rsThread.getRegionServer(); + if (rs != rsServedTable && rs.getOnlineTables().contains(TABLE_NAME)) { + regions.addAll(rs.getRegions(TABLE_NAME)); + } + } + assertEquals("Table should only have one region", 1, regions.size()); + region = regions.get(0); + store = region.getStore(FAMILY); + // The compacted storefile should be cleaned and only have 1 storefile + assertEquals(1, store.getStorefilesCount()); + } + + private void writeDataAndFlush(int fileNum, HRegion region) throws Exception { + for (int i = 0; i < fileNum; i++) { + for (int j = 0; j < 100; j++) { + table.put(new Put(concat(ROW, j)).addColumn(FAMILY, QUALIFIER, concat(VALUE, j))); + } + region.flush(true); + } + } + + private byte[] concat(byte[] base, int index) { + return Bytes.toBytes(Bytes.toString(base) + "-" + index); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java index 564e5c7c388..6ae68f834e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupCompactedFileOnRegionClose.java @@ -22,19 +22,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import java.io.IOException; import java.util.Collection; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -151,60 +147,4 @@ public class TestCleanupCompactedFileOnRegionClose { ((HStore)region.getStore(familyNameBytes)).getStoreEngine().getStoreFileManager() .getCompactedfiles().size()); } - - @Test - public void testIOExceptionThrownOnClose() throws Exception { - byte[] filler = new byte[128000]; - TableName tableName = TableName.valueOf("testIOExceptionThrownOnClose"); - String familyName = "f"; - byte[] familyNameBytes = Bytes.toBytes(familyName); - util.createTable(tableName, familyName); - - Table table = util.getConnection().getTable(tableName); - - HRegionServer rs = util.getRSForFirstRegionInTable(tableName); - Region region = rs.getRegions(tableName).get(0); - - int refSFCount = 4; - for (int i = 0; i < refSFCount; i++) { - for (int j = 0; j < refSFCount; j++) { - Put put = new Put(Bytes.toBytes(j)); - put.addColumn(familyNameBytes, Bytes.toBytes(i), filler); - table.put(put); - } - util.flush(tableName); - } - assertEquals(refSFCount, region.getStoreFileList(new byte[][]{familyNameBytes}).size()); - - HStore store = ((HRegion) region).getStore(familyNameBytes); - HStoreFile hsf = ((Collection)region.getStore(familyNameBytes).getStorefiles()) - .iterator().next(); - long readPt = ((HRegion)region).getReadPoint(IsolationLevel.READ_COMMITTED); - StoreFileScanner preadScanner = hsf.getPreadScanner(false, readPt, 0, false); - StoreFileScanner streamScanner = - hsf.getStreamScanner(false, false, false, readPt, 0, false); - preadScanner.seek(KeyValue.LOWESTKEY); - streamScanner.seek(KeyValue.LOWESTKEY); - - //Major compact to produce compacted storefiles that need to be cleaned up - util.compact(tableName, true); - assertNotNull(preadScanner.next()); - assertNotNull(streamScanner.next()); - store.closeAndArchiveCompactedFiles(true); - - try { - assertNotNull(preadScanner.next()); - fail("Expected IOException"); - }catch (IOException ex) { - ex.printStackTrace(); - } - - //Wait a bit for file to be remove from - try { - assertNotNull(streamScanner.next()); - fail("Expected IOException"); - } catch (IOException ex) { - ex.printStackTrace(); - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java index c1cecf8b590..f0372f61674 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java @@ -23,12 +23,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Set; import java.util.concurrent.ThreadLocalRandom; -import java.util.stream.Collectors; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -38,7 +34,6 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Scan.ReadType; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterBase; @@ -48,7 +43,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; import org.junit.Ignore; @@ -105,49 +99,6 @@ public class TestSwitchToStreamRead { UTIL.cleanupTestDir(); } - private Set getStreamReaders() { - List stores = REGION.getStores(); - Assert.assertEquals(1, stores.size()); - HStore firstStore = stores.get(0); - Assert.assertNotNull(firstStore); - Collection storeFiles = firstStore.getStorefiles(); - Assert.assertEquals(1, storeFiles.size()); - HStoreFile firstSToreFile = storeFiles.iterator().next(); - Assert.assertNotNull(firstSToreFile); - return Collections.unmodifiableSet(firstSToreFile.streamReaders); - } - - /** - * Test Case for HBASE-21551 - */ - @Test - public void testStreamReadersCleanup() throws IOException { - Set streamReaders = getStreamReaders(); - Assert.assertEquals(0, getStreamReaders().size()); - try (RegionScannerImpl scanner = REGION.getScanner(new Scan().setReadType(ReadType.STREAM))) { - StoreScanner storeScanner = - (StoreScanner) (scanner).getStoreHeapForTesting().getCurrentForTesting(); - List sfScanners = storeScanner.getAllScannersForTesting().stream() - .filter(kvs -> kvs instanceof StoreFileScanner).map(kvs -> (StoreFileScanner) kvs) - .collect(Collectors.toList()); - Assert.assertEquals(1, sfScanners.size()); - StoreFileScanner sfScanner = sfScanners.get(0); - Assert.assertFalse(sfScanner.getReader().shared); - - // There should be a stream reader - Assert.assertEquals(1, getStreamReaders().size()); - } - Assert.assertEquals(0, getStreamReaders().size()); - - // The streamsReader should be clear after region close even if there're some opened stream - // scanner. - RegionScannerImpl scanner = REGION.getScanner(new Scan().setReadType(ReadType.STREAM)); - Assert.assertNotNull(scanner); - Assert.assertEquals(1, getStreamReaders().size()); - REGION.close(); - Assert.assertEquals(0, streamReaders.size()); - } - @Test public void test() throws IOException { try (RegionScannerImpl scanner = REGION.getScanner(new Scan())) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java index f8df8700653..2dbf00c4447 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java @@ -25,6 +25,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyLong; @@ -111,6 +112,13 @@ public class TestCompactor { return null; } }).when(writer).appendMetadata(anyLong(), anyBoolean()); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + realWriter.hasMetadata = true; + return null; + } + }).when(writer).appendMetadata(anyLong(), anyBoolean(), anyCollection()); doAnswer(new Answer() { @Override public Path answer(InvocationOnMock invocation) throws Throwable {