You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2022/01/31 16:22:41 UTC

[cassandra] branch trunk updated: Remove use of deprecated File classes in tests

This is an automated email from the ASF dual-hosted git repository.

brandonwilliams 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 23e88ad  Remove use of deprecated File classes in tests
23e88ad is described below

commit 23e88ad0c80b5910ad5734fe29032ff579956eee
Author: nvharikrishna <n....@gmail.com>
AuthorDate: Sun Jan 9 12:35:09 2022 +0530

    Remove use of deprecated File classes in tests
    
    Patch by Venkata Harikrishna Nukala; reviewed by brandonwilliams and
    bereng for CASSANDRA-17126
---
 .../cassandra/distributed/impl/FileLogAction.java  |  37 ++-----
 .../test/MultipleDataDirectoryTest.java            |   2 +-
 .../org/apache/cassandra/cql3/CorruptionTest.java  |   8 +-
 .../db/commitlog/CommitLogStressTest.java          |   2 +-
 .../cassandra/simulator/debug/Reconcile.java       |  11 +-
 .../apache/cassandra/simulator/debug/Record.java   |   8 +-
 test/unit/org/apache/cassandra/Util.java           |  26 +++++
 .../config/YamlConfigurationLoaderTest.java        |   5 +-
 test/unit/org/apache/cassandra/db/ImportTest.java  |  15 +--
 .../unit/org/apache/cassandra/db/MmapFileTest.java |  19 +---
 test/unit/org/apache/cassandra/db/ScrubTest.java   |   8 +-
 .../unit/org/apache/cassandra/db/SnapshotTest.java |   2 +-
 test/unit/org/apache/cassandra/db/VerifyTest.java  |  35 ++++---
 .../commitlog/CommitLogSegmentManagerCDCTest.java  |   3 +-
 .../cassandra/db/commitlog/CommitLogTest.java      |   7 +-
 .../db/commitlog/CommitLogUpgradeTest.java         |   2 +-
 .../db/commitlog/CommitLogUpgradeTestMaker.java    |   2 +-
 .../CorruptedSSTablesCompactionsTest.java          |  22 ++--
 .../db/compaction/SingleSSTableLCSTaskTest.java    |  13 ++-
 .../cassandra/db/lifecycle/LogTransactionTest.java |   8 +-
 .../apache/cassandra/index/sasi/SASIIndexTest.java |  12 +--
 .../compress/CompressedRandomAccessReaderTest.java |   2 +-
 .../io/sstable/SSTableCorruptionDetectionTest.java |  29 +++---
 .../io/util/ChecksummedRandomAccessReaderTest.java |   9 +-
 .../apache/cassandra/io/util/DataOutputTest.java   |   4 +-
 .../org/apache/cassandra/io/util/FileTest.java     |  33 +++---
 .../apache/cassandra/io/util/FileUtilsTest.java    |   6 +-
 .../cassandra/io/util/NIODataInputStreamTest.java  |  10 +-
 .../net/AsyncStreamingOutputPlusTest.java          |   4 +-
 .../org/apache/cassandra/schema/MockSchema.java    |   7 +-
 .../cassandra/security/EncryptionUtilsTest.java    |   5 +-
 .../service/FileSystemOwnershipCheckTest.java      | 115 +++++++++++----------
 .../service/StorageServiceServerM3PTest.java       |   2 +-
 .../service/StorageServiceServerTest.java          |   4 +-
 .../service/snapshot/SnapshotManifestTest.java     |   1 -
 .../service/snapshot/TableSnapshotTest.java        |   1 -
 .../compression/CompressedInputStreamTest.java     |   8 +-
 .../org/apache/cassandra/utils/KeyGenerator.java   |   6 +-
 38 files changed, 245 insertions(+), 248 deletions(-)

diff --git a/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java b/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java
index 14db561..db2dfaa 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/FileLogAction.java
@@ -18,9 +18,7 @@
 
 package org.apache.cassandra.distributed.impl;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.io.UncheckedIOException;
 import java.util.Objects;
 import java.util.function.Predicate;
@@ -28,6 +26,7 @@ import java.util.function.Predicate;
 import com.google.common.io.Closeables;
 
 import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.distributed.api.LogAction;
 import org.apache.cassandra.distributed.api.LineIterator;
