This is an automated email from the ASF dual-hosted git repository. samt pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/trunk by this push: new 54d297a Make sure MockSchema.sstable() disposes of its FileHandles properly 54d297a is described below commit 54d297a192ca452dab5640f33fd6c22fd31e2f9c Author: Caleb Rackliffe <calebrackli...@gmail.com> AuthorDate: Wed Sep 9 16:35:38 2020 -0500 Make sure MockSchema.sstable() disposes of its FileHandles properly Patch by Caleb Rackcliffe; reviewed by Marcus Eriksson and Sam Tunnicliffe for CASSANDRA-16119 --- .../org/apache/cassandra/schema/MockSchema.java | 48 +++++++++++----------- 1 file changed, 25 insertions(+), 23 deletions(-) diff --git a/test/unit/org/apache/cassandra/schema/MockSchema.java b/test/unit/org/apache/cassandra/schema/MockSchema.java index 40b0f87..5ce8520 100644 --- a/test/unit/org/apache/cassandra/schema/MockSchema.java +++ b/test/unit/org/apache/cassandra/schema/MockSchema.java @@ -111,35 +111,37 @@ public class MockSchema } } // .complete() with size to make sstable.onDiskLength work - @SuppressWarnings("resource") - FileHandle fileHandle = new FileHandle.Builder(new ChannelProxy(tempFile)).bufferSize(size).complete(size); - if (size > 0) + try (FileHandle.Builder builder = new FileHandle.Builder(new ChannelProxy(tempFile)).bufferSize(size); + FileHandle fileHandle = builder.complete(size)) { - try + if (size > 0) { - File file = new File(descriptor.filenameFor(Component.DATA)); - try (RandomAccessFile raf = new RandomAccessFile(file, "rw")) + try { - raf.setLength(size); + File file = new File(descriptor.filenameFor(Component.DATA)); + try (RandomAccessFile raf = new RandomAccessFile(file, "rw")) + { + raf.setLength(size); + } + } + catch (IOException e) + { + throw new RuntimeException(e); } } - catch (IOException e) - { - throw new RuntimeException(e); - } + SerializationHeader header = SerializationHeader.make(cfs.metadata(), Collections.emptyList()); + StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata().comparator) + .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, UNREPAIRED_SSTABLE, null, false, header) + .get(MetadataType.STATS); + SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata, + fileHandle.sharedCopy(), fileHandle.sharedCopy(), indexSummary.sharedCopy(), + new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL, header); + reader.first = readerBounds(firstToken); + reader.last = readerBounds(lastToken); + if (!keepRef) + reader.selfRef().release(); + return reader; } - SerializationHeader header = SerializationHeader.make(cfs.metadata(), Collections.emptyList()); - StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata().comparator) - .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, UNREPAIRED_SSTABLE, null, false, header) - .get(MetadataType.STATS); - SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata, - fileHandle.sharedCopy(), fileHandle.sharedCopy(), indexSummary.sharedCopy(), - new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL, header); - reader.first = readerBounds(firstToken); - reader.last = readerBounds(lastToken); - if (!keepRef) - reader.selfRef().release(); - return reader; } public static ColumnFamilyStore newCFS() --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org