From 83c45873e6ca92c3cffc0f7531e7cc04e665f294 Mon Sep 17 00:00:00 2001 From: Rob Hague Date: Sun, 29 Jun 2025 18:58:50 +0200 Subject: [PATCH 1/3] Build the read-ahead mechanism into SftpFileStream This change unifies the SFTP download implementations that exist via DownloadFile and via SftpFileStream, by rewriting SftpFileStream to perform the same "read-aheads" as DownloadFile. This brings the performance of downloads via SftpFileStream in line with DownloadFile, such that the latter is now effectively SftpFileStream.CopyTo. It also brings the recently added DownloadFileAsync up to speed since that was implemented via SftpFileStream.CopyToAsync. The methodology is a mix of the previous one and that within OpenSSH: the first call to SftpFileStream.Read sends one read request to the server. The second sends two and when not interrupted by Write or similar, the number of in-flight read requests continues to scale up in this fashion. I have measured CopyTo to be 3-20x faster than before, depending on file size and server round-trip time. --- src/Renci.SshNet/Common/Extensions.cs | 24 + src/Renci.SshNet/IServiceFactory.cs | 12 - src/Renci.SshNet/ServiceFactory.cs | 47 - src/Renci.SshNet/Sftp/ISftpFileReader.cs | 23 - src/Renci.SshNet/Sftp/ISftpSession.cs | 17 +- src/Renci.SshNet/Sftp/SftpFileReader.cs | 535 ++----- src/Renci.SshNet/Sftp/SftpFileStream.cs | 1257 ++++++----------- src/Renci.SshNet/Sftp/SftpSession.cs | 32 +- src/Renci.SshNet/SftpClient.cs | 153 +- .../SftpClientTest.Download.cs | 2 +- .../SftpTests.cs | 235 ++- ...tpFileReader_EndLStatThrowsSshException.cs | 99 -- ...izeIsAlmostSixTimesGreaterThanChunkSize.cs | 101 -- ...tpFileReader_FileSizeIsEqualToChunkSize.cs | 101 -- ...eIsExactlyFiveTimesGreaterThanChunkSize.cs | 101 -- ...pFileReader_FileSizeIsLessThanChunkSize.cs | 101 -- ...leMoreThanFiveTimesGreaterThanChunkSize.cs | 101 -- ...IsMoreThanMaxPendingReadsTimesChunkSize.cs | 103 -- ...est_CreateSftpFileReader_FileSizeIsZero.cs | 101 -- .../Classes/Sftp/SftpFileReaderTestBase.cs | 63 - ...st_DisposeShouldUnblockReadAndReadAhead.cs | 173 --- ...ReaderTest_Dispose_SftpSessionIsNotOpen.cs | 132 -- ...SessionIsOpen_BeginCloseThrowsException.cs | 136 -- ...tpSessionIsOpen_EndCloseThrowsException.cs | 141 -- ...ReaderTest_LastChunkBeforeEofIsComplete.cs | 168 --- ...eReaderTest_LastChunkBeforeEofIsPartial.cs | 167 --- ...iousChunkIsIncompleteAndEofIsNotReached.cs | 323 ----- ...reviousChunkIsIncompleteAndEofIsReached.cs | 207 --- ...eReaderTest_ReadAheadBeginReadException.cs | 6 - ...vokeException_DiscardsFurtherReadAheads.cs | 213 --- ...vokeException_PreventsFurtherReadAheads.cs | 189 --- ...leReaderTest_ReadBackBeginReadException.cs | 6 - ...leReaderTest_ReadBackEndInvokeException.cs | 6 - ...Test_Read_ReadAheadExceptionInBeginRead.cs | 172 --- ...dExceptionInWaitOnHandle_ChunkAvailable.cs | 144 -- ...xceptionInWaitOnHandle_NoChunkAvailable.cs | 128 -- .../Sftp/SftpFileStreamAsyncTestBase.cs | 70 - .../Classes/Sftp/SftpFileStreamTest.cs | 53 +- 38 files changed, 928 insertions(+), 4714 deletions(-) delete mode 100644 src/Renci.SshNet/Sftp/ISftpFileReader.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_EndLStatThrowsSshException.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsAlmostSixTimesGreaterThanChunkSize.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsEqualToChunkSize.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsExactlyFiveTimesGreaterThanChunkSize.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLessThanChunkSize.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLittleMoreThanFiveTimesGreaterThanChunkSize.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsMoreThanMaxPendingReadsTimesChunkSize.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsZero.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTestBase.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_DisposeShouldUnblockReadAndReadAhead.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsNotOpen.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_BeginCloseThrowsException.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_EndCloseThrowsException.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsComplete.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsPartial.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsNotReached.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsReached.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadBeginReadException.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_DiscardsFurtherReadAheads.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_PreventsFurtherReadAheads.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackBeginReadException.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackEndInvokeException.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInBeginRead.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_ChunkAvailable.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_NoChunkAvailable.cs delete mode 100644 test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamAsyncTestBase.cs diff --git a/src/Renci.SshNet/Common/Extensions.cs b/src/Renci.SshNet/Common/Extensions.cs index 6cc65a779..1d3749c7b 100644 --- a/src/Renci.SshNet/Common/Extensions.cs +++ b/src/Renci.SshNet/Common/Extensions.cs @@ -3,6 +3,7 @@ using System.Globalization; #if !NET using System.IO; +using System.Threading.Tasks; #endif using System.Net; using System.Net.Sockets; @@ -398,6 +399,29 @@ internal static void ReadExactly(this Stream stream, byte[] buffer, int offset, totalRead += read; } } + + internal static Task WaitAsync(this Task task, CancellationToken cancellationToken) + { + if (task.IsCompleted || !cancellationToken.CanBeCanceled) + { + return task; + } + + return WaitCore(); + + async Task WaitCore() + { + TaskCompletionSource tcs = new(TaskCreationOptions.RunContinuationsAsynchronously); + + using var reg = cancellationToken.Register( + () => tcs.TrySetCanceled(cancellationToken), + useSynchronizationContext: false); + + var completedTask = await Task.WhenAny(task, tcs.Task).ConfigureAwait(false); + + return await completedTask.ConfigureAwait(false); + } + } #endif } } diff --git a/src/Renci.SshNet/IServiceFactory.cs b/src/Renci.SshNet/IServiceFactory.cs index 7051942a1..681d69da9 100644 --- a/src/Renci.SshNet/IServiceFactory.cs +++ b/src/Renci.SshNet/IServiceFactory.cs @@ -83,18 +83,6 @@ internal partial interface IServiceFactory /// No key exchange algorithm is supported by both client and server. IKeyExchange CreateKeyExchange(IDictionary> clientAlgorithms, string[] serverAlgorithms); - /// - /// Creates an for the specified file and with the specified - /// buffer size. - /// - /// The file to read. - /// The SFTP session to use. - /// The size of buffer. - /// - /// An . - /// - ISftpFileReader CreateSftpFileReader(string fileName, ISftpSession sftpSession, uint bufferSize); - /// /// Creates a new instance. /// diff --git a/src/Renci.SshNet/ServiceFactory.cs b/src/Renci.SshNet/ServiceFactory.cs index ab3ae16cb..e8e5a2c35 100644 --- a/src/Renci.SshNet/ServiceFactory.cs +++ b/src/Renci.SshNet/ServiceFactory.cs @@ -4,8 +4,6 @@ using System.Net.Sockets; using System.Text; -using Microsoft.Extensions.Logging; - using Renci.SshNet.Common; using Renci.SshNet.Connection; using Renci.SshNet.Messages.Transport; @@ -118,51 +116,6 @@ public INetConfSession CreateNetConfSession(ISession session, int operationTimeo return new NetConfSession(session, operationTimeout); } - /// - /// Creates an for the specified file and with the specified - /// buffer size. - /// - /// The file to read. - /// The SFTP session to use. - /// The size of buffer. - /// - /// An . - /// - public ISftpFileReader CreateSftpFileReader(string fileName, ISftpSession sftpSession, uint bufferSize) - { - const int defaultMaxPendingReads = 10; - - // Issue #292: Avoid overlapping SSH_FXP_OPEN and SSH_FXP_LSTAT requests for the same file as this - // causes a performance degradation on Sun SSH - var openAsyncResult = sftpSession.BeginOpen(fileName, Flags.Read, callback: null, state: null); - var handle = sftpSession.EndOpen(openAsyncResult); - - var statAsyncResult = sftpSession.BeginLStat(fileName, callback: null, state: null); - - long? fileSize; - int maxPendingReads; - - var chunkSize = sftpSession.CalculateOptimalReadLength(bufferSize); - - // fallback to a default maximum of pending reads when remote server does not allow us to obtain - // the attributes of the file - try - { - var fileAttributes = sftpSession.EndLStat(statAsyncResult); - fileSize = fileAttributes.Size; - maxPendingReads = Math.Min(100, (int)Math.Ceiling((double)fileAttributes.Size / chunkSize) + 1); - } - catch (SshException ex) - { - fileSize = null; - maxPendingReads = defaultMaxPendingReads; - - sftpSession.SessionLoggerFactory.CreateLogger().LogInformation(ex, "Failed to obtain size of file. Allowing maximum {MaxPendingReads} pending reads", maxPendingReads); - } - - return sftpSession.CreateFileReader(handle, sftpSession, chunkSize, maxPendingReads, fileSize); - } - /// /// Creates a new instance. /// diff --git a/src/Renci.SshNet/Sftp/ISftpFileReader.cs b/src/Renci.SshNet/Sftp/ISftpFileReader.cs deleted file mode 100644 index 823b2e23a..000000000 --- a/src/Renci.SshNet/Sftp/ISftpFileReader.cs +++ /dev/null @@ -1,23 +0,0 @@ -using System; - -using Renci.SshNet.Common; - -namespace Renci.SshNet.Sftp -{ - /// - /// Reads a given file. - /// - internal interface ISftpFileReader : IDisposable - { - /// - /// Reads a sequence of bytes from the current file and advances the position within the file by the number of bytes read. - /// - /// - /// The sequence of bytes read from the file, or a zero-length array if the end of the file - /// has been reached. - /// - /// The current is disposed. - /// Attempting to read beyond the end of the file. - byte[] Read(); - } -} diff --git a/src/Renci.SshNet/Sftp/ISftpSession.cs b/src/Renci.SshNet/Sftp/ISftpSession.cs index b9baf43a5..ca281e355 100644 --- a/src/Renci.SshNet/Sftp/ISftpSession.cs +++ b/src/Renci.SshNet/Sftp/ISftpSession.cs @@ -67,11 +67,10 @@ internal interface ISftpSession : ISubsystemSession /// Asynchronously performs a SSH_FXP_FSTAT request. /// /// The handle. - /// If set to , is returned in case of an error. /// /// The file attributes. /// - SftpFileAttributes RequestFStat(byte[] handle, bool nullOnError); + SftpFileAttributes RequestFStat(byte[] handle); /// /// Asynchronously performs a SSH_FXP_FSTAT request. @@ -522,19 +521,5 @@ void RequestWrite(byte[] handle, /// Currently, we do not take the remote window size into account. /// uint CalculateOptimalWriteLength(uint bufferSize, byte[] handle); - - /// - /// Creates an for reading the content of the file represented by a given . - /// - /// The handle of the file to read. - /// The SFTP session. - /// The maximum number of bytes to read with each chunk. - /// The maximum number of pending reads. - /// The size of the file or when the size could not be determined. - /// - /// An for reading the content of the file represented by the - /// specified . - /// - ISftpFileReader CreateFileReader(byte[] handle, ISftpSession sftpSession, uint chunkSize, int maxPendingReads, long? fileSize); } } diff --git a/src/Renci.SshNet/Sftp/SftpFileReader.cs b/src/Renci.SshNet/Sftp/SftpFileReader.cs index 1f3fe396e..a2b884019 100644 --- a/src/Renci.SshNet/Sftp/SftpFileReader.cs +++ b/src/Renci.SshNet/Sftp/SftpFileReader.cs @@ -1,469 +1,178 @@ -using System; +#nullable enable +using System; using System.Collections.Generic; -using System.Globalization; +using System.Diagnostics; using System.Runtime.ExceptionServices; using System.Threading; +using System.Threading.Tasks; -using Microsoft.Extensions.Logging; - -using Renci.SshNet.Abstractions; +#if !NET using Renci.SshNet.Common; +#endif namespace Renci.SshNet.Sftp { - internal sealed class SftpFileReader : ISftpFileReader + public sealed partial class SftpFileStream { - private const int ReadAheadWaitTimeoutInMilliseconds = 1000; - - private readonly byte[] _handle; - private readonly ISftpSession _sftpSession; - private readonly uint _chunkSize; - private readonly SemaphoreSlim _semaphore; - private readonly object _readLock; - private readonly ManualResetEvent _disposingWaitHandle; - private readonly ManualResetEvent _readAheadCompleted; - private readonly Dictionary _queue; - private readonly WaitHandle[] _waitHandles; - private readonly ILogger _logger; - - /// - /// Holds the size of the file, when available. - /// - private readonly long? _fileSize; - - private ulong _offset; - private int _readAheadChunkIndex; - private ulong _readAheadOffset; - private int _nextChunkIndex; - - /// - /// Holds a value indicating whether EOF has already been signaled by the SSH server. - /// - private bool _endOfFileReceived; - - /// - /// Holds a value indicating whether the client has read up to the end of the file. - /// - private bool _isEndOfFileRead; - - private bool _disposingOrDisposed; - - private Exception _exception; - - /// - /// Initializes a new instance of the class with the specified handle, - /// and the maximum number of pending reads. - /// - /// The file handle. - /// The SFT session. - /// The size of a individual read-ahead chunk. - /// The maximum number of pending reads. - /// The size of the file, if known; otherwise, . - public SftpFileReader(byte[] handle, ISftpSession sftpSession, uint chunkSize, int maxPendingReads, long? fileSize) + private sealed class SftpFileReader : IDisposable { - _handle = handle; - _sftpSession = sftpSession; - _chunkSize = chunkSize; - _fileSize = fileSize; - _semaphore = new SemaphoreSlim(maxPendingReads); - _queue = new Dictionary(maxPendingReads); - _readLock = new object(); - _readAheadCompleted = new ManualResetEvent(initialState: false); - _disposingWaitHandle = new ManualResetEvent(initialState: false); - _waitHandles = _sftpSession.CreateWaitHandleArray(_disposingWaitHandle, _semaphore.AvailableWaitHandle); - _logger = sftpSession.SessionLoggerFactory.CreateLogger(); - - StartReadAhead(); - } - - public byte[] Read() - { - ThrowHelper.ThrowObjectDisposedIf(_disposingOrDisposed, this); - - if (_exception is not null) - { - ExceptionDispatchInfo.Capture(_exception).Throw(); - } - - if (_isEndOfFileRead) - { - throw new SshException("Attempting to read beyond the end of the file."); - } - - BufferedRead nextChunk; - - lock (_readLock) - { - // wait until either the next chunk is available, an exception has occurred or the current - // instance is already disposed - while (!_queue.TryGetValue(_nextChunkIndex, out nextChunk) && _exception is null) + private readonly byte[] _handle; + private readonly ISftpSession _sftpSession; + private readonly int _maxPendingReads; + private readonly ulong? _fileSize; + private readonly Dictionary _requests = []; + private readonly CancellationTokenSource _cts; + + private uint _chunkSize; + + private ulong _offset; + private ulong _readAheadOffset; + private int _currentMaxRequests; + private ExceptionDispatchInfo? _exception; + + /// + /// Initializes a new instance of the class with the specified handle, + /// and the maximum number of pending reads. + /// + /// The file handle. + /// The SFTP session. + /// The size of a individual read-ahead chunk. + /// The starting offset in the file. + /// The maximum number of pending reads. + /// The size of the file, if known; otherwise, . + /// The initial number of pending reads. + public SftpFileReader(byte[] handle, ISftpSession sftpSession, int chunkSize, long position, int maxPendingReads, ulong? fileSize = null, int initialMaxRequests = 1) + { + Debug.Assert(chunkSize > 0); + Debug.Assert(position >= 0); + Debug.Assert(initialMaxRequests >= 1); + + _handle = handle; + _sftpSession = sftpSession; + _chunkSize = (uint)chunkSize; + _offset = _readAheadOffset = (ulong)position; + _maxPendingReads = maxPendingReads; + _fileSize = fileSize; + _currentMaxRequests = initialMaxRequests; + + _cts = new CancellationTokenSource(); + } + + public async Task ReadAsync(CancellationToken cancellationToken) + { + _exception?.Throw(); + + try { - _ = Monitor.Wait(_readLock); - } - - // throw when exception occured in read-ahead, or the current instance is already disposed - if (_exception != null) - { - ExceptionDispatchInfo.Capture(_exception).Throw(); - } - - var data = nextChunk.Data; - - if (nextChunk.Offset == _offset) - { - // have we reached EOF? - if (data.Length == 0) - { - // PERF: we do not bother updating all of the internal state when we've reached EOF - _isEndOfFileRead = true; - } - else + // Fill up the requests buffer with as many requests as we currently allow. + // On the first call to Read, that number is 1. On the second it is 2 etc. + while (_requests.Count < _currentMaxRequests) { - // remove processed chunk - _ = _queue.Remove(_nextChunkIndex); - - // update offset - _offset += (ulong)data.Length; + AddRequest(_readAheadOffset, _chunkSize); - // move to next chunk - _nextChunkIndex++; + _readAheadOffset += _chunkSize; } - // unblock wait in read-ahead - _ = _semaphore.Release(); + var request = _requests[_offset]; - return data; - } + var data = await request.Task.WaitAsync(cancellationToken).ConfigureAwait(false); - // When we received an EOF for the next chunk and the size of the file is known, then - // we only complete the current chunk if we haven't already read up to the file size. - // This way we save an extra round-trip to the server. - if (data.Length == 0 && _fileSize.HasValue && _offset == (ulong)_fileSize.Value) - { - // avoid future reads - _isEndOfFileRead = true; - - // unblock wait in read-ahead - _ = _semaphore.Release(); - - // signal EOF to caller - return nextChunk.Data; - } - } - - /* - * When the server returned less bytes than requested (for the previous chunk) - * we'll synchronously request the remaining data. - * - * Due to the optimization above, we'll only get here in one of the following cases: - * - an EOF situation for files for which we were unable to obtain the file size - * - fewer bytes that requested were returned - * - * According to the SSH specification, this last case should never happen for normal - * disk files (but can happen for device files). In practice, OpenSSH - for example - - * returns less bytes than requested when requesting more than 64 KB. - * - * Important: - * To avoid a deadlock, this read must be done outside of the read lock. - */ - - var bytesToCatchUp = nextChunk.Offset - _offset; - - /* - * TODO: break loop and interrupt blocking wait in case of exception - */ - - var read = _sftpSession.RequestRead(_handle, _offset, (uint)bytesToCatchUp); - if (read.Length == 0) - { - // process data in read lock to avoid ObjectDisposedException while releasing semaphore - lock (_readLock) - { - // a zero-length (EOF) response is only valid for the read-back when EOF has - // been signaled for the next read-ahead chunk - if (nextChunk.Data.Length == 0) - { - _isEndOfFileRead = true; - - // ensure we've not yet disposed the current instance - if (!_disposingOrDisposed) - { - // unblock wait in read-ahead - _ = _semaphore.Release(); - } - - // signal EOF to caller - return read; - } - - // move reader to error state - _exception = new SshException("Unexpectedly reached end of file."); - - // ensure we've not yet disposed the current instance - if (!_disposingOrDisposed) + if (data.Length == 0) { - // unblock wait in read-ahead - _ = _semaphore.Release(); + // EOF. We effectively disable this instance - further reads will + // continue to return EOF. + _currentMaxRequests = 0; + return data; } - // notify caller of error - throw _exception; - } - } - - _offset += (uint)read.Length; - - return read; - } - - public void Dispose() - { - Dispose(disposing: true); - GC.SuppressFinalize(this); - } - - /// - /// Releases unmanaged and - optionally - managed resources. - /// - /// to release both managed and unmanaged resources; to release only unmanaged resources. - private void Dispose(bool disposing) - { - if (_disposingOrDisposed) - { - return; - } - - // transition to disposing state - _disposingOrDisposed = true; - - if (disposing) - { - // record exception to break prevent future Read() - _exception = new ObjectDisposedException(GetType().FullName); - - // signal that we're disposing to interrupt wait in read-ahead - _ = _disposingWaitHandle.Set(); + _ = _requests.Remove(_offset); - // wait until the read-ahead thread has completed - _ = _readAheadCompleted.WaitOne(); + _offset += (ulong)data.Length; - // unblock the Read() - lock (_readLock) - { - // dispose semaphore in read lock to ensure we don't run into an ObjectDisposedException - // in Read() - _semaphore.Dispose(); - - // awake Read - Monitor.PulseAll(_readLock); - } - - _readAheadCompleted.Dispose(); - _disposingWaitHandle.Dispose(); - - if (_sftpSession.IsOpen) - { - try - { - var closeAsyncResult = _sftpSession.BeginClose(_handle, callback: null, state: null); - _sftpSession.EndClose(closeAsyncResult); - } - catch (Exception ex) + if (data.Length < request.Count) { - _logger.LogInformation(ex, "Failure closing handle"); - } - } - } - } + // We didn't receive all the data we requested. + // Add another request to fill in the gap. + AddRequest(_offset, request.Count - (uint)data.Length); - private void StartReadAhead() - { - ThreadAbstraction.ExecuteThread(() => - { - while (!_endOfFileReceived && _exception is null) - { - // check if we should continue with the read-ahead loop - // note that the EOF and exception check are not included - // in this check as they do not require Read() to be - // unblocked (or have already done this) - if (!ContinueReadAhead()) - { - // unblock the Read() - lock (_readLock) + if (data.Length < _chunkSize) { - Monitor.PulseAll(_readLock); + // Right-size the buffer to match the amount that the server + // is willing to return. + // Note that this also happens near EOF. + _chunkSize = Math.Max(512, (uint)data.Length); } - - // break the read-ahead loop - break; } - // attempt to obtain the semaphore; this may time out when all semaphores are - // in use due to pending read-aheads (which in turn can happen when the server - // is slow to respond or when the session is broken) - if (!_semaphore.Wait(ReadAheadWaitTimeoutInMilliseconds)) + if (_currentMaxRequests > 0) { - // re-evaluate whether an exception occurred, and - if not - wait again - continue; - } - - // don't bother reading any more chunks if we received EOF, an exception has occurred - // or the current instance is disposed - if (_endOfFileReceived || _exception != null) - { - break; - } - - // start reading next chunk - var bufferedRead = new BufferedRead(_readAheadChunkIndex, _readAheadOffset); - - try - { - // even if we know the size of the file and have read up to EOF, we still want - // to keep reading (ahead) until we receive zero bytes from the remote host as - // we do not want to rely purely on the reported file size - // - // if the offset of the read-ahead chunk is greater than that file size, then - // we can expect to be reading the last (zero-byte) chunk and switch to synchronous - // mode to avoid having multiple read-aheads that read beyond EOF - if (_fileSize != null && (long)_readAheadOffset > _fileSize.Value) + if (_readAheadOffset > _fileSize) { - var asyncResult = _sftpSession.BeginRead(_handle, _readAheadOffset, _chunkSize, callback: null, bufferedRead); - var data = _sftpSession.EndRead(asyncResult); - ReadCompletedCore(bufferedRead, data); + _currentMaxRequests = 1; } - else + else if (_currentMaxRequests < _maxPendingReads) { - _ = _sftpSession.BeginRead(_handle, _readAheadOffset, _chunkSize, ReadCompleted, bufferedRead); + _currentMaxRequests++; } } - catch (Exception ex) - { - HandleFailure(ex); - break; - } - - // advance read-ahead offset - _readAheadOffset += _chunkSize; - // increment index of read-ahead chunk - _readAheadChunkIndex++; + return data; } - - _ = _readAheadCompleted.Set(); - }); - } - - /// - /// Returns a value indicating whether the read-ahead loop should be continued. - /// - /// - /// if the read-ahead loop should be continued; otherwise, . - /// - private bool ContinueReadAhead() - { - try - { - var waitResult = _sftpSession.WaitAny(_waitHandles, _sftpSession.OperationTimeout); - switch (waitResult) + catch (Exception ex) when (!(ex is OperationCanceledException oce && oce.CancellationToken == cancellationToken)) { - case 0: // disposing - return false; - case 1: // semaphore available - return true; - default: - throw new NotImplementedException(string.Format(CultureInfo.InvariantCulture, "WaitAny return value '{0}' is not implemented.", waitResult)); + // If the wait was cancelled then we will allow subsequent reads as normal. + // For any other errors, we prevent further read requests, effectively disabling + // this instance. + _currentMaxRequests = 0; + _exception = ExceptionDispatchInfo.Capture(ex); + throw; } } - catch (Exception ex) - { - _ = Interlocked.CompareExchange(ref _exception, ex, comparand: null); - return false; - } - } - private void ReadCompleted(IAsyncResult result) - { - if (_disposingOrDisposed) + private void AddRequest(ulong offset, uint count) { - // skip further processing if we're disposing the current instance - // to avoid accessing disposed members - return; + _requests.Add( + offset, + new Request( + offset, + count, + _sftpSession.RequestReadAsync(_handle, offset, count, _cts.Token))); } - var readAsyncResult = (SftpReadAsyncResult)result; - - byte[] data; - - try - { - data = readAsyncResult.EndInvoke(); - } - catch (Exception ex) + public void Dispose() { - HandleFailure(ex); - return; - } + _exception ??= ExceptionDispatchInfo.Capture(new ObjectDisposedException(GetType().FullName)); - // a read that completes with a zero-byte result signals EOF - // but there may be pending reads before that read - var bufferedRead = (BufferedRead)readAsyncResult.AsyncState; - ReadCompletedCore(bufferedRead, data); - } - - private void ReadCompletedCore(BufferedRead bufferedRead, byte[] data) - { - bufferedRead.Complete(data); - - lock (_readLock) - { - // add item to queue - _queue.Add(bufferedRead.ChunkIndex, bufferedRead); - - // Signal that a chunk has been read or EOF has been reached. - // In both cases, Read() will eventually also unblock the "read-ahead" thread. - Monitor.PulseAll(_readLock); - } + if (_requests.Count > 0) + { + // Cancel outstanding requests and observe the exception on them + // as an effort to prevent unhandled exceptions. - // check if server signaled EOF - if (data.Length == 0) - { - // set a flag to stop read-aheads - _endOfFileReceived = true; - } - } + _cts.Cancel(); - private void HandleFailure(Exception cause) - { - _ = Interlocked.CompareExchange(ref _exception, cause, comparand: null); + foreach (var request in _requests.Values) + { + _ = request.Task.Exception; + } - // unblock read-ahead - _ = _semaphore.Release(); + _requests.Clear(); + } - // unblock Read() - lock (_readLock) - { - Monitor.PulseAll(_readLock); + _cts.Dispose(); } - } - internal sealed class BufferedRead - { - public int ChunkIndex { get; } - - public byte[] Data { get; private set; } - - public ulong Offset { get; } - - public BufferedRead(int chunkIndex, ulong offset) + private sealed class Request { - ChunkIndex = chunkIndex; - Offset = offset; - } + public Request(ulong offset, uint count, Task task) + { + Offset = offset; + Count = count; + Task = task; + } - public void Complete(byte[] data) - { - Data = data; + public ulong Offset { get; } + public uint Count { get; } + public Task Task { get; } } } } diff --git a/src/Renci.SshNet/Sftp/SftpFileStream.cs b/src/Renci.SshNet/Sftp/SftpFileStream.cs index c5e486a9a..e33b71b9b 100644 --- a/src/Renci.SshNet/Sftp/SftpFileStream.cs +++ b/src/Renci.SshNet/Sftp/SftpFileStream.cs @@ -1,75 +1,54 @@ -using System; +#nullable enable +using System; +using System.ComponentModel; using System.Diagnostics; -using System.Diagnostics.CodeAnalysis; -using System.Globalization; using System.IO; using System.Threading; using System.Threading.Tasks; +using Microsoft.Extensions.Logging; + using Renci.SshNet.Common; namespace Renci.SshNet.Sftp { /// - /// Exposes a around a remote SFTP file, supporting both synchronous and asynchronous read and write operations. + /// Exposes a around a remote SFTP file, supporting + /// both synchronous and asynchronous read and write operations. /// - /// -#pragma warning disable IDE0079 // We intentionally want to suppress the below warning. - [SuppressMessage("Performance", "CA1844: Provide memory-based overrides of async methods when subclassing 'Stream'", Justification = "TODO: This should be addressed in the future.")] -#pragma warning restore IDE0079 - public sealed class SftpFileStream : Stream + public sealed partial class SftpFileStream : Stream { - private readonly Lock _lock = new Lock(); + private const int MaxPendingReads = 100; + + private readonly ISftpSession _session; + private readonly FileAccess _access; + private readonly bool _canSeek; private readonly int _readBufferSize; - private readonly int _writeBufferSize; - // Internal state. - private byte[] _handle; - private ISftpSession _session; + private SftpFileReader? _sftpFileReader; + private ReadOnlyMemory _readBuffer; + private System.Net.ArrayBuffer _writeBuffer; - // Buffer information. - private byte[] _readBuffer; - private byte[] _writeBuffer; - private int _bufferPosition; - private int _bufferLen; private long _position; - private bool _bufferOwnedByWrite; - private bool _canRead; - private bool _canSeek; - private bool _canWrite; private TimeSpan _timeout; + private bool _disposed; - /// - /// Gets a value indicating whether the current stream supports reading. - /// - /// - /// if the stream supports reading; otherwise, . - /// + /// public override bool CanRead { - get { return _canRead; } + get { return !_disposed && (_access & FileAccess.Read) == FileAccess.Read; } } - /// - /// Gets a value indicating whether the current stream supports seeking. - /// - /// - /// if the stream supports seeking; otherwise, . - /// + /// public override bool CanSeek { - get { return _canSeek; } + get { return !_disposed && _canSeek; } } - /// - /// Gets a value indicating whether the current stream supports writing. - /// - /// - /// if the stream supports writing; otherwise, . - /// + /// public override bool CanWrite { - get { return _canWrite; } + get { return !_disposed && (_access & FileAccess.Write) == FileAccess.Write; } } /// @@ -80,66 +59,32 @@ public override bool CanWrite /// public override bool CanTimeout { - get { return true; } + get { return false; } } - /// - /// Gets the length in bytes of the stream. - /// - /// A long value representing the length of the stream in bytes. - /// A class derived from Stream does not support seeking. - /// Methods were called after the stream was closed. - /// IO operation failed. + /// public override long Length { get { - // Lock down the file stream while we do this. - lock (_lock) - { - CheckSessionIsOpen(); + ThrowIfNotSeekable(); - if (!CanSeek) - { - throw new NotSupportedException("Seek operation is not supported."); - } + Flush(); - // Flush the write buffer, because it may - // affect the length of the stream. - if (_bufferOwnedByWrite) - { - FlushWriteBuffer(); - } + var size = _session.RequestFStat(Handle).Size; - // obtain file attributes - var attributes = _session.RequestFStat(_handle, nullOnError: true); - if (attributes != null) - { - return attributes.Size; - } + Debug.Assert(size >= 0, "fstat should return size as checked in ctor"); - throw new IOException("Seek operation failed."); - } + return size; } } - /// - /// Gets or sets the position within the current stream. - /// - /// The current position within the stream. - /// An I/O error occurs. - /// The stream does not support seeking. - /// Methods were called after the stream was closed. + /// public override long Position { get { - CheckSessionIsOpen(); - - if (!CanSeek) - { - throw new NotSupportedException("Seek operation not supported."); - } + ThrowIfNotSeekable(); return _position; } @@ -155,7 +100,7 @@ public override long Position /// /// The name of the path that was used to construct the current . /// - public string Name { get; private set; } + public string Name { get; } /// /// Gets the operating system file handle for the file that the current encapsulates. @@ -163,14 +108,7 @@ public override long Position /// /// The operating system file handle for the file that the current encapsulates. /// - public byte[] Handle - { - get - { - Flush(); - return _handle; - } - } + public byte[] Handle { get; } /// /// Gets or sets the operation timeout. @@ -178,6 +116,7 @@ public byte[] Handle /// /// The timeout. /// + [EditorBrowsable(EditorBrowsableState.Never)] // Unused public TimeSpan Timeout { get @@ -192,33 +131,63 @@ public TimeSpan Timeout } } - private SftpFileStream(ISftpSession session, string path, FileAccess access, int readBufferSize, int writeBufferSize, byte[] handle, long position) + private SftpFileStream( + ISftpSession session, + string path, + FileAccess access, + bool canSeek, + int readBufferSize, + int writeBufferSize, + byte[] handle, + long position, + SftpFileReader? initialReader) { Timeout = TimeSpan.FromSeconds(30); Name = path; _session = session; - _canRead = (access & FileAccess.Read) == FileAccess.Read; - _canSeek = true; - _canWrite = (access & FileAccess.Write) == FileAccess.Write; + _access = access; + _canSeek = canSeek; - _handle = handle; + Handle = handle; _readBufferSize = readBufferSize; - _writeBufferSize = writeBufferSize; _position = position; + _writeBuffer = new System.Net.ArrayBuffer(writeBufferSize); + _sftpFileReader = initialReader; } - internal static SftpFileStream Open(ISftpSession session, string path, FileMode mode, FileAccess access, int bufferSize) + internal static SftpFileStream Open( + ISftpSession? session, + string path, + FileMode mode, + FileAccess access, + int bufferSize, + bool isDownloadFile = false) { - return Open(session, path, mode, access, bufferSize, isAsync: false, CancellationToken.None).GetAwaiter().GetResult(); + return Open(session, path, mode, access, bufferSize, isDownloadFile, isAsync: false, CancellationToken.None).GetAwaiter().GetResult(); } - internal static Task OpenAsync(ISftpSession session, string path, FileMode mode, FileAccess access, int bufferSize, CancellationToken cancellationToken) + internal static Task OpenAsync( + ISftpSession? session, + string path, + FileMode mode, + FileAccess access, + int bufferSize, + CancellationToken cancellationToken, + bool isDownloadFile = false) { - return Open(session, path, mode, access, bufferSize, isAsync: true, cancellationToken); + return Open(session, path, mode, access, bufferSize, isDownloadFile, isAsync: true, cancellationToken); } - private static async Task Open(ISftpSession session, string path, FileMode mode, FileAccess access, int bufferSize, bool isAsync, CancellationToken cancellationToken) + private static async Task Open( + ISftpSession? session, + string path, + FileMode mode, + FileAccess access, + int bufferSize, + bool isDownloadFile, + bool isAsync, + CancellationToken cancellationToken) { Debug.Assert(isAsync || cancellationToken == default); @@ -234,44 +203,27 @@ private static async Task Open(ISftpSession session, string path throw new SshConnectionException("Client not connected."); } - var flags = Flags.None; - - switch (access) + var flags = access switch { - case FileAccess.Read: - flags |= Flags.Read; - break; - case FileAccess.Write: - flags |= Flags.Write; - break; - case FileAccess.ReadWrite: - flags |= Flags.Read; - flags |= Flags.Write; - break; - default: - throw new ArgumentOutOfRangeException(nameof(access)); - } + FileAccess.Read => Flags.Read, + FileAccess.Write => Flags.Write, + FileAccess.ReadWrite => Flags.Read | Flags.Write, + _ => throw new ArgumentOutOfRangeException(nameof(access)) + }; - if ((access & FileAccess.Read) == FileAccess.Read && mode == FileMode.Append) + if (mode == FileMode.Append && access != FileAccess.Write) { - throw new ArgumentException(string.Format(CultureInfo.InvariantCulture, - "{0} mode can be requested only when combined with write-only access.", - mode.ToString("G")), - nameof(mode)); + throw new ArgumentException( + "Append mode can be requested only with write-only access.", + nameof(access)); } - if ((access & FileAccess.Write) != FileAccess.Write) + if (access == FileAccess.Read && + mode is FileMode.Create or FileMode.CreateNew or FileMode.Truncate or FileMode.Append) { - if (mode is FileMode.Create or FileMode.CreateNew or FileMode.Truncate or FileMode.Append) - { - throw new ArgumentException(string.Format(CultureInfo.InvariantCulture, - "Combining {0}: {1} with {2}: {3} is invalid.", - nameof(FileMode), - mode, - nameof(FileAccess), - access), - nameof(mode)); - } + throw new ArgumentException( + $"Combining {nameof(FileMode)}: {mode} with {nameof(FileAccess)}: {access} is invalid.", + nameof(access)); } switch (mode) @@ -317,869 +269,526 @@ private static async Task Open(ISftpSession session, string path var readBufferSize = (int)session.CalculateOptimalReadLength((uint)bufferSize); var writeBufferSize = (int)session.CalculateOptimalWriteLength((uint)bufferSize, handle); - long position = 0; - if (mode == FileMode.Append) - { - SftpFileAttributes attributes; + SftpFileAttributes? attributes; + try + { if (isAsync) { attributes = await session.RequestFStatAsync(handle, cancellationToken).ConfigureAwait(false); } else { - attributes = session.RequestFStat(handle, nullOnError: false); + attributes = session.RequestFStat(handle); } + } + catch (SshException ex) + { + session.SessionLoggerFactory.CreateLogger().LogInformation( + ex, "fstat failed after opening {Path}. Will set CanSeek=false.", path); - position = attributes.Size; + attributes = null; } - return new SftpFileStream(session, path, access, readBufferSize, writeBufferSize, handle, position); - } + bool canSeek; + long position = 0; + SftpFileReader? initialReader = null; - /// - /// Clears all buffers for this stream and causes any buffered data to be written to the file. - /// - /// An I/O error occurs. - /// Stream is closed. - public override void Flush() - { - lock (_lock) + if (attributes?.Size >= 0) { - CheckSessionIsOpen(); + canSeek = true; - if (_bufferOwnedByWrite) + if (mode == FileMode.Append) { - FlushWriteBuffer(); + position = attributes.Size; } - else + else if (isDownloadFile) + { + // If we are in a call to SftpClient.DownloadFile, then we know that we will read the whole file, + // so we can let there be several in-flight requests from the get go. + // This optimisation is mostly only beneficial to smaller files on higher latency connections. + var initialPendingReads = (int)Math.Max(1, Math.Min(MaxPendingReads, 1 + (attributes.Size / readBufferSize))); + + initialReader = new(handle, session, readBufferSize, position, MaxPendingReads, (ulong)attributes.Size, initialPendingReads); + } + else if ((access & FileAccess.Read) == FileAccess.Read) { - FlushReadBuffer(); + // The reader can use the size information to reduce in-flight requests near the expected EOF, + // so pass it in here. + initialReader = new(handle, session, readBufferSize, position, MaxPendingReads, (ulong)attributes.Size); } } - } - - /// - /// Asynchronously clears all buffers for this stream and causes any buffered data to be written to the file. - /// - /// The to observe. - /// A that represents the asynchronous flush operation. - /// An I/O error occurs. - /// Stream is closed. - public override Task FlushAsync(CancellationToken cancellationToken) - { - CheckSessionIsOpen(); - - if (_bufferOwnedByWrite) + else { - return FlushWriteBufferAsync(cancellationToken); + // Either fstat is failing or it doesn't return the size, in which case we can't support Length, + // so CanSeek must return false. + canSeek = false; } - FlushReadBuffer(); - - return Task.CompletedTask; + return new SftpFileStream(session, path, access, canSeek, readBufferSize, writeBufferSize, handle, position, initialReader); } - /// - /// Reads a sequence of bytes from the current stream and advances the position within the stream by the - /// number of bytes read. - /// - /// An array of bytes. When this method returns, the buffer contains the specified byte array with the values between and ( + - 1) replaced by the bytes read from the current source. - /// The zero-based byte offset in at which to begin storing the data read from the current stream. - /// The maximum number of bytes to be read from the current stream. - /// - /// The total number of bytes read into the buffer. This can be less than the number of bytes requested - /// if that many bytes are not currently available, or zero (0) if the end of the stream has been reached. - /// - /// The sum of and is larger than the buffer length. - /// is . - /// or is negative. - /// An I/O error occurs. - /// The stream does not support reading. - /// Methods were called after the stream was closed. - /// - /// - /// This method attempts to read up to bytes. This either from the buffer, from the - /// server (using one or more SSH_FXP_READ requests) or using a combination of both. - /// - /// - /// The read loop is interrupted when either bytes are read, the server returns zero - /// bytes (EOF) or less bytes than the read buffer size. - /// - /// - /// When a server returns less number of bytes than the read buffer size, this may indicate that EOF has - /// been reached. A subsequent (SSH_FXP_READ) server request is necessary to make sure EOF has effectively - /// been reached. Breaking out of the read loop avoids reading from the server twice to determine EOF: once in - /// the read loop, and once upon the next or invocation. - /// - /// - public override int Read(byte[] buffer, int offset, int count) + /// + public override void Flush() { -#if !NET - ThrowHelper. -#endif - ValidateBufferArguments(buffer, offset, count); + ThrowHelper.ThrowObjectDisposedIf(_disposed, this); - var readLen = 0; + var writeLength = _writeBuffer.ActiveLength; - // Lock down the file stream while we do this. - lock (_lock) + if (writeLength == 0) { - CheckSessionIsOpen(); - - // Set up for the read operation. - SetupRead(); - - // Read data into the caller's buffer. - while (count > 0) - { - // How much data do we have available in the buffer? - var bytesAvailableInBuffer = _bufferLen - _bufferPosition; - if (bytesAvailableInBuffer <= 0) - { - var data = _session.RequestRead(_handle, (ulong)_position, (uint)_readBufferSize); - - if (data.Length == 0) - { - _bufferPosition = 0; - _bufferLen = 0; - - break; - } - - var bytesToWriteToCallerBuffer = count; - if (bytesToWriteToCallerBuffer >= data.Length) - { - // write all data read to caller-provided buffer - bytesToWriteToCallerBuffer = data.Length; - - // reset buffer since we will skip buffering - _bufferPosition = 0; - _bufferLen = 0; - } - else - { - // determine number of bytes that we should write into read buffer - var bytesToWriteToReadBuffer = data.Length - bytesToWriteToCallerBuffer; - - // write remaining bytes to read buffer - Buffer.BlockCopy(data, count, GetOrCreateReadBuffer(), 0, bytesToWriteToReadBuffer); - - // update position in read buffer - _bufferPosition = 0; - - // update number of bytes in read buffer - _bufferLen = bytesToWriteToReadBuffer; - } - - // write bytes to caller-provided buffer - Buffer.BlockCopy(data, 0, buffer, offset, bytesToWriteToCallerBuffer); - - // update stream position - _position += bytesToWriteToCallerBuffer; - - // record total number of bytes read into caller-provided buffer - readLen += bytesToWriteToCallerBuffer; + return; + } - // break out of the read loop when the server returned less than the request number of bytes - // as that *may* indicate that we've reached EOF - // - // doing this avoids reading from server twice to determine EOF: once in the read loop, and - // once upon the next Read or ReadByte invocation by the caller - if (data.Length < _readBufferSize) - { - break; - } + // Under normal usage the offset will be nonnegative, but we nevertheless + // perform a checked conversion to prevent writing to a very large offset + // in case of corruption due to e.g. invalid multithreaded usage. + var serverOffset = checked((ulong)(_position - writeLength)); - // advance offset to start writing bytes into caller-provided buffer - offset += bytesToWriteToCallerBuffer; + using (var wait = new AutoResetEvent(initialState: false)) + { + _session.RequestWrite( + Handle, + serverOffset, + _writeBuffer.DangerousGetUnderlyingBuffer(), + _writeBuffer.ActiveStartOffset, + writeLength, + wait); - // update number of bytes left to read into caller-provided buffer - count -= bytesToWriteToCallerBuffer; - } - else - { - // limit the number of bytes to use from read buffer to the caller-request number of bytes - if (bytesAvailableInBuffer > count) - { - bytesAvailableInBuffer = count; - } + _writeBuffer.Discard(writeLength); + } + } - // copy data from read buffer to the caller-provided buffer - Buffer.BlockCopy(GetOrCreateReadBuffer(), _bufferPosition, buffer, offset, bytesAvailableInBuffer); + /// + public override async Task FlushAsync(CancellationToken cancellationToken) + { + ThrowHelper.ThrowObjectDisposedIf(_disposed, this); - // update position in read buffer - _bufferPosition += bytesAvailableInBuffer; + var writeLength = _writeBuffer.ActiveLength; - // update stream position - _position += bytesAvailableInBuffer; + if (writeLength == 0) + { + return; + } - // record total number of bytes read into caller-provided buffer - readLen += bytesAvailableInBuffer; + // Under normal usage the offset will be nonnegative, but we nevertheless + // perform a checked conversion to prevent writing to a very large offset + // in case of corruption due to e.g. invalid multithreaded usage. + var serverOffset = checked((ulong)(_position - writeLength)); - // advance offset to start writing bytes into caller-provided buffer - offset += bytesAvailableInBuffer; + await _session.RequestWriteAsync( + Handle, + serverOffset, + _writeBuffer.DangerousGetUnderlyingBuffer(), + _writeBuffer.ActiveStartOffset, + writeLength, + cancellationToken).ConfigureAwait(false); - // update number of bytes left to read - count -= bytesAvailableInBuffer; - } - } - } + _writeBuffer.Discard(writeLength); + } - // return the number of bytes that were read to the caller. - return readLen; + private void InvalidateReads() + { + _readBuffer = ReadOnlyMemory.Empty; + _sftpFileReader?.Dispose(); + _sftpFileReader = null; } - /// - /// Asynchronously reads a sequence of bytes from the current stream and advances the position within the stream by the - /// number of bytes read. - /// - /// An array of bytes. When this method returns, the buffer contains the specified byte array with the values between and ( + - 1) replaced by the bytes read from the current source. - /// The zero-based byte offset in at which to begin storing the data read from the current stream. - /// The maximum number of bytes to be read from the current stream. - /// The to observe. - /// A that represents the asynchronous read operation. - public override async Task ReadAsync(byte[] buffer, int offset, int count, CancellationToken cancellationToken) + /// + public override int Read(byte[] buffer, int offset, int count) { #if !NET ThrowHelper. #endif ValidateBufferArguments(buffer, offset, count); - cancellationToken.ThrowIfCancellationRequested(); - - var readLen = 0; - - CheckSessionIsOpen(); + return Read(buffer.AsSpan(offset, count)); + } - // Set up for the read operation. - SetupRead(); +#if NET + /// + public override int Read(Span buffer) +#else + private int Read(Span buffer) +#endif + { + ThrowIfNotReadable(); - // Read data into the caller's buffer. - while (count > 0) + if (_readBuffer.IsEmpty) { - // How much data do we have available in the buffer? - var bytesAvailableInBuffer = _bufferLen - _bufferPosition; - if (bytesAvailableInBuffer <= 0) + if (_sftpFileReader is null) { - var data = await _session.RequestReadAsync(_handle, (ulong)_position, (uint)_readBufferSize, cancellationToken).ConfigureAwait(false); + Flush(); + _sftpFileReader = new(Handle, _session, _readBufferSize, _position, MaxPendingReads); + } - if (data.Length == 0) - { - _bufferPosition = 0; - _bufferLen = 0; + _readBuffer = _sftpFileReader.ReadAsync(CancellationToken.None).GetAwaiter().GetResult(); - break; - } - - var bytesToWriteToCallerBuffer = count; - if (bytesToWriteToCallerBuffer >= data.Length) - { - // write all data read to caller-provided buffer - bytesToWriteToCallerBuffer = data.Length; + if (_readBuffer.IsEmpty) + { + // If we've hit EOF then throw away this reader instance. + // If Read is called again we will create a new reader. + // This takes care of the case when a file is expanding + // during reading. + _sftpFileReader.Dispose(); + _sftpFileReader = null; + } + } - // reset buffer since we will skip buffering - _bufferPosition = 0; - _bufferLen = 0; - } - else - { - // determine number of bytes that we should write into read buffer - var bytesToWriteToReadBuffer = data.Length - bytesToWriteToCallerBuffer; + Debug.Assert(_writeBuffer.ActiveLength == 0, "Write buffer should be empty when reading."); - // write remaining bytes to read buffer - Buffer.BlockCopy(data, count, GetOrCreateReadBuffer(), 0, bytesToWriteToReadBuffer); + var bytesRead = Math.Min(buffer.Length, _readBuffer.Length); - // update position in read buffer - _bufferPosition = 0; + _readBuffer.Span.Slice(0, bytesRead).CopyTo(buffer); + _readBuffer = _readBuffer.Slice(bytesRead); - // update number of bytes in read buffer - _bufferLen = bytesToWriteToReadBuffer; - } + _position += bytesRead; - // write bytes to caller-provided buffer - Buffer.BlockCopy(data, 0, buffer, offset, bytesToWriteToCallerBuffer); + return bytesRead; + } - // update stream position - _position += bytesToWriteToCallerBuffer; + /// + public override Task ReadAsync(byte[] buffer, int offset, int count, CancellationToken cancellationToken) + { +#if !NET + ThrowHelper. +#endif + ValidateBufferArguments(buffer, offset, count); - // record total number of bytes read into caller-provided buffer - readLen += bytesToWriteToCallerBuffer; + return ReadAsync(buffer.AsMemory(offset, count), cancellationToken).AsTask(); + } - // break out of the read loop when the server returned less than the request number of bytes - // as that *may* indicate that we've reached EOF - // - // doing this avoids reading from server twice to determine EOF: once in the read loop, and - // once upon the next Read or ReadByte invocation by the caller - if (data.Length < _readBufferSize) - { - break; - } +#if NET + /// + public override async ValueTask ReadAsync(Memory buffer, CancellationToken cancellationToken = default) +#else + private async ValueTask ReadAsync(Memory buffer, CancellationToken cancellationToken) +#endif + { + ThrowIfNotReadable(); - // advance offset to start writing bytes into caller-provided buffer - offset += bytesToWriteToCallerBuffer; + if (_readBuffer.IsEmpty) + { + if (_sftpFileReader is null) + { + await FlushAsync(cancellationToken).ConfigureAwait(false); - // update number of bytes left to read into caller-provided buffer - count -= bytesToWriteToCallerBuffer; + _sftpFileReader = new(Handle, _session, _readBufferSize, _position, MaxPendingReads); } - else - { - // limit the number of bytes to use from read buffer to the caller-request number of bytes - if (bytesAvailableInBuffer > count) - { - bytesAvailableInBuffer = count; - } - // copy data from read buffer to the caller-provided buffer - Buffer.BlockCopy(GetOrCreateReadBuffer(), _bufferPosition, buffer, offset, bytesAvailableInBuffer); + _readBuffer = await _sftpFileReader.ReadAsync(cancellationToken).ConfigureAwait(false); - // update position in read buffer - _bufferPosition += bytesAvailableInBuffer; + if (_readBuffer.IsEmpty) + { + // If we've hit EOF then throw away this reader instance. + // If Read is called again we will create a new reader. + // This takes care of the case when a file is expanding + // during reading. + _sftpFileReader.Dispose(); + _sftpFileReader = null; + } + } - // update stream position - _position += bytesAvailableInBuffer; + Debug.Assert(_writeBuffer.ActiveLength == 0, "Write buffer should be empty when reading."); - // record total number of bytes read into caller-provided buffer - readLen += bytesAvailableInBuffer; + var bytesRead = Math.Min(buffer.Length, _readBuffer.Length); - // advance offset to start writing bytes into caller-provided buffer - offset += bytesAvailableInBuffer; + _readBuffer.Slice(0, bytesRead).CopyTo(buffer); + _readBuffer = _readBuffer.Slice(bytesRead); - // update number of bytes left to read - count -= bytesAvailableInBuffer; - } - } + _position += bytesRead; - // return the number of bytes that were read to the caller. - return readLen; + return bytesRead; } - /// - /// Reads a byte from the stream and advances the position within the stream by one byte, or returns -1 if at the end of the stream. - /// - /// - /// The unsigned byte cast to an , or -1 if at the end of the stream. - /// - /// The stream does not support reading. - /// Methods were called after the stream was closed. - /// Read operation failed. +#if NET + /// public override int ReadByte() { - // Lock down the file stream while we do this. - lock (_lock) - { - CheckSessionIsOpen(); - - // Setup the object for reading. - SetupRead(); - - byte[] readBuffer; - - // Read more data into the internal buffer if necessary. - if (_bufferPosition >= _bufferLen) - { - var data = _session.RequestRead(_handle, (ulong)_position, (uint)_readBufferSize); - if (data.Length == 0) - { - // We've reached EOF. - return -1; - } + byte b = default; + var read = Read(new Span(ref b)); + return read == 0 ? -1 : b; + } +#endif - readBuffer = GetOrCreateReadBuffer(); - Buffer.BlockCopy(data, 0, readBuffer, 0, data.Length); + /// + public override IAsyncResult BeginRead(byte[] buffer, int offset, int count, AsyncCallback? callback, object? state) + { + return TaskToAsyncResult.Begin(ReadAsync(buffer, offset, count), callback, state); + } - _bufferPosition = 0; - _bufferLen = data.Length; - } - else - { - readBuffer = GetOrCreateReadBuffer(); - } + /// + public override int EndRead(IAsyncResult asyncResult) + { + return TaskToAsyncResult.End(asyncResult); + } - // Extract the next byte from the buffer. - ++_position; + /// + public override void Write(byte[] buffer, int offset, int count) + { +#if !NET + ThrowHelper. +#endif + ValidateBufferArguments(buffer, offset, count); - return readBuffer[_bufferPosition++]; - } + Write(buffer.AsSpan(offset, count)); } - /// - /// Sets the position within the current stream. - /// - /// A byte offset relative to the parameter. - /// A value of type indicating the reference point used to obtain the new position. - /// - /// The new position within the current stream. - /// - /// An I/O error occurs. - /// The stream does not support seeking, such as if the stream is constructed from a pipe or console output. - /// Methods were called after the stream was closed. - public override long Seek(long offset, SeekOrigin origin) +#if NET + /// + public override void Write(ReadOnlySpan buffer) +#else + private void Write(ReadOnlySpan buffer) +#endif { - long newPosn; + ThrowIfNotWriteable(); - // Lock down the file stream while we do this. - lock (_lock) - { - CheckSessionIsOpen(); + InvalidateReads(); - if (!CanSeek) - { - throw new NotSupportedException("Seek is not supported."); - } - - // Don't do anything if the position won't be moving. - if (origin == SeekOrigin.Begin && offset == _position) - { - return offset; - } + while (!buffer.IsEmpty) + { + var byteCount = Math.Min(buffer.Length, _writeBuffer.AvailableLength); - if (origin == SeekOrigin.Current && offset == 0) - { - return _position; - } + buffer.Slice(0, byteCount).CopyTo(_writeBuffer.AvailableSpan); - // The behaviour depends upon the read/write mode. - if (_bufferOwnedByWrite) - { - // Flush the write buffer and then seek. - FlushWriteBuffer(); - } - else - { - // Determine if the seek is to somewhere inside - // the current read buffer bounds. - if (origin == SeekOrigin.Begin) - { - newPosn = _position - _bufferPosition; - if (offset >= newPosn && offset < (newPosn + _bufferLen)) - { - _bufferPosition = (int)(offset - newPosn); - _position = offset; - return _position; - } - } - else if (origin == SeekOrigin.Current) - { - newPosn = _position + offset; - if (newPosn >= (_position - _bufferPosition) && - newPosn < (_position - _bufferPosition + _bufferLen)) - { - _bufferPosition = (int)(newPosn - (_position - _bufferPosition)); - _position = newPosn; - return _position; - } - } + buffer = buffer.Slice(byteCount); - // Abandon the read buffer. - _bufferPosition = 0; - _bufferLen = 0; - } + _writeBuffer.Commit(byteCount); - // Seek to the new position. - switch (origin) - { - case SeekOrigin.Begin: - newPosn = offset; - break; - case SeekOrigin.Current: - newPosn = _position + offset; - break; - case SeekOrigin.End: - var attributes = _session.RequestFStat(_handle, nullOnError: false); - newPosn = attributes.Size + offset; - break; - default: - throw new ArgumentException("Invalid seek origin.", nameof(origin)); - } + _position += byteCount; - if (newPosn < 0) + if (_writeBuffer.AvailableLength == 0) { - throw new EndOfStreamException(); + Flush(); } - - _position = newPosn; - return _position; } } - /// - /// Sets the length of the current stream. - /// - /// The desired length of the current stream in bytes. - /// An I/O error occurs. - /// The stream does not support both writing and seeking. - /// Methods were called after the stream was closed. - /// must be greater than zero. - /// - /// - /// Buffers are first flushed. - /// - /// - /// If the specified value is less than the current length of the stream, the stream is truncated and - if the - /// current position is greater than the new length - the current position is moved to the last byte of the stream. - /// - /// - /// If the given value is greater than the current length of the stream, the stream is expanded and the current - /// position remains the same. - /// - /// - public override void SetLength(long value) + /// + public override void WriteByte(byte value) { - ThrowHelper.ThrowIfNegative(value); - - // Lock down the file stream while we do this. - lock (_lock) - { - CheckSessionIsOpen(); - - if (!CanSeek) - { - throw new NotSupportedException("Seek is not supported."); - } - - if (_bufferOwnedByWrite) - { - FlushWriteBuffer(); - } - else - { - SetupWrite(); - } - - var attributes = _session.RequestFStat(_handle, nullOnError: false); - attributes.Size = value; - _session.RequestFSetStat(_handle, attributes); - - if (_position > value) - { - _position = value; - } - } + Write([value]); } - /// - /// Writes a sequence of bytes to the current stream and advances the current position within this stream by the number of bytes written. - /// - /// An array of bytes. This method copies bytes from to the current stream. - /// The zero-based byte offset in at which to begin copying bytes to the current stream. - /// The number of bytes to be written to the current stream. - /// The sum of and is greater than the buffer length. - /// is . - /// or is negative. - /// An I/O error occurs. - /// The stream does not support writing. - /// Methods were called after the stream was closed. - public override void Write(byte[] buffer, int offset, int count) + /// + public override Task WriteAsync(byte[] buffer, int offset, int count, CancellationToken cancellationToken) { #if !NET ThrowHelper. #endif ValidateBufferArguments(buffer, offset, count); - // Lock down the file stream while we do this. - lock (_lock) - { - CheckSessionIsOpen(); + return WriteAsync(buffer.AsMemory(offset, count), cancellationToken).AsTask(); + } - // Setup this object for writing. - SetupWrite(); +#if NET + /// + public override async ValueTask WriteAsync(ReadOnlyMemory buffer, CancellationToken cancellationToken = default) +#else + private async ValueTask WriteAsync(ReadOnlyMemory buffer, CancellationToken cancellationToken) +#endif + { + ThrowIfNotWriteable(); - // Write data to the file stream. - while (count > 0) - { - // Determine how many bytes we can write to the buffer. - var tempLen = _writeBufferSize - _bufferPosition; - if (tempLen <= 0) - { - // flush write buffer, and mark it empty - FlushWriteBuffer(); + InvalidateReads(); - // we can now write or buffer the full buffer size - tempLen = _writeBufferSize; - } + while (!buffer.IsEmpty) + { + var byteCount = Math.Min(buffer.Length, _writeBuffer.AvailableLength); - // limit the number of bytes to write to the actual number of bytes requested - if (tempLen > count) - { - tempLen = count; - } + buffer.Slice(0, byteCount).CopyTo(_writeBuffer.AvailableMemory); - // Can we short-cut the internal buffer? - if (_bufferPosition == 0 && tempLen == _writeBufferSize) - { - using (var wait = new AutoResetEvent(initialState: false)) - { - _session.RequestWrite(_handle, (ulong)_position, buffer, offset, tempLen, wait); - } - } - else - { - // No: copy the data to the write buffer first. - Buffer.BlockCopy(buffer, offset, GetOrCreateWriteBuffer(), _bufferPosition, tempLen); - _bufferPosition += tempLen; - } + buffer = buffer.Slice(byteCount); - // Advance the buffer and stream positions. - _position += tempLen; - offset += tempLen; - count -= tempLen; - } + _writeBuffer.Commit(byteCount); - // If the buffer is full, then do a speculative flush now, - // rather than waiting for the next call to this method. - if (_bufferPosition >= _writeBufferSize) - { - using (var wait = new AutoResetEvent(initialState: false)) - { - _session.RequestWrite(_handle, (ulong)(_position - _bufferPosition), GetOrCreateWriteBuffer(), 0, _bufferPosition, wait); - } + _position += byteCount; - _bufferPosition = 0; + if (_writeBuffer.AvailableLength == 0) + { + await FlushAsync(cancellationToken).ConfigureAwait(false); } } } - /// - /// Asynchronously writes a sequence of bytes to the current stream and advances the current position within this stream by the number of bytes written. - /// - /// An array of bytes. This method copies bytes from to the current stream. - /// The zero-based byte offset in at which to begin copying bytes to the current stream. - /// The number of bytes to be written to the current stream. - /// The to observe. - /// A that represents the asynchronous write operation. - /// The sum of and is greater than the buffer length. - /// is . - /// or is negative. - /// An I/O error occurs. - /// The stream does not support writing. - /// Methods were called after the stream was closed. - public override async Task WriteAsync(byte[] buffer, int offset, int count, CancellationToken cancellationToken) + /// + public override IAsyncResult BeginWrite(byte[] buffer, int offset, int count, AsyncCallback? callback, object? state) { -#if !NET - ThrowHelper. -#endif - ValidateBufferArguments(buffer, offset, count); + return TaskToAsyncResult.Begin(WriteAsync(buffer, offset, count), callback, state); + } - cancellationToken.ThrowIfCancellationRequested(); + /// + public override void EndWrite(IAsyncResult asyncResult) + { + TaskToAsyncResult.End(asyncResult); + } - CheckSessionIsOpen(); + /// + public override long Seek(long offset, SeekOrigin origin) + { + ThrowIfNotSeekable(); - // Setup this object for writing. - SetupWrite(); + Flush(); - // Write data to the file stream. - while (count > 0) + var newPosition = origin switch { - // Determine how many bytes we can write to the buffer. - var tempLen = _writeBufferSize - _bufferPosition; - if (tempLen <= 0) - { - // flush write buffer, and mark it empty - await FlushWriteBufferAsync(cancellationToken).ConfigureAwait(false); - - // we can now write or buffer the full buffer size - tempLen = _writeBufferSize; - } + SeekOrigin.Begin => offset, + SeekOrigin.Current => _position + offset, + SeekOrigin.End => _session.RequestFStat(Handle).Size + offset, + _ => throw new ArgumentOutOfRangeException(nameof(origin)) + }; - // limit the number of bytes to write to the actual number of bytes requested - if (tempLen > count) - { - tempLen = count; - } + if (newPosition < 0) + { + throw new IOException("An attempt was made to move the position before the beginning of the stream."); + } - // Can we short-cut the internal buffer? - if (_bufferPosition == 0 && tempLen == _writeBufferSize) - { - await _session.RequestWriteAsync(_handle, (ulong)_position, buffer, offset, tempLen, cancellationToken).ConfigureAwait(false); - } - else - { - // No: copy the data to the write buffer first. - Buffer.BlockCopy(buffer, offset, GetOrCreateWriteBuffer(), _bufferPosition, tempLen); - _bufferPosition += tempLen; - } + var readBufferStart = _position; // inclusive + var readBufferEnd = _position + _readBuffer.Length; // exclusive - // Advance the buffer and stream positions. - _position += tempLen; - offset += tempLen; - count -= tempLen; + if (readBufferStart <= newPosition && newPosition <= readBufferEnd) + { + _readBuffer = _readBuffer.Slice((int)(newPosition - readBufferStart)); } - - // If the buffer is full, then do a speculative flush now, - // rather than waiting for the next call to this method. - if (_bufferPosition >= _writeBufferSize) + else { - await _session.RequestWriteAsync(_handle, (ulong)(_position - _bufferPosition), GetOrCreateWriteBuffer(), 0, _bufferPosition, cancellationToken).ConfigureAwait(false); - _bufferPosition = 0; + InvalidateReads(); } + + return _position = newPosition; } - /// - /// Writes a byte to the current position in the stream and advances the position within the stream by one byte. - /// - /// The byte to write to the stream. - /// An I/O error occurs. - /// The stream does not support writing, or the stream is already closed. - /// Methods were called after the stream was closed. - public override void WriteByte(byte value) + /// + public override void SetLength(long value) { - // Lock down the file stream while we do this. - lock (_lock) - { - CheckSessionIsOpen(); - - // Setup the object for writing. - SetupWrite(); - - var writeBuffer = GetOrCreateWriteBuffer(); + ThrowHelper.ThrowIfNegative(value); + ThrowIfNotWriteable(); + ThrowIfNotSeekable(); - // Flush the current buffer if it is full. - if (_bufferPosition >= _writeBufferSize) - { - using (var wait = new AutoResetEvent(initialState: false)) - { - _session.RequestWrite(_handle, (ulong)(_position - _bufferPosition), writeBuffer, 0, _bufferPosition, wait); - } + Flush(); + InvalidateReads(); - _bufferPosition = 0; - } + var attributes = _session.RequestFStat(Handle); + attributes.Size = value; + _session.RequestFSetStat(Handle, attributes); - // Write the byte into the buffer and advance the posn. - writeBuffer[_bufferPosition++] = value; - ++_position; + if (_position > value) + { + _position = value; } } - /// - /// Releases the unmanaged resources used by the and optionally releases the managed resources. - /// - /// to release both managed and unmanaged resources; to release only unmanaged resources. + /// protected override void Dispose(bool disposing) { - base.Dispose(disposing); + if (_disposed) + { + return; + } - if (_session != null) + try { - if (disposing) + if (disposing && _session.IsOpen) { - lock (_lock) + try { - if (_session != null) + Flush(); + } + finally + { + if (_session.IsOpen) { - _canRead = false; - _canSeek = false; - _canWrite = false; - - if (_handle != null) - { - if (_session.IsOpen) - { - if (_bufferOwnedByWrite) - { - FlushWriteBuffer(); - } - - _session.RequestClose(_handle); - } - - _handle = null; - } - - _session = null; + _session.RequestClose(Handle); } } } } + finally + { + _disposed = true; + InvalidateReads(); + base.Dispose(disposing); + } } - private byte[] GetOrCreateReadBuffer() - { - _readBuffer ??= new byte[_readBufferSize]; - return _readBuffer; - } - - private byte[] GetOrCreateWriteBuffer() - { - _writeBuffer ??= new byte[_writeBufferSize]; - return _writeBuffer; - } - - /// - /// Flushes the read data from the buffer. - /// - private void FlushReadBuffer() +#if NET + /// +#pragma warning disable CA2215 // Dispose methods should call base class dispose + public override async ValueTask DisposeAsync() +#pragma warning restore CA2215 // Dispose methods should call base class dispose +#else + internal async ValueTask DisposeAsync() +#endif { - _bufferPosition = 0; - _bufferLen = 0; - } + if (_disposed) + { + return; + } - /// - /// Flush any buffered write data to the file. - /// - private void FlushWriteBuffer() - { - if (_bufferPosition > 0) + try { - using (var wait = new AutoResetEvent(initialState: false)) + if (_session.IsOpen) { - _session.RequestWrite(_handle, (ulong)(_position - _bufferPosition), _writeBuffer, 0, _bufferPosition, wait); + try + { + await FlushAsync().ConfigureAwait(false); + } + finally + { + if (_session.IsOpen) + { + await _session.RequestCloseAsync(Handle, CancellationToken.None).ConfigureAwait(false); + } + } } - - _bufferPosition = 0; } - } - - private async Task FlushWriteBufferAsync(CancellationToken cancellationToken) - { - if (_bufferPosition > 0) + finally { - await _session.RequestWriteAsync(_handle, (ulong)(_position - _bufferPosition), _writeBuffer, 0, _bufferPosition, cancellationToken).ConfigureAwait(false); - _bufferPosition = 0; + _disposed = true; + InvalidateReads(); + base.Dispose(disposing: false); } } - /// - /// Setups the read. - /// - private void SetupRead() + private void ThrowIfNotSeekable() { - if (!CanRead) + if (!CanSeek) { - throw new NotSupportedException("Read not supported."); + ThrowHelper.ThrowObjectDisposedIf(_disposed, this); + Throw(); } - if (_bufferOwnedByWrite) + static void Throw() { - FlushWriteBuffer(); - _bufferOwnedByWrite = false; + throw new NotSupportedException("Stream does not support seeking."); } } - /// - /// Setups the write. - /// - private void SetupWrite() + private void ThrowIfNotWriteable() { if (!CanWrite) { - throw new NotSupportedException("Write not supported."); + ThrowHelper.ThrowObjectDisposedIf(_disposed, this); + Throw(); } - if (!_bufferOwnedByWrite) + static void Throw() { - FlushReadBuffer(); - _bufferOwnedByWrite = true; + throw new NotSupportedException("Stream does not support writing."); } } - private void CheckSessionIsOpen() + private void ThrowIfNotReadable() { - ThrowHelper.ThrowObjectDisposedIf(_session is null, this); + if (!CanRead) + { + ThrowHelper.ThrowObjectDisposedIf(_disposed, this); + Throw(); + } - if (!_session.IsOpen) + static void Throw() { - throw new ObjectDisposedException(GetType().FullName, "Cannot access a closed SFTP session."); + throw new NotSupportedException("Stream does not support reading."); } } } diff --git a/src/Renci.SshNet/Sftp/SftpSession.cs b/src/Renci.SshNet/Sftp/SftpSession.cs index 2b4d8c00e..0f04e85f2 100644 --- a/src/Renci.SshNet/Sftp/SftpSession.cs +++ b/src/Renci.SshNet/Sftp/SftpSession.cs @@ -251,23 +251,6 @@ public async Task GetCanonicalPathAsync(string path, CancellationToken c return canonizedPath + slash + pathParts[pathParts.Length - 1]; } - /// - /// Creates an for reading the content of the file represented by a given . - /// - /// The handle of the file to read. - /// The SFTP session. - /// The maximum number of bytes to read with each chunk. - /// The maximum number of pending reads. - /// The size of the file or when the size could not be determined. - /// - /// An for reading the content of the file represented by the - /// specified . - /// - public ISftpFileReader CreateFileReader(byte[] handle, ISftpSession sftpSession, uint chunkSize, int maxPendingReads, long? fileSize) - { - return new SftpFileReader(handle, sftpSession, chunkSize, maxPendingReads, fileSize); - } - internal string GetFullRemotePath(string path) { var fullPath = path; @@ -820,6 +803,8 @@ public byte[] RequestRead(byte[] handle, ulong offset, uint length) /// public Task RequestReadAsync(byte[] handle, ulong offset, uint length, CancellationToken cancellationToken) { + Debug.Assert(length > 0, "This implementation cannot distinguish between EOF and zero-length reads"); + if (cancellationToken.IsCancellationRequested) { return Task.FromCanceled(cancellationToken); @@ -1075,15 +1060,8 @@ public SftpFileAttributes EndLStat(SFtpStatAsyncResult asyncResult) } } - /// - /// Performs SSH_FXP_FSTAT request. - /// - /// The handle. - /// If set to , returns instead of throwing an exception. - /// - /// File attributes. - /// - public SftpFileAttributes RequestFStat(byte[] handle, bool nullOnError) + /// + public SftpFileAttributes RequestFStat(byte[] handle) { SshException exception = null; SftpFileAttributes attributes = null; @@ -1109,7 +1087,7 @@ public SftpFileAttributes RequestFStat(byte[] handle, bool nullOnError) WaitOnHandle(wait, OperationTimeout); } - if (!nullOnError && exception is not null) + if (exception is not null) { throw exception; } diff --git a/src/Renci.SshNet/SftpClient.cs b/src/Renci.SshNet/SftpClient.cs index 949c64b3e..322c1e7da 100644 --- a/src/Renci.SshNet/SftpClient.cs +++ b/src/Renci.SshNet/SftpClient.cs @@ -1,5 +1,6 @@ #nullable enable using System; +using System.Buffers; using System.Collections.Generic; using System.Diagnostics; using System.Diagnostics.CodeAnalysis; @@ -899,17 +900,33 @@ public async Task ExistsAsync(string path, CancellationToken cancellationT /// public void DownloadFile(string path, Stream output, Action? downloadCallback = null) { + ThrowHelper.ThrowIfNullOrWhiteSpace(path); + ThrowHelper.ThrowIfNull(output); CheckDisposed(); - InternalDownloadFile(path, output, asyncResult: null, downloadCallback); + InternalDownloadFile( + path, + output, + asyncResult: null, + downloadCallback, + isAsync: false, + CancellationToken.None).GetAwaiter().GetResult(); } /// public Task DownloadFileAsync(string path, Stream output, CancellationToken cancellationToken = default) { + ThrowHelper.ThrowIfNullOrWhiteSpace(path); + ThrowHelper.ThrowIfNull(output); CheckDisposed(); - return InternalDownloadFileAsync(path, output, cancellationToken); + return InternalDownloadFile( + path, + output, + asyncResult: null, + downloadCallback: null, + isAsync: true, + cancellationToken); } /// @@ -976,17 +993,25 @@ public IAsyncResult BeginDownloadFile(string path, Stream output, AsyncCallback? /// public IAsyncResult BeginDownloadFile(string path, Stream output, AsyncCallback? asyncCallback, object? state, Action? downloadCallback = null) { - CheckDisposed(); ThrowHelper.ThrowIfNullOrWhiteSpace(path); ThrowHelper.ThrowIfNull(output); + CheckDisposed(); var asyncResult = new SftpDownloadAsyncResult(asyncCallback, state); - ThreadAbstraction.ExecuteThread(() => + _ = DoDownloadAndSetResult(); + + async Task DoDownloadAndSetResult() { try { - InternalDownloadFile(path, output, asyncResult, downloadCallback); + await InternalDownloadFile( + path, + output, + asyncResult, + downloadCallback, + isAsync: true, + CancellationToken.None).ConfigureAwait(false); asyncResult.SetAsCompleted(exception: null, completedSynchronously: false); } @@ -994,7 +1019,7 @@ public IAsyncResult BeginDownloadFile(string path, Stream output, AsyncCallback? { asyncResult.SetAsCompleted(exp, completedSynchronously: false); } - }); + } return asyncResult; } @@ -1050,7 +1075,7 @@ public void UploadFile(Stream input, string path, bool canOverride, Action @@ -2233,32 +2258,59 @@ private List InternalListDirectory(string path, SftpListDirectoryAsyn return result; } - /// - /// Internals the download file. - /// - /// The path. - /// The output. - /// An that references the asynchronous request. - /// The download callback. - /// is . - /// is or contains whitespace. - /// Client not connected. - private void InternalDownloadFile(string path, Stream output, SftpDownloadAsyncResult? asyncResult, Action? downloadCallback) +#pragma warning disable S6966 // Awaitable method should be used + private async Task InternalDownloadFile( + string path, + Stream output, + SftpDownloadAsyncResult? asyncResult, + Action? downloadCallback, + bool isAsync, + CancellationToken cancellationToken) { - ThrowHelper.ThrowIfNull(output); - ThrowHelper.ThrowIfNullOrWhiteSpace(path); + Debug.Assert(!string.IsNullOrWhiteSpace(path)); + Debug.Assert(output is not null); + Debug.Assert(isAsync || cancellationToken == default); if (_sftpSession is null) { throw new SshConnectionException("Client not connected."); } - var fullPath = _sftpSession.GetCanonicalPath(path); + SftpFileStream sftpStream; + + if (isAsync) + { + var fullPath = await _sftpSession.GetCanonicalPathAsync(path, cancellationToken).ConfigureAwait(false); - using (var fileReader = ServiceFactory.CreateSftpFileReader(fullPath, _sftpSession, _bufferSize)) + sftpStream = await SftpFileStream.OpenAsync( + _sftpSession, + fullPath, + FileMode.Open, + FileAccess.Read, + (int)_bufferSize, + cancellationToken, + isDownloadFile: true).ConfigureAwait(false); + } + else { - var totalBytesRead = 0UL; + var fullPath = _sftpSession.GetCanonicalPath(path); + + sftpStream = SftpFileStream.Open( + _sftpSession, + fullPath, + FileMode.Open, + FileAccess.Read, + (int)_bufferSize, + isDownloadFile: true); + } + + // The below is effectively sftpStream.CopyTo{Async}(output) with consideration + // for downloadCallback/asyncResult. + var buffer = ArrayPool.Shared.Rent(81920); + try + { + ulong totalBytesRead = 0; while (true) { // Cancel download @@ -2267,15 +2319,33 @@ private void InternalDownloadFile(string path, Stream output, SftpDownloadAsyncR break; } - var data = fileReader.Read(); - if (data.Length == 0) + var bytesRead = isAsync +#if NET + ? await sftpStream.ReadAsync(buffer, cancellationToken).ConfigureAwait(false) +#else + ? await sftpStream.ReadAsync(buffer, 0, buffer.Length, cancellationToken).ConfigureAwait(false) +#endif + : sftpStream.Read(buffer, 0, buffer.Length); + + if (bytesRead == 0) { break; } - output.Write(data, 0, data.Length); + if (isAsync) + { +#if NET + await output.WriteAsync(buffer.AsMemory(0, bytesRead), cancellationToken).ConfigureAwait(false); +#else + await output.WriteAsync(buffer, 0, bytesRead, cancellationToken).ConfigureAwait(false); +#endif + } + else + { + output.Write(buffer, 0, bytesRead); + } - totalBytesRead += (ulong)data.Length; + totalBytesRead += (ulong)bytesRead; asyncResult?.Update(totalBytesRead); @@ -2289,28 +2359,21 @@ private void InternalDownloadFile(string path, Stream output, SftpDownloadAsyncR } } } - } - - private async Task InternalDownloadFileAsync(string path, Stream output, CancellationToken cancellationToken) - { - ThrowHelper.ThrowIfNull(output); - ThrowHelper.ThrowIfNullOrWhiteSpace(path); - - if (_sftpSession is null) + finally { - throw new SshConnectionException("Client not connected."); - } - - cancellationToken.ThrowIfCancellationRequested(); + ArrayPool.Shared.Return(buffer); - var fullPath = await _sftpSession.GetCanonicalPathAsync(path, cancellationToken).ConfigureAwait(false); - var openStreamTask = SftpFileStream.OpenAsync(_sftpSession, fullPath, FileMode.Open, FileAccess.Read, (int)_bufferSize, cancellationToken); - - using (var input = await openStreamTask.ConfigureAwait(false)) - { - await input.CopyToAsync(output, 81920, cancellationToken).ConfigureAwait(false); + if (isAsync) + { + await sftpStream.DisposeAsync().ConfigureAwait(false); + } + else + { + sftpStream.Dispose(); + } } } +#pragma warning restore S6966 // Awaitable method should be used #pragma warning disable S6966 // Awaitable method should be used private async Task InternalUploadFile( diff --git a/test/Renci.SshNet.IntegrationTests/OldIntegrationTests/SftpClientTest.Download.cs b/test/Renci.SshNet.IntegrationTests/OldIntegrationTests/SftpClientTest.Download.cs index 8a2be6bae..e6e62cd5d 100644 --- a/test/Renci.SshNet.IntegrationTests/OldIntegrationTests/SftpClientTest.Download.cs +++ b/test/Renci.SshNet.IntegrationTests/OldIntegrationTests/SftpClientTest.Download.cs @@ -65,7 +65,7 @@ public async Task Test_Sftp_DownloadAsync_Cancellation_Requested() var cancelledToken = new CancellationToken(true); - await Assert.ThrowsExactlyAsync(() => sftp.DownloadFileAsync("/xxx/eee/yyy", Stream.Null, cancelledToken)); + await Assert.ThrowsAsync(() => sftp.DownloadFileAsync("/xxx/eee/yyy", Stream.Null, cancelledToken)); } } diff --git a/test/Renci.SshNet.IntegrationTests/SftpTests.cs b/test/Renci.SshNet.IntegrationTests/SftpTests.cs index 3c65bb0c3..b673c6808 100644 --- a/test/Renci.SshNet.IntegrationTests/SftpTests.cs +++ b/test/Renci.SshNet.IntegrationTests/SftpTests.cs @@ -4353,11 +4353,11 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginBegin() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset - 1]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4397,11 +4397,11 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginBegin() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset - 1]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4438,7 +4438,7 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginBegin() Assert.AreEqual(0x00, fs.ReadByte()); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(writeBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4474,11 +4474,11 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginBegin() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[550 - 1]; - Assert.AreEqual(550 - 1, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[550 - 1].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[550 - 1], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(writeBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4599,11 +4599,11 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginEnd() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4641,11 +4641,11 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginEnd() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4681,11 +4681,11 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginEnd() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(writeBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4722,11 +4722,11 @@ public void Sftp_SftpFileStream_Seek_BeyondEndOfFile_SeekOriginEnd() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(writeBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4813,7 +4813,7 @@ public void Sftp_SftpFileStream_Seek_NegativeOffSet_SeekOriginEnd() Assert.AreEqual(writeBuffer.Length, fs.Length); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(writeBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -4844,8 +4844,8 @@ public void Sftp_SftpFileStream_Seek_NegativeOffSet_SeekOriginEnd() Assert.AreEqual(writeBuffer.Length + 1, fs.Length); var readBuffer = new byte[writeBuffer.Length - 3]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); - Assert.IsTrue(readBuffer.SequenceEqual(writeBuffer.Take(readBuffer.Length))); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); + CollectionAssert.AreEqual(writeBuffer.Take(readBuffer.Length), readBuffer); Assert.AreEqual(0x01, fs.ReadByte()); Assert.AreEqual(0x05, fs.ReadByte()); @@ -4884,8 +4884,8 @@ public void Sftp_SftpFileStream_Seek_NegativeOffSet_SeekOriginEnd() // First part of file should not have been touched var readBuffer = new byte[(int)client.BufferSize * 2]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); - Assert.IsTrue(readBuffer.SequenceEqual(writeBuffer.Take(readBuffer.Length))); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); + CollectionAssert.AreEqual(writeBuffer.Take(readBuffer.Length), readBuffer); // Check part that should have been updated Assert.AreEqual(0x01, fs.ReadByte()); @@ -4895,8 +4895,10 @@ public void Sftp_SftpFileStream_Seek_NegativeOffSet_SeekOriginEnd() // Remaining bytes should not have been touched readBuffer = new byte[((int)client.BufferSize * 2) - 4]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); - Assert.IsTrue(readBuffer.SequenceEqual(writeBuffer.Skip(((int)client.BufferSize * 2) + 4).Take(readBuffer.Length))); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); + CollectionAssert.AreEqual( + writeBuffer.Skip(((int)client.BufferSize * 2) + 4).Take(readBuffer.Length).ToArray(), + readBuffer); // Ensure we've reached end of the stream Assert.AreEqual(-1, fs.ReadByte()); @@ -4987,7 +4989,7 @@ public void Sftp_SftpFileStream_Seek_WithinReadBuffer() { var readBuffer = new byte[200]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); var newPosition = fs.Seek(offset: 3L, SeekOrigin.Begin); @@ -5062,11 +5064,11 @@ public void Sftp_SftpFileStream_Seek_WithinReadBuffer() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset - 1]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -5104,11 +5106,11 @@ public void Sftp_SftpFileStream_Seek_WithinReadBuffer() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset - 1]; - Assert.AreEqual(soughtOverReadBuffer.Length, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[soughtOverReadBuffer.Length].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[soughtOverReadBuffer.Length], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(readBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -5148,7 +5150,7 @@ public void Sftp_SftpFileStream_Seek_WithinReadBuffer() Assert.AreEqual(0x00, fs.ReadByte()); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(writeBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -5187,11 +5189,11 @@ public void Sftp_SftpFileStream_Seek_WithinReadBuffer() Assert.AreEqual(0x04, fs.ReadByte()); var soughtOverReadBuffer = new byte[seekOffset - 1]; - Assert.AreEqual(seekOffset - 1, fs.Read(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length)); - Assert.IsTrue(new byte[seekOffset - 1].IsEqualTo(soughtOverReadBuffer)); + fs.ReadExactly(soughtOverReadBuffer, offset: 0, soughtOverReadBuffer.Length); + CollectionAssert.AreEqual(new byte[seekOffset - 1], soughtOverReadBuffer); var readBuffer = new byte[writeBuffer.Length]; - Assert.AreEqual(writeBuffer.Length, fs.Read(readBuffer, offset: 0, readBuffer.Length)); + fs.ReadExactly(readBuffer, offset: 0, readBuffer.Length); CollectionAssert.AreEqual(writeBuffer, readBuffer); // Ensure we've reached end of the stream @@ -6203,6 +6205,136 @@ public void Sftp_SetLastWriteTimeUtc() } } + [TestMethod] + public void Sftp_SftpFileStream_Fuzz() + { + const int OperationCount = 100; + const int MaxBufferSize = 1000; + const int MaxFileSize = 15_000; + + int seed = Environment.TickCount; + + Console.WriteLine("Using seed " + seed); + + var random = new Random(seed); + + using var client = new SftpClient(_connectionInfoFactory.Create()) + { + BufferSize = 100 + }; + client.Connect(); + + // We will perform operations on an SftpFileStream and a local + // System.IO.FileStream, and check that the results are the same. + // This could use a MemoryStream for the local side, except for the + // fact that performing a 0-byte write at a position beyond the length + // of the MemoryStream causes its length to increase, which is not the + // case for FileStream. Since we've got 'FileStream' in the name, we + // check that we align with FileStream's behaviour. + + string remoteFilePath = GenerateUniqueRemoteFileName(); + string localFilePath = Path.GetTempFileName(); + + byte[] fileBytes = new byte[1024]; + random.NextBytes(fileBytes); + + File.WriteAllBytes(localFilePath, fileBytes); + client.WriteAllBytes(remoteFilePath, fileBytes); + + try + { + using (var local = File.Open(localFilePath, FileMode.Open, FileAccess.ReadWrite)) + using (var remote = client.Open(remoteFilePath, FileMode.Open, FileAccess.ReadWrite)) + { + for (int i = 0; i < OperationCount; i++) + { +#pragma warning disable IDE0010 // Add missing cases + int op = random.Next(5); + switch (op) + { + case 0 when local.Length < MaxFileSize: // Write + { + var buffer = new byte[random.Next(0, MaxBufferSize)]; + random.NextBytes(buffer); + int offset = random.Next(0, buffer.Length + 1); + int count = random.Next(0, buffer.Length - offset + 1); + + remote.Write(buffer, offset, count); + local.Write(buffer, offset, count); + break; + } + case 1: // Read + { + var remoteBuffer = new byte[random.Next(0, MaxBufferSize)]; + var localBuffer = new byte[remoteBuffer.Length]; + int offset = random.Next(0, remoteBuffer.Length + 1); + int count = random.Next(0, remoteBuffer.Length - offset + 1); + + int remoteRead = ReadExactly(remote, remoteBuffer, offset, count); + int localRead = ReadExactly(local, localBuffer, offset, count); + + Assert.AreEqual(localRead, remoteRead); + CollectionAssert.AreEqual(localBuffer, remoteBuffer); + break; + } + case 2 when local.Length < MaxFileSize: // Seek + { + int position = (int)local.Position; + int length = (int)local.Length; + + SeekOrigin origin = (SeekOrigin)random.Next(0, 3); + long offset = 0; + switch (origin) + { + case SeekOrigin.Begin: + offset = random.Next(0, length * 2); + break; + case SeekOrigin.Current: + offset = random.Next(-position, position); + break; + case SeekOrigin.End: + offset = random.Next(-length, length); + break; + } + long newPosRemote = remote.Seek(offset, origin); + long newPosLocal = local.Seek(offset, origin); + Assert.AreEqual(newPosLocal, newPosRemote); + Assert.AreEqual(local.Length, remote.Length); + break; + } + case 3: // SetLength + { + long newLength = random.Next(0, MaxFileSize); + remote.SetLength(newLength); + local.SetLength(newLength); + Assert.AreEqual(local.Length, remote.Length); + Assert.AreEqual(local.Position, remote.Position); + break; + } + case 4: // Flush + { + remote.Flush(); + local.Flush(); + break; + } + } +#pragma warning restore IDE0010 // Add missing cases + } + } + + CollectionAssert.AreEqual(File.ReadAllBytes(localFilePath), client.ReadAllBytes(remoteFilePath)); + } + finally + { + File.Delete(localFilePath); + + if (client.Exists(remoteFilePath)) + { + client.DeleteFile(remoteFilePath); + } + } + } + private static IEnumerable GetSftpUploadFileFileStreamData() { yield return new object[] { 0 }; @@ -6292,21 +6424,24 @@ private static decimal CalculateTransferSpeed(long length, long elapsedMilliseco return (length / 1024m) / (elapsedMilliseconds / 1000m); } - private static void SftpCreateRemoteFile(SftpClient client, string remoteFile, int size) + /// + /// Similar to the netcore ReadExactly but without throwing on end of stream. + /// + private static int ReadExactly(Stream stream, byte[] buffer, int offset, int count) { - var file = CreateTempFile(size); - - try + int totalRead = 0; + while (totalRead < count) { - using (var fs = new FileStream(file, FileMode.Open, FileAccess.Read, FileShare.Read)) + int read = stream.Read(buffer, offset + totalRead, count - totalRead); + if (read == 0) { - client.UploadFile(fs, remoteFile); + return totalRead; } + + totalRead += read; } - finally - { - File.Delete(file); - } + + return totalRead; } private static byte[] GenerateRandom(int size) diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_EndLStatThrowsSshException.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_EndLStatThrowsSshException.cs deleted file mode 100644 index 77e226b3c..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_EndLStatThrowsSshException.cs +++ /dev/null @@ -1,99 +0,0 @@ -using System; - -using Microsoft.Extensions.Logging.Abstractions; -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_EndLStatThrowsSshException - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private ISftpFileReader _actual; - - private void SetupData() - { - var random = new Random(); - - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1, int.MaxValue); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpSessionMock.Setup(p => p.SessionLoggerFactory).Returns(NullLoggerFactory.Instance); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Throws(new SshException()); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, 10, null)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsAlmostSixTimesGreaterThanChunkSize.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsAlmostSixTimesGreaterThanChunkSize.cs deleted file mode 100644 index a869f48f8..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsAlmostSixTimesGreaterThanChunkSize.cs +++ /dev/null @@ -1,101 +0,0 @@ -using System; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; -using Renci.SshNet.Tests.Common; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_FileSizeIsAlmostSixTimesGreaterThanChunkSize - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private SftpFileAttributes _fileAttributes; - private long _fileSize; - private ISftpFileReader _actual; - - private void SetupData() - { - var random = new Random(); - - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1000, 5000); - _fileSize = (_chunkSize * 6) - 10; - _fileAttributes = new SftpFileAttributesBuilder().WithSize(_fileSize).Build(); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Returns(_fileAttributes); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, 7, _fileSize)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsEqualToChunkSize.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsEqualToChunkSize.cs deleted file mode 100644 index 29a0c9c71..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsEqualToChunkSize.cs +++ /dev/null @@ -1,101 +0,0 @@ -using System; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; -using Renci.SshNet.Tests.Common; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_FileSizeIsEqualToChunkSize - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private SftpFileAttributes _fileAttributes; - private long _fileSize; - private ISftpFileReader _actual; - - private void SetupData() - { - var random = new Random(); - - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1000, int.MaxValue); - _fileSize = _chunkSize; - _fileAttributes = new SftpFileAttributesBuilder().WithSize(_fileSize).Build(); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Returns(_fileAttributes); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, 2, _fileSize)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsExactlyFiveTimesGreaterThanChunkSize.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsExactlyFiveTimesGreaterThanChunkSize.cs deleted file mode 100644 index 47a446c17..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsExactlyFiveTimesGreaterThanChunkSize.cs +++ /dev/null @@ -1,101 +0,0 @@ -using System; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; -using Renci.SshNet.Tests.Common; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_FileSizeIsExactlyFiveTimesGreaterThanChunkSize - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private SftpFileAttributes _fileAttributes; - private long _fileSize; - private ISftpFileReader _actual; - - private void SetupData() - { - var random = new Random(); - - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1000, 5000); - _fileSize = _chunkSize * 5; - _fileAttributes = new SftpFileAttributesBuilder().WithSize(_fileSize).Build(); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Returns(_fileAttributes); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, 6, _fileSize)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLessThanChunkSize.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLessThanChunkSize.cs deleted file mode 100644 index b2202b183..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLessThanChunkSize.cs +++ /dev/null @@ -1,101 +0,0 @@ -using System; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; -using Renci.SshNet.Tests.Common; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLessThanChunkSize - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private SftpFileAttributes _fileAttributes; - private long _fileSize; - private ISftpFileReader _actual; - - private void SetupData() - { - var random = new Random(); - - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1000, int.MaxValue); - _fileSize = _chunkSize - random.Next(1, 10); - _fileAttributes = new SftpFileAttributesBuilder().WithSize(_fileSize).Build(); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Returns(_fileAttributes); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, 2, _fileSize)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLittleMoreThanFiveTimesGreaterThanChunkSize.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLittleMoreThanFiveTimesGreaterThanChunkSize.cs deleted file mode 100644 index c0439cba9..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLittleMoreThanFiveTimesGreaterThanChunkSize.cs +++ /dev/null @@ -1,101 +0,0 @@ -using System; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; -using Renci.SshNet.Tests.Common; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_FileSizeIsLittleMoreThanFiveTimesGreaterThanChunkSize - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private SftpFileAttributes _fileAttributes; - private long _fileSize; - private ISftpFileReader _actual; - - private void SetupData() - { - var random = new Random(); - - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1000, 5000); - _fileSize = (_chunkSize * 5) + 10; - _fileAttributes = new SftpFileAttributesBuilder().WithSize(_fileSize).Build(); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Returns(_fileAttributes); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, 7, _fileSize)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsMoreThanMaxPendingReadsTimesChunkSize.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsMoreThanMaxPendingReadsTimesChunkSize.cs deleted file mode 100644 index 105085a25..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsMoreThanMaxPendingReadsTimesChunkSize.cs +++ /dev/null @@ -1,103 +0,0 @@ -using System; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; -using Renci.SshNet.Tests.Common; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_FileSizeIsMoreThanMaxPendingReadsTimesChunkSize - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private SftpFileAttributes _fileAttributes; - private long _fileSize; - private ISftpFileReader _actual; - private int _maxPendingReads; - - private void SetupData() - { - var random = new Random(); - - _maxPendingReads = 100; - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1000, 5000); - _fileSize = _chunkSize * random.Next(_maxPendingReads + 1, _maxPendingReads * 2); - _fileAttributes = new SftpFileAttributesBuilder().WithSize(_fileSize).Build(); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Returns(_fileAttributes); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, _maxPendingReads, _fileSize)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsZero.cs b/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsZero.cs deleted file mode 100644 index 068865800..000000000 --- a/test/Renci.SshNet.Tests/Classes/ServiceFactoryTest_CreateSftpFileReader_FileSizeIsZero.cs +++ /dev/null @@ -1,101 +0,0 @@ -using System; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; -using Renci.SshNet.Tests.Common; - -namespace Renci.SshNet.Tests.Classes -{ - [TestClass] - public class ServiceFactoryTest_CreateSftpFileReader_FileSizeIsZero - { - private ServiceFactory _serviceFactory; - private Mock _sftpSessionMock; - private Mock _sftpFileReaderMock; - private uint _bufferSize; - private string _fileName; - private SftpOpenAsyncResult _openAsyncResult; - private byte[] _handle; - private SFtpStatAsyncResult _statAsyncResult; - private uint _chunkSize; - private long _fileSize; - private SftpFileAttributes _fileAttributes; - private ISftpFileReader _actual; - - private void SetupData() - { - var random = new Random(); - - _bufferSize = (uint)random.Next(1, int.MaxValue); - _openAsyncResult = new SftpOpenAsyncResult(null, null); - _handle = CryptoAbstraction.GenerateRandom(random.Next(1, 10)); - _statAsyncResult = new SFtpStatAsyncResult(null, null); - _fileName = random.Next().ToString(); - _chunkSize = (uint)random.Next(1, int.MaxValue); - _fileSize = 0L; - _fileAttributes = new SftpFileAttributesBuilder().WithSize(_fileSize).Build(); - } - - private void CreateMocks() - { - _sftpSessionMock = new Mock(MockBehavior.Strict); - _sftpFileReaderMock = new Mock(MockBehavior.Strict); - } - - private void SetupMocks() - { - var seq = new MockSequence(); - - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginOpen(_fileName, Flags.Read, null, null)) - .Returns(_openAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndOpen(_openAsyncResult)) - .Returns(_handle); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.BeginLStat(_fileName, null, null)) - .Returns(_statAsyncResult); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CalculateOptimalReadLength(_bufferSize)) - .Returns(_chunkSize); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.EndLStat(_statAsyncResult)) - .Returns(_fileAttributes); - _sftpSessionMock.InSequence(seq) - .Setup(p => p.CreateFileReader(_handle, _sftpSessionMock.Object, _chunkSize, 1, _fileSize)) - .Returns(_sftpFileReaderMock.Object); - } - - private void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - - _serviceFactory = new ServiceFactory(); - } - - [TestInitialize] - public void Initialize() - { - Arrange(); - Act(); - } - - private void Act() - { - _actual = _serviceFactory.CreateSftpFileReader(_fileName, _sftpSessionMock.Object, _bufferSize); - } - - [TestMethod] - public void CreateSftpFileReaderShouldReturnCreatedInstance() - { - Assert.IsNotNull(_actual); - Assert.AreSame(_sftpFileReaderMock.Object, _actual); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTestBase.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTestBase.cs deleted file mode 100644 index 1cb1207b6..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTestBase.cs +++ /dev/null @@ -1,63 +0,0 @@ -using System; -using System.Threading; - -using Microsoft.Extensions.Logging.Abstractions; -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - public abstract class SftpFileReaderTestBase - { - internal Mock SftpSessionMock { get; private set; } - - protected abstract void SetupData(); - - protected void CreateMocks() - { - SftpSessionMock = new Mock(MockBehavior.Strict); - SftpSessionMock.Setup(s => s.SessionLoggerFactory).Returns(NullLoggerFactory.Instance); - } - - protected abstract void SetupMocks(); - - protected virtual void Arrange() - { - SetupData(); - CreateMocks(); - SetupMocks(); - } - - [TestInitialize] - public void SetUp() - { - Arrange(); - Act(); - } - - protected abstract void Act(); - - protected static byte[] CreateByteArray(Random random, int length) - { - var chunk = new byte[length]; - random.NextBytes(chunk); - return chunk; - } - - protected static int WaitAny(WaitHandle[] waitHandles, int millisecondsTimeout) - { - var result = WaitHandle.WaitAny(waitHandles, millisecondsTimeout); - - if (result == WaitHandle.WaitTimeout) - { - throw new SshOperationTimeoutException(); - } - - return result; - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_DisposeShouldUnblockReadAndReadAhead.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_DisposeShouldUnblockReadAndReadAhead.cs deleted file mode 100644 index b49de6e34..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_DisposeShouldUnblockReadAndReadAhead.cs +++ /dev/null @@ -1,173 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_DisposeShouldUnblockReadAndReadAhead : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private SftpFileReader _reader; - private ObjectDisposedException _actualException; - private AsyncCallback _readAsyncCallback; - private EventWaitHandle _disposeCompleted; - - [TestCleanup] - public void TearDown() - { - _disposeCompleted?.Dispose(); - } - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _fileSize = 5000; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - _disposeCompleted = new ManualResetEvent(false); - _readAsyncCallback = null; - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Returns((handle, offset, length, callback, state) => - { - _readAsyncCallback = callback; - return null; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.IsOpen) - .Returns(true); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginClose(_handle, null, null)) - .Returns(_closeAsyncResult); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.EndClose(_closeAsyncResult)); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 1, _fileSize); - } - - protected override void Act() - { - ThreadAbstraction.ExecuteThread(() => - { - Thread.Sleep(500); - _reader.Dispose(); - _ = _disposeCompleted.Set(); - }); - - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (ObjectDisposedException ex) - { - _actualException = ex; - } - - // Dispose may unblock Read() before the dispose has fully completed, so - // let's wait until it has completed - _ = _disposeCompleted.WaitOne(500); - } - - [TestMethod] - public void ReadShouldHaveThrownObjectDisposedException() - { - Assert.IsNotNull(_actualException); - Assert.AreEqual(typeof(SftpFileReader).FullName, _actualException.ObjectName); - } - - [TestMethod] - public void ReadAfterDisposeShouldThrowObjectDisposedException() - { - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (ObjectDisposedException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual(typeof(SftpFileReader).FullName, ex.ObjectName); - } - } - - [TestMethod] - public void HandleShouldHaveBeenClosed() - { - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - - [TestMethod] - public void DisposeShouldCompleteImmediatelyAndNotAttemptToCloseHandleAgain() - { - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - - [TestMethod] - public void InvokeOfReadAheadCallbackShouldCompleteImmediately() - { - Assert.IsNotNull(_readAsyncCallback); - - _readAsyncCallback(new SftpReadAsyncResult(null, null)); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsNotOpen.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsNotOpen.cs deleted file mode 100644 index 6b8612ab7..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsNotOpen.cs +++ /dev/null @@ -1,132 +0,0 @@ -using System; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_Dispose_SftpSessionIsNotOpen : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpFileReader _reader; - private AsyncCallback _readAsyncCallback; - private ManualResetEvent _beginReadInvoked; - private EventWaitHandle _disposeCompleted; - - [TestCleanup] - public void TearDown() - { - _beginReadInvoked?.Dispose(); - _disposeCompleted?.Dispose(); - } - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _fileSize = 5000; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _beginReadInvoked = new ManualResetEvent(false); - _disposeCompleted = new ManualResetEvent(false); - _readAsyncCallback = null; - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback(() => - { - // harden test by making sure that we've invoked BeginRead before Dispose is invoked - _ = _beginReadInvoked.Set(); - }) - .Returns((handle, offset, length, callback, state) => - { - _readAsyncCallback = callback; - return null; - }) - .Callback(() => - { - // wait until Dispose has been invoked on reader to allow us to harden test, and - // verify whether Dispose will prevent us from entering the read-ahead loop again - _ = _waitHandleArray[0].WaitOne(); - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.IsOpen) - .Returns(false); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 1, _fileSize); - } - - protected override void Act() - { - Assert.IsTrue(_beginReadInvoked.WaitOne(5000)); - _reader.Dispose(); - } - - [TestMethod] - public void ReadAfterDisposeShouldThrowObjectDisposedException() - { - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (ObjectDisposedException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual(typeof(SftpFileReader).FullName, ex.ObjectName); - } - } - - [TestMethod] - public void InvokeOfReadAheadCallbackShouldCompleteImmediately() - { - Assert.IsNotNull(_readAsyncCallback); - - _readAsyncCallback(new SftpReadAsyncResult(null, null)); - } - - [TestMethod] - public void BeginCloseOnSftpSessionShouldNeverHaveBeenInvoked() - { - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Never); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_BeginCloseThrowsException.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_BeginCloseThrowsException.cs deleted file mode 100644 index a678dfffb..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_BeginCloseThrowsException.cs +++ /dev/null @@ -1,136 +0,0 @@ -using System; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_Dispose_SftpSessionIsOpen_BeginCloseThrowsException : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpFileReader _reader; - private AsyncCallback _readAsyncCallback; - private ManualResetEvent _beginReadInvoked; - private EventWaitHandle _disposeCompleted; - - [TestCleanup] - public void TearDown() - { - _beginReadInvoked?.Dispose(); - _disposeCompleted?.Dispose(); - } - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _fileSize = 5000; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _beginReadInvoked = new ManualResetEvent(false); - _disposeCompleted = new ManualResetEvent(false); - _readAsyncCallback = null; - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback(() => - { - // harden test by making sure that we've invoked BeginRead before Dispose is invoked - _ = _beginReadInvoked.Set(); - }) - .Returns((handle, offset, length, callback, state) => - { - _readAsyncCallback = callback; - return null; - }) - .Callback(() => - { - // wait until Dispose has been invoked on reader to allow us to harden test, and - // verify whether Dispose will prevent us from entering the read-ahead loop again - _ = _waitHandleArray[0].WaitOne(); - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.IsOpen) - .Returns(true); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginClose(_handle, null, null)) - .Throws(new SshException()); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 1, _fileSize); - } - - protected override void Act() - { - Assert.IsTrue(_beginReadInvoked.WaitOne(5000)); - _reader.Dispose(); - } - - [TestMethod] - public void ReadAfterDisposeShouldThrowObjectDisposedException() - { - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (ObjectDisposedException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual(typeof(SftpFileReader).FullName, ex.ObjectName); - } - } - - [TestMethod] - public void InvokeOfReadAheadCallbackShouldCompleteImmediately() - { - Assert.IsNotNull(_readAsyncCallback); - - _readAsyncCallback(new SftpReadAsyncResult(null, null)); - } - - [TestMethod] - public void BeginCloseOnSftpSessionShouldHaveBeenInvokedOnce() - { - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_EndCloseThrowsException.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_EndCloseThrowsException.cs deleted file mode 100644 index d2585e2bf..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Dispose_SftpSessionIsOpen_EndCloseThrowsException.cs +++ /dev/null @@ -1,141 +0,0 @@ -using System; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_Dispose_SftpSessionIsOpen_EndCloseThrowsException : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private SftpFileReader _reader; - private AsyncCallback _readAsyncCallback; - private ManualResetEvent _beginReadInvoked; - private EventWaitHandle _disposeCompleted; - - [TestCleanup] - public void TearDown() - { - _beginReadInvoked?.Dispose(); - _disposeCompleted?.Dispose(); - } - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _fileSize = 5000; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - _beginReadInvoked = new ManualResetEvent(false); - _disposeCompleted = new ManualResetEvent(false); - _readAsyncCallback = null; - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback(() => - { - // harden test by making sure that we've invoked BeginRead before Dispose is invoked - _ = _beginReadInvoked.Set(); - }) - .Returns((handle, offset, length, callback, state) => - { - _readAsyncCallback = callback; - return null; - }) - .Callback(() => - { - // wait until Dispose has been invoked on reader to allow us to harden test, and - // verify whether Dispose will prevent us from entering the read-ahead loop again - _ = _waitHandleArray[0].WaitOne(); - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.IsOpen) - .Returns(true); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginClose(_handle, null, null)) - .Returns(_closeAsyncResult); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.EndClose(_closeAsyncResult)) - .Throws(new SshException()); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 1, _fileSize); - } - - protected override void Act() - { - Assert.IsTrue(_beginReadInvoked.WaitOne(5000)); - _reader.Dispose(); - } - - [TestMethod] - public void ReadAfterDisposeShouldThrowObjectDisposedException() - { - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (ObjectDisposedException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual(typeof(SftpFileReader).FullName, ex.ObjectName); - } - } - - [TestMethod] - public void InvokeOfReadAheadCallbackShouldCompleteImmediately() - { - Assert.IsNotNull(_readAsyncCallback); - - _readAsyncCallback(new SftpReadAsyncResult(null, null)); - } - - [TestMethod] - public void EndCloseOnSftpSessionShouldHaveBeenInvokedOnce() - { - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsComplete.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsComplete.cs deleted file mode 100644 index 7acd13fa0..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsComplete.cs +++ /dev/null @@ -1,168 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_LastChunkBeforeEofIsComplete : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpReadAsyncResult _readAsyncResultBeyondEof; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private byte[] _chunk2; - private byte[] _chunk3; - private SftpFileReader _reader; - private byte[] _actualChunk1; - private byte[] _actualChunk2; - private byte[] _actualChunk3; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _chunk1 = CreateByteArray(random, ChunkLength); - // chunk is less than the requested length, but - together with chunk 1 - contains all data up to the EOF - _chunk2 = CreateByteArray(random, ChunkLength - 10); - _chunk3 = new byte[0]; - _fileSize = _chunk1.Length + _chunk2.Length; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - _readAsyncResultBeyondEof = new SftpReadAsyncResult(null, null); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk2, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, null, It.IsAny())) - .Returns(_readAsyncResultBeyondEof); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.EndRead(_readAsyncResultBeyondEof)) - .Returns(_chunk3); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 15, _fileSize); - } - - protected override void Act() - { - _actualChunk1 = _reader.Read(); - _actualChunk2 = _reader.Read(); - _actualChunk3 = _reader.Read(); - } - - [TestMethod] - public void FirstReadShouldReturnChunk1() - { - Assert.IsNotNull(_actualChunk1); - Assert.AreSame(_chunk1, _actualChunk1); - } - - [TestMethod] - public void SecondReadShouldReturnChunk2() - { - Assert.IsNotNull(_actualChunk2); - Assert.AreSame(_chunk2, _actualChunk2); - } - - [TestMethod] - public void ThirdReadShouldReturnChunk3() - { - Assert.IsNotNull(_actualChunk3); - Assert.AreSame(_chunk3, _actualChunk3); - } - - [TestMethod] - public void ReadAfterEndOfFileShouldThrowSshException() - { - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual("Attempting to read beyond the end of the file.", ex.Message); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - SftpSessionMock.InSequence(_seq).Setup(p => p.IsOpen).Returns(true); - SftpSessionMock.InSequence(_seq).Setup(p => p.BeginClose(_handle, null, null)).Returns(_closeAsyncResult); - SftpSessionMock.InSequence(_seq).Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsPartial.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsPartial.cs deleted file mode 100644 index 2c42ee226..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_LastChunkBeforeEofIsPartial.cs +++ /dev/null @@ -1,167 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_LastChunkBeforeEofIsPartial : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private byte[] _chunk2; - private byte[] _chunk3; - private SftpFileReader _reader; - private byte[] _actualChunk1; - private byte[] _actualChunk2; - private byte[] _actualChunk3; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _chunk1 = CreateByteArray(random, ChunkLength); - _chunk2 = CreateByteArray(random, ChunkLength); - _chunk3 = new byte[0]; - _fileSize = _chunk1.Length + _chunk2.Length; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk2, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk3, false); - }) - .Returns((SftpReadAsyncResult)null); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 15, _fileSize); - } - - protected override void Act() - { - _actualChunk1 = _reader.Read(); - _actualChunk2 = _reader.Read(); - _actualChunk3 = _reader.Read(); - } - - [TestMethod] - public void FirstReadShouldReturnChunk1() - { - Assert.IsNotNull(_actualChunk1); - Assert.AreSame(_chunk1, _actualChunk1); - } - - [TestMethod] - public void SecondReadShouldReturnChunk2() - { - Assert.IsNotNull(_actualChunk2); - Assert.AreSame(_chunk2, _actualChunk2); - } - - [TestMethod] - public void ThirdReadShouldReturnChunk3() - { - Assert.IsNotNull(_actualChunk3); - Assert.AreSame(_chunk3, _actualChunk3); - } - - [TestMethod] - public void ReadAfterEndOfFileShouldThrowSshException() - { - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual("Attempting to read beyond the end of the file.", ex.Message); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - SftpSessionMock.InSequence(_seq).Setup(p => p.IsOpen).Returns(true); - SftpSessionMock.InSequence(_seq).Setup(p => p.BeginClose(_handle, null, null)).Returns(_closeAsyncResult); - SftpSessionMock.InSequence(_seq).Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsNotReached.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsNotReached.cs deleted file mode 100644 index b364f60dd..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsNotReached.cs +++ /dev/null @@ -1,323 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsNotReached : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private byte[] _chunk2; - private byte[] _chunk2CatchUp1; - private byte[] _chunk2CatchUp2; - private byte[] _chunk3; - private byte[] _chunk4; - private byte[] _chunk5; - private SftpFileReader _reader; - private byte[] _actualChunk1; - private byte[] _actualChunk2; - private byte[] _actualChunk3; - private ManualResetEvent _chunk1BeginRead; - private ManualResetEvent _chunk2BeginRead; - private ManualResetEvent _chunk3BeginRead; - private ManualResetEvent _chunk4BeginRead; - private ManualResetEvent _chunk5BeginRead; - private ManualResetEvent _waitBeforeChunk6; - private ManualResetEvent _chunk6BeginRead; - private byte[] _actualChunk4; - private byte[] _actualChunk2CatchUp1; - private byte[] _actualChunk2CatchUp2; - private byte[] _chunk6; - private byte[] _actualChunk5; - private byte[] _actualChunk6; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 3); - _chunk1 = CreateByteArray(random, ChunkLength); - _chunk2 = CreateByteArray(random, ChunkLength - 17); - _chunk2CatchUp1 = CreateByteArray(random, 10); - _chunk2CatchUp2 = CreateByteArray(random, 7); - _chunk3 = CreateByteArray(random, ChunkLength); - _chunk4 = CreateByteArray(random, ChunkLength); - _chunk5 = CreateByteArray(random, ChunkLength); - _chunk6 = new byte[0]; - _chunk1BeginRead = new ManualResetEvent(false); - _chunk2BeginRead = new ManualResetEvent(false); - _chunk3BeginRead = new ManualResetEvent(false); - _chunk4BeginRead = new ManualResetEvent(false); - _chunk5BeginRead = new ManualResetEvent(false); - _waitBeforeChunk6 = new ManualResetEvent(false); - _chunk6BeginRead = new ManualResetEvent(false); - _fileSize = _chunk1.Length + _chunk2.Length + _chunk2CatchUp1.Length + _chunk2CatchUp2.Length + _chunk3.Length + _chunk4.Length + _chunk5.Length; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk1BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk2BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk2, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk3BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk3, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 3 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk4BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk4, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 4 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk5BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk5, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Callback(() => _waitBeforeChunk6.Set()) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.RequestRead(_handle, (2 * ChunkLength) - 17, 17)) - .Returns(_chunk2CatchUp1); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.RequestRead(_handle, (2 * ChunkLength) - 7, 7)) - .Returns(_chunk2CatchUp2); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 5 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk6BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk6, false); - }) - .Returns((SftpReadAsyncResult)null); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 3, _fileSize); - } - - protected override void Act() - { - // reader is configured to read-ahead max. 3 chunks, so chunk4 should not have been read - Assert.IsFalse(_chunk4BeginRead.WaitOne(0)); - // consume chunk 1 - _actualChunk1 = _reader.Read(); - // consuming chunk1 allows chunk4 to be read-ahead - Assert.IsTrue(_chunk4BeginRead.WaitOne(200)); - // verify that chunk5 has not yet been read-ahead - Assert.IsFalse(_chunk5BeginRead.WaitOne(0)); - // consume chunk 2 - _actualChunk2 = _reader.Read(); - // consuming chunk2 allows chunk5 to be read-ahead - Assert.IsTrue(_chunk5BeginRead.WaitOne(200)); - // pauze until the read-ahead has started waiting a semaphore to become available - Assert.IsTrue(_waitBeforeChunk6.WaitOne(200)); - // consume remaining parts of chunk 2 - _actualChunk2CatchUp1 = _reader.Read(); - _actualChunk2CatchUp2 = _reader.Read(); - // verify that chunk6 has not yet been read-ahead - Assert.IsFalse(_chunk6BeginRead.WaitOne(0)); - // consume chunk 3 - _actualChunk3 = _reader.Read(); - // consuming chunk3 allows chunk6 to be read-ahead - Assert.IsTrue(_chunk6BeginRead.WaitOne(200)); - // consume chunk 4 - _actualChunk4 = _reader.Read(); - // consume chunk 5 - _actualChunk5 = _reader.Read(); - // consume chunk 6 - _actualChunk6 = _reader.Read(); - } - - [TestMethod] - public void FirstReadShouldReturnChunk1() - { - Assert.IsNotNull(_actualChunk1); - Assert.AreSame(_chunk1, _actualChunk1); - } - - [TestMethod] - public void SecondReadShouldReturnChunk2() - { - Assert.IsNotNull(_actualChunk2); - Assert.AreSame(_chunk2, _actualChunk2); - } - - [TestMethod] - public void ThirdReadShouldReturnChunk2CatchUp1() - { - Assert.IsNotNull(_actualChunk2CatchUp1); - Assert.AreSame(_chunk2CatchUp1, _actualChunk2CatchUp1); - } - - [TestMethod] - public void FourthReadShouldReturnChunk2CatchUp2() - { - Assert.IsNotNull(_actualChunk2CatchUp2); - Assert.AreSame(_chunk2CatchUp2, _actualChunk2CatchUp2); - } - - [TestMethod] - public void FifthReadShouldReturnChunk3() - { - Assert.IsNotNull(_actualChunk3); - Assert.AreSame(_chunk3, _actualChunk3); - } - - [TestMethod] - public void SixthReadShouldReturnChunk4() - { - Assert.IsNotNull(_actualChunk4); - Assert.AreSame(_chunk4, _actualChunk4); - } - - [TestMethod] - public void SeventhReadShouldReturnChunk5() - { - Assert.IsNotNull(_actualChunk5); - Assert.AreSame(_chunk5, _actualChunk5); - } - - [TestMethod] - public void EightReadShouldReturnChunk6() - { - Assert.IsNotNull(_actualChunk6); - Assert.AreSame(_chunk6, _actualChunk6); - } - - [TestMethod] - public void ReadAfterEndOfFileShouldThrowSshException() - { - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual("Attempting to read beyond the end of the file.", ex.Message); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.IsOpen) - .Returns(true); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginClose(_handle, null, null)) - .Returns(_closeAsyncResult); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsReached.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsReached.cs deleted file mode 100644 index 5d1f69121..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsReached.cs +++ /dev/null @@ -1,207 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_PreviousChunkIsIncompleteAndEofIsReached : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private byte[] _chunk2; - private byte[] _chunk2CatchUp; - private byte[] _chunk3; - private SftpFileReader _reader; - private byte[] _actualChunk1; - private byte[] _actualChunk2; - private byte[] _actualChunk2CatchUp; - private byte[] _actualChunk3; - private ManualResetEvent _chunk1BeginRead; - private ManualResetEvent _chunk2BeginRead; - private ManualResetEvent _chunk3BeginRead; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 3); - _chunk1 = CreateByteArray(random, ChunkLength); - _chunk2 = CreateByteArray(random, ChunkLength - 10); - _chunk2CatchUp = CreateByteArray(random, 10); - _chunk3 = new byte[0]; - _chunk1BeginRead = new ManualResetEvent(false); - _chunk2BeginRead = new ManualResetEvent(false); - _chunk3BeginRead = new ManualResetEvent(false); - _fileSize = _chunk1.Length + _chunk2.Length + _chunk2CatchUp.Length + _chunk3.Length; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk1BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk2BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk2, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _ = _chunk3BeginRead.Set(); - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk3, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.RequestRead(_handle, (2 * ChunkLength) - 10, 10)) - .Returns(_chunk2CatchUp); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 5, _fileSize); - } - - protected override void Act() - { - // consume chunk 1 - _actualChunk1 = _reader.Read(); - // consume chunk 2 - _actualChunk2 = _reader.Read(); - // wait until chunk3 has been read-ahead - Assert.IsTrue(_chunk3BeginRead.WaitOne(200)); - // consume remaining parts of chunk 2 - _actualChunk2CatchUp = _reader.Read(); - // consume chunk 3 - _actualChunk3 = _reader.Read(); - } - - [TestMethod] - public void FirstReadShouldReturnChunk1() - { - Assert.IsNotNull(_actualChunk1); - Assert.AreSame(_chunk1, _actualChunk1); - } - - [TestMethod] - public void SecondReadShouldReturnChunk2() - { - Assert.IsNotNull(_actualChunk2); - Assert.AreSame(_chunk2, _actualChunk2); - } - - [TestMethod] - public void ThirdReadShouldReturnChunk2CatchUp() - { - Assert.IsNotNull(_actualChunk2CatchUp); - Assert.AreSame(_chunk2CatchUp, _actualChunk2CatchUp); - } - - [TestMethod] - public void FourthReadShouldReturnChunk3() - { - Assert.IsNotNull(_actualChunk3); - Assert.AreSame(_chunk3, _actualChunk3); - } - - [TestMethod] - public void ReadAfterEndOfFileShouldThrowSshException() - { - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.IsNull(ex.InnerException); - Assert.AreEqual("Attempting to read beyond the end of the file.", ex.Message); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.IsOpen) - .Returns(true); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginClose(_handle, null, null)) - .Returns(_closeAsyncResult); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadBeginReadException.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadBeginReadException.cs deleted file mode 100644 index 9707a50ba..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadBeginReadException.cs +++ /dev/null @@ -1,6 +0,0 @@ -namespace Renci.SshNet.Tests.Classes.Sftp -{ - class SftpFileReaderTest_ReadAheadBeginReadException - { - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_DiscardsFurtherReadAheads.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_DiscardsFurtherReadAheads.cs deleted file mode 100644 index 6026e94d0..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_DiscardsFurtherReadAheads.cs +++ /dev/null @@ -1,213 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - /// - /// Runs a reader with max. 2 pending reads. - /// The read-ahead of chunk1 starts followed by the read-ahead of chunk2. - /// The read-ahead of chunk1 completes successfully and the resulting chunk is read. - /// The read of this first chunk allows a third ahead-head to start. - /// The second read-ahead uses signals to forcefully block a failure completion until the read - /// ahead of the third chunk has completed and the semaphore is waiting for a slot to start - /// the read-ahead of chunk4. - /// The second read does not consume check3 as it is out of order, but instead waits for - /// the outcome of the read-ahead of chunk2. - /// - /// The completion with exception of chunk2 causes the second read to throw that same exception, and - /// signals the semaphore that was waiting to start the read-ahead of chunk4. However, due to the fact - /// that chunk2 completed with an exception, the read-ahead loop is stopped. - /// - [TestClass] - public class SftpFileReaderTest_ReadAheadEndInvokeException_DiscardsFurtherReadAheads : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private byte[] _chunk3; - private ManualResetEvent _readAheadChunk2Completed; - private ManualResetEvent _readAheadChunk3Completed; - private ManualResetEvent _waitingForSemaphoreAfterCompletingChunk3; - private SftpFileReader _reader; - private SshException _exception; - private SshException _actualException; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _chunk1 = CreateByteArray(random, ChunkLength); - _chunk3 = CreateByteArray(random, ChunkLength); - _fileSize = 4 * ChunkLength; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - - _readAheadChunk2Completed = new ManualResetEvent(false); - _readAheadChunk3Completed = new ManualResetEvent(false); - _waitingForSemaphoreAfterCompletingChunk3 = new ManualResetEvent(false); - - _exception = new SshException(); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - ThreadAbstraction.ExecuteThread(() => - { - // wait until the read-ahead for chunk3 has completed; this should allow - // the read-ahead of chunk4 to start - _readAheadChunk3Completed.WaitOne(TimeSpan.FromSeconds(3)); - // wait until the semaphore wait to start with chunk4 has started - _waitingForSemaphoreAfterCompletingChunk3.WaitOne(TimeSpan.FromSeconds(7)); - // complete async read of chunk2 with exception - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_exception, false); - // signal that read-ahead of chunk 2 has completed - _readAheadChunk2Completed.Set(); - }); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk3, false); - // signal that we've completed the read-ahead for chunk3 - _readAheadChunk3Completed.Set(); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Callback(() => _waitingForSemaphoreAfterCompletingChunk3.Set()) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 2, _fileSize); - } - - protected override void Act() - { - _reader.Read(); - - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - _actualException = ex; - } - } - - [TestMethod] - public void ReadOfSecondChunkShouldThrowExceptionThatOccurredInReadAhead() - { - Assert.IsNotNull(_actualException); - Assert.AreSame(_exception, _actualException); - } - - [TestMethod] - public void ReahAheadOfChunk3ShouldHaveStarted() - { - SftpSessionMock.Verify(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny()), Times.Once); - } - - [TestMethod] - public void ReadAfterReadAheadExceptionShouldRethrowExceptionThatOccurredInReadAhead() - { - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.AreSame(_exception, ex); - } - } - - [TestMethod] - public void WaitAnyOFSftpSessionShouldHaveBeenInvokedFourTimes() - { - SftpSessionMock.Verify(p => p.WaitAny(_waitHandleArray, _operationTimeout), Times.Exactly(4)); - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - SftpSessionMock.InSequence(_seq).Setup(p => p.IsOpen).Returns(true); - SftpSessionMock.InSequence(_seq).Setup(p => p.BeginClose(_handle, null, null)).Returns(_closeAsyncResult); - SftpSessionMock.InSequence(_seq).Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_PreventsFurtherReadAheads.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_PreventsFurtherReadAheads.cs deleted file mode 100644 index 881b52296..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadAheadEndInvokeException_PreventsFurtherReadAheads.cs +++ /dev/null @@ -1,189 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Abstractions; -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_ReadAheadEndInvokeException_PreventsFurtherReadAheads : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private SftpFileReader _reader; - private ManualResetEvent _readAheadChunk2; - private ManualResetEvent _readChunk2; - private SshException _exception; - private SshException _actualException; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _chunk1 = CreateByteArray(random, ChunkLength); - _fileSize = 3 * _chunk1.Length; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - - _readAheadChunk2 = new ManualResetEvent(false); - _readChunk2 = new ManualResetEvent(false); - - _exception = new SshException(); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - ThreadAbstraction.ExecuteThread(() => - { - // signal that we're in the read-ahead for chunk2 - _ = _readAheadChunk2.Set(); - // wait for client to start reading this chunk - _ = _readChunk2.WaitOne(TimeSpan.FromSeconds(5)); - // sleep a short time to make sure the client is in the blocking wait - Thread.Sleep(500); - // complete async read of chunk2 with exception - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_exception, false); - }); - }) - .Returns((SftpReadAsyncResult)null); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.OperationTimeout) - .Returns(_operationTimeout); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - } - - protected override void Arrange() - { - base.Arrange(); - - // use a max. read-ahead of 1 to allow us to verify that the next read-ahead is not done - // when a read-ahead has failed - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 1, _fileSize); - } - - protected override void Act() - { - _ = _reader.Read(); - - // wait until SftpFileReader has starting reading ahead chunk 2 - Assert.IsTrue(_readAheadChunk2.WaitOne(TimeSpan.FromSeconds(5))); - // signal that we are about to read chunk 2 - _ = _readChunk2.Set(); - - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - _actualException = ex; - } - } - - [TestMethod] - public void ReadOfSecondChunkShouldThrowExceptionThatOccurredInReadAhead() - { - Assert.IsNotNull(_actualException); - Assert.AreSame(_exception, _actualException); - } - - [TestMethod] - public void ReadAfterReadAheadExceptionShouldRethrowExceptionThatOccurredInReadAhead() - { - try - { - _ = _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.AreSame(_exception, ex); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.IsOpen) - .Returns(true); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginClose(_handle, null, null)) - .Returns(_closeAsyncResult); - _ = SftpSessionMock.InSequence(_seq) - .Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - - [TestMethod] - public void ExceptionInReadAheadShouldPreventFurtherReadAheads() - { - SftpSessionMock.Verify(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny()), Times.Never); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackBeginReadException.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackBeginReadException.cs deleted file mode 100644 index a89d4c977..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackBeginReadException.cs +++ /dev/null @@ -1,6 +0,0 @@ -namespace Renci.SshNet.Tests.Classes.Sftp -{ - class SftpFileReaderTest_ReadBackBeginReadException - { - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackEndInvokeException.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackEndInvokeException.cs deleted file mode 100644 index 324dbd3d4..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_ReadBackEndInvokeException.cs +++ /dev/null @@ -1,6 +0,0 @@ -namespace Renci.SshNet.Tests.Classes.Sftp -{ - class SftpFileReaderTest_ReadBackEndInvokeException - { - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInBeginRead.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInBeginRead.cs deleted file mode 100644 index c7022a548..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInBeginRead.cs +++ /dev/null @@ -1,172 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_Read_ReadAheadExceptionInBeginRead : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private byte[] _chunk2; - private SftpFileReader _reader; - private ManualResetEvent _readAheadChunk3; - private ManualResetEvent _readChunk3; - private SshException _exception; - private SshException _actualException; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _chunk1 = CreateByteArray(random, ChunkLength); - _chunk2 = CreateByteArray(random, ChunkLength); - _fileSize = _chunk1.Length + _chunk2.Length + 1; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - - _readAheadChunk3 = new ManualResetEvent(false); - _readChunk3 = new ManualResetEvent(false); - - _exception = new SshException(); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk2, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 2 * ChunkLength, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - _readAheadChunk3.Set(); - _readChunk3.WaitOne(TimeSpan.FromSeconds(5)); - // sleep a short time to make sure the client is in the blocking wait - Thread.Sleep(500); - }) - .Throws(_exception); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 3, _fileSize); - } - - protected override void Act() - { - _reader.Read(); - _reader.Read(); - - // wait until we've the SftpFileReader has starting reading ahead chunk 3 - Assert.IsTrue(_readAheadChunk3.WaitOne(TimeSpan.FromSeconds(5))); - // signal that we are about to read chunk 3 - _readChunk3.Set(); - - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - _actualException = ex; - } - } - - [TestMethod] - public void ReadOfThirdChunkShouldThrowExceptionThatOccurredInReadAhead() - { - Assert.IsNotNull(_actualException); - Assert.AreSame(_exception, _actualException); - } - - [TestMethod] - public void ReadAfterReadAheadExceptionShouldRethrowExceptionThatOccurredInReadAhead() - { - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.AreSame(_exception, ex); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - SftpSessionMock.InSequence(_seq).Setup(p => p.IsOpen).Returns(true); - SftpSessionMock.InSequence(_seq).Setup(p => p.BeginClose(_handle, null, null)).Returns(_closeAsyncResult); - SftpSessionMock.InSequence(_seq).Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_ChunkAvailable.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_ChunkAvailable.cs deleted file mode 100644 index 45ceb7079..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_ChunkAvailable.cs +++ /dev/null @@ -1,144 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_ChunkAvailable : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private byte[] _chunk1; - private byte[] _chunk2; - private SftpFileReader _reader; - private SshException _exception; - private ManualResetEvent _exceptionSignaled; - private SshException _actualException; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _chunk1 = CreateByteArray(random, ChunkLength); - _chunk2 = CreateByteArray(random, ChunkLength); - _fileSize = _chunk1.Length + _chunk2.Length + 1; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - - _exception = new SshException(); - _exceptionSignaled = new ManualResetEvent(false); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Callback((handle, offset, length, callback, state) => - { - var asyncResult = new SftpReadAsyncResult(callback, state); - asyncResult.SetAsCompleted(_chunk1, false); - }) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Callback(() => _exceptionSignaled.Set()) - .Throws(_exception); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 2, _fileSize); - } - - protected override void Act() - { - // wait for the exception to be signaled by the second call to WaitAny - _exceptionSignaled.WaitOne(5000); - // allow a little time to allow SftpFileReader to process exception - Thread.Sleep(100); - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - _actualException = ex; - } - } - - [TestMethod] - public void ReadShouldHaveRethrownExceptionThrownByWaitAny() - { - Assert.IsNotNull(_actualException); - Assert.AreSame(_exception, _actualException); - } - - [TestMethod] - public void ReadShouldRethrowExceptionThrownByWaitAny() - { - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.AreSame(_exception, ex); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - SftpSessionMock.InSequence(_seq).Setup(p => p.IsOpen).Returns(true); - SftpSessionMock.InSequence(_seq).Setup(p => p.BeginClose(_handle, null, null)).Returns(_closeAsyncResult); - SftpSessionMock.InSequence(_seq).Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_NoChunkAvailable.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_NoChunkAvailable.cs deleted file mode 100644 index 3456a5f94..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_NoChunkAvailable.cs +++ /dev/null @@ -1,128 +0,0 @@ -using System; -using System.Diagnostics; -using System.Threading; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Common; -using Renci.SshNet.Sftp; - -using BufferedRead = Renci.SshNet.Sftp.SftpFileReader.BufferedRead; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - [TestClass] - public class SftpFileReaderTest_Read_ReadAheadExceptionInWaitOnHandle_NoChunkAvailable : SftpFileReaderTestBase - { - private const int ChunkLength = 32 * 1024; - - private MockSequence _seq; - private byte[] _handle; - private int _fileSize; - private WaitHandle[] _waitHandleArray; - private int _operationTimeout; - private SftpCloseAsyncResult _closeAsyncResult; - private SftpFileReader _reader; - private SshException _exception; - private SshException _actualException; - - protected override void SetupData() - { - var random = new Random(); - - _handle = CreateByteArray(random, 5); - _fileSize = 1234; - _waitHandleArray = new WaitHandle[2]; - _operationTimeout = random.Next(10000, 20000); - _closeAsyncResult = new SftpCloseAsyncResult(null, null); - - _exception = new SshException(); - } - - protected override void SetupMocks() - { - _seq = new MockSequence(); - - SftpSessionMock.InSequence(_seq) - .Setup(p => p.CreateWaitHandleArray(It.IsNotNull(), It.IsNotNull())) - .Returns((disposingWaitHandle, semaphoreAvailableWaitHandle) => - { - _waitHandleArray[0] = disposingWaitHandle; - _waitHandleArray[1] = semaphoreAvailableWaitHandle; - return _waitHandleArray; - }); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Returns(() => WaitAny(_waitHandleArray, _operationTimeout)); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.BeginRead(_handle, 0, ChunkLength, It.IsNotNull(), It.IsAny())) - .Returns((SftpReadAsyncResult)null); - SftpSessionMock.InSequence(_seq).Setup(p => p.OperationTimeout).Returns(_operationTimeout); - SftpSessionMock.InSequence(_seq) - .Setup(p => p.WaitAny(_waitHandleArray, _operationTimeout)) - .Throws(_exception); - } - - protected override void Arrange() - { - base.Arrange(); - - _reader = new SftpFileReader(_handle, SftpSessionMock.Object, ChunkLength, 1, _fileSize); - } - - protected override void Act() - { - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - _actualException = ex; - } - } - - [TestMethod] - public void ReadShouldHaveRethrownExceptionThrownByWaitOnHandle() - { - Assert.IsNotNull(_actualException); - Assert.AreSame(_exception, _actualException); - } - - [TestMethod] - public void ReadShouldRethrowExceptionThrownByWaitOnHandle() - { - try - { - _reader.Read(); - Assert.Fail(); - } - catch (SshException ex) - { - Assert.AreSame(_exception, ex); - } - } - - [TestMethod] - public void DisposeShouldCloseHandleAndCompleteImmediately() - { - SftpSessionMock.InSequence(_seq).Setup(p => p.IsOpen).Returns(true); - SftpSessionMock.InSequence(_seq).Setup(p => p.BeginClose(_handle, null, null)).Returns(_closeAsyncResult); - SftpSessionMock.InSequence(_seq).Setup(p => p.EndClose(_closeAsyncResult)); - - var stopwatch = Stopwatch.StartNew(); - _reader.Dispose(); - stopwatch.Stop(); - - Assert.IsTrue(stopwatch.ElapsedMilliseconds < 200, "Dispose took too long to complete: " + stopwatch.ElapsedMilliseconds); - - SftpSessionMock.Verify(p => p.IsOpen, Times.Once); - SftpSessionMock.Verify(p => p.BeginClose(_handle, null, null), Times.Once); - SftpSessionMock.Verify(p => p.EndClose(_closeAsyncResult), Times.Once); - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamAsyncTestBase.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamAsyncTestBase.cs deleted file mode 100644 index 950b69ec7..000000000 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamAsyncTestBase.cs +++ /dev/null @@ -1,70 +0,0 @@ -using System; -using System.Threading.Tasks; - -using Microsoft.VisualStudio.TestTools.UnitTesting; - -using Moq; - -using Renci.SshNet.Sftp; - -namespace Renci.SshNet.Tests.Classes.Sftp -{ - public abstract class SftpFileStreamAsyncTestBase - { - internal Mock SftpSessionMock; - protected MockSequence MockSequence; - - protected virtual Task ArrangeAsync() - { - SetupData(); - CreateMocks(); - SetupMocks(); - return Task.CompletedTask; - } - - protected virtual void SetupData() - { - MockSequence = new MockSequence(); - } - - protected abstract void SetupMocks(); - - private void CreateMocks() - { - SftpSessionMock = new Mock(MockBehavior.Strict); - } - - [TestInitialize] - public async Task SetUpAsync() - { - await ArrangeAsync(); - await ActAsync(); - } - - protected abstract Task ActAsync(); - - protected byte[] GenerateRandom(int length) - { - return GenerateRandom(length, new Random()); - } - - protected byte[] GenerateRandom(int length, Random random) - { - var buffer = new byte[length]; - random.NextBytes(buffer); - return buffer; - } - - protected byte[] GenerateRandom(uint length) - { - return GenerateRandom(length, new Random()); - } - - protected byte[] GenerateRandom(uint length, Random random) - { - var buffer = new byte[length]; - random.NextBytes(buffer); - return buffer; - } - } -} diff --git a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamTest.cs b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamTest.cs index 7b83f4ee0..603aabc7d 100644 --- a/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamTest.cs +++ b/test/Renci.SshNet.Tests/Classes/Sftp/SftpFileStreamTest.cs @@ -4,6 +4,7 @@ using System.Threading; using System.Threading.Tasks; +using Microsoft.Extensions.Logging.Abstractions; using Microsoft.VisualStudio.TestTools.UnitTesting; using Moq; @@ -85,7 +86,7 @@ public async Task InvalidModeAccessCombination_ThrowsArgumentException(FileMode SftpFileStream.Open(new Mock().Object, "file.txt", mode, access, bufferSize: 1024)); } - Assert.AreEqual("mode", ex.ParamName); + Assert.AreEqual("access", ex.ParamName); } [TestMethod] @@ -93,6 +94,7 @@ public void ReadWithWriteAccess_ThrowsNotSupportedException() { var sessionMock = new Mock(); + sessionMock.Setup(s => s.CalculateOptimalWriteLength(It.IsAny(), It.IsAny())).Returns((x, _) => x); sessionMock.Setup(s => s.IsOpen).Returns(true); SetupRemoteSize(sessionMock, 128); @@ -118,6 +120,7 @@ public void WriteWithReadAccess_ThrowsNotSupportedException() { var sessionMock = new Mock(); + sessionMock.Setup(s => s.CalculateOptimalWriteLength(It.IsAny(), It.IsAny())).Returns((x, _) => x); sessionMock.Setup(s => s.IsOpen).Returns(true); var s = SftpFileStream.Open(sessionMock.Object, "file.txt", FileMode.Open, FileAccess.Read, bufferSize: 1024); @@ -135,7 +138,6 @@ public void WriteWithReadAccess_ThrowsNotSupportedException() Assert.Throws(() => s.SetLength(1024)); } - [Ignore("TODO Currently throws EndOfStreamException in all cases.")] [TestMethod] [DataRow(-1, SeekOrigin.Begin)] [DataRow(-1, SeekOrigin.Current)] @@ -144,6 +146,8 @@ public void SeekBeforeBeginning_ThrowsIOException(long offset, SeekOrigin origin { var sessionMock = new Mock(); + sessionMock.Setup(s => s.CalculateOptimalReadLength(It.IsAny())).Returns(x => x); + sessionMock.Setup(s => s.CalculateOptimalWriteLength(It.IsAny(), It.IsAny())).Returns((x, _) => x); sessionMock.Setup(s => s.IsOpen).Returns(true); SetupRemoteSize(sessionMock, 128); @@ -155,7 +159,7 @@ public void SeekBeforeBeginning_ThrowsIOException(long offset, SeekOrigin origin private static void SetupRemoteSize(Mock sessionMock, long size) { - sessionMock.Setup(s => s.RequestFStat(It.IsAny(), It.IsAny())).Returns(new SftpFileAttributes( + sessionMock.Setup(s => s.RequestFStat(It.IsAny())).Returns(new SftpFileAttributes( default, default, size: size, default, default, default, default )); } @@ -210,6 +214,7 @@ private void TestSendsBufferedWrites(Action flushAction) { var sessionMock = new Mock(); + sessionMock.Setup(s => s.CalculateOptimalReadLength(It.IsAny())).Returns(x => x); sessionMock.Setup(s => s.CalculateOptimalWriteLength(It.IsAny(), It.IsAny())).Returns((x, _) => x); sessionMock.Setup(s => s.IsOpen).Returns(true); SetupRemoteSize(sessionMock, 0); @@ -246,12 +251,12 @@ public void Dispose() { var sessionMock = new Mock(); + sessionMock.Setup(s => s.CalculateOptimalWriteLength(It.IsAny(), It.IsAny())).Returns((x, _) => x); sessionMock.Setup(s => s.IsOpen).Returns(true); var s = SftpFileStream.Open(sessionMock.Object, "file.txt", FileMode.Create, FileAccess.ReadWrite, bufferSize: 1024); Assert.IsTrue(s.CanRead); - Assert.IsTrue(s.CanSeek); Assert.IsTrue(s.CanWrite); s.Dispose(); @@ -276,6 +281,46 @@ public void Dispose() sessionMock.Verify(p => p.RequestClose(It.IsAny()), Times.Once); } + [TestMethod] + public void FstatFailure_DisablesSeek() + { + TestFstatFailure(fstat => fstat.Throws()); + } + + [TestMethod] + public void FstatSizeNotReturned_DisablesSeek() + { + TestFstatFailure(fstat => fstat.Returns(SftpFileAttributes.FromBytes([0, 0, 0, 0]))); + } + + private void TestFstatFailure(Action> fstatSetup) + { + var sessionMock = new Mock(); + + sessionMock.Setup(s => s.CalculateOptimalReadLength(It.IsAny())).Returns(x => x); + sessionMock.Setup(s => s.CalculateOptimalWriteLength(It.IsAny(), It.IsAny())).Returns((x, _) => x); + sessionMock.Setup(p => p.SessionLoggerFactory).Returns(NullLoggerFactory.Instance); + sessionMock.Setup(s => s.IsOpen).Returns(true); + + fstatSetup(sessionMock.Setup(s => s.RequestFStat(It.IsAny()))); + + var s = SftpFileStream.Open(sessionMock.Object, "file.txt", FileMode.Open, FileAccess.ReadWrite, bufferSize: 1024); + + Assert.IsFalse(s.CanSeek); + Assert.IsTrue(s.CanRead); + Assert.IsTrue(s.CanWrite); + + Assert.Throws(() => s.Position); + Assert.Throws(() => s.Length); + Assert.Throws(() => s.Seek(0, SeekOrigin.Begin)); + Assert.Throws(() => s.SetLength(1024)); + + // Reads and writes still succeed. + _ = s.Read(new byte[16], 0, 16); + s.Write(new byte[16], 0, 16); + s.Flush(); + } + private static void VerifyRequestWrite(Mock sessionMock, ReadOnlyMemory newData, int serverOffset) { sessionMock.Verify(s => s.RequestWrite( From 9a1b94726069db26d637e151ef540f0bc1642190 Mon Sep 17 00:00:00 2001 From: Robert Hague Date: Mon, 20 Oct 2025 13:31:59 +0200 Subject: [PATCH 2/3] Check CanSeek in ReadAllBytes --- src/Renci.SshNet/Sftp/SftpFileStream.cs | 2 +- src/Renci.SshNet/SftpClient.cs | 16 ++++++++++++++-- test/Renci.SshNet.IntegrationTests/SftpTests.cs | 14 +++++++++++--- 3 files changed, 26 insertions(+), 6 deletions(-) diff --git a/src/Renci.SshNet/Sftp/SftpFileStream.cs b/src/Renci.SshNet/Sftp/SftpFileStream.cs index e33b71b9b..ecbcc36ad 100644 --- a/src/Renci.SshNet/Sftp/SftpFileStream.cs +++ b/src/Renci.SshNet/Sftp/SftpFileStream.cs @@ -55,7 +55,7 @@ public override bool CanWrite /// Gets a value indicating whether timeout properties are usable for . /// /// - /// in all cases. + /// in all cases. /// public override bool CanTimeout { diff --git a/src/Renci.SshNet/SftpClient.cs b/src/Renci.SshNet/SftpClient.cs index 322c1e7da..2139ff76a 100644 --- a/src/Renci.SshNet/SftpClient.cs +++ b/src/Renci.SshNet/SftpClient.cs @@ -1714,8 +1714,20 @@ public byte[] ReadAllBytes(string path) { using (var stream = OpenRead(path)) { - var buffer = new byte[stream.Length]; - stream.ReadExactly(buffer, 0, buffer.Length); + byte[] buffer; + + if (stream.CanSeek) + { + buffer = new byte[stream.Length]; + stream.ReadExactly(buffer, 0, buffer.Length); + } + else + { + MemoryStream ms = new(); + stream.CopyTo(ms); + buffer = ms.ToArray(); + } + return buffer; } } diff --git a/test/Renci.SshNet.IntegrationTests/SftpTests.cs b/test/Renci.SshNet.IntegrationTests/SftpTests.cs index b673c6808..ad7a19319 100644 --- a/test/Renci.SshNet.IntegrationTests/SftpTests.cs +++ b/test/Renci.SshNet.IntegrationTests/SftpTests.cs @@ -6360,7 +6360,7 @@ private void DownloadFileRandomMethod(SftpClient client, string path, Stream out { Console.Write($"Downloading '{path}'"); - var random = new Random().Next(1, 6); + var random = new Random().Next(1, 7); switch (random) { case 1: @@ -6390,8 +6390,7 @@ private void DownloadFileRandomMethod(SftpClient client, string path, Stream out } break; - default: - Debug.Assert(random == 5); + case 5: Console.WriteLine($" with {nameof(SftpFileStream.CopyToAsync)}"); using (var fs = client.OpenAsync(path, FileMode.Open, FileAccess.Read, CancellationToken.None).GetAwaiter().GetResult()) @@ -6399,6 +6398,15 @@ private void DownloadFileRandomMethod(SftpClient client, string path, Stream out fs.CopyToAsync(output).GetAwaiter().GetResult(); } + break; + default: + Debug.Assert(random == 6); + Console.WriteLine($" with {nameof(SftpClient.ReadAllBytes)}"); + + byte[] bytes = client.ReadAllBytes(path); + + output.Write(bytes, 0, bytes.Length); + break; } } From e99f5a50c6f2b3451fe35a68cbf3ed63ec80c322 Mon Sep 17 00:00:00 2001 From: Robert Hague Date: Mon, 20 Oct 2025 23:53:45 +0200 Subject: [PATCH 3/3] Squeeze out some performance --- src/Renci.SshNet/Sftp/SftpFileReader.cs | 31 +++++++++++++++++++++++-- src/Renci.SshNet/Sftp/SftpFileStream.cs | 3 ++- 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/src/Renci.SshNet/Sftp/SftpFileReader.cs b/src/Renci.SshNet/Sftp/SftpFileReader.cs index a2b884019..76e55849b 100644 --- a/src/Renci.SshNet/Sftp/SftpFileReader.cs +++ b/src/Renci.SshNet/Sftp/SftpFileReader.cs @@ -92,7 +92,29 @@ public async Task ReadAsync(CancellationToken cancellationToken) if (data.Length < request.Count) { // We didn't receive all the data we requested. - // Add another request to fill in the gap. + + // If we've read exactly up to our known file size and the next + // request is already in-flight, then wait for it and if it signals + // EOF (as is likely), then call EOF here and omit a final round-trip. + // This optimisation is mostly only beneficial to smaller files on + // higher latency connections. + + var nextRequestOffset = _offset - (ulong)data.Length + request.Count; + + if (_offset == _fileSize && + _requests.TryGetValue(nextRequestOffset, out var nextRequest)) + { + var nextRequestData = await nextRequest.Task.WaitAsync(cancellationToken).ConfigureAwait(false); + + if (nextRequestData.Length == 0) + { + _offset = nextRequestOffset; + _currentMaxRequests = 0; + return data; + } + } + + // Otherwise, add another request to fill in the gap. AddRequest(_offset, request.Count - (uint)data.Length); if (data.Length < _chunkSize) @@ -106,8 +128,13 @@ public async Task ReadAsync(CancellationToken cancellationToken) if (_currentMaxRequests > 0) { - if (_readAheadOffset > _fileSize) + if (_readAheadOffset > _fileSize + _chunkSize) { + // If the file size is known and we've got requests + // out beyond that (plus a buffer for EOD read), then + // restrict the number of outgoing requests. + // This does nothing for the performance of this download + // but may reduce traffic for other downloads. _currentMaxRequests = 1; } else if (_currentMaxRequests < _maxPendingReads) diff --git a/src/Renci.SshNet/Sftp/SftpFileStream.cs b/src/Renci.SshNet/Sftp/SftpFileStream.cs index ecbcc36ad..4eb0a2627 100644 --- a/src/Renci.SshNet/Sftp/SftpFileStream.cs +++ b/src/Renci.SshNet/Sftp/SftpFileStream.cs @@ -307,7 +307,8 @@ private static async Task Open( // If we are in a call to SftpClient.DownloadFile, then we know that we will read the whole file, // so we can let there be several in-flight requests from the get go. // This optimisation is mostly only beneficial to smaller files on higher latency connections. - var initialPendingReads = (int)Math.Max(1, Math.Min(MaxPendingReads, 1 + (attributes.Size / readBufferSize))); + // The +2 is +1 for rounding up to cover the whole file, and +1 for the final request to receive EOF. + var initialPendingReads = (int)Math.Max(1, Math.Min(MaxPendingReads, 2 + (attributes.Size / readBufferSize))); initialReader = new(handle, session, readBufferSize, position, MaxPendingReads, (ulong)attributes.Size, initialPendingReads); }