@@ -50,36 +49,21 @@ public class FileLogAction implements LogAction
     @Override
     public LineIterator match(long startPosition, Predicate<String> fn)
     {
-        RandomAccessFile reader;
-        try
-        {
-            reader = new RandomAccessFile(file.toJavaIOFile(), "r");
-        }
-        catch (FileNotFoundException e)
-        {
-            // if file isn't present, don't return an empty stream as it looks the same as no log lines matched
-            throw new UncheckedIOException(e);
-        }
+        RandomAccessReader reader;
+        reader = RandomAccessReader.open(file);
         if (startPosition > 0) // -1 used to disable, so ignore any negative values or 0 (default offset)
         {
-            try
-            {
-                reader.seek(startPosition);
-            }
-            catch (IOException e)
-            {
-                throw new UncheckedIOException("Unable to seek to " + startPosition, e);
-            }
+            reader.seek(startPosition);
         }
         return new FileLineIterator(reader, fn);
     }
 
     private static final class FileLineIterator extends AbstractIterator<String> implements LineIterator
     {
-        private final RandomAccessFile reader;
+        private final RandomAccessReader reader;
         private final Predicate<String> fn;
 
-        private FileLineIterator(RandomAccessFile reader, Predicate<String> fn)
+        private FileLineIterator(RandomAccessReader reader, Predicate<String> fn)
         {
             this.reader = reader;
             this.fn = fn;
@@ -88,14 +72,7 @@ public class FileLogAction implements LogAction
         @Override
         public long mark()
         {
-            try
-            {
-                return reader.getFilePointer();
-            }
-            catch (IOException e)
-            {
-                throw new UncheckedIOException(e);
-            }
+            return reader.getFilePointer();
         }
 
         @Override
diff --git a/test/distributed/org/apache/cassandra/distributed/test/MultipleDataDirectoryTest.java b/test/distributed/org/apache/cassandra/distributed/test/MultipleDataDirectoryTest.java
index a2f4aab..0826954 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/MultipleDataDirectoryTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/MultipleDataDirectoryTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.distributed.test;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.util.Iterator;
@@ -38,6 +37,7 @@ import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.File;
 
 public class MultipleDataDirectoryTest extends TestBaseImpl
 {
diff --git a/test/long/org/apache/cassandra/cql3/CorruptionTest.java b/test/long/org/apache/cassandra/cql3/CorruptionTest.java
index 8068a2d..78f5871 100644
--- a/test/long/org/apache/cassandra/cql3/CorruptionTest.java
+++ b/test/long/org/apache/cassandra/cql3/CorruptionTest.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.cql3;
 
 
-import java.io.FileWriter;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -37,6 +36,7 @@ import com.datastax.driver.core.policies.Policies;
 import com.datastax.driver.core.utils.Bytes;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.util.FileWriter;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.EmbeddedCassandraService;
@@ -145,10 +145,12 @@ public class CorruptionTest extends SchemaLoader
                     String basename = "bad-data-tid" + Thread.currentThread().getId();
                     File put = new File(basename+"-put");
                     File get = new File(basename+"-get");
-                    try(FileWriter pw = new FileWriter(put.toJavaIOFile())) {
+                    try (FileWriter pw = put.newWriter(File.WriteMode.OVERWRITE))
+                    {
                         pw.write(new String(putdata));
                     }
-                    try(FileWriter pw = new FileWriter(get.toJavaIOFile())) {
+                    try (FileWriter pw = get.newWriter(File.WriteMode.OVERWRITE))
+                    {
                         pw.write(new String(getdata));
                     }
                 }
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index 493f33e..9531dd2 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.db.commitlog;
  *
  */
 
-import java.io.*;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
diff --git a/test/simulator/main/org/apache/cassandra/simulator/debug/Reconcile.java b/test/simulator/main/org/apache/cassandra/simulator/debug/Reconcile.java
index d35bf77..64c61cb 100644
--- a/test/simulator/main/org/apache/cassandra/simulator/debug/Reconcile.java
+++ b/test/simulator/main/org/apache/cassandra/simulator/debug/Reconcile.java
@@ -20,17 +20,13 @@ package org.apache.cassandra.simulator.debug;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.function.Supplier;
 import java.util.regex.Pattern;
@@ -40,6 +36,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.simulator.ClusterSimulation;
 import org.apache.cassandra.simulator.RandomSource;
 import org.apache.cassandra.utils.Closeable;
@@ -358,8 +355,8 @@ public class Reconcile
         File eventFile = new File(new File(loadFromDir), Long.toHexString(seed) + ".gz");
         File rngFile = new File(new File(loadFromDir), Long.toHexString(seed) + ".rng.gz");
 
-        try (BufferedReader eventIn = new BufferedReader(new InputStreamReader(new GZIPInputStream(new FileInputStream(eventFile))));
-             DataInputPlus.DataInputStreamPlus rngIn = new DataInputPlus.DataInputStreamPlus(rngFile.exists() ? new GZIPInputStream(new FileInputStream(rngFile)) : new ByteArrayInputStream(new byte[0])))
+        try (BufferedReader eventIn = new BufferedReader(new InputStreamReader(new GZIPInputStream(eventFile.newInputStream())));
+             DataInputPlus.DataInputStreamPlus rngIn = new DataInputPlus.DataInputStreamPlus(rngFile.exists() ? new GZIPInputStream(rngFile.newInputStream()) : new ByteArrayInputStream(new byte[0])))
         {
             boolean inputHasWaitSites, inputHasWakeSites, inputHasRngCallSites;
             {
@@ -397,7 +394,7 @@ public class Reconcile
 
             class Line { int line = 1; } Line line = new Line(); // box for heap dump analysis
             try (ClusterSimulation<?> cluster = builder.create(seed);
-                 CloseableIterator<?> iter = cluster.simulation.iterator();)
+                 CloseableIterator<?> iter = cluster.simulation.iterator())
             {
                 try
                 {
diff --git a/test/simulator/main/org/apache/cassandra/simulator/debug/Record.java b/test/simulator/main/org/apache/cassandra/simulator/debug/Record.java
index 631e28f..cca2c5e 100644
--- a/test/simulator/main/org/apache/cassandra/simulator/debug/Record.java
+++ b/test/simulator/main/org/apache/cassandra/simulator/debug/Record.java
@@ -19,8 +19,6 @@
 package org.apache.cassandra.simulator.debug;
 
 import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.nio.channels.Channels;
@@ -40,12 +38,14 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.simulator.ClusterSimulation;
 import org.apache.cassandra.simulator.RandomSource;
 import org.apache.cassandra.utils.Closeable;
 import org.apache.cassandra.utils.CloseableIterator;
 import org.apache.cassandra.utils.concurrent.Threads;
 
+import static org.apache.cassandra.io.util.File.WriteMode.OVERWRITE;
 import static org.apache.cassandra.simulator.SimulatorUtils.failWithOOM;
 
 public class Record
@@ -72,8 +72,8 @@ public class Record
             logger.error("Seed 0x{} ({}) (With: {})", Long.toHexString(seed), eventFile, modifiers);
         }
 
-        try (PrintWriter eventOut = new PrintWriter(new GZIPOutputStream(new FileOutputStream(eventFile), 1 << 16));
-             DataOutputStreamPlus rngOut = new BufferedDataOutputStreamPlus(Channels.newChannel(withRng ? new GZIPOutputStream(new FileOutputStream(rngFile), 1 << 16) : new ByteArrayOutputStream(0))))
+        try (PrintWriter eventOut = new PrintWriter(new GZIPOutputStream(eventFile.newOutputStream(OVERWRITE), 1 << 16));
+             DataOutputStreamPlus rngOut = new BufferedDataOutputStreamPlus(Channels.newChannel(withRng ? new GZIPOutputStream(rngFile.newOutputStream(OVERWRITE), 1 << 16) : new ByteArrayOutputStream(0))))
         {
             eventOut.println("modifiers:"
                              + (withRng ? "rng," : "") + (withRngCallSites ? "rngCallSites," : "")
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 2f696ba..8a1a62b 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -22,8 +22,11 @@ package org.apache.cassandra;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.IOError;
+import java.io.IOException;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.NoSuchFileException;
 import java.time.Duration;
 import java.util.*;
 import java.util.concurrent.Callable;
@@ -850,4 +853,27 @@ public class Util
         }
         Gossiper.instance.expireUpgradeFromVersion();
     }
+
+    /**
+     * Sets the length of the file to given size. File will be created if not exist.
+     *
+     * @param file file for which length needs to be set
+     * @param size new szie
+     * @throws IOException on any I/O error.
+     */
+    public static void setFileLength(File file, long size) throws IOException
+    {
+        try (FileChannel fileChannel = file.newReadWriteChannel())
+        {
+            if (file.length() >= size)
+            {
+                fileChannel.truncate(size);
+            }
+            else
+            {
+                fileChannel.position(size - 1);
+                fileChannel.write(ByteBuffer.wrap(new byte[1]));
+            }
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java
index e036c0a..06671df 100644
--- a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java
+++ b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.config;
 
-import java.io.File;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.Arrays;
@@ -28,6 +27,8 @@ import java.util.Map;
 import com.google.common.collect.ImmutableMap;
 import org.junit.Test;
 
+import org.apache.cassandra.io.util.File;
+
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 
@@ -120,7 +121,7 @@ public class YamlConfigurationLoaderTest
         {
             try
             {
-                url = new File(path).toURI().toURL();
+                url = new File(path).toPath().toUri().toURL();
             }
             catch (MalformedURLException e)
             {
diff --git a/test/unit/org/apache/cassandra/db/ImportTest.java b/test/unit/org/apache/cassandra/db/ImportTest.java
index 85d3251..c223f20 100644
--- a/test/unit/org/apache/cassandra/db/ImportTest.java
+++ b/test/unit/org/apache/cassandra/db/ImportTest.java
@@ -19,7 +19,7 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.Collections;
@@ -46,6 +46,7 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
@@ -314,10 +315,10 @@ public class ImportTest extends CQLTester
         getCurrentColumnFamilyStore().clearUnsafe();
 
         String filenameToCorrupt = sstableToCorrupt.descriptor.filenameFor(Component.STATS);
-        try (RandomAccessFile file = new RandomAccessFile(filenameToCorrupt, "rw"))
+        try (FileChannel fileChannel = new File(filenameToCorrupt).newReadWriteChannel())
         {
-            file.seek(0);
-            file.writeBytes(StringUtils.repeat('z', 2));
+            fileChannel.position(0);
+            fileChannel.write(ByteBufferUtil.bytes(StringUtils.repeat('z', 2)));
         }
 
         File backupdir = moveToBackupDir(sstables);
@@ -574,10 +575,10 @@ public class ImportTest extends CQLTester
         // corrupt the sstable which is still in the data directory
         SSTableReader sstableToCorrupt = sstables.iterator().next();
         String filenameToCorrupt = sstableToCorrupt.descriptor.filenameFor(Component.STATS);
-        try (RandomAccessFile file = new RandomAccessFile(filenameToCorrupt, "rw"))
+        try (FileChannel fileChannel = new File(filenameToCorrupt).newReadWriteChannel())
         {
-            file.seek(0);
-            file.writeBytes(StringUtils.repeat('z', 2));
+            fileChannel.position(0);
+            fileChannel.write(ByteBufferUtil.bytes(StringUtils.repeat('z', 2)));
         }
 
         for (int i = 10; i < 20; i++)
diff --git a/test/unit/org/apache/cassandra/db/MmapFileTest.java b/test/unit/org/apache/cassandra/db/MmapFileTest.java
index c8338cb..a666426 100644
--- a/test/unit/org/apache/cassandra/db/MmapFileTest.java
+++ b/test/unit/org/apache/cassandra/db/MmapFileTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.RandomAccessFile;
 import java.lang.management.ManagementFactory;
 import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
@@ -25,6 +24,7 @@ import java.nio.file.StandardOpenOption;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.io.util.File;
 import org.junit.Assert;
 import org.junit.Test;
@@ -56,20 +56,9 @@ public class MmapFileTest
         {
             int size = 1024 * 1024;
 
-            try (RandomAccessFile raf = new RandomAccessFile(f1.toJavaIOFile(), "rw"))
-            {
-                raf.setLength(size);
-            }
-
-            try (RandomAccessFile raf = new RandomAccessFile(f2.toJavaIOFile(), "rw"))
-            {
-                raf.setLength(size);
-            }
-
-            try (RandomAccessFile raf = new RandomAccessFile(f3.toJavaIOFile(), "rw"))
-            {
-                raf.setLength(size);
-            }
+            Util.setFileLength(f1, size);
+            Util.setFileLength(f2, size);
+            Util.setFileLength(f3,size);
 
             try (FileChannel channel = FileChannel.open(f1.toPath(), StandardOpenOption.WRITE, StandardOpenOption.READ))
             {
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index a0d971c..bbdea66 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.db;
 
 import java.io.IOError;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -443,10 +443,10 @@ public class ScrubTest
 
     private static void overrideWithGarbage(SSTableReader sstable, long startPosition, long endPosition) throws IOException
     {
-        try (RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw"))
+        try (FileChannel fileChannel = new File(sstable.getFilename()).newReadWriteChannel())
         {
-            file.seek(startPosition);
-            file.writeBytes(StringUtils.repeat('z', (int) (endPosition - startPosition)));
+            fileChannel.position(startPosition);
+            fileChannel.write(ByteBufferUtil.bytes(StringUtils.repeat('z', (int) (endPosition - startPosition))));
         }
         if (ChunkCache.instance != null)
             ChunkCache.instance.invalidateFile(sstable.getFilename());
diff --git a/test/unit/org/apache/cassandra/db/SnapshotTest.java b/test/unit/org/apache/cassandra/db/SnapshotTest.java
index a406048..c14a6c7 100644
--- a/test/unit/org/apache/cassandra/db/SnapshotTest.java
+++ b/test/unit/org/apache/cassandra/db/SnapshotTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db;
 
-import java.io.File;
 import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
 
@@ -27,6 +26,7 @@ import org.junit.Test;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.File;
 
 public class SnapshotTest extends CQLTester
 {
diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java
index 571d5ef..c86b988 100644
--- a/test/unit/org/apache/cassandra/db/VerifyTest.java
+++ b/test/unit/org/apache/cassandra/db/VerifyTest.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.db;
 
 import java.io.BufferedWriter;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.net.UnknownHostException;
+import java.nio.channels.FileChannel;
 import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -56,6 +56,7 @@ import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileInputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.schema.CompressionParams;
@@ -306,9 +307,9 @@ public class VerifyTest
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
 
 
-        try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.DIGEST), "rw"))
+        try (RandomAccessReader file = RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.DIGEST))))
         {
-            Long correctChecksum = Long.valueOf(file.readLine());
+            Long correctChecksum = file.readLong();
 
             writeChecksum(++correctChecksum, sstable.descriptor.filenameFor(Component.DIGEST));
         }
@@ -348,9 +349,9 @@ public class VerifyTest
         long startPosition = row0Start < row1Start ? row0Start : row1Start;
         long endPosition = row0Start < row1Start ? row1Start : row0Start;
 
-        RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw");
-        file.seek(startPosition);
-        file.writeBytes(StringUtils.repeat('z', (int) 2));
+        FileChannel file = new File(sstable.getFilename()).newReadWriteChannel();
+        file.position(startPosition);
+        file.write(ByteBufferUtil.bytes(StringUtils.repeat('z', 2)));
         file.close();
         if (ChunkCache.instance != null)
             ChunkCache.instance.invalidateFile(sstable.getFilename());
@@ -400,9 +401,9 @@ public class VerifyTest
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
 
         String filenameToCorrupt = sstable.descriptor.filenameFor(Component.STATS);
-        RandomAccessFile file = new RandomAccessFile(filenameToCorrupt, "rw");
-        file.seek(0);
-        file.writeBytes(StringUtils.repeat('z', 2));
+        FileChannel file = new File(filenameToCorrupt).newReadWriteChannel();
+        file.position(0);
+        file.write(ByteBufferUtil.bytes(StringUtils.repeat('z', 2)));
         file.close();
         try (Verifier verifier = new Verifier(cfs, sstable, false, Verifier.options().invokeDiskFailurePolicy(true).build()))
         {
@@ -439,9 +440,9 @@ public class VerifyTest
 
         // break the sstable:
         Long correctChecksum;
-        try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.DIGEST), "rw"))
+        try (RandomAccessReader file = RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.DIGEST))))
         {
-            correctChecksum = Long.parseLong(file.readLine());
+            correctChecksum = file.readLong();
         }
         writeChecksum(++correctChecksum, sstable.descriptor.filenameFor(Component.DIGEST));
         try (Verifier verifier = new Verifier(cfs, sstable, false, Verifier.options().mutateRepairStatus(false).invokeDiskFailurePolicy(true).build()))
@@ -508,9 +509,9 @@ public class VerifyTest
 
         sstable = cfs.getLiveSSTables().iterator().next();
         Long correctChecksum;
-        try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.DIGEST), "rw"))
+        try (RandomAccessReader file = RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.DIGEST))))
         {
-            correctChecksum = Long.parseLong(file.readLine());
+            correctChecksum = file.readLong();
         }
         writeChecksum(++correctChecksum, sstable.descriptor.filenameFor(Component.DIGEST));
         try (Verifier verifier = new Verifier(cfs, sstable, false, Verifier.options().invokeDiskFailurePolicy(true).mutateRepairStatus(true).build()))
