From 43938f899c9a1dff1e9e6a76c1e5061fa6ba617c Mon Sep 17 00:00:00 2001 From: Sebastian Stenzel Date: Thu, 23 Mar 2023 13:01:17 +0100 Subject: [PATCH 01/32] use a single collection for both stale and active chunks, facilitate cache weight to keep active chunks indefinitely --- .../cryptomator/cryptofs/fh/ChunkCache.java | 119 ++++++------------ .../cryptofs/fh/ChunkCacheTest.java | 16 +-- 2 files changed, 40 insertions(+), 95 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java index 2364005f..b8c605cd 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java @@ -10,11 +10,8 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.ByteBuffer; -import java.util.concurrent.ConcurrentHashMap; +import java.nio.channels.NonWritableChannelException; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.BiFunction; @OpenFileScoped public class ChunkCache { @@ -26,14 +23,7 @@ public class ChunkCache { private final CryptoFileSystemStats stats; private final BufferPool bufferPool; private final ExceptionsDuringWrite exceptionsDuringWrite; - private final ConcurrentMap staleChunks; - private final ConcurrentMap activeChunks; - - /** - * This lock ensures no chunks are passed between stale and active state while flushing, - * as flushing requires iteration over both sets. - */ - private final ReadWriteLock flushLock = new ReentrantReadWriteLock(); + private final ConcurrentMap cachedChunks; @Inject public ChunkCache(ChunkLoader chunkLoader, ChunkSaver chunkSaver, CryptoFileSystemStats stats, BufferPool bufferPool, ExceptionsDuringWrite exceptionsDuringWrite) { @@ -42,12 +32,20 @@ public ChunkCache(ChunkLoader chunkLoader, ChunkSaver chunkSaver, CryptoFileSyst this.stats = stats; this.bufferPool = bufferPool; this.exceptionsDuringWrite = exceptionsDuringWrite; - this.staleChunks = Caffeine.newBuilder() // - .maximumSize(MAX_CACHED_CLEARTEXT_CHUNKS) // + this.cachedChunks = Caffeine.newBuilder() // + .maximumWeight(MAX_CACHED_CLEARTEXT_CHUNKS) // + .weigher(this::weigh) // .evictionListener(this::evictStaleChunk) // .build() // .asMap(); - this.activeChunks = new ConcurrentHashMap<>(); + } + + private int weigh(Long index, Chunk chunk) { + if (chunk.currentAccesses().get() > 0) { + return 0; // zero, if currently in use -> avoid maximum size eviction + } else { + return 1; + } } /** @@ -59,13 +57,7 @@ public ChunkCache(ChunkLoader chunkLoader, ChunkSaver chunkSaver, CryptoFileSyst * @throws IllegalArgumentException If {@code chunkData}'s remaining bytes is not equal to the number of bytes fitting into a chunk */ public Chunk putChunk(long chunkIndex, ByteBuffer chunkData) throws IllegalArgumentException { - return activeChunks.compute(chunkIndex, (index, chunk) -> { - // stale chunk for this index is obsolete: - var staleChunk = staleChunks.remove(index); - if (staleChunk != null) { - bufferPool.recycle(staleChunk.data()); - } - // either create completely new chunk or replace all data of existing active chunk: + return cachedChunks.compute(chunkIndex, (index, chunk) -> { if (chunk == null) { chunk = new Chunk(chunkData, true, () -> releaseChunk(chunkIndex)); } else { @@ -87,31 +79,18 @@ public Chunk putChunk(long chunkIndex, ByteBuffer chunkData) throws IllegalArgum * @throws IOException If reading or decrypting the chunk failed */ public Chunk getChunk(long chunkIndex) throws IOException { - var lock = flushLock.readLock(); - lock.lock(); + stats.addChunkCacheAccess(); try { - stats.addChunkCacheAccess(); - return activeChunks.compute(chunkIndex, this::acquireInternal); + return cachedChunks.compute(chunkIndex, (idx, chunk) -> { + if (chunk == null) { + chunk = loadChunk(idx); + } + chunk.currentAccesses().incrementAndGet(); + return chunk; + }); } catch (UncheckedIOException | AuthenticationFailedException e) { throw new IOException(e); - } finally { - lock.unlock(); - } - } - - private Chunk acquireInternal(Long index, Chunk activeChunk) throws AuthenticationFailedException, UncheckedIOException { - Chunk result = activeChunk; - if (result == null) { - result = staleChunks.remove(index); - assert result == null || result.currentAccesses().get() == 0; - } - if (result == null) { - result = loadChunk(index); } - - assert result != null; - result.currentAccesses().incrementAndGet(); - return result; } private Chunk loadChunk(long chunkIndex) throws AuthenticationFailedException, UncheckedIOException { @@ -125,23 +104,10 @@ private Chunk loadChunk(long chunkIndex) throws AuthenticationFailedException, U @SuppressWarnings("resource") private void releaseChunk(long chunkIndex) { - var lock = flushLock.readLock(); - lock.lock(); - try { - activeChunks.compute(chunkIndex, (index, chunk) -> { - assert chunk != null; - var accessCnt = chunk.currentAccesses().decrementAndGet(); - if (accessCnt == 0) { - staleChunks.put(index, chunk); - return null; //chunk is stale, remove from active - } else { - assert accessCnt > 0; - return chunk; //keep active - } - }); - } finally { - lock.unlock(); - } + cachedChunks.computeIfPresent(chunkIndex, (idx, chunk) -> { + chunk.currentAccesses().decrementAndGet(); + return chunk; + }); } /** @@ -149,37 +115,26 @@ private void releaseChunk(long chunkIndex) { * @see #invalidateAll() */ public void flush() throws IOException { - var lock = flushLock.writeLock(); - lock.lock(); - BiFunction saveUnchecked = (index, chunk) -> { - try { - chunkSaver.save(index, chunk); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - return chunk; - }; try { - activeChunks.replaceAll(saveUnchecked); - staleChunks.replaceAll(saveUnchecked); + cachedChunks.forEach((index, chunk) -> { + try { + chunkSaver.save(index, chunk); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); } catch (UncheckedIOException e) { throw new IOException(e); - } finally { - lock.unlock(); } } /** * Removes stale chunks from cache. */ - public void invalidateAll() { - var lock = flushLock.writeLock(); - lock.lock(); - try { - staleChunks.clear(); - } finally { - lock.unlock(); - } + public void invalidateAll() { // TODO rename to invalidateStale() + // This may not be atomic, however this method is only called during truncation. + // If chunks are added (and become stale) concurrently, behavior is undefined anyway + cachedChunks.entrySet().removeIf(entry -> entry.getValue().currentAccesses().get() == 0); } // visible for testing diff --git a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java index d4cf0615..24d5e29d 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java @@ -170,8 +170,9 @@ public void testClosingActiveChunkTriggersEvictionOfStaleChunk() throws IOExcept Assertions.assertTimeoutPreemptively(Duration.ofMillis(100), () -> { cdl.await(); }); - verify(chunkSaver).save(42L, staleChunk42); - verify(bufferPool).recycle(staleChunk42.data()); + // we can't know _which_ stale chunk gets evicted. see https://github.com/ben-manes/caffeine/issues/583 + verify(chunkSaver).save(Mockito.anyLong(), Mockito.any()); + verify(bufferPool).recycle(Mockito.any()); verifyNoMoreInteractions(chunkSaver); } @@ -253,17 +254,6 @@ public void testPutChunkReturnsActiveChunk() { Assertions.assertTrue(chunk.isDirty()); } - @Test - @DisplayName("putChunk() recycles stale chunk if present") - public void testPutChunkRecyclesStaleChunk() { - var chunk = inTest.putChunk(42L, ByteBuffer.allocate(0)); - - Assertions.assertNotSame(staleChunk42, chunk); - Assertions.assertEquals(1, chunk.currentAccesses().get()); - Assertions.assertTrue(chunk.isDirty()); - verify(bufferPool).recycle(staleChunk42.data()); - } - @Test @DisplayName("putChunk() returns new chunk if neither stale nor active") public void testPutChunkReturnsNewChunk() { From 071a1f8d6809c6e6fac2ba70ff44dd404970b094 Mon Sep 17 00:00:00 2001 From: Sebastian Stenzel Date: Thu, 23 Mar 2023 13:32:53 +0100 Subject: [PATCH 02/32] expose cache.cleanup() to avoid waiting for async maintenance during tests --- .../org/cryptomator/cryptofs/fh/ChunkCache.java | 13 ++++++++++--- .../cryptomator/cryptofs/fh/ChunkCacheTest.java | 14 +++----------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java index b8c605cd..a99adbbf 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java @@ -1,5 +1,6 @@ package org.cryptomator.cryptofs.fh; +import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.RemovalCause; import com.google.common.base.Preconditions; @@ -23,6 +24,7 @@ public class ChunkCache { private final CryptoFileSystemStats stats; private final BufferPool bufferPool; private final ExceptionsDuringWrite exceptionsDuringWrite; + private final Cache chunkCache; private final ConcurrentMap cachedChunks; @Inject @@ -32,12 +34,12 @@ public ChunkCache(ChunkLoader chunkLoader, ChunkSaver chunkSaver, CryptoFileSyst this.stats = stats; this.bufferPool = bufferPool; this.exceptionsDuringWrite = exceptionsDuringWrite; - this.cachedChunks = Caffeine.newBuilder() // + this.chunkCache = Caffeine.newBuilder() // .maximumWeight(MAX_CACHED_CLEARTEXT_CHUNKS) // .weigher(this::weigh) // .evictionListener(this::evictStaleChunk) // - .build() // - .asMap(); + .build(); + this.cachedChunks = chunkCache.asMap(); } private int weigh(Long index, Chunk chunk) { @@ -137,6 +139,11 @@ public void invalidateAll() { // TODO rename to invalidateStale() cachedChunks.entrySet().removeIf(entry -> entry.getValue().currentAccesses().get() == 0); } + // visible for testing + void cleanup() { + chunkCache.cleanUp(); + } + // visible for testing void evictStaleChunk(Long index, Chunk chunk, RemovalCause removalCause) { assert removalCause != RemovalCause.EXPLICIT; // as per spec of Caffeine#evictionListener(RemovalListener) diff --git a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java index 24d5e29d..1d171233 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java @@ -156,20 +156,12 @@ public void testClosingActiveChunkThatIsReferencedTwice() throws IOException, Au verifyNoMoreInteractions(bufferPool); } - @RepeatedTest(30) - @DisplayName("chunk.close() triggers eviction of LRU stale chunk") + @Test + @DisplayName("chunk.close() triggers eviction of some stale chunk") public void testClosingActiveChunkTriggersEvictionOfStaleChunk() throws IOException, AuthenticationFailedException { - var cdl = new CountDownLatch(1); - Mockito.doAnswer(invocation -> { - cdl.countDown(); - return null; - }).when(chunkSaver).save(Mockito.anyLong(), Mockito.any()); - activeChunk1.close(); - Assertions.assertTimeoutPreemptively(Duration.ofMillis(100), () -> { - cdl.await(); - }); + inTest.cleanup(); // evict now, don't wait for async task // we can't know _which_ stale chunk gets evicted. see https://github.com/ben-manes/caffeine/issues/583 verify(chunkSaver).save(Mockito.anyLong(), Mockito.any()); verify(bufferPool).recycle(Mockito.any()); From 4dad1ceceaba7681ba29e362043c4b64e86bcf55 Mon Sep 17 00:00:00 2001 From: Sebastian Stenzel Date: Thu, 23 Mar 2023 13:33:43 +0100 Subject: [PATCH 03/32] renamed method --- .../cryptomator/cryptofs/ch/CleartextFileChannel.java | 2 +- .../java/org/cryptomator/cryptofs/fh/ChunkCache.java | 4 ++-- .../java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java | 2 +- .../java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java | 9 +++------ .../org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java | 2 +- 5 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index 883247df..0bf1a18f 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -205,7 +205,7 @@ protected void truncateLocked(long newSize) throws IOException { } long ciphertextFileSize = cryptor.fileHeaderCryptor().headerSize() + cryptor.fileContentCryptor().ciphertextSize(newSize); chunkCache.flush(); - chunkCache.invalidateAll(); // make sure no chunks _after_ newSize exist that would otherwise be written during the next cache eviction + chunkCache.invalidateStale(); // make sure no chunks _after_ newSize exist that would otherwise be written during the next cache eviction ciphertextFileChannel.truncate(ciphertextFileSize); position = min(newSize, position); fileSize.set(newSize); diff --git a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java index a99adbbf..17f0599c 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java @@ -114,7 +114,7 @@ private void releaseChunk(long chunkIndex) { /** * Flushes cached data (but keeps them cached). - * @see #invalidateAll() + * @see #invalidateStale() */ public void flush() throws IOException { try { @@ -133,7 +133,7 @@ public void flush() throws IOException { /** * Removes stale chunks from cache. */ - public void invalidateAll() { // TODO rename to invalidateStale() + public void invalidateStale() { // This may not be atomic, however this method is only called during truncation. // If chunks are added (and become stale) concurrently, behavior is undefined anyway cachedChunks.entrySet().removeIf(entry -> entry.getValue().currentAccesses().get() == 0); diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java index 24d03020..b567db1d 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java @@ -69,7 +69,7 @@ public synchronized FileChannel newFileChannel(EffectiveOpenOptions options, Fil Path path = currentFilePath.get(); if (options.truncateExisting()) { - chunkCache.invalidateAll(); + chunkCache.invalidateStale(); } FileChannel ciphertextFileChannel = null; diff --git a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java index 1d171233..0898cf47 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java @@ -8,15 +8,12 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Nested; -import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.file.AccessDeniedException; -import java.time.Duration; -import java.util.concurrent.CountDownLatch; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -214,11 +211,11 @@ public void testFlushKeepsItemInCacheDespiteIOException() throws IOException, Au } @Test - @DisplayName("invalidateAll() flushes stale chunks but keeps active chunks") - public void testInvalidateAll() throws IOException, AuthenticationFailedException { + @DisplayName("invalidateStale() flushes stale chunks but keeps active chunks") + public void testInvalidateStale() throws IOException, AuthenticationFailedException { when(chunkLoader.load(Mockito.anyLong())).thenReturn(ByteBuffer.allocate(0)); - inTest.invalidateAll(); + inTest.invalidateStale(); Assertions.assertSame(activeChunk1, inTest.getChunk(1L)); Assertions.assertNotSame(staleChunk42, inTest.getChunk(42L)); diff --git a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java index afd5616d..35886a0e 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java @@ -171,7 +171,7 @@ public void testTruncateExistingInvalidatesChunkCache() throws IOException { Files.write(CURRENT_FILE_PATH.get(), new byte[0]); EffectiveOpenOptions options = EffectiveOpenOptions.from(EnumSet.of(StandardOpenOption.TRUNCATE_EXISTING, StandardOpenOption.WRITE), readonlyFlag); openCryptoFile.newFileChannel(options); - verify(chunkCache).invalidateAll(); + verify(chunkCache).invalidateStale(); } @Test From 0406814a3d08862f19533f4074a8e63b66b35b0c Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Fri, 31 Mar 2023 15:18:52 +0200 Subject: [PATCH 04/32] Fixes #165 --- .../cryptofs/EffectiveOpenOptions.java | 1 + .../cryptofs/ch/ChannelComponent.java | 2 -- .../cryptofs/ch/CleartextFileChannel.java | 22 +++++++----- .../cryptofs/ch/MustWriteHeader.java | 13 ------- .../cryptofs/fh/FileHeaderHolder.java | 13 ++++--- .../cryptofs/fh/OpenCryptoFile.java | 30 ++++++++-------- .../cryptofs/fh/OpenCryptoFileModule.java | 2 ++ .../cryptofs/ch/CleartextFileChannelTest.java | 34 +++++++++++++++---- .../cryptofs/fh/OpenCryptoFileTest.java | 9 +++-- 9 files changed, 74 insertions(+), 52 deletions(-) delete mode 100644 src/main/java/org/cryptomator/cryptofs/ch/MustWriteHeader.java diff --git a/src/main/java/org/cryptomator/cryptofs/EffectiveOpenOptions.java b/src/main/java/org/cryptomator/cryptofs/EffectiveOpenOptions.java index fadf1df1..7d6be085 100644 --- a/src/main/java/org/cryptomator/cryptofs/EffectiveOpenOptions.java +++ b/src/main/java/org/cryptomator/cryptofs/EffectiveOpenOptions.java @@ -182,6 +182,7 @@ public Set createOpenOptionsForEncryptedFile() { result.add(READ); // also needed during write result.remove(LinkOption.NOFOLLOW_LINKS); result.remove(APPEND); + result.remove(TRUNCATE_EXISTING); return result; } diff --git a/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java b/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java index 706a9185..7cd64e56 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java @@ -17,8 +17,6 @@ public interface ChannelComponent { interface Factory { ChannelComponent create(@BindsInstance FileChannel ciphertextChannel, // - @BindsInstance FileHeader fileHeader, // - @BindsInstance @MustWriteHeader boolean mustWriteHeader, // @BindsInstance EffectiveOpenOptions options, // @BindsInstance ChannelCloseListener listener); // } diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index 883247df..70d1a7f9 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -7,6 +7,7 @@ import org.cryptomator.cryptofs.fh.Chunk; import org.cryptomator.cryptofs.fh.ChunkCache; import org.cryptomator.cryptofs.fh.ExceptionsDuringWrite; +import org.cryptomator.cryptofs.fh.FileHeaderHolder; import org.cryptomator.cryptofs.fh.OpenFileModifiedDate; import org.cryptomator.cryptofs.fh.OpenFileSize; import org.cryptomator.cryptolib.api.Cryptor; @@ -25,7 +26,6 @@ import java.nio.file.attribute.BasicFileAttributeView; import java.nio.file.attribute.FileTime; import java.time.Instant; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; @@ -40,7 +40,7 @@ public class CleartextFileChannel extends AbstractFileChannel { private static final Logger LOG = LoggerFactory.getLogger(CleartextFileChannel.class); private final FileChannel ciphertextFileChannel; - private final FileHeader fileHeader; + private final FileHeaderHolder fileHeaderHolder; private final Cryptor cryptor; private final ChunkCache chunkCache; private final BufferPool bufferPool; @@ -51,13 +51,12 @@ public class CleartextFileChannel extends AbstractFileChannel { private final ExceptionsDuringWrite exceptionsDuringWrite; private final ChannelCloseListener closeListener; private final CryptoFileSystemStats stats; - private final AtomicBoolean mustWriteHeader; @Inject - public CleartextFileChannel(FileChannel ciphertextFileChannel, FileHeader fileHeader, @MustWriteHeader boolean mustWriteHeader, ReadWriteLock readWriteLock, Cryptor cryptor, ChunkCache chunkCache, BufferPool bufferPool, EffectiveOpenOptions options, @OpenFileSize AtomicLong fileSize, @OpenFileModifiedDate AtomicReference lastModified, Supplier attrViewProvider, ExceptionsDuringWrite exceptionsDuringWrite, ChannelCloseListener closeListener, CryptoFileSystemStats stats) { + public CleartextFileChannel(FileChannel ciphertextFileChannel, FileHeaderHolder fileHeaderHolder, ReadWriteLock readWriteLock, Cryptor cryptor, ChunkCache chunkCache, BufferPool bufferPool, EffectiveOpenOptions options, @OpenFileSize AtomicLong fileSize, @OpenFileModifiedDate AtomicReference lastModified, Supplier attrViewProvider, ExceptionsDuringWrite exceptionsDuringWrite, ChannelCloseListener closeListener, CryptoFileSystemStats stats) { super(readWriteLock); this.ciphertextFileChannel = ciphertextFileChannel; - this.fileHeader = fileHeader; + this.fileHeaderHolder = fileHeaderHolder; this.cryptor = cryptor; this.chunkCache = chunkCache; this.bufferPool = bufferPool; @@ -71,7 +70,6 @@ public CleartextFileChannel(FileChannel ciphertextFileChannel, FileHeader fileHe if (options.append()) { position = fileSize.get(); } - this.mustWriteHeader = new AtomicBoolean(mustWriteHeader); if (options.createNew() || options.create()) { lastModified.compareAndSet(Instant.EPOCH, Instant.now()); } @@ -183,10 +181,16 @@ private long writeLockedInternal(ByteSource src, long position) throws IOExcepti } private void writeHeaderIfNeeded() throws IOException { - if (mustWriteHeader.getAndSet(false)) { + if (!fileHeaderHolder.headerIsPersisted().getAndSet(true)) { LOG.trace("{} - Writing file header.", this); - ByteBuffer encryptedHeader = cryptor.fileHeaderCryptor().encryptHeader(fileHeader); - ciphertextFileChannel.write(encryptedHeader, 0); + ByteBuffer encryptedHeader = cryptor.fileHeaderCryptor().encryptHeader(fileHeaderHolder.get()); + //TODO: what if write fails? Should the encrypted header be cached? Because it cannot be encrypted again (NONCE reuse!) + try { + ciphertextFileChannel.write(encryptedHeader, 0); + } catch (IOException e) { + fileHeaderHolder.headerIsPersisted().set(false); + throw e; + } } } diff --git a/src/main/java/org/cryptomator/cryptofs/ch/MustWriteHeader.java b/src/main/java/org/cryptomator/cryptofs/ch/MustWriteHeader.java deleted file mode 100644 index 9df5388a..00000000 --- a/src/main/java/org/cryptomator/cryptofs/ch/MustWriteHeader.java +++ /dev/null @@ -1,13 +0,0 @@ -package org.cryptomator.cryptofs.ch; - -import javax.inject.Qualifier; -import java.lang.annotation.Documented; -import java.lang.annotation.Retention; - -import static java.lang.annotation.RetentionPolicy.RUNTIME; - -@Qualifier -@Documented -@Retention(RUNTIME) -@interface MustWriteHeader { -} diff --git a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java index 34abc1e1..5bfd9202 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java @@ -11,6 +11,7 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @OpenFileScoped @@ -21,6 +22,7 @@ public class FileHeaderHolder { private final Cryptor cryptor; private final AtomicReference path; private final AtomicReference header = new AtomicReference<>(); + private final AtomicBoolean isPersisted = new AtomicBoolean(); @Inject public FileHeaderHolder(Cryptor cryptor, @CurrentOpenFilePath AtomicReference path) { @@ -29,11 +31,7 @@ public FileHeaderHolder(Cryptor cryptor, @CurrentOpenFilePath AtomicReference... attrs) throws IOException { Path path = currentFilePath.get(); - if (options.truncateExisting()) { - chunkCache.invalidateAll(); - } - FileChannel ciphertextFileChannel = null; CleartextFileChannel cleartextFileChannel = null; try { - ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); - final FileHeader header; - final boolean isNewHeader; - if (ciphertextFileChannel.size() == 0l) { - header = headerHolder.createNew(); - isNewHeader = true; + if (headerHolder.get() != null) { + //file already loaded, use already loaded header + ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); + } else if (Files.notExists(path)) { + //file does not exist, create new header + ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); + headerHolder.createNew(); } else { - header = headerHolder.loadExisting(ciphertextFileChannel); - isNewHeader = false; + //file already exists, load header + ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); + headerHolder.loadExisting(ciphertextFileChannel); + } + if (options.truncateExisting()) { + chunkCache.invalidateAll(); + ciphertextFileChannel.truncate(cryptor.fileHeaderCryptor().headerSize()); } initFileSize(ciphertextFileChannel); cleartextFileChannel = component.newChannelComponent() // - .create(ciphertextFileChannel, header, isNewHeader, options, this::channelClosed) // + .create(ciphertextFileChannel, options, this::channelClosed) // .channel(); } finally { if (cleartextFileChannel == null) { // i.e. something didn't work diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java index 83d587ee..48e30c46 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java @@ -2,6 +2,8 @@ import dagger.Module; import dagger.Provides; +import org.cryptomator.cryptolib.api.Cryptor; +import org.cryptomator.cryptolib.api.FileHeader; import java.io.IOException; import java.nio.file.Files; diff --git a/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java b/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java index fe9ac96f..5e067420 100644 --- a/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java +++ b/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java @@ -6,9 +6,9 @@ import org.cryptomator.cryptofs.fh.Chunk; import org.cryptomator.cryptofs.fh.ChunkCache; import org.cryptomator.cryptofs.fh.ExceptionsDuringWrite; +import org.cryptomator.cryptofs.fh.FileHeaderHolder; import org.cryptomator.cryptolib.api.Cryptor; import org.cryptomator.cryptolib.api.FileContentCryptor; -import org.cryptomator.cryptolib.api.FileHeader; import org.cryptomator.cryptolib.api.FileHeaderCryptor; import org.hamcrest.MatcherAssert; import org.junit.jupiter.api.Assertions; @@ -35,6 +35,7 @@ import java.nio.file.attribute.BasicFileAttributeView; import java.nio.file.attribute.FileTime; import java.time.Instant; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; @@ -43,6 +44,9 @@ import static org.hamcrest.CoreMatchers.is; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -59,8 +63,8 @@ public class CleartextFileChannelTest { private FileHeaderCryptor fileHeaderCryptor = mock(FileHeaderCryptor.class); private FileContentCryptor fileContentCryptor = mock(FileContentCryptor.class); private FileChannel ciphertextFileChannel = mock(FileChannel.class); - private FileHeader header = mock(FileHeader.class); - private boolean mustWriteHeader = true; + private FileHeaderHolder headerHolder = mock(FileHeaderHolder.class); + private AtomicBoolean headerIsPersisted = mock(AtomicBoolean.class); private EffectiveOpenOptions options = mock(EffectiveOpenOptions.class); private AtomicLong fileSize = new AtomicLong(100); private AtomicReference lastModified = new AtomicReference<>(Instant.ofEpochMilli(0)); @@ -80,13 +84,15 @@ public void setUp() throws IOException { when(chunkCache.putChunk(Mockito.anyLong(), Mockito.any())).thenAnswer(invocation -> new Chunk(invocation.getArgument(1), true, () -> {})); when(bufferPool.getCleartextBuffer()).thenAnswer(invocation -> ByteBuffer.allocate(100)); when(fileHeaderCryptor.headerSize()).thenReturn(50); + when(headerHolder.headerIsPersisted()).thenReturn(headerIsPersisted); + when(headerIsPersisted.getAndSet(anyBoolean())).thenReturn(true); when(fileContentCryptor.cleartextChunkSize()).thenReturn(100); when(fileContentCryptor.ciphertextChunkSize()).thenReturn(110); when(attributeViewSupplier.get()).thenReturn(attributeView); when(readWriteLock.readLock()).thenReturn(readLock); when(readWriteLock.writeLock()).thenReturn(writeLock); - inTest = new CleartextFileChannel(ciphertextFileChannel, header, mustWriteHeader, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); + inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); } @Test @@ -339,7 +345,7 @@ public void testReadFromMultipleChunks() throws IOException { fileSize.set(5_000_000_100l); // initial cleartext size will be 5_000_000_100l when(options.readable()).thenReturn(true); - inTest = new CleartextFileChannel(ciphertextFileChannel, header, mustWriteHeader, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); + inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); ByteBuffer buf = ByteBuffer.allocate(10); // A read from frist chunk: @@ -483,6 +489,8 @@ public void writeAfterEof() throws IOException { public void testWriteHeaderIfNeeded() throws IOException { when(options.writable()).thenReturn(true); + when(headerIsPersisted.getAndSet(anyBoolean())).thenReturn(false).thenReturn(true).thenReturn(true); + inTest.force(true); inTest.force(true); inTest.force(true); @@ -490,11 +498,25 @@ public void testWriteHeaderIfNeeded() throws IOException { Mockito.verify(ciphertextFileChannel, Mockito.times(1)).write(Mockito.any(), Mockito.eq(0l)); } + @Test + @DisplayName("If writing header fails, it is indicated as not persistet") + public void testWriteHeaderFailsResetsPersistenceState() throws IOException { + when(options.writable()).thenReturn(true); + when(headerIsPersisted.getAndSet(anyBoolean())).thenReturn(false); + doNothing().when(headerIsPersisted).set(anyBoolean()); + when(ciphertextFileChannel.write(any(), anyLong())).thenThrow(new IOException("writing failed")); + + Assertions.assertThrows(IOException.class, () -> inTest.force(true)); + + Mockito.verify(ciphertextFileChannel, Mockito.times(1)).write(Mockito.any(), Mockito.eq(0l)); + Mockito.verify(headerIsPersisted, Mockito.times(1)).set(false); + } + @Test @DisplayName("don't write header if it is already written") public void testDontRewriteHeader() throws IOException { when(options.writable()).thenReturn(true); - inTest = new CleartextFileChannel(ciphertextFileChannel, header, false, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); + inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); inTest.force(true); diff --git a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java index afd5616d..89ae9592 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java @@ -9,6 +9,7 @@ import org.cryptomator.cryptofs.ch.CleartextFileChannel; import org.cryptomator.cryptolib.api.Cryptor; import org.cryptomator.cryptolib.api.FileHeader; +import org.cryptomator.cryptolib.api.FileHeaderCryptor; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; @@ -46,8 +47,8 @@ public class OpenCryptoFileTest { private FileCloseListener closeListener = mock(FileCloseListener.class); private ChunkCache chunkCache = mock(ChunkCache.class); private Cryptor cryptor = mock(Cryptor.class); + private FileHeaderCryptor fileHeaderCryptor = mock(FileHeaderCryptor.class); private FileHeaderHolder headerHolder = mock(FileHeaderHolder.class); - private FileHeader header = mock(FileHeader.class); private ChunkIO chunkIO = mock(ChunkIO.class); private AtomicLong fileSize = new AtomicLong(-1l); private AtomicReference lastModified = new AtomicReference(Instant.ofEpochMilli(0)); @@ -107,9 +108,9 @@ public void setup() throws IOException { ciphertextChannel = new AtomicReference<>(); Mockito.when(openCryptoFileComponent.newChannelComponent()).thenReturn(channelComponentFactory); - Mockito.when(channelComponentFactory.create(Mockito.any(), Mockito.any(), Mockito.anyBoolean(), Mockito.any(), Mockito.any())).thenAnswer(invocation -> { + Mockito.when(channelComponentFactory.create(Mockito.any(), Mockito.any(), Mockito.any())).thenAnswer(invocation -> { ciphertextChannel.set(invocation.getArgument(0)); - listener.set(invocation.getArgument(4)); + listener.set(invocation.getArgument(2)); return channelComponent; }); Mockito.when(channelComponent.channel()).thenReturn(cleartextFileChannel); @@ -168,6 +169,8 @@ public void testGetSizeAfterCreatingSecondFileChannel() { @Order(20) @DisplayName("TRUNCATE_EXISTING leads to chunk cache invalidation") public void testTruncateExistingInvalidatesChunkCache() throws IOException { + Mockito.when(cryptor.fileHeaderCryptor()).thenReturn(fileHeaderCryptor); + Mockito.when(fileHeaderCryptor.headerSize()).thenReturn(43); Files.write(CURRENT_FILE_PATH.get(), new byte[0]); EffectiveOpenOptions options = EffectiveOpenOptions.from(EnumSet.of(StandardOpenOption.TRUNCATE_EXISTING, StandardOpenOption.WRITE), readonlyFlag); openCryptoFile.newFileChannel(options); From b45f642347ec623581821ed061585dfbc0040056 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Sat, 1 Apr 2023 23:53:16 +0200 Subject: [PATCH 05/32] add javadoc to method and removing questionable assertion --- .../org/cryptomator/cryptofs/fh/FileHeaderHolder.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java index 5bfd9202..8dedcfe1 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java @@ -41,11 +41,18 @@ public FileHeader createNew() { return newHeader; } + + /** + * Reads, decrypts and caches the file header from the given file channel. + * + * @param ch File channel to the encrypted file + * @return {@link FileHeader} of the encrypted file + * @throws IOException if the file header cannot be read or decrypted + */ public FileHeader loadExisting(FileChannel ch) throws IOException { LOG.trace("Reading file header from {}", path.get()); ByteBuffer existingHeaderBuf = ByteBuffer.allocate(cryptor.fileHeaderCryptor().headerSize()); - int read = ch.read(existingHeaderBuf, 0); - assert read == existingHeaderBuf.capacity(); + ch.read(existingHeaderBuf, 0); existingHeaderBuf.flip(); try { FileHeader existingHeader = cryptor.fileHeaderCryptor().decryptHeader(existingHeaderBuf); From 289b7249fbd83bed97c7787b2ce4abb1bc48863f Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Sat, 1 Apr 2023 23:57:42 +0200 Subject: [PATCH 06/32] refactor logic of file header initialization --- .../cryptofs/fh/OpenCryptoFile.java | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java index 8b81af6d..2328dfe1 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java @@ -18,7 +18,6 @@ import java.io.Closeable; import java.io.IOException; import java.nio.channels.FileChannel; -import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.attribute.FileAttribute; import java.nio.file.attribute.FileTime; @@ -71,18 +70,8 @@ public synchronized FileChannel newFileChannel(EffectiveOpenOptions options, Fil FileChannel ciphertextFileChannel = null; CleartextFileChannel cleartextFileChannel = null; try { - if (headerHolder.get() != null) { - //file already loaded, use already loaded header - ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); - } else if (Files.notExists(path)) { - //file does not exist, create new header - ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); - headerHolder.createNew(); - } else { - //file already exists, load header - ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); - headerHolder.loadExisting(ciphertextFileChannel); - } + ciphertextFileChannel = path.getFileSystem().provider().newFileChannel(path, options.createOpenOptionsForEncryptedFile(), attrs); + initFileHeader(options, ciphertextFileChannel); if (options.truncateExisting()) { chunkCache.invalidateAll(); ciphertextFileChannel.truncate(cryptor.fileHeaderCryptor().headerSize()); @@ -107,6 +96,20 @@ public synchronized FileChannel newFileChannel(EffectiveOpenOptions options, Fil return cleartextFileChannel; } + //TODO test + private void initFileHeader(EffectiveOpenOptions options, FileChannel ciphertextFileChannel) throws IOException { + if (headerHolder.get() == null) { + //first file channel to file, no header present + if (options.createNew() || (options.create() && ciphertextFileChannel.size() == 0)) { + //file did not exist, create new header + headerHolder.createNew(); + } else { + //file must exist, load header from file + headerHolder.loadExisting(ciphertextFileChannel); + } + } + } + private void closeQuietly(Closeable closeable) { if (closeable != null) { try { From 1bcb06a01f4edb3e351ca00844e34ccf2b788fc2 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Sun, 2 Apr 2023 12:53:46 +0200 Subject: [PATCH 07/32] remove unused imports --- src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java | 1 - .../java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java | 2 -- 2 files changed, 3 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java b/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java index 7cd64e56..a22b0fa4 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/ChannelComponent.java @@ -3,7 +3,6 @@ import dagger.BindsInstance; import dagger.Subcomponent; import org.cryptomator.cryptofs.EffectiveOpenOptions; -import org.cryptomator.cryptolib.api.FileHeader; import java.nio.channels.FileChannel; diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java index 48e30c46..83d587ee 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java @@ -2,8 +2,6 @@ import dagger.Module; import dagger.Provides; -import org.cryptomator.cryptolib.api.Cryptor; -import org.cryptomator.cryptolib.api.FileHeader; import java.io.IOException; import java.nio.file.Files; From c827486def21a687a08c16c5526427391b732094 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Sun, 2 Apr 2023 12:54:44 +0200 Subject: [PATCH 08/32] add unit test covering truncate_existing is removed when opening ciphertext filechannel --- .../cryptofs/EffectiveOpenOptionsTest.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/test/java/org/cryptomator/cryptofs/EffectiveOpenOptionsTest.java b/src/test/java/org/cryptomator/cryptofs/EffectiveOpenOptionsTest.java index f8de714a..08af690f 100644 --- a/src/test/java/org/cryptomator/cryptofs/EffectiveOpenOptionsTest.java +++ b/src/test/java/org/cryptomator/cryptofs/EffectiveOpenOptionsTest.java @@ -294,6 +294,24 @@ public void testTruncateExisting() throws IOException { MatcherAssert.assertThat(inTest.createOpenOptionsForEncryptedFile(), containsInAnyOrder(READ)); } + @Test + public void testTruncateExistingAndWrite() throws IOException { + EffectiveOpenOptions inTest = EffectiveOpenOptions.from(Set.of(TRUNCATE_EXISTING, WRITE), falseReadonlyFlag); + + Assertions.assertFalse(inTest.append()); + Assertions.assertFalse(inTest.create()); + Assertions.assertFalse(inTest.createNew()); + Assertions.assertFalse(inTest.deleteOnClose()); + Assertions.assertFalse(inTest.noFollowLinks()); + Assertions.assertFalse(inTest.readable()); + Assertions.assertFalse(inTest.syncData()); + Assertions.assertFalse(inTest.syncDataAndMetadata()); + Assertions.assertTrue(inTest.truncateExisting()); + Assertions.assertTrue(inTest.writable()); + + MatcherAssert.assertThat(inTest.createOpenOptionsForEncryptedFile(), containsInAnyOrder(READ, WRITE)); + } + @Test public void testWrite() throws IOException { EffectiveOpenOptions inTest = EffectiveOpenOptions.from(Set.of(WRITE), falseReadonlyFlag); From 9328dd387dfc39be34a865a97aa4144de4cf49cc Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Sun, 2 Apr 2023 12:59:34 +0200 Subject: [PATCH 09/32] correct javadoc --- src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java index 2328dfe1..a87c2079 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java @@ -121,7 +121,7 @@ private void closeQuietly(Closeable closeable) { } /** - * Called by {@link #newFileChannel(EffectiveOpenOptions)} to determine the fileSize. + * Called by {@link #newFileChannel(EffectiveOpenOptions, FileAttribute[])} to determine the fileSize. *

* Before the size is initialized (i.e. before a channel has been created), {@link #size()} must not be called. *

@@ -146,7 +146,7 @@ private void initFileSize(FileChannel ciphertextFileChannel) throws IOException } /** - * @return The size of the opened file. Note that the filesize is unknown until a {@link #newFileChannel(EffectiveOpenOptions) file channel is opened}. In this case this method returns an empty optional. + * @return The size of the opened file. Note that the filesize is unknown until a {@link #newFileChannel(EffectiveOpenOptions, FileAttribute[])} is opened. In this case this method returns an empty optional. */ public Optional size() { long val = fileSize.get(); From 68dcf1e7790297bf790ef195dd1c9cf3f589a1c1 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Sun, 2 Apr 2023 13:32:35 +0200 Subject: [PATCH 10/32] resolve TODO --- .../cryptofs/fh/OpenCryptoFile.java | 4 +- .../cryptofs/fh/OpenCryptoFileTest.java | 78 +++++++++++++++++++ 2 files changed, 80 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java index a87c2079..e40c3120 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java @@ -96,8 +96,8 @@ public synchronized FileChannel newFileChannel(EffectiveOpenOptions options, Fil return cleartextFileChannel; } - //TODO test - private void initFileHeader(EffectiveOpenOptions options, FileChannel ciphertextFileChannel) throws IOException { + //visible for testing + void initFileHeader(EffectiveOpenOptions options, FileChannel ciphertextFileChannel) throws IOException { if (headerHolder.get() == null) { //first file channel to file, no header present if (options.createNew() || (options.create() && ciphertextFileChannel.size() == 0)) { diff --git a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java index 89ae9592..da615fc4 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java @@ -14,6 +14,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.MethodOrderer; import org.junit.jupiter.api.Nested; @@ -36,7 +37,10 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; public class OpenCryptoFileTest { @@ -89,6 +93,80 @@ public void testCloseImmediatelyIfOpeningFirstChannelFails() { verify(closeListener).close(CURRENT_FILE_PATH.get(), openCryptoFile); } + @Nested + @DisplayName("Testing ::initFileHeader") + public class InitFilHeaderTests { + + EffectiveOpenOptions options = Mockito.mock(EffectiveOpenOptions.class); + FileChannel cipherFileChannel = Mockito.mock(FileChannel.class, "cipherFilechannel"); + OpenCryptoFile inTest = new OpenCryptoFile(closeListener, chunkCache, cryptor, headerHolder, chunkIO, CURRENT_FILE_PATH, fileSize, lastModified, openCryptoFileComponent); + + @Test + @DisplayName("Skip file header init, if the file header already exists in memory") + public void testInitFileHeaderExisting() throws IOException { + var header = Mockito.mock(FileHeader.class); + Mockito.when(headerHolder.get()).thenReturn(header); + + inTest.initFileHeader(options, cipherFileChannel); + + Mockito.verify(headerHolder, never()).loadExisting(any()); + Mockito.verify(headerHolder, never()).createNew(); + } + + @Test + @DisplayName("Load file header from file, if not present and neither create nor create_new set") + public void testInitFileHeaderLoad() throws IOException { + Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(options.createNew()).thenReturn(false); + Mockito.when(options.create()).thenReturn(false); + + inTest.initFileHeader(options, cipherFileChannel); + + Mockito.verify(headerHolder, times(1)).loadExisting(cipherFileChannel); + Mockito.verify(headerHolder, never()).createNew(); + } + + @Test + @DisplayName("Create new file header, if not present and create_new set") + public void testInitFileHeaderCreateNew() throws IOException { + Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(options.createNew()).thenReturn(true); + + inTest.initFileHeader(options, cipherFileChannel); + + Mockito.verify(headerHolder, times(1)).createNew(); + Mockito.verify(headerHolder, never()).loadExisting(any()); + } + + @Test + @DisplayName("Create new file header, if not present, create set and channel.size() == 0") + public void testInitFileHeaderCreateAndSize0() throws IOException { + Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(options.createNew()).thenReturn(false); + Mockito.when(options.create()).thenReturn(true); + Mockito.when(cipherFileChannel.size()).thenReturn(0L); + + inTest.initFileHeader(options, cipherFileChannel); + + Mockito.verify(headerHolder, times(1)).createNew(); + Mockito.verify(headerHolder, never()).loadExisting(any()); + } + + @Test + @DisplayName("Load file header, if create is set but channel has size > 0") + public void testInitFileHeaderCreateAndSizeGreater0() throws IOException { + Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(options.createNew()).thenReturn(false); + Mockito.when(options.create()).thenReturn(true); + Mockito.when(cipherFileChannel.size()).thenReturn(42L); + + inTest.initFileHeader(options, cipherFileChannel); + + Mockito.verify(headerHolder, times(1)).loadExisting(cipherFileChannel); + Mockito.verify(headerHolder, never()).createNew(); + } + } + @Nested @TestInstance(TestInstance.Lifecycle.PER_CLASS) @TestMethodOrder(MethodOrderer.OrderAnnotation.class) From c1b02cfd60569229627865a2d880ec3cbb175546 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Sun, 2 Apr 2023 13:38:28 +0200 Subject: [PATCH 11/32] restrict visiblity of file header load/creation methods --- .../java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java index 8dedcfe1..45fcfcc0 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java @@ -34,7 +34,7 @@ public FileHeader get() { return header.get(); } - public FileHeader createNew() { + FileHeader createNew() { LOG.trace("Generating file header for {}", path.get()); FileHeader newHeader = cryptor.fileHeaderCryptor().create(); header.set(newHeader); @@ -49,7 +49,7 @@ public FileHeader createNew() { * @return {@link FileHeader} of the encrypted file * @throws IOException if the file header cannot be read or decrypted */ - public FileHeader loadExisting(FileChannel ch) throws IOException { + FileHeader loadExisting(FileChannel ch) throws IOException { LOG.trace("Reading file header from {}", path.get()); ByteBuffer existingHeaderBuf = ByteBuffer.allocate(cryptor.fileHeaderCryptor().headerSize()); ch.read(existingHeaderBuf, 0); From a9b6f130463ae797107e7c29ba9c78d038a5c21b Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Mon, 3 Apr 2023 15:37:19 +0200 Subject: [PATCH 12/32] apply suggestions from code review --- .../cryptofs/ch/CleartextFileChannel.java | 17 ++++++----------- .../cryptofs/fh/FileHeaderHolder.java | 6 +++++- .../cryptomator/cryptofs/fh/OpenCryptoFile.java | 3 ++- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index 70d1a7f9..818dacf8 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -11,7 +11,6 @@ import org.cryptomator.cryptofs.fh.OpenFileModifiedDate; import org.cryptomator.cryptofs.fh.OpenFileSize; import org.cryptomator.cryptolib.api.Cryptor; -import org.cryptomator.cryptolib.api.FileHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -181,17 +180,13 @@ private long writeLockedInternal(ByteSource src, long position) throws IOExcepti } private void writeHeaderIfNeeded() throws IOException { - if (!fileHeaderHolder.headerIsPersisted().getAndSet(true)) { - LOG.trace("{} - Writing file header.", this); - ByteBuffer encryptedHeader = cryptor.fileHeaderCryptor().encryptHeader(fileHeaderHolder.get()); - //TODO: what if write fails? Should the encrypted header be cached? Because it cannot be encrypted again (NONCE reuse!) - try { - ciphertextFileChannel.write(encryptedHeader, 0); - } catch (IOException e) { - fileHeaderHolder.headerIsPersisted().set(false); - throw e; - } + if (fileHeaderHolder.headerIsPersisted().get()) { + return; } + LOG.trace("{} - Writing file header.", this); + ByteBuffer encryptedHeader = cryptor.fileHeaderCryptor().encryptHeader(fileHeaderHolder.get()); + ciphertextFileChannel.write(encryptedHeader, 0); + fileHeaderHolder.headerIsPersisted().set(true); } @Override diff --git a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java index 45fcfcc0..5e2bb712 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java @@ -31,7 +31,11 @@ public FileHeaderHolder(Cryptor cryptor, @CurrentOpenFilePath AtomicReference Date: Mon, 3 Apr 2023 15:53:14 +0200 Subject: [PATCH 13/32] react to ISE appropriately when init file header --- .../java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java | 4 +++- .../org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java | 8 ++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java index a2e1dff7..b936234f 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java @@ -98,7 +98,9 @@ public synchronized FileChannel newFileChannel(EffectiveOpenOptions options, Fil //visible for testing void initFileHeader(EffectiveOpenOptions options, FileChannel ciphertextFileChannel) throws IOException { - if (headerHolder.get() == null) { + try { + headerHolder.get(); + } catch (IllegalStateException e) { //first file channel to file if (options.createNew() || (options.create() && ciphertextFileChannel.size() == 0)) { //file did not exist, create new header diff --git a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java index da615fc4..1d686151 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java @@ -116,7 +116,7 @@ public void testInitFileHeaderExisting() throws IOException { @Test @DisplayName("Load file header from file, if not present and neither create nor create_new set") public void testInitFileHeaderLoad() throws IOException { - Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(headerHolder.get()).thenThrow(new IllegalStateException("no Header set")); Mockito.when(options.createNew()).thenReturn(false); Mockito.when(options.create()).thenReturn(false); @@ -129,7 +129,7 @@ public void testInitFileHeaderLoad() throws IOException { @Test @DisplayName("Create new file header, if not present and create_new set") public void testInitFileHeaderCreateNew() throws IOException { - Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(headerHolder.get()).thenThrow(new IllegalStateException("no Header set")); Mockito.when(options.createNew()).thenReturn(true); inTest.initFileHeader(options, cipherFileChannel); @@ -141,7 +141,7 @@ public void testInitFileHeaderCreateNew() throws IOException { @Test @DisplayName("Create new file header, if not present, create set and channel.size() == 0") public void testInitFileHeaderCreateAndSize0() throws IOException { - Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(headerHolder.get()).thenThrow(new IllegalStateException("no Header set")); Mockito.when(options.createNew()).thenReturn(false); Mockito.when(options.create()).thenReturn(true); Mockito.when(cipherFileChannel.size()).thenReturn(0L); @@ -155,7 +155,7 @@ public void testInitFileHeaderCreateAndSize0() throws IOException { @Test @DisplayName("Load file header, if create is set but channel has size > 0") public void testInitFileHeaderCreateAndSizeGreater0() throws IOException { - Mockito.when(headerHolder.get()).thenReturn(null); + Mockito.when(headerHolder.get()).thenThrow(new IllegalStateException("no Header set")); Mockito.when(options.createNew()).thenReturn(false); Mockito.when(options.create()).thenReturn(true); Mockito.when(cipherFileChannel.size()).thenReturn(42L); From 48a6b889f75c7578b5778b144ee2d0b3cfda8329 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Tue, 4 Apr 2023 13:22:01 +0200 Subject: [PATCH 14/32] caffeine cache uses a probalisitc eviction approach --- .../org/cryptomator/cryptofs/fh/ChunkCacheTest.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java index d4cf0615..e2165300 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java @@ -10,6 +10,7 @@ import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import java.io.IOException; @@ -156,8 +157,8 @@ public void testClosingActiveChunkThatIsReferencedTwice() throws IOException, Au verifyNoMoreInteractions(bufferPool); } - @RepeatedTest(30) - @DisplayName("chunk.close() triggers eviction of LRU stale chunk") + @RepeatedTest(100) + @DisplayName("chunk.close() triggers eviction of stale chunk") public void testClosingActiveChunkTriggersEvictionOfStaleChunk() throws IOException, AuthenticationFailedException { var cdl = new CountDownLatch(1); Mockito.doAnswer(invocation -> { @@ -170,8 +171,11 @@ public void testClosingActiveChunkTriggersEvictionOfStaleChunk() throws IOExcept Assertions.assertTimeoutPreemptively(Duration.ofMillis(100), () -> { cdl.await(); }); - verify(chunkSaver).save(42L, staleChunk42); - verify(bufferPool).recycle(staleChunk42.data()); + + ArgumentCaptor chunkCaptor = ArgumentCaptor.forClass(Chunk.class); + ArgumentCaptor indexCaptor = ArgumentCaptor.forClass(Long.class); + verify(chunkSaver).save(indexCaptor.capture(), chunkCaptor.capture()); + verify(bufferPool).recycle(chunkCaptor.getValue().data()); verifyNoMoreInteractions(chunkSaver); } From 8e644e1ae339847a39451cbfd785c63f8329a638 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Wed, 5 Apr 2023 10:36:01 +0200 Subject: [PATCH 15/32] only let one thread at a time write the file header --- .../java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index 818dacf8..fabe2fca 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -179,7 +179,7 @@ private long writeLockedInternal(ByteSource src, long position) throws IOExcepti return written; } - private void writeHeaderIfNeeded() throws IOException { + private synchronized void writeHeaderIfNeeded() throws IOException { if (fileHeaderHolder.headerIsPersisted().get()) { return; } From a91267b93847a8e9c9663b253843b536e2a9fc20 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Wed, 5 Apr 2023 11:05:01 +0200 Subject: [PATCH 16/32] directly encrypt new headers to prevent NONCE reuse --- .../cryptomator/cryptofs/ch/CleartextFileChannel.java | 3 +-- .../org/cryptomator/cryptofs/fh/FileHeaderHolder.java | 11 +++++++++++ .../cryptofs/ch/CleartextFileChannelTest.java | 9 +++++---- .../cryptomator/cryptofs/fh/FileHeaderHolderTest.java | 7 +++++++ 4 files changed, 24 insertions(+), 6 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index fabe2fca..3ada2786 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -184,8 +184,7 @@ private synchronized void writeHeaderIfNeeded() throws IOException { return; } LOG.trace("{} - Writing file header.", this); - ByteBuffer encryptedHeader = cryptor.fileHeaderCryptor().encryptHeader(fileHeaderHolder.get()); - ciphertextFileChannel.write(encryptedHeader, 0); + ciphertextFileChannel.write(fileHeaderHolder.getEncrypted(), 0); fileHeaderHolder.headerIsPersisted().set(true); } diff --git a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java index 5e2bb712..822b8740 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/FileHeaderHolder.java @@ -22,6 +22,7 @@ public class FileHeaderHolder { private final Cryptor cryptor; private final AtomicReference path; private final AtomicReference header = new AtomicReference<>(); + private final AtomicReference encryptedHeader = new AtomicReference<>(); private final AtomicBoolean isPersisted = new AtomicBoolean(); @Inject @@ -38,9 +39,18 @@ public FileHeader get() { return result; } + public ByteBuffer getEncrypted() { + var result = encryptedHeader.get(); + if (result == null) { + throw new IllegalStateException("Header not set."); + } + return result; + } + FileHeader createNew() { LOG.trace("Generating file header for {}", path.get()); FileHeader newHeader = cryptor.fileHeaderCryptor().create(); + encryptedHeader.set(cryptor.fileHeaderCryptor().encryptHeader(newHeader).asReadOnlyBuffer()); //to prevent NONCE reuse, we already encrypt the header and cache it header.set(newHeader); return newHeader; } @@ -60,6 +70,7 @@ FileHeader loadExisting(FileChannel ch) throws IOException { existingHeaderBuf.flip(); try { FileHeader existingHeader = cryptor.fileHeaderCryptor().decryptHeader(existingHeaderBuf); + encryptedHeader.set(existingHeaderBuf.flip().asReadOnlyBuffer()); //for consistency header.set(existingHeader); isPersisted.set(true); return existingHeader; diff --git a/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java b/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java index 5e067420..9009ae48 100644 --- a/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java +++ b/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java @@ -489,7 +489,7 @@ public void writeAfterEof() throws IOException { public void testWriteHeaderIfNeeded() throws IOException { when(options.writable()).thenReturn(true); - when(headerIsPersisted.getAndSet(anyBoolean())).thenReturn(false).thenReturn(true).thenReturn(true); + when(headerIsPersisted.get()).thenReturn(false).thenReturn(true).thenReturn(true); inTest.force(true); inTest.force(true); @@ -499,23 +499,24 @@ public void testWriteHeaderIfNeeded() throws IOException { } @Test - @DisplayName("If writing header fails, it is indicated as not persistet") + @DisplayName("If writing header fails, it is indicated as not persistent") public void testWriteHeaderFailsResetsPersistenceState() throws IOException { when(options.writable()).thenReturn(true); - when(headerIsPersisted.getAndSet(anyBoolean())).thenReturn(false); + when(headerIsPersisted.get()).thenReturn(false); doNothing().when(headerIsPersisted).set(anyBoolean()); when(ciphertextFileChannel.write(any(), anyLong())).thenThrow(new IOException("writing failed")); Assertions.assertThrows(IOException.class, () -> inTest.force(true)); Mockito.verify(ciphertextFileChannel, Mockito.times(1)).write(Mockito.any(), Mockito.eq(0l)); - Mockito.verify(headerIsPersisted, Mockito.times(1)).set(false); + Mockito.verify(headerIsPersisted, Mockito.never()).set(anyBoolean()); } @Test @DisplayName("don't write header if it is already written") public void testDontRewriteHeader() throws IOException { when(options.writable()).thenReturn(true); + when(headerIsPersisted.get()).thenReturn(true); inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); inTest.force(true); diff --git a/src/test/java/org/cryptomator/cryptofs/fh/FileHeaderHolderTest.java b/src/test/java/org/cryptomator/cryptofs/fh/FileHeaderHolderTest.java index bad4f2cd..ed93fac4 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/FileHeaderHolderTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/FileHeaderHolderTest.java @@ -42,6 +42,7 @@ public class FileHeaderHolderTest { @BeforeEach public void setup() throws IOException { when(cryptor.fileHeaderCryptor()).thenReturn(fileHeaderCryptor); + when(fileHeaderCryptor.encryptHeader(Mockito.any())).thenReturn(ByteBuffer.wrap(new byte[0])); } @Nested @@ -75,6 +76,9 @@ public void testLoadExisting() throws IOException, AuthenticationFailedException Assertions.assertSame(headerToLoad, loadedHeader3); verify(fileHeaderCryptor, times(1)).decryptHeader(Mockito.any()); + Assertions.assertNotNull(inTest.get()); + Assertions.assertNotNull(inTest.getEncrypted()); + Assertions.assertTrue(inTest.headerIsPersisted().get()); } } @@ -106,6 +110,9 @@ public void testCreateNew() { Assertions.assertSame(headerToCreate, createdHeader3); verify(fileHeaderCryptor, times(1)).create(); + Assertions.assertNotNull(inTest.get()); + Assertions.assertNotNull(inTest.getEncrypted()); + Assertions.assertFalse(inTest.headerIsPersisted().get()); } } From 2b2330d63b1a1a72d37339e2b51784b8fe7d12c9 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Thu, 6 Apr 2023 11:01:01 +0200 Subject: [PATCH 17/32] remove synchronisation when writing file header --- .../java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index 3ada2786..1326d464 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -179,7 +179,7 @@ private long writeLockedInternal(ByteSource src, long position) throws IOExcepti return written; } - private synchronized void writeHeaderIfNeeded() throws IOException { + private void writeHeaderIfNeeded() throws IOException { if (fileHeaderHolder.headerIsPersisted().get()) { return; } From ab4a809297b5a969a48bef62aa8c6d148aa44c6e Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Thu, 6 Apr 2023 11:01:50 +0200 Subject: [PATCH 18/32] adjust superseded test --- .../CryptoFileChannelWriteReadIntegrationTest.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java index 93216d8b..bc1e1ea5 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java @@ -8,7 +8,6 @@ *******************************************************************************/ package org.cryptomator.cryptofs; -import com.google.common.jimfs.Configuration; import com.google.common.jimfs.Jimfs; import org.cryptomator.cryptofs.util.ByteBuffers; import org.cryptomator.cryptolib.api.Masterkey; @@ -251,17 +250,17 @@ public void testWriteFromSecondChannelWhileStillOpen() throws IOException { } } - // tests https://github.com/cryptomator/cryptofs/issues/160 + //tests changes made in https://github.com/cryptomator/cryptofs/pull/166 @Test - @DisplayName("TRUNCATE_EXISTING leads to new file header") + @DisplayName("TRUNCATE_EXISTING does not produce invalid ciphertext") public void testNewFileHeaderWhenTruncateExisting() throws IOException { try (var ch1 = FileChannel.open(file, CREATE_NEW, WRITE)) { ch1.write(StandardCharsets.UTF_8.encode("this content will be truncated soon"), 0); ch1.force(true); - try (var ch2 = FileChannel.open(file, CREATE, WRITE, TRUNCATE_EXISTING)) { // re-roll file header + try (var ch2 = FileChannel.open(file, CREATE, WRITE, TRUNCATE_EXISTING)) { ch2.write(StandardCharsets.UTF_8.encode("hello"), 0); } - ch1.write(StandardCharsets.UTF_8.encode(" world"), 5); // should use new file key + ch1.write(StandardCharsets.UTF_8.encode(" world"), 5); } try (var ch3 = FileChannel.open(file, READ)) { From f79f84728e4c4f857895ef05d165a301d17c1aa7 Mon Sep 17 00:00:00 2001 From: Sebastian Stenzel Date: Wed, 12 Apr 2023 09:29:56 +0200 Subject: [PATCH 19/32] re-add ReadWriteLock --- .../cryptomator/cryptofs/fh/ChunkCache.java | 99 +++++++++++++------ 1 file changed, 70 insertions(+), 29 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java index 17f0599c..24f268c8 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java @@ -13,6 +13,9 @@ import java.nio.ByteBuffer; import java.nio.channels.NonWritableChannelException; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; @OpenFileScoped public class ChunkCache { @@ -27,6 +30,23 @@ public class ChunkCache { private final Cache chunkCache; private final ConcurrentMap cachedChunks; + /** + * We have to deal with two forms of access to the {@link #chunkCache}: + *

    + *
  1. Accessing a single chunk (with a known index), e.g. during {@link #getChunk(long)}
  2. + *
  3. Accessing multiple chunks, e.g. during {@link #invalidateStale()}
  4. + *
+ * + * While the former can be handled by the cache implementation (based on {@link ConcurrentMap}) just fine, + * we need to make sure no concurrent modification will happen accessing multiple chunks (e.g. when iterating over the entry set). + * + * This is achieved using this {@link ReadWriteLock}, where holding the {@link ReadWriteLock#readLock() shared lock} is + * sufficient for index-based access, while the {@link ReadWriteLock#writeLock() exclusive lock} is necessary otherwise. + */ + private final ReadWriteLock lock = new ReentrantReadWriteLock(); + private final Lock sharedLock = lock.readLock(); // required when accessing a single chunk + private final Lock exclusiveLock = lock.writeLock(); // required when accessing multiple chunks at once + @Inject public ChunkCache(ChunkLoader chunkLoader, ChunkSaver chunkSaver, CryptoFileSystemStats stats, BufferPool bufferPool, ExceptionsDuringWrite exceptionsDuringWrite) { this.chunkLoader = chunkLoader; @@ -59,18 +79,23 @@ private int weigh(Long index, Chunk chunk) { * @throws IllegalArgumentException If {@code chunkData}'s remaining bytes is not equal to the number of bytes fitting into a chunk */ public Chunk putChunk(long chunkIndex, ByteBuffer chunkData) throws IllegalArgumentException { - return cachedChunks.compute(chunkIndex, (index, chunk) -> { - if (chunk == null) { - chunk = new Chunk(chunkData, true, () -> releaseChunk(chunkIndex)); - } else { - var dst = chunk.data().duplicate().clear(); - Preconditions.checkArgument(chunkData.remaining() == dst.remaining()); - dst.put(chunkData); - chunk.dirty().set(true); - } - chunk.currentAccesses().incrementAndGet(); - return chunk; - }); + sharedLock.lock(); + try { + return cachedChunks.compute(chunkIndex, (index, chunk) -> { + if (chunk == null) { + chunk = new Chunk(chunkData, true, () -> releaseChunk(chunkIndex)); + } else { + var dst = chunk.data().duplicate().clear(); + Preconditions.checkArgument(chunkData.remaining() == dst.remaining()); + dst.put(chunkData); + chunk.dirty().set(true); + } + chunk.currentAccesses().incrementAndGet(); + return chunk; + }); + } finally { + sharedLock.unlock(); + } } /** @@ -81,17 +106,22 @@ public Chunk putChunk(long chunkIndex, ByteBuffer chunkData) throws IllegalArgum * @throws IOException If reading or decrypting the chunk failed */ public Chunk getChunk(long chunkIndex) throws IOException { - stats.addChunkCacheAccess(); + sharedLock.lock(); try { - return cachedChunks.compute(chunkIndex, (idx, chunk) -> { - if (chunk == null) { - chunk = loadChunk(idx); - } - chunk.currentAccesses().incrementAndGet(); - return chunk; - }); - } catch (UncheckedIOException | AuthenticationFailedException e) { - throw new IOException(e); + stats.addChunkCacheAccess(); + try { + return cachedChunks.compute(chunkIndex, (idx, chunk) -> { + if (chunk == null) { + chunk = loadChunk(idx); + } + chunk.currentAccesses().incrementAndGet(); + return chunk; + }); + } catch (UncheckedIOException | AuthenticationFailedException e) { + throw new IOException(e); + } + } finally { + sharedLock.unlock(); } } @@ -106,10 +136,15 @@ private Chunk loadChunk(long chunkIndex) throws AuthenticationFailedException, U @SuppressWarnings("resource") private void releaseChunk(long chunkIndex) { - cachedChunks.computeIfPresent(chunkIndex, (idx, chunk) -> { - chunk.currentAccesses().decrementAndGet(); - return chunk; - }); + sharedLock.lock(); + try { + cachedChunks.computeIfPresent(chunkIndex, (idx, chunk) -> { + chunk.currentAccesses().decrementAndGet(); + return chunk; + }); + } finally { + sharedLock.unlock(); + } } /** @@ -117,6 +152,7 @@ private void releaseChunk(long chunkIndex) { * @see #invalidateStale() */ public void flush() throws IOException { + exclusiveLock.lock(); try { cachedChunks.forEach((index, chunk) -> { try { @@ -127,6 +163,8 @@ public void flush() throws IOException { }); } catch (UncheckedIOException e) { throw new IOException(e); + } finally { + exclusiveLock.unlock(); } } @@ -134,9 +172,12 @@ public void flush() throws IOException { * Removes stale chunks from cache. */ public void invalidateStale() { - // This may not be atomic, however this method is only called during truncation. - // If chunks are added (and become stale) concurrently, behavior is undefined anyway - cachedChunks.entrySet().removeIf(entry -> entry.getValue().currentAccesses().get() == 0); + exclusiveLock.lock(); + try { + cachedChunks.entrySet().removeIf(entry -> entry.getValue().currentAccesses().get() == 0); + } finally { + exclusiveLock.unlock(); + } } // visible for testing From 6b6f3f8ee43e21520ac5caaca9ddc83557124c4c Mon Sep 17 00:00:00 2001 From: Sebastian Stenzel Date: Wed, 12 Apr 2023 09:30:27 +0200 Subject: [PATCH 20/32] JavaDoc formatting [ci skip] --- src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java index 24f268c8..e24b7a22 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java @@ -36,10 +36,10 @@ public class ChunkCache { *
  • Accessing a single chunk (with a known index), e.g. during {@link #getChunk(long)}
  • *
  • Accessing multiple chunks, e.g. during {@link #invalidateStale()}
  • * - * + *

    * While the former can be handled by the cache implementation (based on {@link ConcurrentMap}) just fine, * we need to make sure no concurrent modification will happen accessing multiple chunks (e.g. when iterating over the entry set). - * + *

    * This is achieved using this {@link ReadWriteLock}, where holding the {@link ReadWriteLock#readLock() shared lock} is * sufficient for index-based access, while the {@link ReadWriteLock#writeLock() exclusive lock} is necessary otherwise. */ @@ -149,6 +149,7 @@ private void releaseChunk(long chunkIndex) { /** * Flushes cached data (but keeps them cached). + * * @see #invalidateStale() */ public void flush() throws IOException { From 7bbd77d61ef72432b8d733bd3094ee8ebcdf7d12 Mon Sep 17 00:00:00 2001 From: Sebastian Stenzel Date: Wed, 12 Apr 2023 15:12:20 +0200 Subject: [PATCH 21/32] run eviction on same thread --- src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java | 6 +----- .../java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java | 1 - 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java index e24b7a22..9e8e2a20 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/ChunkCache.java @@ -57,6 +57,7 @@ public ChunkCache(ChunkLoader chunkLoader, ChunkSaver chunkSaver, CryptoFileSyst this.chunkCache = Caffeine.newBuilder() // .maximumWeight(MAX_CACHED_CLEARTEXT_CHUNKS) // .weigher(this::weigh) // + .executor(Runnable::run) // run `evictStaleChunk` in same thread -> see https://github.com/cryptomator/cryptofs/pull/163#issuecomment-1505249736 .evictionListener(this::evictStaleChunk) // .build(); this.cachedChunks = chunkCache.asMap(); @@ -181,11 +182,6 @@ public void invalidateStale() { } } - // visible for testing - void cleanup() { - chunkCache.cleanUp(); - } - // visible for testing void evictStaleChunk(Long index, Chunk chunk, RemovalCause removalCause) { assert removalCause != RemovalCause.EXPLICIT; // as per spec of Caffeine#evictionListener(RemovalListener) diff --git a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java index 7fc3b165..6f39f9f4 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/ChunkCacheTest.java @@ -159,7 +159,6 @@ public void testClosingActiveChunkThatIsReferencedTwice() throws IOException, Au public void testClosingActiveChunkTriggersEvictionOfStaleChunk() throws IOException, AuthenticationFailedException { activeChunk1.close(); - inTest.cleanup(); // evict now, don't wait for async task // we can't know _which_ stale chunk gets evicted. see https://github.com/ben-manes/caffeine/issues/583 ArgumentCaptor chunkCaptor = ArgumentCaptor.forClass(Chunk.class); ArgumentCaptor indexCaptor = ArgumentCaptor.forClass(Long.class); From 279d5670be533447c29928b880cd593d7b011bff Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Wed, 12 Apr 2023 15:52:21 +0200 Subject: [PATCH 22/32] fix wrong variable shadowing --- .../java/org/cryptomator/cryptofs/fh/ExceptionsDuringWrite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/ExceptionsDuringWrite.java b/src/main/java/org/cryptomator/cryptofs/fh/ExceptionsDuringWrite.java index b0cf0931..0f24688b 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/ExceptionsDuringWrite.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/ExceptionsDuringWrite.java @@ -19,7 +19,7 @@ public ExceptionsDuringWrite() { } public synchronized void add(Exception e) { - e.addSuppressed(e); + this.e.addSuppressed(e); } public synchronized void throwIfPresent() throws IOException { From d4ce926f5c07bd2020e89def8b7599d76134cbb0 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Wed, 12 Apr 2023 15:53:55 +0200 Subject: [PATCH 23/32] fix filesize not zero'd on TRUNCATE_EXISTING --- .../cryptofs/fh/OpenCryptoFile.java | 1 + .../cryptofs/fh/OpenCryptoFileTest.java | 22 ++++++++++++++++--- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java index 378c5e08..71f1afb5 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java @@ -74,6 +74,7 @@ public synchronized FileChannel newFileChannel(EffectiveOpenOptions options, Fil if (options.truncateExisting()) { chunkCache.invalidateStale(); ciphertextFileChannel.truncate(cryptor.fileHeaderCryptor().headerSize()); + fileSize.set(0); } initFileSize(ciphertextFileChannel); cleartextFileChannel = component.newChannelComponent() // diff --git a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java index 3b0d26f2..fc7cc326 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java @@ -14,7 +14,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.MethodOrderer; import org.junit.jupiter.api.Nested; @@ -54,7 +53,7 @@ public class OpenCryptoFileTest { private FileHeaderCryptor fileHeaderCryptor = mock(FileHeaderCryptor.class); private FileHeaderHolder headerHolder = mock(FileHeaderHolder.class); private ChunkIO chunkIO = mock(ChunkIO.class); - private AtomicLong fileSize = new AtomicLong(-1l); + private AtomicLong fileSize = Mockito.mock(AtomicLong.class); private AtomicReference lastModified = new AtomicReference(Instant.ofEpochMilli(0)); private OpenCryptoFileComponent openCryptoFileComponent = mock(OpenCryptoFileComponent.class); private ChannelComponent.Factory channelComponentFactory = mock(ChannelComponent.Factory.class); @@ -93,6 +92,22 @@ public void testCloseImmediatelyIfOpeningFirstChannelFails() { verify(closeListener).close(CURRENT_FILE_PATH.get(), openCryptoFile); } + @Test + @DisplayName("Opening a file channel with TRUNCATE_EXISTING sets the file size to 0") + public void testFileSizeZerodOnTruncateExisting() throws IOException { + EffectiveOpenOptions options = EffectiveOpenOptions.from(EnumSet.of(StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING), readonlyFlag); + Mockito.when(headerHolder.get()).thenReturn(Mockito.mock(FileHeader.class)); + Mockito.when(cryptor.fileHeaderCryptor()).thenReturn(fileHeaderCryptor); + Mockito.when(fileHeaderCryptor.headerSize()).thenReturn(42); + Mockito.when(openCryptoFileComponent.newChannelComponent()).thenReturn(channelComponentFactory); + Mockito.when(channelComponentFactory.create(any(), any(), any())).thenReturn(channelComponent); + Mockito.when(channelComponent.channel()).thenReturn(mock(CleartextFileChannel.class)); + OpenCryptoFile openCryptoFile = new OpenCryptoFile(closeListener, chunkCache, cryptor, headerHolder, chunkIO, CURRENT_FILE_PATH, fileSize, lastModified, openCryptoFileComponent); + + openCryptoFile.newFileChannel(options); + verify(fileSize).set(0L); + } + @Nested @DisplayName("Testing ::initFileHeader") public class InitFilHeaderTests { @@ -173,6 +188,7 @@ public void testInitFileHeaderCreateAndSizeGreater0() throws IOException { @DisplayName("FileChannels") public class FileChannelFactoryTest { + private final AtomicLong realFileSize = new AtomicLong(-1L); private OpenCryptoFile openCryptoFile; private CleartextFileChannel cleartextFileChannel; private AtomicReference listener; @@ -180,7 +196,7 @@ public class FileChannelFactoryTest { @BeforeAll public void setup() throws IOException { - openCryptoFile = new OpenCryptoFile(closeListener, chunkCache, cryptor, headerHolder, chunkIO, CURRENT_FILE_PATH, fileSize, lastModified, openCryptoFileComponent); + openCryptoFile = new OpenCryptoFile(closeListener, chunkCache, cryptor, headerHolder, chunkIO, CURRENT_FILE_PATH, realFileSize, lastModified, openCryptoFileComponent); cleartextFileChannel = mock(CleartextFileChannel.class); listener = new AtomicReference<>(); ciphertextChannel = new AtomicReference<>(); From a5ea170059fb1a409fbdb85e113ee4891da23625 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Wed, 12 Apr 2023 16:01:11 +0200 Subject: [PATCH 24/32] fix unit tests --- .../java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java index fc7cc326..3de6a2f9 100644 --- a/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java +++ b/src/test/java/org/cryptomator/cryptofs/fh/OpenCryptoFileTest.java @@ -196,6 +196,8 @@ public class FileChannelFactoryTest { @BeforeAll public void setup() throws IOException { + FS = Jimfs.newFileSystem("OpenCryptoFileTest.FileChannelFactoryTest", Configuration.unix().toBuilder().setAttributeViews("basic", "posix").build()); + CURRENT_FILE_PATH = new AtomicReference<>(FS.getPath("currentFile")); openCryptoFile = new OpenCryptoFile(closeListener, chunkCache, cryptor, headerHolder, chunkIO, CURRENT_FILE_PATH, realFileSize, lastModified, openCryptoFileComponent); cleartextFileChannel = mock(CleartextFileChannel.class); listener = new AtomicReference<>(); From 3297356a3da735731ab2b4856044c56c888c71f0 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Thu, 13 Apr 2023 12:10:33 +0200 Subject: [PATCH 25/32] closes #169 --- .../cryptomator/cryptofs/ch/CleartextFileChannel.java | 8 +++++++- .../CryptoFileChannelWriteReadIntegrationTest.java | 10 ++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index b06aa970..c3400f81 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -227,6 +227,7 @@ private void forceInternal(boolean metaData) throws IOException { /** * Writes in-memory contents to the ciphertext file + * * @throws IOException */ private void flush() throws IOException { @@ -313,7 +314,12 @@ long beginOfChunk(long cleartextPos) { protected void implCloseChannel() throws IOException { try { flush(); - persistLastModified(); + try { + persistLastModified(); + } catch (IOException e) { + //no-op, see https://github.com/cryptomator/cryptofs/issues/169 + LOG.warn("Failed to persist last modified timestamp for encrypted file: {}", e.getMessage()); + } } finally { super.implCloseChannel(); closeListener.closed(this); diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java index bc1e1ea5..161a908f 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java @@ -546,6 +546,16 @@ public void testConcurrentRead() throws IOException, InterruptedException { })); } + @Test + public void testClosingChannelOfDeletedFileDoesNotThrow() { + Assertions.assertDoesNotThrow(() -> { + try (var ch = FileChannel.open(file, CREATE_NEW, WRITE)) { + ch.write(ByteBuffer.wrap("delete me".getBytes(StandardCharsets.UTF_8))); + Files.delete(file); + } + }); + Assertions.assertTrue(Files.notExists(file)); + } } } From 694f6e0eda03feb71b0e751bddb76b123cde939a Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Fri, 14 Apr 2023 11:16:00 +0200 Subject: [PATCH 26/32] Fixes #170 --- .../cryptofs/CryptoFileSystemImpl.java | 7 ++++++- .../cryptomator/cryptofs/fh/OpenCryptoFile.java | 16 +++++++++++++--- .../cryptomator/cryptofs/fh/OpenCryptoFiles.java | 16 +++++++++++++++- ...ryptoFileChannelWriteReadIntegrationTest.java | 16 ++++++++++++++++ .../cryptofs/CryptoFileSystemImplTest.java | 5 +++++ 5 files changed, 55 insertions(+), 5 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java b/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java index 887bb189..4f8397ce 100644 --- a/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java +++ b/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java @@ -421,10 +421,15 @@ void delete(CryptoPath cleartextPath) throws IOException { CiphertextFilePath ciphertextPath = cryptoPathMapper.getCiphertextFilePath(cleartextPath); switch (ciphertextFileType) { case DIRECTORY -> deleteDirectory(cleartextPath, ciphertextPath); - case FILE, SYMLINK -> Files.walkFileTree(ciphertextPath.getRawPath(), DeletingFileVisitor.INSTANCE); + case FILE, SYMLINK -> deleteFileOrSymlink(ciphertextPath); } } + private void deleteFileOrSymlink(CiphertextFilePath ciphertextPath) throws IOException { + openCryptoFiles.delete(ciphertextPath.getFilePath()); + Files.walkFileTree(ciphertextPath.getRawPath(), DeletingFileVisitor.INSTANCE); + } + private void deleteDirectory(CryptoPath cleartextPath, CiphertextFilePath ciphertextPath) throws IOException { Path ciphertextDir = cryptoPathMapper.getCiphertextDir(cleartextPath).path; Path ciphertextDirFile = ciphertextPath.getDirFilePath(); diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java index 71f1afb5..a07b5db2 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFile.java @@ -66,6 +66,9 @@ public OpenCryptoFile(FileCloseListener listener, ChunkCache chunkCache, Cryptor */ public synchronized FileChannel newFileChannel(EffectiveOpenOptions options, FileAttribute... attrs) throws IOException { Path path = currentFilePath.get(); + if (path == null) { + throw new IllegalStateException("Cannot create file channel to deleted file"); + } FileChannel ciphertextFileChannel = null; CleartextFileChannel cleartextFileChannel = null; try { @@ -172,8 +175,12 @@ public Path getCurrentFilePath() { return currentFilePath.get(); } - public void setCurrentFilePath(Path currentFilePath) { - this.currentFilePath.set(currentFilePath); + /** + * Updates the current ciphertext file path, if it is not already set to null (i.e., the openCryptoFile is deleted) + * @param newFilePath new ciphertext path + */ + public void updateCurrentFilePath(Path newFilePath) { + currentFilePath.updateAndGet(p -> p == null ? null : newFilePath); } private synchronized void channelClosed(CleartextFileChannel cleartextFileChannel) throws IOException { @@ -192,7 +199,10 @@ private synchronized void channelClosed(CleartextFileChannel cleartextFileChanne @Override public void close() { - listener.close(currentFilePath.get(), this); + var p = currentFilePath.get(); + if(p != null) { + listener.close(p, this); + } } @Override diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFiles.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFiles.java index 4186e248..4e872250 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFiles.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFiles.java @@ -81,6 +81,20 @@ public ByteBuffer readCiphertextFile(Path ciphertextPath, EffectiveOpenOptions o } } + /** + * Removes a ciphertextPath to {@link OpenCryptoFile} mapping, if it exists, and sets the path of the openCryptoFile to null. + * + * @param ciphertextPath The ciphertext file path to invalidate + */ + public void delete(Path ciphertextPath) { + openCryptoFiles.compute(ciphertextPath, (p, openFile) -> { + if (openFile != null) { + openFile.updateCurrentFilePath(null); + } + return null; + }); + } + /** * Prepares to update any open file references during a move operation. * MUST be invoked using a try-with-resource statement and committed after the physical file move succeeded. @@ -137,7 +151,7 @@ public void commit() { throw new IllegalStateException(); } if (openCryptoFile != null) { - openCryptoFile.setCurrentFilePath(dst); + openCryptoFile.updateCurrentFilePath(dst); } openCryptoFiles.remove(src, openCryptoFile); committed = true; diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java index 161a908f..5e34c9f6 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java @@ -48,6 +48,7 @@ import java.util.Arrays; import java.util.List; import java.util.Random; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; @@ -556,6 +557,21 @@ public void testClosingChannelOfDeletedFileDoesNotThrow() { }); Assertions.assertTrue(Files.notExists(file)); } + + //https://github.com/cryptomator/cryptofs/issues/170 + @Test + public void testWriteThenDeleteThenRead() throws IOException { + var bufToWrite = ByteBuffer.wrap("delete me".getBytes(StandardCharsets.UTF_8)); + final int bytesRead; + try (var ch = FileChannel.open(file, CREATE_NEW, WRITE)) { + ch.write(bufToWrite); + Files.delete(file); + try (var ch2 = fileSystem.provider().newFileChannel(file, Set.of(CREATE, READ, WRITE))) { + bytesRead = ch2.read(ByteBuffer.allocate(bufToWrite.capacity())); + } + } + Assertions.assertEquals(-1, bytesRead); + } } } diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java index a133999f..5fc8aa32 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java @@ -72,6 +72,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -558,6 +559,7 @@ public class Delete { private final CryptoPath cleartextPath = mock(CryptoPath.class, "cleartext"); private final Path ciphertextRawPath = mock(Path.class, "d/00/00/path.c9r"); private final Path ciphertextDirFilePath = mock(Path.class, "d/00/00/path.c9r/dir.c9r"); + private final Path ciphertextFilePath = mock(Path.class, "d/00/00/path.c9r"); private final Path ciphertextDirPath = mock(Path.class, "d/FF/FF/"); private final CiphertextFilePath ciphertextPath = mock(CiphertextFilePath.class, "ciphertext"); private final FileSystem physicalFs = mock(FileSystem.class); @@ -574,6 +576,7 @@ public void setup() throws IOException { when(ciphertextRawPath.resolve("dir.c9r")).thenReturn(ciphertextDirFilePath); when(cryptoPathMapper.getCiphertextFilePath(cleartextPath)).thenReturn(ciphertextPath); when(ciphertextPath.getRawPath()).thenReturn(ciphertextRawPath); + when(ciphertextPath.getFilePath()).thenReturn(ciphertextFilePath); when(ciphertextPath.getDirFilePath()).thenReturn(ciphertextDirFilePath); when(cryptoPathMapper.getCiphertextDir(cleartextPath)).thenReturn(new CiphertextDirectory("foo", ciphertextDirPath)); when(physicalFsProv.readAttributes(ciphertextRawPath, BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS)).thenReturn(ciphertextPathAttr); @@ -590,10 +593,12 @@ public void testDeleteRootFails() { public void testDeleteExistingFile() throws IOException { when(cryptoPathMapper.getCiphertextFileType(cleartextPath)).thenReturn(CiphertextFileType.FILE); when(physicalFsProv.deleteIfExists(ciphertextRawPath)).thenReturn(true); + doNothing().when(openCryptoFiles).delete(Mockito.any()); inTest.delete(cleartextPath); verify(readonlyFlag).assertWritable(); + verify(openCryptoFiles).delete(ciphertextFilePath); verify(physicalFsProv).deleteIfExists(ciphertextRawPath); } From f9f3d007d69f41d751ea7f6fb4cd64e5bc93e803 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Fri, 14 Apr 2023 11:52:49 +0200 Subject: [PATCH 27/32] rework persistLastModified() related to #169 --- .../cryptofs/ch/CleartextFileChannel.java | 22 ++++++++++++++----- .../cryptofs/fh/OpenCryptoFileModule.java | 11 +--------- .../cryptofs/ch/CleartextFileChannelTest.java | 18 ++++++++++----- 3 files changed, 30 insertions(+), 21 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java index c3400f81..97791365 100644 --- a/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java +++ b/src/main/java/org/cryptomator/cryptofs/ch/CleartextFileChannel.java @@ -6,6 +6,7 @@ import org.cryptomator.cryptofs.fh.BufferPool; import org.cryptomator.cryptofs.fh.Chunk; import org.cryptomator.cryptofs.fh.ChunkCache; +import org.cryptomator.cryptofs.fh.CurrentOpenFilePath; import org.cryptomator.cryptofs.fh.ExceptionsDuringWrite; import org.cryptomator.cryptofs.fh.FileHeaderHolder; import org.cryptomator.cryptofs.fh.OpenFileModifiedDate; @@ -22,13 +23,14 @@ import java.nio.channels.FileLock; import java.nio.channels.NonReadableChannelException; import java.nio.channels.NonWritableChannelException; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributeView; import java.nio.file.attribute.FileTime; import java.time.Instant; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; -import java.util.function.Supplier; import static java.lang.Math.max; import static java.lang.Math.min; @@ -44,15 +46,15 @@ public class CleartextFileChannel extends AbstractFileChannel { private final ChunkCache chunkCache; private final BufferPool bufferPool; private final EffectiveOpenOptions options; + private final AtomicReference currentFilePath; private final AtomicLong fileSize; private final AtomicReference lastModified; - private final Supplier attrViewProvider; private final ExceptionsDuringWrite exceptionsDuringWrite; private final ChannelCloseListener closeListener; private final CryptoFileSystemStats stats; @Inject - public CleartextFileChannel(FileChannel ciphertextFileChannel, FileHeaderHolder fileHeaderHolder, ReadWriteLock readWriteLock, Cryptor cryptor, ChunkCache chunkCache, BufferPool bufferPool, EffectiveOpenOptions options, @OpenFileSize AtomicLong fileSize, @OpenFileModifiedDate AtomicReference lastModified, Supplier attrViewProvider, ExceptionsDuringWrite exceptionsDuringWrite, ChannelCloseListener closeListener, CryptoFileSystemStats stats) { + public CleartextFileChannel(FileChannel ciphertextFileChannel, FileHeaderHolder fileHeaderHolder, ReadWriteLock readWriteLock, Cryptor cryptor, ChunkCache chunkCache, BufferPool bufferPool, EffectiveOpenOptions options, @OpenFileSize AtomicLong fileSize, @OpenFileModifiedDate AtomicReference lastModified, @CurrentOpenFilePath AtomicReference currentPath, ExceptionsDuringWrite exceptionsDuringWrite, ChannelCloseListener closeListener, CryptoFileSystemStats stats) { super(readWriteLock); this.ciphertextFileChannel = ciphertextFileChannel; this.fileHeaderHolder = fileHeaderHolder; @@ -60,9 +62,9 @@ public CleartextFileChannel(FileChannel ciphertextFileChannel, FileHeaderHolder this.chunkCache = chunkCache; this.bufferPool = bufferPool; this.options = options; + this.currentFilePath = currentPath; this.fileSize = fileSize; this.lastModified = lastModified; - this.attrViewProvider = attrViewProvider; this.exceptionsDuringWrite = exceptionsDuringWrite; this.closeListener = closeListener; this.stats = stats; @@ -246,7 +248,13 @@ private void flush() throws IOException { private void persistLastModified() throws IOException { FileTime lastModifiedTime = isWritable() ? FileTime.from(lastModified.get()) : null; FileTime lastAccessTime = FileTime.from(Instant.now()); - attrViewProvider.get().setTimes(lastModifiedTime, lastAccessTime, null); + var p = currentFilePath.get(); + if (p != null) { + p.getFileSystem().provider()// + .getFileAttributeView(p, BasicFileAttributeView.class) + .setTimes(lastModifiedTime, lastAccessTime, null); + } + } @Override @@ -316,8 +324,10 @@ protected void implCloseChannel() throws IOException { flush(); try { persistLastModified(); - } catch (IOException e) { + } catch (NoSuchFileException nsfe) { //no-op, see https://github.com/cryptomator/cryptofs/issues/169 + } catch (IOException e) { + //only best effort attempt LOG.warn("Failed to persist last modified timestamp for encrypted file: {}", e.getMessage()); } } finally { diff --git a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java index 83d587ee..aceb7484 100644 --- a/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java +++ b/src/main/java/org/cryptomator/cryptofs/fh/OpenCryptoFileModule.java @@ -28,20 +28,11 @@ public ReadWriteLock provideReadWriteLock() { @Provides @OpenFileScoped - @CurrentOpenFilePath // TODO: do we still need this? only used in logging. + @CurrentOpenFilePath public AtomicReference provideCurrentPath(@OriginalOpenFilePath Path originalPath) { return new AtomicReference<>(originalPath); } - @Provides - @OpenFileScoped - public Supplier provideBasicFileAttributeViewSupplier(@CurrentOpenFilePath AtomicReference currentPath) { - return () -> { - Path path = currentPath.get(); - return path.getFileSystem().provider().getFileAttributeView(path, BasicFileAttributeView.class); - }; - } - @Provides @OpenFileScoped @OpenFileModifiedDate diff --git a/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java b/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java index 9009ae48..8b07565a 100644 --- a/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java +++ b/src/test/java/org/cryptomator/cryptofs/ch/CleartextFileChannelTest.java @@ -32,8 +32,11 @@ import java.nio.channels.NonWritableChannelException; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; +import java.nio.file.FileSystem; +import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributeView; import java.nio.file.attribute.FileTime; +import java.nio.file.spi.FileSystemProvider; import java.time.Instant; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -66,9 +69,10 @@ public class CleartextFileChannelTest { private FileHeaderHolder headerHolder = mock(FileHeaderHolder.class); private AtomicBoolean headerIsPersisted = mock(AtomicBoolean.class); private EffectiveOpenOptions options = mock(EffectiveOpenOptions.class); + private Path filePath = Mockito.mock(Path.class,"/foo/bar"); + private AtomicReference currentFilePath = new AtomicReference<>(filePath); private AtomicLong fileSize = new AtomicLong(100); private AtomicReference lastModified = new AtomicReference<>(Instant.ofEpochMilli(0)); - private Supplier attributeViewSupplier = mock(Supplier.class); private BasicFileAttributeView attributeView = mock(BasicFileAttributeView.class); private ExceptionsDuringWrite exceptionsDuringWrite = mock(ExceptionsDuringWrite.class); private ChannelCloseListener closeListener = mock(ChannelCloseListener.class); @@ -88,11 +92,15 @@ public void setUp() throws IOException { when(headerIsPersisted.getAndSet(anyBoolean())).thenReturn(true); when(fileContentCryptor.cleartextChunkSize()).thenReturn(100); when(fileContentCryptor.ciphertextChunkSize()).thenReturn(110); - when(attributeViewSupplier.get()).thenReturn(attributeView); + var fs = Mockito.mock(FileSystem.class); + var fsProvider = Mockito.mock(FileSystemProvider.class); + when(filePath.getFileSystem()).thenReturn(fs); + when(fs.provider()).thenReturn(fsProvider); + when(fsProvider.getFileAttributeView(filePath,BasicFileAttributeView.class)).thenReturn(attributeView); when(readWriteLock.readLock()).thenReturn(readLock); when(readWriteLock.writeLock()).thenReturn(writeLock); - inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); + inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, currentFilePath, exceptionsDuringWrite, closeListener, stats); } @Test @@ -345,7 +353,7 @@ public void testReadFromMultipleChunks() throws IOException { fileSize.set(5_000_000_100l); // initial cleartext size will be 5_000_000_100l when(options.readable()).thenReturn(true); - inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); + inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, currentFilePath, exceptionsDuringWrite, closeListener, stats); ByteBuffer buf = ByteBuffer.allocate(10); // A read from frist chunk: @@ -517,7 +525,7 @@ public void testWriteHeaderFailsResetsPersistenceState() throws IOException { public void testDontRewriteHeader() throws IOException { when(options.writable()).thenReturn(true); when(headerIsPersisted.get()).thenReturn(true); - inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, attributeViewSupplier, exceptionsDuringWrite, closeListener, stats); + inTest = new CleartextFileChannel(ciphertextFileChannel, headerHolder, readWriteLock, cryptor, chunkCache, bufferPool, options, fileSize, lastModified, currentFilePath, exceptionsDuringWrite, closeListener, stats); inTest.force(true); From 2a22af17abc35d271992836146964f9939c25d9b Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Fri, 14 Apr 2023 16:42:40 +0200 Subject: [PATCH 28/32] Apply suggestions from code review Co-authored-by: Sebastian Stenzel --- .../cryptofs/CryptoFileChannelWriteReadIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java index 5e34c9f6..aa725db0 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java @@ -561,7 +561,7 @@ public void testClosingChannelOfDeletedFileDoesNotThrow() { //https://github.com/cryptomator/cryptofs/issues/170 @Test public void testWriteThenDeleteThenRead() throws IOException { - var bufToWrite = ByteBuffer.wrap("delete me".getBytes(StandardCharsets.UTF_8)); + var bufToWrite = StandardCharsets.UTF_8.encode("delete me"); final int bytesRead; try (var ch = FileChannel.open(file, CREATE_NEW, WRITE)) { ch.write(bufToWrite); From 3b638246a7990e7d24fab51218d4989b0f9f69f5 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Fri, 14 Apr 2023 17:28:51 +0200 Subject: [PATCH 29/32] add more file channel integration tests --- ...toFileChannelWriteReadIntegrationTest.java | 62 ++++++++++++++++++- 1 file changed, 61 insertions(+), 1 deletion(-) diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java index aa725db0..b9e43e2f 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java @@ -32,6 +32,7 @@ import org.mockito.Mockito; import java.io.IOException; +import java.io.UncheckedIOException; import java.lang.invoke.MethodHandles; import java.net.URI; import java.nio.ByteBuffer; @@ -51,6 +52,8 @@ import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -547,6 +550,33 @@ public void testConcurrentRead() throws IOException, InterruptedException { })); } + //https://github.com/cryptomator/cryptofs/issues/168 + @Test + @DisplayName("Opening two file channels simultaneously retains ciphertext readability") + public void testOpeningTwoChannelsRetainsCiphertextReadability() throws IOException { + var content = StandardCharsets.UTF_8.encode("two channels sitting on the wall"); + AtomicInteger numBytesRead = new AtomicInteger(-1); + ByteBuffer bytesRead = ByteBuffer.allocate(content.limit()); + + try (var ch = FileChannel.open(file, READ, WRITE, CREATE_NEW)) { + System.out.println("Openend channel " + ch); + try (var ch2 = FileChannel.open(file, WRITE)) { + } + ch.write(content, 0); + } + + Assertions.assertDoesNotThrow(() -> { + try (var ch = FileChannel.open(file, READ)) { + int read = ch.read(bytesRead, 0); + numBytesRead.set(read); + } + }); + + Assertions.assertEquals(content.limit(), numBytesRead.get()); + Assertions.assertArrayEquals(content.array(), bytesRead.array()); + } + + //https://github.com/cryptomator/cryptofs/issues/169 @Test public void testClosingChannelOfDeletedFileDoesNotThrow() { Assertions.assertDoesNotThrow(() -> { @@ -572,6 +602,36 @@ public void testWriteThenDeleteThenRead() throws IOException { } Assertions.assertEquals(-1, bytesRead); } - } + @RepeatedTest(50) + public void testConcurrentWriteAndTruncate() throws IOException { + AtomicBoolean keepWriting = new AtomicBoolean(true); + ByteBuffer buf = ByteBuffer.wrap("the quick brown fox jumps over the lazy dog".getBytes(StandardCharsets.UTF_8)); + var executor = Executors.newCachedThreadPool(); + try (FileChannel writingChannel = FileChannel.open(file, WRITE, CREATE)) { + executor.submit(() -> { + while (keepWriting.get()) { + try { + writingChannel.write(buf); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + buf.flip(); + } + }); + try (FileChannel truncatingChannel = FileChannel.open(file, WRITE, TRUNCATE_EXISTING)) { + keepWriting.set(false); + } + executor.shutdown(); + } + + Assertions.assertDoesNotThrow(() -> { + try (FileChannel readingChannel = FileChannel.open(file, READ)) { + var dst = ByteBuffer.allocate(buf.capacity()); + readingChannel.read(dst); + } + }); + } + + } } From de3d3f02762878040f7baa8e0cd8cfb2d5d9e630 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Fri, 14 Apr 2023 17:35:51 +0200 Subject: [PATCH 30/32] reformat code --- .../java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java | 4 ++-- .../org/cryptomator/cryptofs/CryptoFileSystemImplTest.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java b/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java index 4f8397ce..bd10dc34 100644 --- a/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java +++ b/src/main/java/org/cryptomator/cryptofs/CryptoFileSystemImpl.java @@ -140,11 +140,11 @@ public Path getPathToVault() { public Path getCiphertextPath(Path cleartextPath) throws IOException { var p = CryptoPath.castAndAssertAbsolute(cleartextPath); var nodeType = cryptoPathMapper.getCiphertextFileType(p); - if( nodeType == CiphertextFileType.DIRECTORY) { + if (nodeType == CiphertextFileType.DIRECTORY) { return cryptoPathMapper.getCiphertextDir(p).path; } var cipherFile = cryptoPathMapper.getCiphertextFilePath(p); - if( nodeType == CiphertextFileType.SYMLINK) { + if (nodeType == CiphertextFileType.SYMLINK) { return cipherFile.getSymlinkFilePath(); } else { return cipherFile.getFilePath(); diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java index 5fc8aa32..12cc3ef2 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileSystemImplTest.java @@ -206,7 +206,7 @@ public void testCleartextDirectory() throws IOException { Path result = inTest.getCiphertextPath(cleartext); Assertions.assertEquals(ciphertext, result); - Mockito.verify(cryptoPathMapper,never()).getCiphertextFilePath(any()); + Mockito.verify(cryptoPathMapper, never()).getCiphertextFilePath(any()); } } From c7e1636c4f66eb18ca0f39e28f852883bdbb9fc5 Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Fri, 14 Apr 2023 18:18:14 +0200 Subject: [PATCH 31/32] simplify test --- .../CryptoFileChannelWriteReadIntegrationTest.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java index b9e43e2f..02bda852 100644 --- a/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java +++ b/src/test/java/org/cryptomator/cryptofs/CryptoFileChannelWriteReadIntegrationTest.java @@ -552,10 +552,9 @@ public void testConcurrentRead() throws IOException, InterruptedException { //https://github.com/cryptomator/cryptofs/issues/168 @Test - @DisplayName("Opening two file channels simultaneously retains ciphertext readability") + @DisplayName("Opening two file channels simultaneously and close afterwards retains ciphertext readability") public void testOpeningTwoChannelsRetainsCiphertextReadability() throws IOException { - var content = StandardCharsets.UTF_8.encode("two channels sitting on the wall"); - AtomicInteger numBytesRead = new AtomicInteger(-1); + var content = StandardCharsets.UTF_8.encode("two channels sitting on the wall").asReadOnlyBuffer(); ByteBuffer bytesRead = ByteBuffer.allocate(content.limit()); try (var ch = FileChannel.open(file, READ, WRITE, CREATE_NEW)) { @@ -567,13 +566,9 @@ public void testOpeningTwoChannelsRetainsCiphertextReadability() throws IOExcept Assertions.assertDoesNotThrow(() -> { try (var ch = FileChannel.open(file, READ)) { - int read = ch.read(bytesRead, 0); - numBytesRead.set(read); + ch.read(bytesRead, 0); } }); - - Assertions.assertEquals(content.limit(), numBytesRead.get()); - Assertions.assertArrayEquals(content.array(), bytesRead.array()); } //https://github.com/cryptomator/cryptofs/issues/169 From 13355c0fb330edb6cc358bf11d6570978bb0939c Mon Sep 17 00:00:00 2001 From: Armin Schrenk Date: Mon, 17 Apr 2023 10:22:20 +0200 Subject: [PATCH 32/32] prepare 2.6.4 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 949e3af9..6a07e495 100644 --- a/pom.xml +++ b/pom.xml @@ -2,7 +2,7 @@ 4.0.0 org.cryptomator cryptofs - 2.7.0-SNAPSHOT + 2.6.4 Cryptomator Crypto Filesystem This library provides the Java filesystem provider used by Cryptomator. https://github.com/cryptomator/cryptofs