@@ -554,9 +555,9 @@ public class VerifyTest
             verifier.verify(); //still not corrupt, should pass
         }
         String filenameToCorrupt = sstable.descriptor.filenameFor(componentToBreak);
-        try (RandomAccessFile file = new RandomAccessFile(filenameToCorrupt, "rw"))
+        try(FileChannel fileChannel = new File(filenameToCorrupt).newReadWriteChannel())
         {
-            file.setLength(3);
+            fileChannel.truncate(3);
         }
 
         try (Verifier verifier = new Verifier(cfs, sstable, false, Verifier.options().invokeDiskFailurePolicy(true).build()))
@@ -584,9 +585,9 @@ public class VerifyTest
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
 
 
-        try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.DIGEST), "rw"))
+        try (RandomAccessReader file = RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.DIGEST))))
         {
-            Long correctChecksum = Long.valueOf(file.readLine());
+            Long correctChecksum = file.readLong();
 
             writeChecksum(++correctChecksum, sstable.descriptor.filenameFor(Component.DIGEST));
         }
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
index a6e5ab1..6c1d375 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
@@ -18,7 +18,8 @@
 
 package org.apache.cassandra.db.commitlog;
 
-import java.io.*;
+import java.io.BufferedReader;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
 import java.nio.file.Path;
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index c0a70c6..31b44cd 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -18,8 +18,13 @@
 */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.*;
 import org.apache.cassandra.io.util.File;
+
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.*;
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
index 5747a38..3228a5c 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.db.commitlog;
  *
  */
 
-import java.io.*;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Properties;
 
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
index b067faa..b97cad2 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.db.commitlog;
  *
  */
 
-import java.io.*;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
diff --git a/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java
index f5a9e6a..9fe20c8 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java
@@ -21,7 +21,8 @@ package org.apache.cassandra.db.compaction;
  */
 
 
-import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.util.*;
 
 import org.junit.After;
@@ -44,6 +45,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.*;
 
@@ -177,29 +179,29 @@ public class CorruptedSSTablesCompactionsTest
             if (currentSSTable + 1 > SSTABLES_TO_CORRUPT)
                 break;
 
-            RandomAccessFile raf = null;
+            FileChannel fc = null;
 
             try
             {
                 int corruptionSize = 100;
-                raf = new RandomAccessFile(sstable.getFilename(), "rw");
-                assertNotNull(raf);
-                assertTrue(raf.length() > corruptionSize);
-                long pos = random.nextInt((int)(raf.length() - corruptionSize));
-                logger.info("Corrupting sstable {} [{}] at pos {} / {}", currentSSTable, sstable.getFilename(), pos, raf.length());
-                raf.seek(pos);
+                fc = new File(sstable.getFilename()).newReadWriteChannel();
+                assertNotNull(fc);
+                assertTrue(fc.size() > corruptionSize);
+                long pos = random.nextInt((int)(fc.size() - corruptionSize));
+                logger.info("Corrupting sstable {} [{}] at pos {} / {}", currentSSTable, sstable.getFilename(), pos, fc.size());
+                fc.position(pos);
                 // We want to write something large enough that the corruption cannot get undetected
                 // (even without compression)
                 byte[] corruption = new byte[corruptionSize];
                 random.nextBytes(corruption);
-                raf.write(corruption);
+                fc.write(ByteBuffer.wrap(corruption));
                 if (ChunkCache.instance != null)
                     ChunkCache.instance.invalidateFile(sstable.getFilename());
 
             }
             finally
             {
-                FileUtils.closeQuietly(raf);
+                FileUtils.closeQuietly(fc);
             }
 
             currentSSTable++;
diff --git a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java
index 61cf302..fe35599 100644
--- a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.cassandra.db.compaction;
 
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.util.Random;
 
 import org.apache.commons.lang3.StringUtils;
@@ -30,6 +30,8 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -127,10 +129,11 @@ public class SingleSSTableLCSTaskTest extends CQLTester
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
 
         String filenameToCorrupt = sstable.descriptor.filenameFor(Component.STATS);
-        RandomAccessFile file = new RandomAccessFile(filenameToCorrupt, "rw");
-        file.seek(0);
-        file.writeBytes(StringUtils.repeat('z', 2));
-        file.close();
+        try(FileChannel fc = new File(filenameToCorrupt).newReadWriteChannel())
+        {
+            fc.position(0);
+            fc.write(ByteBufferUtil.bytes(StringUtils.repeat('z', 2)));
+        }
         boolean gotException = false;
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstable, OperationType.COMPACTION))
         {
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
index a3b3700..b459d05 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.lifecycle;
 
 
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.file.Files;
 import java.util.*;
 import java.util.function.BiConsumer;
@@ -29,6 +28,8 @@ import java.util.stream.Collectors;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
+
+import org.apache.cassandra.Util;
 import org.apache.cassandra.io.util.File;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -1226,10 +1227,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
             if (!file.exists())
                 assertTrue(file.createFileIfNotExists());
 
-            try (RandomAccessFile raf = new RandomAccessFile(file.toJavaIOFile(), "rw"))
-            {
-                raf.setLength(size);
-            }
+            Util.setFileLength(file, size);
         }
 
         FileHandle dFile = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).complete();
diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index e9fb34a..15dd5d8 100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@ -17,14 +17,15 @@
  */
 package org.apache.cassandra.index.sasi;
 
-import java.io.FileWriter;
 import java.io.IOException;
-import java.io.Writer;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.FileSystems;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.*;
 import java.util.concurrent.ExecutorService;
@@ -95,9 +96,6 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Uninterruptibles;
 
-import org.json.simple.JSONArray;
-import org.json.simple.JSONObject;
-import org.json.simple.parser.JSONParser;
 import org.junit.*;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -2045,9 +2043,9 @@ public class SASIIndexTest
         Path path = FileSystems.getDefault().getPath(ssTable.getFilename().replace("-Data", "-SI_" + CLUSTERING_CF_NAME_1 + "_age"));
 
         // Overwrite index file with garbage
-        try (Writer writer = new FileWriter(path.toFile(), false))
+        try(FileChannel fc = FileChannel.open(path, StandardOpenOption.WRITE))
         {
-            writer.write("garbage");
+            fc.truncate(8).write(ByteBuffer.wrap("grabage".getBytes(StandardCharsets.UTF_8)));
         }
 
         long size1 = Files.readAttributes(path, BasicFileAttributes.class).size();
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 87927cf..c398ac4 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.io.compress;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.io.RandomAccessFile;
+import java.io.RandomAccessFile; //checkstyle: permit this import
 import java.util.Arrays;
 import java.util.Random;
 
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java
index 15d77f0..525020c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java
@@ -19,8 +19,8 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.util.*;
 import java.util.function.*;
 
@@ -148,18 +148,18 @@ public class SSTableCorruptionDetectionTest extends SSTableWriterTestBase
 
     private void bruteForceCorruptionTest(SSTableReader ssTableReader, Consumer<SSTableReader> walker) throws Throwable
     {
-        RandomAccessFile raf = new RandomAccessFile(ssTableReader.getFilename(), "rw");
+        FileChannel fc = new File(ssTableReader.getFilename()).newReadWriteChannel();
 
         int corruptedCounter = 0;
 
-        int fileLength = (int)raf.length(); // in current test, it does fit into int
+        int fileLength = (int)fc.size(); // in current test, it does fit into int
         for (int i = 0; i < numberOfRuns; i++)
         {
             final int corruptionPosition = random.nextInt(fileLength - 1); //ensure at least one byte will be corrupted
             // corrupt max from position to end of file
             final int corruptionSize = Math.min(maxCorruptionSize, random.nextInt(fileLength - corruptionPosition));
 
-            byte[] backup = corruptSstable(raf, corruptionPosition, corruptionSize);
+            byte[] backup = corruptSstable(fc, corruptionPosition, corruptionSize);
 
             try
             {
@@ -174,12 +174,12 @@ public class SSTableCorruptionDetectionTest extends SSTableWriterTestBase
                 if (ChunkCache.instance != null)
                     ChunkCache.instance.invalidateFile(ssTableReader.getFilename());
 
-                restore(raf, corruptionPosition, backup);
+                restore(fc, corruptionPosition, backup);
             }
         }
 
         assertTrue(corruptedCounter > 0);
-        FileUtils.closeQuietly(raf);
+        FileUtils.closeQuietly(fc);
     }
 
     private Consumer<SSTableReader> sstableScanner()
@@ -230,29 +230,28 @@ public class SSTableCorruptionDetectionTest extends SSTableWriterTestBase
                             // no-op read
                         }
                     }
-                    rowIter.close();
                 }
             }
         };
     }
 
-    private byte[] corruptSstable(RandomAccessFile raf, int position, int corruptionSize) throws IOException
+    private byte[] corruptSstable(FileChannel fc, int position, int corruptionSize) throws IOException
     {
         byte[] backup = new byte[corruptionSize];
-        raf.seek(position);
-        raf.read(backup);
+        fc.position(position);
+        fc.read(ByteBuffer.wrap(backup));
 
-        raf.seek(position);
+        fc.position(position);
         byte[] corruption = new byte[corruptionSize];
         random.nextBytes(corruption);
-        raf.write(corruption);
+        fc.write(ByteBuffer.wrap(corruption));
 
         return backup;
     }
 
-    private void restore(RandomAccessFile raf, int position, byte[] backup) throws IOException
+    private void restore(FileChannel fc, int position, byte[] backup) throws IOException
     {
-        raf.seek(position);
-        raf.write(backup);
+        fc.position(position);
+        fc.write(ByteBuffer.wrap(backup));
     }
 }
diff --git a/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java
index 91584ef..196a0b2 100644
--- a/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java
@@ -19,7 +19,8 @@
 package org.apache.cassandra.io.util;
 
 import java.io.IOException;
-import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.util.Arrays;
 import java.util.concurrent.ThreadLocalRandom;
 
@@ -118,10 +119,10 @@ public class ChecksummedRandomAccessReaderTest
         assert data.exists();
 
         // simulate corruption of file
-        try (RandomAccessFile dataFile = new RandomAccessFile(data.toJavaIOFile(), "rw"))
+        try (FileChannel dataFile = data.newReadWriteChannel())
         {
-            dataFile.seek(1024);
-            dataFile.write((byte) 5);
+            dataFile.position(1024);
+            dataFile.write(ByteBuffer.wrap(new byte[] {5}));
         }
 
         try (RandomAccessReader reader = ChecksummedRandomAccessReader.open(data, crc))
diff --git a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
index 4c3c6bb..5f3553b 100644
--- a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
+++ b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
@@ -25,7 +25,6 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
@@ -363,8 +362,7 @@ public class DataOutputTest
         try
         {
             @SuppressWarnings("resource")
-            final RandomAccessFile raf = new RandomAccessFile(file.toJavaIOFile(), "rw");
-            DataOutputStreamPlus write = new BufferedDataOutputStreamPlus(raf.getChannel());
+            DataOutputStreamPlus write = new BufferedDataOutputStreamPlus(file.newReadWriteChannel());
             DataInput canon = testWrite(write);
             write.close();
             DataInputStream test = new DataInputStream(new FileInputStreamPlus(file));
diff --git a/test/unit/org/apache/cassandra/io/util/FileTest.java b/test/unit/org/apache/cassandra/io/util/FileTest.java
index d12565c..0fb415a 100644
--- a/test/unit/org/apache/cassandra/io/util/FileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/FileTest.java
@@ -44,32 +44,33 @@ public class FileTest
     private static final java.io.File dir;
     static
     {
-        java.io.File parent = new java.io.File(JAVA_IO_TMPDIR.getString());
+        java.io.File parent = new java.io.File(JAVA_IO_TMPDIR.getString()); //checkstyle: permit this instantiation
         String dirName = Long.toHexString(ThreadLocalRandom.current().nextLong());
-        while (new java.io.File(parent, dirName).exists())
+        while (new java.io.File(parent, dirName).exists()) //checkstyle: permit this instantiation
             dirName = Long.toHexString(ThreadLocalRandom.current().nextLong());
-        dir = new java.io.File(parent, dirName);
+        dir = new java.io.File(parent, dirName); //checkstyle: permit this instantiation
         dir.mkdirs();
         new File(dir).deleteRecursiveOnExit();
     }
 
+
     @Test
     public void testEquivalence() throws IOException
     {
-        java.io.File notExists = new java.io.File(dir, "notExists");
-        java.io.File regular = new java.io.File(dir, "regular");
+        java.io.File notExists = new java.io.File(dir, "notExists"); //checkstyle: permit this instantiation
+        java.io.File regular = new java.io.File(dir, "regular"); //checkstyle: permit this instantiation
         regular.createNewFile();
-        java.io.File regularLink = new java.io.File(dir, "regularLink");
+        java.io.File regularLink = new java.io.File(dir, "regularLink"); //checkstyle: permit this instantiation
         Files.createSymbolicLink(regularLink.toPath(), regular.toPath());
-        java.io.File emptySubdir = new java.io.File(dir, "empty");
-        java.io.File emptySubdirLink = new java.io.File(dir, "emptyLink");
+        java.io.File emptySubdir = new java.io.File(dir, "empty"); //checkstyle: permit this instantiation
+        java.io.File emptySubdirLink = new java.io.File(dir, "emptyLink"); //checkstyle: permit this instantiation
         emptySubdir.mkdir();
         Files.createSymbolicLink(emptySubdirLink.toPath(), emptySubdir.toPath());
-        java.io.File nonEmptySubdir = new java.io.File(dir, "nonEmpty");
-        java.io.File nonEmptySubdirLink = new java.io.File(dir, "nonEmptyLink");
+        java.io.File nonEmptySubdir = new java.io.File(dir, "nonEmpty"); //checkstyle: permit this instantiation
+        java.io.File nonEmptySubdirLink = new java.io.File(dir, "nonEmptyLink"); //checkstyle: permit this instantiation
         nonEmptySubdir.mkdir();
         Files.createSymbolicLink(nonEmptySubdirLink.toPath(), nonEmptySubdir.toPath());
-        new java.io.File(nonEmptySubdir, "something").createNewFile();
+        new java.io.File(nonEmptySubdir, "something").createNewFile(); //checkstyle: permit this instantiation
 
         testEquivalence("");
 
@@ -112,7 +113,7 @@ public class FileTest
 
     private void    testEquivalence(String path) throws IOException
     {
-        java.io.File file = new java.io.File(path);
+        java.io.File file = new java.io.File(path); //checkstyle: permit this instantiation
         if (file.exists()) testExists(path);
         else testNotExists(path);
     }
@@ -136,7 +137,7 @@ public class FileTest
         testEquivalence(path, java.io.File::toPath, File::toPath);
         testEquivalence(path, java.io.File::list, File::tryListNames);
         testEquivalence(path, java.io.File::listFiles, File::tryList);
-        java.io.File file = new java.io.File(path);
+        java.io.File file = new java.io.File(path); //checkstyle: permit this instantiation
         if (file.getParentFile() != null) testBasic(file.getParent());
         if (!file.equals(file.getAbsoluteFile())) testBasic(file.getAbsolutePath());
         if (!file.equals(file.getCanonicalFile())) testBasic(file.getCanonicalPath());
@@ -151,7 +152,7 @@ public class FileTest
         );
         for (Triple<BiFunction<java.io.File, Boolean, Boolean>, BiFunction<File, Boolean, Boolean>, Function<java.io.File, Boolean>> test : tests)
         {
-            java.io.File file = new java.io.File(path);
+            java.io.File file = new java.io.File(path); //checkstyle: permit this instantiation
             boolean cur = test.v3.apply(file);
             boolean canRead = file.canRead();
             boolean canWrite = file.canWrite();
@@ -210,7 +211,7 @@ public class FileTest
 
     private <T> void testEquivalence(String path, IOFn<java.io.File, T> canonical, IOFn<File, T> test, IOBiConsumer<java.io.File, Boolean> afterEach)
     {
-        java.io.File file = new java.io.File(path);
+        java.io.File file = new java.io.File(path); //checkstyle: permit this instantiation
         Object expect;
         try
         {
@@ -246,7 +247,7 @@ public class FileTest
     }
     private void testTryVsConfirm(String path, Predicate<java.io.File> canonical, IOConsumer<File> test, IOBiConsumer<java.io.File, Boolean> afterEach)
     {
-        java.io.File file = new java.io.File(path);
+        java.io.File file = new java.io.File(path); //checkstyle: permit this instantiation
         boolean expect = canonical.test(file);
         try { afterEach.accept(file, expect); } catch (IOException e) { throw new AssertionError(e); }
         boolean actual;
diff --git a/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
index 737434c..46bdded 100644
--- a/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
+++ b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.io.util;
 
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -29,6 +28,7 @@ import java.util.Arrays;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.assertj.core.api.Assertions;
 
@@ -228,9 +228,9 @@ public class FileUtilsTest
 
     private File createFile(File file, long size)
     {
-        try (RandomAccessFile f = new RandomAccessFile(file.toJavaIOFile(), "rw"))
+        try
         {
-            f.setLength(size);
+            Util.setFileLength(file, size);
         }
         catch (Exception e)
         {
diff --git a/test/unit/org/apache/cassandra/io/util/NIODataInputStreamTest.java b/test/unit/org/apache/cassandra/io/util/NIODataInputStreamTest.java
index 7c42082..31fa53c 100644
--- a/test/unit/org/apache/cassandra/io/util/NIODataInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/io/util/NIODataInputStreamTest.java
@@ -26,8 +26,8 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
 import java.nio.channels.ReadableByteChannel;
 import java.util.ArrayDeque;
 import java.util.Queue;
@@ -223,11 +223,11 @@ public class NIODataInputStreamTest
         assertEquals(8190 - 10 - 4096, is.available());
 
         File f = FileUtils.createTempFile("foo", "bar");
-        RandomAccessFile fos = new RandomAccessFile(f.toJavaIOFile(), "rw");
-        fos.write(new byte[10]);
-        fos.seek(0);
+        FileChannel fos = f.newReadWriteChannel();
+        fos.write(ByteBuffer.wrap(new byte[10]));
+        fos.position(0);
 
-        is = new NIODataInputStream(fos.getChannel(), 9);
+        is = new NIODataInputStream(fos, 9);
 
         int remaining = 10;
         assertEquals(10, is.available());
diff --git a/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java b/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java
index 4a2de32..249850c 100644
--- a/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java
+++ b/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.net;
 
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.Files;
@@ -157,8 +156,7 @@ public class AsyncStreamingOutputPlusTest
         StreamManager.StreamRateLimiter limiter = zeroCopy ? StreamManager.getEntireSSTableRateLimiter(FBUtilities.getBroadcastAddressAndPort())
                                                            : StreamManager.getRateLimiter(FBUtilities.getBroadcastAddressAndPort());
 
-        try (RandomAccessFile raf = new RandomAccessFile(file.path(), "r");
-             FileChannel fileChannel = raf.getChannel();
+        try (FileChannel fileChannel = file.newReadChannel();
              AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
         {
             assertTrue(fileChannel.isOpen());
diff --git a/test/unit/org/apache/cassandra/schema/MockSchema.java b/test/unit/org/apache/cassandra/schema/MockSchema.java
index 432e407..d6dc896 100644
--- a/test/unit/org/apache/cassandra/schema/MockSchema.java
+++ b/test/unit/org/apache/cassandra/schema/MockSchema.java
@@ -18,9 +18,9 @@
 */
 package org.apache.cassandra.schema;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.io.util.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
@@ -128,10 +128,7 @@ public class MockSchema
                 try
                 {
                     File file = new File(descriptor.filenameFor(Component.DATA));
-                    try (RandomAccessFile raf = new RandomAccessFile(file.toJavaIOFile(), "rw"))
-                    {
-                        raf.setLength(size);
-                    }
+                    Util.setFileLength(file, size);
                 }
                 catch (IOException e)
                 {
diff --git a/test/unit/org/apache/cassandra/security/EncryptionUtilsTest.java b/test/unit/org/apache/cassandra/security/EncryptionUtilsTest.java
index cb8b138..fdeda05 100644
--- a/test/unit/org/apache/cassandra/security/EncryptionUtilsTest.java
+++ b/test/unit/org/apache/cassandra/security/EncryptionUtilsTest.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.security;
 
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.HashMap;
@@ -82,7 +81,7 @@ public class EncryptionUtilsTest
 
         File f = FileUtils.createTempFile("commitlog-enc-utils-", ".tmp");
         f.deleteOnExit();
-        FileChannel channel = new RandomAccessFile(f.toJavaIOFile(), "rw").getChannel();
+        FileChannel channel = f.newReadWriteChannel();
         EncryptionUtils.encryptAndWrite(ByteBuffer.wrap(buf), channel, true, encryptor);
         channel.close();
 
@@ -111,7 +110,7 @@ public class EncryptionUtilsTest
         Cipher encryptor = cipherFactory.getEncryptor(tdeOptions.cipher, tdeOptions.key_alias);
         File f = FileUtils.createTempFile("commitlog-enc-utils-", ".tmp");
         f.deleteOnExit();
-        FileChannel channel = new RandomAccessFile(f.toJavaIOFile(), "rw").getChannel();
+        FileChannel channel = f.newReadWriteChannel();
         EncryptionUtils.encryptAndWrite(compressedBuffer, channel, true, encryptor);
 
         // decrypt
diff --git a/test/unit/org/apache/cassandra/service/FileSystemOwnershipCheckTest.java b/test/unit/org/apache/cassandra/service/FileSystemOwnershipCheckTest.java
index f59b7dd..22f7268 100644
--- a/test/unit/org/apache/cassandra/service/FileSystemOwnershipCheckTest.java
+++ b/test/unit/org/apache/cassandra/service/FileSystemOwnershipCheckTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.service;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.file.Files;
@@ -33,6 +32,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.cassandra.exceptions.StartupException;
+import org.apache.cassandra.io.util.File;
 
 import static org.apache.cassandra.service.FileSystemOwnershipCheck.*;
 
@@ -49,7 +49,7 @@ public class FileSystemOwnershipCheckTest
     public void setup() throws IOException
     {
         cleanTempDir();
-        tempDir = com.google.common.io.Files.createTempDir();
+        tempDir = new File(com.google.common.io.Files.createTempDir());
         token = makeRandomString(10);
 
         System.setProperty(OWNERSHIP_TOKEN, token);
@@ -71,9 +71,10 @@ public class FileSystemOwnershipCheckTest
 
     private void delete(File file)
     {
-        file.setReadable(true);
-        file.setWritable(true);
-        File[] files = file.listFiles();
+        file.trySetReadable(true);
+        file.trySetWritable(true);
+        file.trySetExecutable(true);
+        File[] files = file.tryList();
         if (files != null)
         {
             for (File child : files)
@@ -118,16 +119,16 @@ public class FileSystemOwnershipCheckTest
     public void noDirectoryStructureOrTokenFilePresent() throws Exception
     {
         // The root directory exists, but is completely empty
-        executeAndFail(checker(tempDir), NO_OWNERSHIP_FILE, quote(tempDir.getAbsolutePath()));
+        executeAndFail(checker(tempDir), NO_OWNERSHIP_FILE, quote(tempDir.absolutePath()));
     }
 
     @Test
     public void directoryStructureButNoTokenFiles() throws Exception
     {
-        File childDir = new File(tempDir, "cassandra/data");
-        assertTrue(childDir.mkdirs());
+        File childDir = new File(tempDir, "cassandra/data"); //checkstyle: permit this instantiation
+        assertTrue(childDir.tryCreateDirectories());
         assertTrue(childDir.exists());
-        executeAndFail(checker(childDir), NO_OWNERSHIP_FILE, quote(childDir.getAbsolutePath()));
+        executeAndFail(checker(childDir), NO_OWNERSHIP_FILE, quote(childDir.absolutePath()));
     }
 
     @Test
@@ -135,7 +136,7 @@ public class FileSystemOwnershipCheckTest
     {
         File leafDir = mkdirs(tempDir, "cassandra/data");
         writeFile(leafDir, 1, token);
-        writeFile(leafDir.getParentFile(), 1, token);
+        writeFile(leafDir.parent(), 1, token);
         executeAndFail(checker(leafDir), MULTIPLE_OWNERSHIP_FILES, leafDir);
     }
 
@@ -149,7 +150,7 @@ public class FileSystemOwnershipCheckTest
                                        mkdirs(tempDir, "d2/commitlogs"),
                                        mkdirs(tempDir, "d3/hints") };
         for (File dir : leafDirs)
-            writeFile(dir.getParentFile(), 3, token);
+            writeFile(dir.parent(), 3, token);
         checker(leafDirs).execute();
     }
 
@@ -177,34 +178,34 @@ public class FileSystemOwnershipCheckTest
 
         executeAndFail(checker(leafDir1, leafDir2, leafDir3),
                        NO_OWNERSHIP_FILE,
-                       quote(leafDir3.getAbsolutePath()));
+                       quote(leafDir3.absolutePath()));
     }
 
     @Test
     public void propsFileUnreadable() throws Exception
     {
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        File tokenFile = writeFile(leafDir.getParentFile(), 1, token);
-        assertTrue(tokenFile.setReadable(false));
+        File tokenFile = writeFile(leafDir.parent(), 1, token);
+        assertTrue(tokenFile.trySetReadable(false));
         executeAndFail(checker(leafDir),
                        READ_EXCEPTION,
-                       leafDir.getAbsolutePath());
+                       leafDir.absolutePath());
     }
 
     @Test
     public void propsFileIllegalContent() throws Exception
     {
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        File propsFile = new File(leafDir, DEFAULT_FS_OWNERSHIP_FILENAME);
-        assertTrue(propsFile.createNewFile());
+        File propsFile = new File(leafDir, DEFAULT_FS_OWNERSHIP_FILENAME); //checkstyle: permit this instantiation
+        assertTrue(propsFile.createFileIfNotExists());
         try (OutputStream os = Files.newOutputStream(propsFile.toPath()))
         {
             os.write(makeRandomString(40).getBytes());
         }
-        assertTrue(propsFile.canRead());
+        assertTrue(propsFile.isReadable());
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, VERSION),
-                       leafDir.getAbsolutePath());
+                       leafDir.absolutePath());
     }
 
     @Test
@@ -213,7 +214,7 @@ public class FileSystemOwnershipCheckTest
         // The props file itself is readable, but its dir is not
         File leafDir = mkdirs(tempDir, "cassandra/data");
         writeFile(leafDir, 1, token);
-        assertTrue(leafDir.setReadable(false));
+        assertTrue(leafDir.trySetReadable(false));
         checker(leafDir).execute();
     }
 
@@ -222,19 +223,19 @@ public class FileSystemOwnershipCheckTest
     {
         // top level dir can't be listed, so no files are found
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), 1, token);
-        assertTrue(tempDir.setExecutable(false));
+        writeFile(leafDir.parent(), 1, token);
+        assertTrue(tempDir.trySetExecutable(false));
         executeAndFail(checker(leafDir),
                        NO_OWNERSHIP_FILE,
-                       quote(leafDir.getAbsolutePath()));
+                       quote(leafDir.absolutePath()));
     }
 
     @Test
     public void overrideFilename() throws Exception
     {
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), "other_file", makeProperties(1, 1, token));
-        executeAndFail(checker(leafDir), NO_OWNERSHIP_FILE, quote(leafDir.getAbsolutePath()));
+        writeFile(leafDir.parent(), "other_file", makeProperties(1, 1, token));
+        executeAndFail(checker(leafDir), NO_OWNERSHIP_FILE, quote(leafDir.absolutePath()));
         System.setProperty(FS_OWNERSHIP_FILENAME_PROPERTY, "other_file");
         checker(leafDir).execute();
     }
@@ -247,11 +248,11 @@ public class FileSystemOwnershipCheckTest
         File file2 = writeFile(mkdirs(tempDir, "d2/commitlogs"), 3, token);
         File file3 = writeFile(mkdirs(tempDir, "d3/hints"), 3, "mismatchingtoken");
         String errorSuffix = String.format("['%s', '%s'], ['%s']",
-                                           file1.getAbsolutePath(),
-                                           file2.getAbsolutePath(),
-                                           file3.getAbsolutePath());
+                                           file1.absolutePath(),
+                                           file2.absolutePath(),
+                                           file3.absolutePath());
 
-        executeAndFail(checker(file1.getParentFile(), file2.getParentFile(), file3.getParentFile()),
+        executeAndFail(checker(file1.parent(), file2.parent(), file3.parent()),
                        INCONSISTENT_FILES_FOUND,
                        errorSuffix);
     }
@@ -263,10 +264,10 @@ public class FileSystemOwnershipCheckTest
         File file2 = writeFile(mkdirs(tempDir, "d2/commitlogs"), 2, token);
         File file3 = writeFile(mkdirs(tempDir, "d3/hints"), 3, "mismatchingtoken");
         String errorSuffix = String.format("['%s'], ['%s'], ['%s']",
-                                           file1.getAbsolutePath(),
-                                           file2.getAbsolutePath(),
-                                           file3.getAbsolutePath());
-        executeAndFail(checker(file1.getParentFile(), file2.getParentFile(), file3.getParentFile()),
+                                           file1.absolutePath(),
+                                           file2.absolutePath(),
+                                           file3.absolutePath());
+        executeAndFail(checker(file1.parent(), file2.parent(), file3.parent()),
                        INCONSISTENT_FILES_FOUND,
                        errorSuffix);
     }
@@ -276,10 +277,10 @@ public class FileSystemOwnershipCheckTest
     public void emptyPropertiesFile() throws Exception
     {
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), DEFAULT_FS_OWNERSHIP_FILENAME, new Properties());
+        writeFile(leafDir.parent(), DEFAULT_FS_OWNERSHIP_FILENAME, new Properties());
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, VERSION),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
@@ -289,10 +290,10 @@ public class FileSystemOwnershipCheckTest
         p.setProperty(VOLUME_COUNT, "1");
         p.setProperty(TOKEN, "foo");
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
+        writeFile(leafDir.parent(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, VERSION),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
@@ -301,10 +302,10 @@ public class FileSystemOwnershipCheckTest
         Properties p = new Properties();
         p.setProperty(VERSION, "abc");
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
+        writeFile(leafDir.parent(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, VERSION),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
@@ -313,10 +314,10 @@ public class FileSystemOwnershipCheckTest
         Properties p = new Properties();
         p.setProperty(VERSION, "99");
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
+        writeFile(leafDir.parent(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
         executeAndFail(checker(leafDir),
                        String.format(UNSUPPORTED_VERSION, "99"),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
@@ -326,10 +327,10 @@ public class FileSystemOwnershipCheckTest
         p.setProperty(VERSION, "1");
         p.setProperty(TOKEN, token);
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
+        writeFile(leafDir.parent(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, VOLUME_COUNT),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
@@ -340,10 +341,10 @@ public class FileSystemOwnershipCheckTest
         p.setProperty(VOLUME_COUNT, "bar");
         p.setProperty(TOKEN, token);
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
+        writeFile(leafDir.parent(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, VOLUME_COUNT),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
@@ -353,20 +354,20 @@ public class FileSystemOwnershipCheckTest
         p.setProperty(VERSION, "1");
         p.setProperty(VOLUME_COUNT, "1");
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
+        writeFile(leafDir.parent(), DEFAULT_FS_OWNERSHIP_FILENAME, p);
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, TOKEN),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
     public void emptyTokenProp() throws Exception
     {
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), 1, "");
+        writeFile(leafDir.parent(), 1, "");
         executeAndFail(checker(leafDir),
                        String.format(INVALID_PROPERTY_VALUE, TOKEN),
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
     @Test
@@ -374,10 +375,10 @@ public class FileSystemOwnershipCheckTest
     {
         // Ownership token file exists in parent, but content doesn't match property
         File leafDir = mkdirs(tempDir, "cassandra/data");
-        writeFile(leafDir.getParentFile(), 1, makeRandomString(15));
+        writeFile(leafDir.parent(), 1, makeRandomString(15));
         executeAndFail(checker(leafDir),
                        MISMATCHING_TOKEN,
-                       leafDir.getParentFile().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
+                       leafDir.parent().toPath().resolve(DEFAULT_FS_OWNERSHIP_FILENAME));
     }
 
 
@@ -440,20 +441,20 @@ public class FileSystemOwnershipCheckTest
 
     private static File writeFile(File dir, String filename, Properties props) throws IOException
     {
-        File tokenFile = new File(dir, filename);
-        assertTrue(tokenFile.createNewFile());
+        File tokenFile = new File(dir, filename); //checkstyle: permit this instantiation
+        assertTrue(tokenFile.createFileIfNotExists());
         try (OutputStream os = Files.newOutputStream(tokenFile.toPath()))
         {
             props.store(os, "Test properties");
         }
-        assertTrue(tokenFile.canRead());
+        assertTrue(tokenFile.isReadable());
         return tokenFile;
     }
 
     private static File mkdirs(File parent, String path)
     {
-        File childDir = new File(parent, path);
-        assertTrue(childDir.mkdirs());
+        File childDir = new File(parent, path); //checkstyle: permit this instantiation
+        assertTrue(childDir.tryCreateDirectories());
         assertTrue(childDir.exists());
         return childDir;
     }
@@ -465,7 +466,7 @@ public class FileSystemOwnershipCheckTest
 
     private static FileSystemOwnershipCheck checker(File...dirs)
     {
-        return checker(() -> Arrays.stream(dirs).map(File::getAbsolutePath).collect(Collectors.toList()));
+        return checker(() -> Arrays.stream(dirs).map(File::absolutePath).collect(Collectors.toList()));
     }
 
     private static FileSystemOwnershipCheck checker(String...dirs)
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerM3PTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerM3PTest.java
index 69b0642..bb78c83 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerM3PTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerM3PTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.service;
 
-import java.io.File;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -28,6 +27,7 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.PropertyFileSnitch;
 
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index f8d3942..d9cf4f2 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -20,7 +20,6 @@
 package org.apache.cassandra.service;
 
 import org.apache.cassandra.io.util.File;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetAddress;
@@ -54,6 +53,7 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.PropertyFileSnitch;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.schema.*;
+import org.apache.cassandra.io.util.FileWriter;
 import org.apache.cassandra.utils.FBUtilities;
 import org.assertj.core.api.Assertions;
 
@@ -138,7 +138,7 @@ public class StorageServiceServerTest
 
         // Check to make sure we don't delete non-temp, non-datafile locations
         WindowsFailedSnapshotTracker.resetForTests();
-        PrintWriter tempPrinter = new PrintWriter(new FileWriter(WindowsFailedSnapshotTracker.TODELETEFILE, true));
+        PrintWriter tempPrinter = new PrintWriter(new FileWriter(new File(WindowsFailedSnapshotTracker.TODELETEFILE), File.WriteMode.APPEND));
         tempPrinter.println(".safeDir");
         tempPrinter.close();
 
diff --git a/test/unit/org/apache/cassandra/service/snapshot/SnapshotManifestTest.java b/test/unit/org/apache/cassandra/service/snapshot/SnapshotManifestTest.java
index 5afae1c..41cb1e7 100644
--- a/test/unit/org/apache/cassandra/service/snapshot/SnapshotManifestTest.java
+++ b/test/unit/org/apache/cassandra/service/snapshot/SnapshotManifestTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.service.snapshot;
 
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.time.Instant;
diff --git a/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java b/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java
index c49b10e..691aa70 100644
--- a/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java
+++ b/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.service.snapshot;
 
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.time.Instant;
 import java.util.Arrays;
diff --git a/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java b/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java
index a7a55cf..7c41fbe 100644
--- a/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/streaming/compression/CompressedInputStreamTest.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.streaming.compression;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
 import java.util.*;
 
 import org.apache.cassandra.io.util.File;
@@ -33,6 +36,7 @@ import org.apache.cassandra.io.compress.CompressedSequentialWriter;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
+import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriterOption;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.io.sstable.Component;
@@ -154,7 +158,7 @@ public class CompressedInputStreamTest
             size += (c.length + 4); // 4bytes CRC
         byte[] toRead = new byte[size];
 
-        try (RandomAccessFile f = new RandomAccessFile(tmp.toJavaIOFile(), "r"))
+        try (RandomAccessReader f = RandomAccessReader.open(tmp))
         {
             int pos = 0;
             for (CompressionMetadata.Chunk c : chunks)
diff --git a/test/unit/org/apache/cassandra/utils/KeyGenerator.java b/test/unit/org/apache/cassandra/utils/KeyGenerator.java
index d84d7aa..9bbfb2c 100644
--- a/test/unit/org/apache/cassandra/utils/KeyGenerator.java
+++ b/test/unit/org/apache/cassandra/utils/KeyGenerator.java
@@ -21,11 +21,11 @@ package org.apache.cassandra.utils;
 import org.apache.cassandra.io.util.FileInputStreamPlus;
 
 import java.io.BufferedReader;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.ByteBuffer;
+import java.nio.file.NoSuchFileException;
 import java.util.Random;
 
 public class KeyGenerator
@@ -160,9 +160,9 @@ public class KeyGenerator
         {
             try 
             {
-                reader = new BufferedReader(new InputStreamReader(new FileInputStream("/usr/share/dict/words")));
+                reader = new BufferedReader(new InputStreamReader(new FileInputStreamPlus("/usr/share/dict/words")));
             } 
-            catch (FileNotFoundException e)
+            catch (NoSuchFileException e)
             {
                 throw new RuntimeException(e);
             }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org