{-# LANGUAGE CPP #-}
{-# LANGUAGE UnliftedFFITypes #-}
-- |
-- Module      : Streamly.Internal.Data.MutArray.Type
-- Copyright   : (c) 2020 Composewell Technologies
-- License     : BSD3-3-Clause
-- Maintainer  : streamly@composewell.com
-- Stability   : experimental
-- Portability : GHC
--
-- Pinned and unpinned mutable array for 'Unboxed' types. Fulfils the following
-- goals:
--
-- * Random access (array)
-- * Efficient storage (unboxed)
-- * Performance (unboxed access)
-- * Performance - in-place operations (mutable)
-- * Performance - GC (pinned, mutable)
-- * interfacing with OS (pinned)
--
-- Stream and Fold APIs allow easy, efficient and convenient operations on
-- arrays.
--
-- Mutable arrays and file system files are quite similar, they can grow and
-- their content is mutable. Therefore, both have similar APIs as well. We
-- strive to keep the API consistent for both. Ideally, you should be able to
-- replace one with another with little changes to the code.

module Streamly.Internal.Data.MutArray.Type
    (
    -- ** Type
    -- $arrayNotes
      MutArray (..)
    , fromMutByteArray
    , toMutByteArray

    -- ** Conversion
    -- *** Pinned and Unpinned
    , pin
    , unpin
    , isPinned

    -- ** Casting
    , cast
    , unsafeCast
    , asBytes
    , unsafeAsPtr

    -- ** Construction
    , empty

    -- *** New
    -- | New arrays are always empty arrays with some reserve capacity to
    -- extend the length without reallocating.
    , emptyOf
    , emptyWithAligned -- XXX emptyAlignAtWith
    , emptyOf'

    -- *** Slicing
    -- | Get a subarray without copying
    , unsafeSliceOffLen
    , sliceOffLen
    , unsafeBreakAt
    , breakAt
    , breakEndByWord8_
    , breakEndBy
    , breakEndBy_
    , revBreakEndBy
    , revBreakEndBy_
    -- , breakSepBy_
    , dropAround
    , dropWhile
    , revDropWhile

    -- *** Stream Folds
    -- | Note: create is just appending to an empty array. So keep the names
    -- consistent with append operations.
    , ArrayUnsafe (..)

    -- With allocator, of capacity
    , unsafeCreateOfWith -- XXX unsafeCreateWithOf
    , createWithOf -- create alloc with

    , unsafeCreateOf
    , createOf
    , createMinOf
    , create -- XXX should we change the min to one elem or one Word?
 -- , createGrowBy

    -- Reverse variants
    , revCreateOf
 -- , revCreate

    -- Pinned variants

    , unsafeCreateOf'
    , createOf'
    , create'

    -- *** From containers
    -- | These can be implemented by appending a stream to an empty array.
    , clone -- XXX fromMutArray or copyMutArray
    , clone'
    , fromListN
    , fromListN'
    , fromList
    , fromList'
    , fromListRevN
    , fromListRev
    , fromStreamN
    , fromStream
    , fromPureStreamN
    , fromPureStream
    , fromCString#
    , fromW16CString#
    , fromPtrN
    , fromChunksK
    , fromChunksRealloced -- fromSmallChunks

    , unsafeCreateWithPtr'

    -- ** Random writes
    , putIndex
 -- , putIndexRev -- or revPutIndex
    , unsafePutIndex
    , putIndices
    -- , putFromThenTo
    -- , putFrom -- start writing at the given position
    -- , putUpto -- write from beginning up to the given position
    -- , putFromTo
    -- , putFromRev
    -- , putUptoRev
    , unsafeModifyIndex
    , modifyIndex
    , modifyIndices
    , modify
    , swapIndices
    , unsafeSwapIndices

    -- ** Reading

    -- *** Indexing
    , getIndex
    , unsafeGetIndex
    , unsafeGetIndexRev
    -- , getFromThenTo
    , getIndexRev -- getRevIndex?
    , indexReader
    , indexReaderWith

    -- *** To Streams
    , read
    , readRev
    , toStreamWith
    , toStreamRevWith
    , toStreamK
    , toStreamKWith
    , toStreamKRev
    , toStreamKRevWith

    -- *** To Containers
    , toList

    -- *** Unfolds
    -- experimental
    , producerWith
    , producer

    , reader
    , readerRevWith
    , readerRev

    -- ** Size and Capacity
    -- *** Size
    , length
    , byteLength

    -- *** Capacity Reporting
    , capacity
    , free
    , byteCapacity
    , bytesFree

    -- *** Capacity Management
    -- There are two ways of growing an array:
    --
    -- * grow: double, align to next power of 2 if large, never shrink
    -- * growBy: align to block size if large, never shrink

    , blockSize
    , arrayChunkBytes
    , allocBytesToElemCount
    , reallocBytes
    , reallocBytesWith

 -- , grow -- double the used capacity and align to power of 2
    , growTo
    , growBy
    , growExp
    , rightSize
    , vacate

    -- ** Folding
    , foldl'
    , foldr
    , fold
    , foldRev -- XXX revFold
    , byteCmp
    , byteEq

    -- ** In-place Mutation Algorithms
    , reverse
    , permute
    , partitionBy
    , shuffleBy
    , divideBy
    , mergeBy
    , bubble
    , rangeBy

    -- ** Growing and Shrinking
    -- | Arrays grow only at the end, though technically it is possible to
    -- grow on both sides and therefore we can have a cons as well as snoc. But
    -- cons is not implemented yet.

    -- *** Appending elements
    -- | snoc is the fundamental operation for growing arrays. Streaming folds,
    -- appending streams can be implemented in terms of snoc.

    -- XXX snoc 128/256/512 bit data using SIMD.
    , snocWith -- XXX snocGrowWith
    , snoc
    , snocGrowBy
    , snocMay
    , unsafeSnoc

 -- , revSnoc -- cons
 -- , revSnocGrowBy  -- consGrowBy

    -- *** Folds for appending streams
    -- | Fundamentally these are a sequence of snoc operations.
    -- Folds are named "append" whereas joining two arrays is named as "splice".

    , appendWith -- XXX replace by pure appendGrowWith

    , unsafeAppendMax -- can be renamed to unsafeAppendN later
    , appendMax -- can be renamed to appendN later
 -- , appendMin -- like createMinOf, supplies a min hint to reduce allocs
 -- , appendGrowWith
    , append2   -- to be renamed to append later
    , appendGrowBy

 -- , revAppend
 -- , revAppendN
 -- , revAppendGrowBy

    -- *** Appending streams
    -- | Fundamentally these are a sequence of snoc operations. These are
    -- convenience operations implemented in terms of folds.
    , unsafeAppendPtrN
    , appendPtrN
    , appendCString
    , appendCString#
 -- , appendStreamGrowWith
    , appendStream
    , appendStreamN
 -- , appendStreamGrowBy

    -- *** Splicing arrays
    -- | TODO: We can replace memcpy with stream copy using Word64. Arrays are
    -- aligned on 64-bit boundaries on 64-bit CPUs. A fast way to copy an
    -- array is to unsafeCast it to Word64, read it as a stream, write the
    -- stream to Word64 array and unsafeCast it again. We can use SIMD
    -- read/write as well.

    , spliceCopy -- XXX freeze and splice instead?
    , splice
    , spliceWith -- XXX spliceGrowWith
    , spliceExp -- XXX spliceGrowExp
 -- , spliceN
 -- , spliceGrowBy
    , unsafeSplice
    -- , putSlice
    -- , appendSlice
    -- , appendSliceFrom

    -- XXX Do not expose these yet, we should perhaps expose only the Peek/Poke
    -- monads instead? Decide after implementing the monads.

    -- ** Serialization using Unbox
    -- | Fixed length serialization.
    -- Serialization operations are essentially a combination of serialization
    -- using Unbox/Serialize type class, followed by snoc. TODO: use SIMD for
    -- snoc.
    , poke
    , pokeMay
 -- , pokeGrowBy
    , unsafePokeSkip -- XXX unsafePoke_
 -- , revPoke

    -- ** Deserialization using Unbox
    -- Fixed length deserialization.
    , peek
    , unsafePeek
    , unsafePeekSkip -- XXX unsafePeek_
 -- , revPeek

    -- Arrays of arrays
    --  We can add dimensionality parameter to the array type to get
    --  multidimensional arrays. Multidimensional arrays would just be a
    --  convenience wrapper on top of single dimensional arrays.

    -- ** Streams of Arrays
    -- *** Chunk
    -- | Group a stream into arrays.
    , chunksOf
    , chunksOf' -- chunksOf'
    -- , timedChunksOf -- see the Streamly.Data.Stream.Prelude module
    , buildChunks
    , chunksEndBy
    , chunksEndBy'
    , chunksEndByLn
    , chunksEndByLn'
    -- , chunksBeginBySeq -- for parsing streams with headers

    -- *** Split
    -- | Split an array into a stream of slices.

    -- Note: some splitting APIs are in MutArray.hs
    , splitEndBy_
    , splitEndBy
    -- , splitSepBy_

    -- *** Concat
    -- | Append the arrays in a stream to form a stream of elements.
    , concat
    -- , concatSepBy
    -- , concatEndBy
    -- , concatEndByLn -- unlines - concat a byte chunk stream using newline byte separator
    , concatWith -- internal
    , concatRev
    , concatRevWith -- internal

    -- *** Compact
    -- | Coalesce arrays together in a stream of arrays to form a stream of
    -- larger arrays.
    , SpliceState (..)
    , compactLeAs -- internal

    -- Creation folds/parsers
    , createCompactMax
    , createCompactMax'
    , createCompactMin
    , createCompactMin'

    -- Stream compaction
    , compactMin
    -- , compactMin'
    , compactExact
    -- , compactExact'

    -- Scans
    , scanCompactMin
    , scanCompactMin'

    -- ** Utilities
    , isPower2
    , roundUpToPower2

    -- * Deprecated
    , unsafeGetSlice
    , getSlice
    , sliceEndBy_
    , strip
    , stripStart
    , stripEnd
    , breakOn
    , splitAt
    , unsafeSplitAt
    , realloc
    , createOfWith
    , peekUncons
    , peekUnconsUnsafe
    , pokeAppend
    , pokeAppendMay
    , castUnsafe
    , newArrayWith
    , getSliceUnsafe
    , putIndexUnsafe
    , modifyIndexUnsafe
    , getIndexUnsafe
    , snocUnsafe
    , spliceUnsafe
    , pokeSkipUnsafe
    , peekSkipUnsafe
    , asPtrUnsafe
    , writeChunks
    , flattenArrays
    , flattenArraysRev
    , fromArrayStreamK
    , fromStreamDN
    , fromStreamD
    , cmp
    , getIndices
    , getIndicesWith
    , resize
    , resizeExp
    , nil
    , new
    , pinnedNew
    , pinnedNewBytes
    , writeAppendNUnsafe
    , writeAppendN
    , writeAppendWith
    , writeAppend
    , writeNWithUnsafe
    , writeNWith
    , writeNUnsafe
    , pinnedWriteNUnsafe
    , writeN
    , pinnedWriteN
    , pinnedWriteNAligned -- XXX not required
    , writeWith
    , write
    , pinnedWrite
    , writeRevN
    , fromByteStr#
    , pCompactLE
    , pPinnedCompactLE
    , fCompactGE
    , fPinnedCompactGE
    , lPinnedCompactGE
    , lCompactGE
    , compactGE
    , pinnedEmptyOf
    , pinnedChunksOf
    , pinnedCreateOf
    , pinnedCreate
    , pinnedFromListN
    , pinnedFromList
    , pinnedClone
    , unsafePinnedCreateOf
    , splitOn
    , pinnedNewAligned
    , unsafePinnedAsPtr
    , grow -- XXX to be deprecated
    , createWith -- XXX to be deprecated
    , snocLinear -- XXX deprecate, replace by snocGrowBy or rename snoc1KB
    , unsafeAppendN -- XXX deprecate, replaced by unsafeAppendMax
    , appendN -- XXX deprecate, replaced by appendMax
    , append -- XXX deprecate, replaced by append2
    )
where

#include "assert.hs"
#include "deprecation.h"
#include "inline.hs"
#include "ArrayMacros.h"
#include "MachDeps.h"

import Control.Monad (when)
import Control.Monad.IO.Class (MonadIO(..))
import Data.Bifunctor (first)
import Data.Bits (shiftR, (.|.), (.&.))
import Data.Char (ord)
import Data.Functor.Identity (Identity(..))
import Data.Proxy (Proxy(..))
import Data.Word (Word8, Word16)
import Foreign.C.Types (CSize(..))
import Foreign.Ptr (plusPtr)
import Streamly.Internal.Data.MutByteArray.Type
    ( MutByteArray(..)
    , PinnedState(..)
    , getMutByteArray#
    , unsafePutSlice
    , blockSize
    , largeObjectThreshold
    , unsafeByteCmp
    )
import Streamly.Internal.Data.Unbox (Unbox(..))
import GHC.Base (noinline)
import GHC.Exts (Addr#, MutableByteArray#, RealWorld)
import GHC.Ptr (Ptr(..))
import GHC.Exts (byteArrayContents#, unsafeCoerce#)

import Streamly.Internal.Data.Fold.Type (Fold(..))
import Streamly.Internal.Data.Producer.Type (Producer (..))
import Streamly.Internal.Data.Scanl.Type (Scanl (..))
import Streamly.Internal.Data.Stream.Type (Stream)
import Streamly.Internal.Data.Parser.Type (Parser (..))
import Streamly.Internal.Data.StreamK.Type (StreamK)
import Streamly.Internal.Data.SVar.Type (adaptState, defState)
import Streamly.Internal.Data.Tuple.Strict (Tuple'(..))
import Streamly.Internal.Data.Unfold.Type (Unfold(..))
import Streamly.Internal.System.IO (arrayPayloadSize, defaultChunkSize)

import qualified Streamly.Internal.Data.Fold.Type as FL
import qualified Streamly.Internal.Data.MutByteArray.Type as Unboxed
import qualified Streamly.Internal.Data.Parser.Type as Parser
-- import qualified Streamly.Internal.Data.Fold.Type as Fold
import qualified Streamly.Internal.Data.Producer as Producer
import qualified Streamly.Internal.Data.Stream.Type as D
import qualified Streamly.Internal.Data.Stream.Lift as D
import qualified Streamly.Internal.Data.Stream.Generate as D
import qualified Streamly.Internal.Data.StreamK.Type as K
import qualified Prelude

import Prelude hiding
    (Foldable(..), concat, read, unlines, splitAt, reverse, truncate, dropWhile)

#include "DocTestDataMutArray.hs"

-------------------------------------------------------------------------------
-- Foreign helpers
-------------------------------------------------------------------------------

-- NOTE: Have to be "ccall unsafe" so that we can pass unpinned memory to
-- these. For passing unpinned memory safely we have to pass unlifted byte
-- array pointers in FFI so that neither the constructor nor the array can
-- become stale if a GC kicks in at any point before the call.

foreign import ccall unsafe "string.h memcpy" c_memcpy_pinned_src
    :: MutableByteArray# RealWorld -> Ptr Word8 -> CSize -> IO (Ptr Word8)

foreign import ccall unsafe "memchr_index" c_memchr_index
    :: MutableByteArray# RealWorld -> CSize -> Word8 -> CSize -> IO CSize

-- XXX Use cstringLength# from GHC.CString in ghc-prim
foreign import ccall unsafe "string.h strlen" c_strlen_pinned
    :: Addr# -> IO CSize

-- | Given an 'Unboxed' type (unused first arg) and a number of bytes, return
-- how many elements of that type will completely fit in those bytes.
--
{-# INLINE bytesToElemCount #-}
bytesToElemCount :: forall a. Unbox a => a -> Int -> Int
bytesToElemCount :: forall a. Unbox a => a -> Int -> Int
bytesToElemCount a
_ Int
n = Int
n Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` SIZE_OF(a)

-------------------------------------------------------------------------------
-- MutArray Data Type
-------------------------------------------------------------------------------

-- Note on using "IO" callbacks:
--
-- The Array APIs should use "IO" callbacks instead of lifted callbacks as the
-- lifted callbacks aren't optimized properly.
--
-- See:
-- https://github.com/composewell/streamly/issues/2820
-- https://github.com/composewell/streamly/issues/2589


-- $arrayNotes
--
-- We can use an 'Unboxed' constraint in the MutArray type and the constraint
-- can be automatically provided to a function that pattern matches on the
-- MutArray type. However, it has huge performance cost, so we do not use it.
-- Investigate a GHC improvement possiblity.

-- | An unboxed mutable array. An array is created with a given length
-- and capacity. Length is the number of valid elements in the array.  Capacity
-- is the maximum number of elements that the array can be expanded to without
-- having to reallocate the memory.
--
-- The elements in the array can be mutated in-place without changing the
-- reference (constructor). However, the length of the array cannot be mutated
-- in-place.  A new array reference is generated when the length changes.  When
-- the length is increased (upto the maximum reserved capacity of the array),
-- the array is not reallocated and the new reference uses the same underlying
-- memory as the old one.
--
-- Several routines in this module allow the programmer to control the capacity
-- of the array. The programmer can control the trade-off between memory usage
-- and performance impact due to reallocations when growing or shrinking the
-- array.
--
data MutArray a =
#ifdef DEVBUILD
    Unbox a =>
#endif
    -- The array is a range into arrContents. arrContents may be a superset of
    -- the slice represented by the array. All offsets are in bytes.
    MutArray
    { forall a. MutArray a -> MutByteArray
arrContents :: {-# UNPACK #-} !MutByteArray
    , forall a. MutArray a -> Int
arrStart :: {-# UNPACK #-} !Int  -- ^ index into arrContents
    , forall a. MutArray a -> Int
arrEnd   :: {-# UNPACK #-} !Int  -- ^ index into arrContents
                                       -- Represents the first invalid index of
                                       -- the array.
    -- XXX rename to arrCapacity to be consistent with ring.
    , forall a. MutArray a -> Int
arrBound :: {-# UNPACK #-} !Int  -- ^ first invalid index of arrContents.
    }

-------------------------------------------------------------------------------
-- Construction and destructuring
-------------------------------------------------------------------------------

{-# INLINE fromMutByteArray #-}
fromMutByteArray :: MonadIO m => MutByteArray -> Int -> Int -> m (MutArray a)
fromMutByteArray :: forall (m :: * -> *) a.
MonadIO m =>
MutByteArray -> Int -> Int -> m (MutArray a)
fromMutByteArray MutByteArray
arr Int
start Int
end = do
    Int
len <- IO Int -> m Int
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO Int -> m Int) -> IO Int -> m Int
forall a b. (a -> b) -> a -> b
$ MutByteArray -> IO Int
Unboxed.length MutByteArray
arr
    MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> m (MutArray a)) -> MutArray a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutArray
        { arrContents :: MutByteArray
arrContents = MutByteArray
arr
        , arrStart :: Int
arrStart = Int
start
        , arrEnd :: Int
arrEnd = Int
end
        , arrBound :: Int
arrBound = Int
len
        }

{-# INLINE toMutByteArray #-}
toMutByteArray :: MutArray a -> (MutByteArray, Int, Int)
toMutByteArray :: forall a. MutArray a -> (MutByteArray, Int, Int)
toMutByteArray MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = (MutByteArray
arrContents, Int
arrStart, Int
arrEnd)

-------------------------------------------------------------------------------
-- Pinning & Unpinning
-------------------------------------------------------------------------------

-- | Return a copy of the array in pinned memory if unpinned, else return the
-- original array.
{-# INLINE pin #-}
pin :: MutArray a -> IO (MutArray a)
pin :: forall a. MutArray a -> IO (MutArray a)
pin arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    if MutByteArray -> Bool
Unboxed.isPinned MutByteArray
arrContents
    then MutArray a -> IO (MutArray a)
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
arr
    else MutArray a -> IO (MutArray a)
forall (m :: * -> *) a. MonadIO m => MutArray a -> m (MutArray a)
clone' MutArray a
arr

-- | Return a copy of the array in unpinned memory if pinned, else return the
-- original array.
{-# INLINE unpin #-}
unpin :: MutArray a -> IO (MutArray a)
unpin :: forall a. MutArray a -> IO (MutArray a)
unpin arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    if MutByteArray -> Bool
Unboxed.isPinned MutByteArray
arrContents
    then MutArray a -> IO (MutArray a)
forall (m :: * -> *) a. MonadIO m => MutArray a -> m (MutArray a)
clone MutArray a
arr
    else MutArray a -> IO (MutArray a)
forall a. a -> IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
arr

-- | Return 'True' if the array is allocated in pinned memory.
{-# INLINE isPinned #-}
isPinned :: MutArray a -> Bool
isPinned :: forall a. MutArray a -> Bool
isPinned MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = MutByteArray -> Bool
Unboxed.isPinned MutByteArray
arrContents

-------------------------------------------------------------------------------
-- Construction
-------------------------------------------------------------------------------

-- XXX Change the names to use "new" instead of "newArray". That way we can use
-- the same names for managed file system objects as well. For unmanaged ones
-- we can use open/create etc as usual.
--
-- A new array is similar to "touch" creating a zero length file. An mmapped
-- array would be similar to a sparse file with holes. TBD: support mmapped
-- files and arrays.

-- GHC always guarantees word-aligned memory, alignment is important only when
-- we need more than that.  See stg_pinnedNewAlignedByteArrayzh and
-- allocatePinned in GHC source.

-- XXX Rename to emptyAlignedWith, alignSize should be first arg.

-- | @emptyWithAligned allocator alignment count@ allocates a new array of zero
-- length and with a capacity to hold @count@ elements, using @allocator
-- size alignment@ as the memory allocator function.
--
-- Alignment must be greater than or equal to machine word size and a power of
-- 2.
--
-- Alignment is ignored if the allocator allocates unpinned memory.
--
-- /Pre-release/
{-# INLINE emptyWithAligned #-}
newArrayWith, emptyWithAligned :: forall m a. (MonadIO m, Unbox a)
    => (Int -> Int -> IO MutByteArray) -> Int -> Int -> m (MutArray a)
emptyWithAligned :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int -> IO MutByteArray) -> Int -> Int -> m (MutArray a)
emptyWithAligned Int -> Int -> IO MutByteArray
alloc Int
alignSize Int
count = IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ do
    let size :: Int
size = Int -> Int -> Int
forall a. Ord a => a -> a -> a
max (Int
count Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)) 0
    MutByteArray
contents <- Int -> Int -> IO MutByteArray
alloc Int
size Int
alignSize
    MutArray a -> IO (MutArray a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> IO (MutArray a)) -> MutArray a -> IO (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutArray
        { arrContents :: MutByteArray
arrContents = MutByteArray
contents
        , arrStart :: Int
arrStart = Int
0
        , arrEnd :: Int
arrEnd   = Int
0
        , arrBound :: Int
arrBound = Int
size
        }

-- For arrays "nil" sounds a bit odd. empty is better. The only problem with
-- empty is that it is also used by the Alternative type class. But assuming we
-- will mostly import the Array module qualified this should be fine.

-- | Create an empty array.
empty ::
#ifdef DEVBUILD
    Unbox a =>
#endif
    MutArray a
empty :: forall a. MutArray a
empty = MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
Unboxed.empty Int
0 Int
0 Int
0

{-# DEPRECATED nil "Please use empty instead." #-}
nil ::
#ifdef DEVBUILD
    Unbox a =>
#endif
    MutArray a
nil :: forall a. MutArray a
nil = MutArray a
forall a. MutArray a
empty

{-# INLINE newBytesAs #-}
newBytesAs :: MonadIO m =>
#ifdef DEVBUILD
    Unbox a =>
#endif
    PinnedState -> Int -> m (MutArray a)
newBytesAs :: forall (m :: * -> *) a.
MonadIO m =>
PinnedState -> Int -> m (MutArray a)
newBytesAs PinnedState
ps Int
bytes = do
    MutByteArray
contents <- IO MutByteArray -> m MutByteArray
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO MutByteArray -> m MutByteArray)
-> IO MutByteArray -> m MutByteArray
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> IO MutByteArray
Unboxed.newAs PinnedState
ps Int
bytes
    MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> m (MutArray a)) -> MutArray a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutArray
        { arrContents :: MutByteArray
arrContents = MutByteArray
contents
        , arrStart :: Int
arrStart = Int
0
        , arrEnd :: Int
arrEnd   = Int
0
        , arrBound :: Int
arrBound = Int
bytes
        }

-- | Allocates a pinned empty array that with a reserved capacity of bytes.
-- The memory of the array is uninitialized and the allocation is aligned as
-- per the 'Unboxed' instance of the type.
--
-- > pinnedNewBytes = (unsafeCast :: Array Word8 -> a) . emptyOf'
--
-- /Pre-release/
{-# INLINE pinnedNewBytes #-}
{-# DEPRECATED pinnedNewBytes "Please use emptyOf' to create a Word8 array and cast it accordingly." #-}
pinnedNewBytes :: MonadIO m =>
#ifdef DEVBUILD
    Unbox a =>
#endif
    Int -> m (MutArray a)
pinnedNewBytes :: forall (m :: * -> *) a. MonadIO m => Int -> m (MutArray a)
pinnedNewBytes = PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
PinnedState -> Int -> m (MutArray a)
newBytesAs PinnedState
Pinned

-- | Like 'emptyWithAligned' but using an allocator is a pinned memory allocator and
-- the alignment is dictated by the 'Unboxed' instance of the type.
--
-- /Internal/
{-# DEPRECATED pinnedNewAligned "Please use emptyOf' to create a Word8 array and cast it accordingly." #-}
{-# INLINE pinnedNewAligned #-}
pinnedNewAligned :: (MonadIO m, Unbox a) => Int -> Int -> m (MutArray a)
pinnedNewAligned :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Int -> m (MutArray a)
pinnedNewAligned = (Int -> Int -> IO MutByteArray) -> Int -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int -> IO MutByteArray) -> Int -> Int -> m (MutArray a)
emptyWithAligned (\Int
s Int
_ -> IO MutByteArray -> IO MutByteArray
forall a. IO a -> IO a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO MutByteArray -> IO MutByteArray)
-> IO MutByteArray -> IO MutByteArray
forall a b. (a -> b) -> a -> b
$ Int -> IO MutByteArray
Unboxed.new' Int
s)

{-# INLINE newAs #-}
newAs :: (MonadIO m, Unbox a) => PinnedState -> Int -> m (MutArray a)
newAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
ps =
    (Int -> Int -> IO MutByteArray) -> Int -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int -> IO MutByteArray) -> Int -> Int -> m (MutArray a)
emptyWithAligned
        (\Int
s Int
_ -> IO MutByteArray -> IO MutByteArray
forall a. IO a -> IO a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO MutByteArray -> IO MutByteArray)
-> IO MutByteArray -> IO MutByteArray
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> IO MutByteArray
Unboxed.newAs PinnedState
ps Int
s)
        ([Char] -> Int
forall a. (?callStack::CallStack) => [Char] -> a
error [Char]
"new: alignment is not used in unpinned arrays.")

-- XXX can unaligned allocation be more efficient when alignment is not needed?

-- | Allocates a pinned array of zero length but growable to the specified
-- capacity without reallocation.
{-# INLINE emptyOf' #-}
pinnedEmptyOf, emptyOf' :: (MonadIO m, Unbox a) => Int -> m (MutArray a)
emptyOf' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
emptyOf' = PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
Pinned
RENAME_PRIME(pinnedEmptyOf,emptyOf)

{-# DEPRECATED pinnedNew "Please use emptyOf' instead." #-}
{-# INLINE pinnedNew #-}
pinnedNew :: forall m a. (MonadIO m, Unbox a) => Int -> m (MutArray a)
pinnedNew :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
pinnedNew = Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
emptyOf'

-- | Allocates an unpinned array of zero length but growable to the specified
-- capacity without reallocation.
--
{-# INLINE emptyOf #-}
emptyOf :: (MonadIO m, Unbox a) => Int -> m (MutArray a)
emptyOf :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
emptyOf = PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
Unpinned

{-# DEPRECATED new "Please use emptyOf instead." #-}
{-# INLINE new #-}
new :: (MonadIO m, Unbox a) => Int -> m (MutArray a)
new :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
new = Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
emptyOf

-------------------------------------------------------------------------------
-- Random writes
-------------------------------------------------------------------------------

-- | Write the given element to the given index of the array. Does not check if
-- the index is out of bounds of the array.
--
-- /Pre-release/
{-# INLINE unsafePutIndex #-}
putIndexUnsafe, unsafePutIndex :: forall m a. (MonadIO m, Unbox a)
    => Int -> MutArray a -> a -> m ()
unsafePutIndex :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m ()
unsafePutIndex Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a
x = do
    let index :: Int
index = Int
INDEX_OF(arrStart, i, a)
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& INDEX_VALID(index, arrEnd, a)) (return ())
    IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
index MutByteArray
arrContents  a
x

invalidIndex :: String -> Int -> a
invalidIndex :: forall a. [Char] -> Int -> a
invalidIndex [Char]
label Int
i =
    [Char] -> a
forall a. (?callStack::CallStack) => [Char] -> a
error ([Char] -> a) -> [Char] -> a
forall a b. (a -> b) -> a -> b
$ [Char]
label [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
": invalid array index " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
i

-- | /O(1)/ Write the given element at the given index in the array.
-- Performs in-place mutation of the array.
--
-- >>> putIndex ix arr val = MutArray.modifyIndex ix arr (const (val, ()))
-- >>> f = MutArray.putIndices
-- >>> putIndex ix arr val = Stream.fold (f arr) (Stream.fromPure (ix, val))
--
{-# INLINE putIndex #-}
putIndex :: forall m a. (MonadIO m, Unbox a) => Int -> MutArray a -> a -> m ()
putIndex :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m ()
putIndex Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a
x = do
    let index :: Int
index = Int
INDEX_OF(arrStart,i,a)
    if Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& INDEX_VALID(index,arrEnd,a)
    then IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
index MutByteArray
arrContents  a
x
    else [Char] -> Int -> m ()
forall a. [Char] -> Int -> a
invalidIndex [Char]
"putIndex" Int
i

-- | Write an input stream of (index, value) pairs to an array. Throws an
-- error if any index is out of bounds.
--
-- /Pre-release/
{-# INLINE putIndices #-}
putIndices :: forall m a. (MonadIO m, Unbox a)
    => MutArray a -> Fold m (Int, a) ()
putIndices :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Fold m (Int, a) ()
putIndices MutArray a
arr = (() -> (Int, a) -> m ()) -> m () -> Fold m (Int, a) ()
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' () -> (Int, a) -> m ()
forall {m :: * -> *}. MonadIO m => () -> (Int, a) -> m ()
step (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())

    where

    step :: () -> (Int, a) -> m ()
step () (Int
i, a
x) = Int -> MutArray a -> a -> m ()
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m ()
putIndex Int
i MutArray a
arr a
x

-- | Modify a given index of an array using a modifier function.
--
-- Unsafe because it does not check the bounds of the array.
--
-- /Pre-release/
modifyIndexUnsafe, unsafeModifyIndex :: forall m a b. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> (a -> (a, b)) -> m b
unsafeModifyIndex :: forall (m :: * -> *) a b.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> (a -> (a, b)) -> m b
unsafeModifyIndex Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a -> (a, b)
f = IO b -> m b
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO b -> m b) -> IO b -> m b
forall a b. (a -> b) -> a -> b
$ do
        let index :: Int
index = Int
INDEX_OF(arrStart,i,a)
        Bool -> IO () -> IO ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& INDEX_NEXT(index,a) <= arrEnd) (return ())
        a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
index MutByteArray
arrContents
        let (a
x, b
res) = a -> (a, b)
f a
r
        Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
index MutByteArray
arrContents  a
x
        b -> IO b
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return b
res

-- | Modify a given index of an array using a modifier function.
--
-- /Pre-release/
modifyIndex :: forall m a b. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> (a -> (a, b)) -> m b
modifyIndex :: forall (m :: * -> *) a b.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> (a -> (a, b)) -> m b
modifyIndex Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a -> (a, b)
f = do
    let index :: Int
index = Int
INDEX_OF(arrStart,i,a)
    if Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& INDEX_VALID(index,arrEnd,a)
    then IO b -> m b
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO b -> m b) -> IO b -> m b
forall a b. (a -> b) -> a -> b
$ do
        a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
index MutByteArray
arrContents
        let (a
x, b
res) = a -> (a, b)
f a
r
        Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
index MutByteArray
arrContents  a
x
        b -> IO b
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return b
res
    else [Char] -> Int -> m b
forall a. [Char] -> Int -> a
invalidIndex [Char]
"modifyIndex" Int
i

-- | Modify the array indices generated by the supplied stream.
--
-- /Pre-release/
{-# INLINE modifyIndices #-}
modifyIndices :: forall m a . (MonadIO m, Unbox a)
    => MutArray a -> (Int -> a -> a) -> Fold m Int ()
modifyIndices :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> (Int -> a -> a) -> Fold m Int ()
modifyIndices MutArray a
arr Int -> a -> a
f = (() -> Int -> m ()) -> m () -> Fold m Int ()
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' () -> Int -> m ()
forall {m :: * -> *}. MonadIO m => () -> Int -> m ()
step m ()
initial

    where

    initial :: m ()
initial = () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()

    step :: () -> Int -> m ()
step () Int
i =
        let f1 :: a -> (a, ())
f1 a
x = (Int -> a -> a
f Int
i a
x, ())
         in Int -> MutArray a -> (a -> (a, ())) -> m ()
forall (m :: * -> *) a b.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> (a -> (a, b)) -> m b
modifyIndex Int
i MutArray a
arr a -> (a, ())
f1

-- | Modify each element of an array using the supplied modifier function.
--
-- This is an in-place equivalent of an immutable map operation.
--
-- /Pre-release/
modify :: forall m a. (MonadIO m, Unbox a)
    => MutArray a -> (a -> a) -> m ()
modify :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> (a -> a) -> m ()
modify MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a -> a
f = IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$
    Int -> IO ()
go Int
arrStart

    where

    go :: Int -> IO ()
go Int
i =
        Bool -> IO () -> IO ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (INDEX_VALID(i,arrEnd,a)) $ do
            r <- peekAt i arrContents
            pokeAt i arrContents (f r)
            go (INDEX_NEXT(i,a))

-- XXX We could specify the number of bytes to swap instead of Proxy. Need
-- to ensure that the memory does not overlap.
{-# INLINE swapArrayByteIndices #-}
swapArrayByteIndices ::
       forall a. Unbox a
    => Proxy a
    -> MutByteArray
    -> Int
    -> Int
    -> IO ()
swapArrayByteIndices :: forall a. Unbox a => Proxy a -> MutByteArray -> Int -> Int -> IO ()
swapArrayByteIndices Proxy a
_ MutByteArray
arrContents Int
i1 Int
i2 = do
    a
r1 <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
i1 MutByteArray
arrContents
    a
r2 <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
i2 MutByteArray
arrContents
    Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
i1 MutByteArray
arrContents (a
r2 :: a)
    Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
i2 MutByteArray
arrContents (a
r1 :: a)

-- | Swap the elements at two indices without validating the indices.
--
-- /Unsafe/: This could result in memory corruption if indices are not valid.
--
-- /Pre-release/
{-# INLINE unsafeSwapIndices #-}
unsafeSwapIndices :: forall m a. (MonadIO m, Unbox a)
    => Int -> Int -> MutArray a -> m ()
unsafeSwapIndices :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Int -> MutArray a -> m ()
unsafeSwapIndices Int
i1 Int
i2 MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ do
        let t1 :: Int
t1 = Int
INDEX_OF(arrStart,i1,a)
            t2 :: Int
t2 = Int
INDEX_OF(arrStart,i2,a)
        Proxy a -> MutByteArray -> Int -> Int -> IO ()
forall a. Unbox a => Proxy a -> MutByteArray -> Int -> Int -> IO ()
swapArrayByteIndices (Proxy a
forall {k} (t :: k). Proxy t
Proxy :: Proxy a) MutByteArray
arrContents Int
t1 Int
t2

-- | Swap the elements at two indices.
--
-- /Pre-release/
swapIndices :: forall m a. (MonadIO m, Unbox a)
    => Int -> Int -> MutArray a -> m ()
swapIndices :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Int -> MutArray a -> m ()
swapIndices Int
i1 Int
i2 MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ do
        let t1 :: Int
t1 = Int
INDEX_OF(arrStart,i1,a)
            t2 :: Int
t2 = Int
INDEX_OF(arrStart,i2,a)
        Bool -> IO () -> IO ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
i1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
0 Bool -> Bool -> Bool
|| INDEX_INVALID(t1,arrEnd,a))
            (IO () -> IO ()) -> IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char] -> Int -> IO ()
forall a. [Char] -> Int -> a
invalidIndex [Char]
"swapIndices" Int
i1
        Bool -> IO () -> IO ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
i2 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
0 Bool -> Bool -> Bool
|| INDEX_INVALID(t2,arrEnd,a))
            (IO () -> IO ()) -> IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ [Char] -> Int -> IO ()
forall a. [Char] -> Int -> a
invalidIndex [Char]
"swapIndices" Int
i2
        Proxy a -> MutByteArray -> Int -> Int -> IO ()
forall a. Unbox a => Proxy a -> MutByteArray -> Int -> Int -> IO ()
swapArrayByteIndices (Proxy a
forall {k} (t :: k). Proxy t
Proxy :: Proxy a) MutByteArray
arrContents Int
t1 Int
t2

-------------------------------------------------------------------------------
-- Rounding
-------------------------------------------------------------------------------

-- XXX Should be done only when we are using the GHC allocator.
-- | Round up an array larger than 'largeObjectThreshold' to use the whole
-- block.
{-# INLINE roundUpLargeArray #-}
roundUpLargeArray :: Int -> Int
roundUpLargeArray :: Int -> Int
roundUpLargeArray Int
size =
    if Int
size Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
largeObjectThreshold
    then
        Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert
            (Int
blockSize Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
/= Int
0 Bool -> Bool -> Bool
&& ((Int
blockSize Int -> Int -> Int
forall a. Bits a => a -> a -> a
.&. (Int
blockSize Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1)) Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0))
            ((Int
size Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
blockSize Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) Int -> Int -> Int
forall a. Bits a => a -> a -> a
.&. Int -> Int
forall a. Num a => a -> a
negate Int
blockSize)
    else Int
size

{-# INLINE isPower2 #-}
isPower2 :: Int -> Bool
isPower2 :: Int -> Bool
isPower2 Int
n = Int
n Int -> Int -> Int
forall a. Bits a => a -> a -> a
.&. (Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0

{-# INLINE roundUpToPower2 #-}
roundUpToPower2 :: Int -> Int
roundUpToPower2 :: Int -> Int
roundUpToPower2 Int
n =
#if WORD_SIZE_IN_BITS == 64
    Int
1 Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
z6
#else
    1 + z5
#endif

    where

    z0 :: Int
z0 = Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1
    z1 :: Int
z1 = Int
z0 Int -> Int -> Int
forall a. Bits a => a -> a -> a
.|. Int
z0 Int -> Int -> Int
forall a. Bits a => a -> Int -> a
`shiftR` Int
1
    z2 :: Int
z2 = Int
z1 Int -> Int -> Int
forall a. Bits a => a -> a -> a
.|. Int
z1 Int -> Int -> Int
forall a. Bits a => a -> Int -> a
`shiftR` Int
2
    z3 :: Int
z3 = Int
z2 Int -> Int -> Int
forall a. Bits a => a -> a -> a
.|. Int
z2 Int -> Int -> Int
forall a. Bits a => a -> Int -> a
`shiftR` Int
4
    z4 :: Int
z4 = Int
z3 Int -> Int -> Int
forall a. Bits a => a -> a -> a
.|. Int
z3 Int -> Int -> Int
forall a. Bits a => a -> Int -> a
`shiftR` Int
8
    z5 :: Int
z5 = Int
z4 Int -> Int -> Int
forall a. Bits a => a -> a -> a
.|. Int
z4 Int -> Int -> Int
forall a. Bits a => a -> Int -> a
`shiftR` Int
16
    z6 :: Int
z6 = Int
z5 Int -> Int -> Int
forall a. Bits a => a -> a -> a
.|. Int
z5 Int -> Int -> Int
forall a. Bits a => a -> Int -> a
`shiftR` Int
32

-- | @allocBytesToBytes elem allocatedBytes@ returns the array size in bytes
-- such that the real allocation is less than or equal to @allocatedBytes@,
-- unless @allocatedBytes@ is less than the size of one array element in which
-- case it returns one element's size.
--
{-# INLINE allocBytesToBytes #-}
allocBytesToBytes :: forall a. Unbox a => a -> Int -> Int
allocBytesToBytes :: forall a. Unbox a => a -> Int -> Int
allocBytesToBytes a
_ Int
n = Int -> Int -> Int
forall a. Ord a => a -> a -> a
max (Int -> Int
arrayPayloadSize Int
n) (SIZE_OF(a))

-- | Given an 'Unboxed' type (unused first arg) and real allocation size
-- (including overhead), return how many elements of that type will completely
-- fit in it, returns at least 1.
--
{-# INLINE allocBytesToElemCount #-}
allocBytesToElemCount :: Unbox a => a -> Int -> Int
allocBytesToElemCount :: forall a. Unbox a => a -> Int -> Int
allocBytesToElemCount a
x Int
bytes =
    let n :: Int
n = a -> Int -> Int
forall a. Unbox a => a -> Int -> Int
bytesToElemCount a
x (a -> Int -> Int
forall a. Unbox a => a -> Int -> Int
allocBytesToBytes a
x Int
bytes)
     in Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
n Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
1) Int
n

-- | The default chunk size by which the array creation routines increase the
-- size of the array when the array is grown linearly.
arrayChunkBytes :: Int
arrayChunkBytes :: Int
arrayChunkBytes = Int
1024

-------------------------------------------------------------------------------
-- Resizing
-------------------------------------------------------------------------------

-- | Round the second argument down to multiples of the first argument.
{-# INLINE roundDownTo #-}
roundDownTo :: Int -> Int -> Int
roundDownTo :: Int -> Int -> Int
roundDownTo Int
elemSize Int
size = Int
size Int -> Int -> Int
forall a. Num a => a -> a -> a
- (Int
size Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` Int
elemSize)

-- NOTE: we are passing elemSize explicitly to avoid an Unboxed constraint.
-- Since this is not inlined, Unboxed constraint leads to dictionary passing
-- which complicates some inspection tests.
--
{-# NOINLINE reallocExplicitAs #-}
reallocExplicitAs :: PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs :: forall a.
PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs PinnedState
ps Int
elemSize Int
newCapacityInBytes MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    assertM(Int
arrEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrBound)

    let newCapMaxInBytes :: Int
newCapMaxInBytes = Int -> Int
roundUpLargeArray Int
newCapacityInBytes
        oldSizeInBytes :: Int
oldSizeInBytes = Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart
        -- XXX Should we round up instead?
        newCapInBytes :: Int
newCapInBytes = Int -> Int -> Int
roundDownTo Int
elemSize Int
newCapMaxInBytes
        newLenInBytes :: Int
newLenInBytes = Int -> Int -> Int
forall a. Ord a => a -> a -> a
min Int
oldSizeInBytes Int
newCapInBytes

    Bool -> IO () -> IO ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
oldSizeInBytes Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` Int
elemSize Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0) (() -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    Bool -> IO () -> IO ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
newLenInBytes Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0) (() -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    Bool -> IO () -> IO ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
newLenInBytes Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` Int
elemSize Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0) (() -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ())

    MutByteArray
contents <-
        PinnedState -> Int -> MutByteArray -> Int -> Int -> IO MutByteArray
Unboxed.reallocSliceAs
            PinnedState
ps Int
newCapInBytes MutByteArray
arrContents Int
arrStart Int
newLenInBytes

    MutArray a -> IO (MutArray a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> IO (MutArray a)) -> MutArray a -> IO (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutArray
        { arrStart :: Int
arrStart = Int
0
        , arrContents :: MutByteArray
arrContents = MutByteArray
contents
        , arrEnd :: Int
arrEnd   = Int
newLenInBytes
        , arrBound :: Int
arrBound = Int
newCapInBytes
        }

-- XXX We may also need reallocAs to allocate as pinned/unpinned explicitly. In
-- fact clone/clone' can be implemented using reallocAs.

-- | @realloc newCapacity array@ reallocates the array to the specified
-- capacity in bytes.
--
-- If the new size is less than the original array the array gets truncated.
-- If the new size is not a multiple of array element size then it is rounded
-- down to multiples of array size.  If the new size is more than
-- 'largeObjectThreshold' then it is rounded up to the block size (4K).
--
-- If the original array is pinned, the newly allocated array is also pinned.
{-# INLINABLE reallocBytes #-}
realloc, reallocBytes :: forall m a. (MonadIO m, Unbox a) => Int -> MutArray a -> m (MutArray a)
reallocBytes :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
reallocBytes Int
bytes MutArray a
arr =
    let ps :: PinnedState
ps =
            if MutArray a -> Bool
forall a. MutArray a -> Bool
isPinned MutArray a
arr
            then PinnedState
Pinned
            else PinnedState
Unpinned
     in IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
forall a.
PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs PinnedState
ps (SIZE_OF(a)) bytes arr

-- | @reallocBytesWith label capSizer minIncrBytes array@. The label is used
-- in error messages and the capSizer is used to determine the capacity of the
-- new array in bytes given the current byte length of the array.
reallocBytesWith :: forall m a. (MonadIO m , Unbox a) =>
       String
    -> (Int -> Int)
    -> Int
    -> MutArray a
    -> m (MutArray a)
reallocBytesWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
reallocBytesWith [Char]
label Int -> Int
capSizer Int
minIncrBytes MutArray a
arr = do
    let oldSizeBytes :: Int
oldSizeBytes = MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
arr Int -> Int -> Int
forall a. Num a => a -> a -> a
- MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
arr
        newCapBytes :: Int
newCapBytes = Int -> Int
capSizer Int
oldSizeBytes
        newSizeBytes :: Int
newSizeBytes = Int
oldSizeBytes Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
minIncrBytes
        safeCapBytes :: Int
safeCapBytes = Int -> Int -> Int
forall a. Ord a => a -> a -> a
max Int
newCapBytes Int
newSizeBytes
    assertM(Int
safeCapBytes Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
newSizeBytes Bool -> Bool -> Bool
|| [Char] -> Bool
forall a. (?callStack::CallStack) => [Char] -> a
error (Int -> [Char]
forall a. Show a => a -> [Char]
badSize Int
newSizeBytes))

    Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
realloc Int
safeCapBytes MutArray a
arr

    where

    badSize :: a -> [Char]
badSize a
newSize =
        [[Char]] -> [Char]
forall (t :: * -> *) a. Foldable t => t [a] -> [a]
Prelude.concat
            [ [Char]
label
            , [Char]
": new array size (in bytes) is less than required size "
            , a -> [Char]
forall a. Show a => a -> [Char]
show a
newSize
            , [Char]
". Please check the sizing function passed."
            ]

-- | @growTo newCapacity array@ changes the total capacity of the array so that
-- it is enough to hold the specified number of elements.  Nothing is done if
-- the specified capacity is less than the length of the array.
--
-- If the capacity is more than 'largeObjectThreshold' then it is rounded up to
-- the block size (4K).
--
-- Nothing is done if the requested capacity is <= 0.
--
-- /Pre-release/
{-# INLINE growTo #-}
growTo, grow :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> m (MutArray a)
growTo :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
growTo Int
nElems arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let req :: Int
req = SIZE_OF(a) * nElems
        cap :: Int
cap = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart
    if Int
req Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
cap
    then MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
arr
    else Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
realloc Int
req MutArray a
arr

{-# INLINE grow #-}
grow :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
grow = Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
growTo

-- | Like 'growTo' but specifies the required reserve (unused) capacity rather
-- than the total capacity. Increases the reserve capacity, if required, to at
-- least the given amount.
--
-- Nothing is done if the requested capacity is <= 0.
--
{-# INLINE growBy #-}
growBy :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> m (MutArray a)
growBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
growBy Int
nElems arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let req :: Int
req = Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ SIZE_OF(a) * nElems
        cap :: Int
cap = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart
    if Int
req Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
cap
    then MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
arr
    else Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
realloc Int
req MutArray a
arr

{-# DEPRECATED resize "Please use growTo instead." #-}
{-# INLINE resize #-}
resize :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> m (MutArray a)
resize :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
resize = Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
grow

-- | Like 'growTo' but if the requested byte capacity is more than
-- 'largeObjectThreshold' then it is rounded up to the closest power of 2.
--
-- Nothing is done if the requested capacity is <= 0.
--
-- /Pre-release/
{-# INLINE growExp #-}
growExp :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> m (MutArray a)
growExp :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
growExp Int
nElems arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let req :: Int
req = Int -> Int
roundUpLargeArray (SIZE_OF(a) * nElems)
        req1 :: Int
req1 =
            if Int
req Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
largeObjectThreshold
            then Int -> Int
roundUpToPower2 Int
req
            else Int
req
        cap :: Int
cap = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart
    if Int
req1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
cap
    then MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
arr
    else Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
realloc Int
req1 MutArray a
arr

{-# DEPRECATED resizeExp "Please use growExp instead." #-}
{-# INLINE resizeExp #-}
resizeExp :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> m (MutArray a)
resizeExp :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
resizeExp = Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
growExp

-- | Resize the allocated memory to drop any reserved free space at the end of
-- the array and reallocate it to reduce wastage.
--
-- Up to 25% wastage is allowed to avoid reallocations.  If the capacity is
-- more than 'largeObjectThreshold' then free space up to the 'blockSize' is
-- retained.
--
-- /Pre-release/
{-# INLINE rightSize #-}
rightSize :: forall m a. (MonadIO m, Unbox a) => MutArray a -> m (MutArray a)
rightSize :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> m (MutArray a)
rightSize arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
arrEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrBound) (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    let start :: Int
start = Int
arrStart
        len :: Int
len = Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
start
        cap :: Int
cap = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
start
        target :: Int
target = Int -> Int
roundUpLargeArray Int
len
        waste :: Int
waste = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrEnd
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
target Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
len) (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
len Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` SIZE_OF(a) == 0) (return ())
    -- We trade off some wastage (25%) to avoid reallocations and copying.
    if Int
target Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
cap Bool -> Bool -> Bool
&& Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
3 Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
waste
    then Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
realloc Int
target MutArray a
arr
    else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
arr

-- | Reset the array end position to start, thus truncating the array to 0
-- length, making it empty. The capacity of the array remains unchanged. The
-- array refers to the same memory as before.
{-# INLINE vacate #-}
vacate :: MutArray a -> MutArray a
vacate :: forall a. MutArray a -> MutArray a
vacate MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
arrContents Int
arrStart Int
arrStart Int
arrBound

-------------------------------------------------------------------------------
-- Snoc
-------------------------------------------------------------------------------

-- XXX We can possibly use a smallMutableByteArray to hold the start, end,
-- bound pointers.  Using fully mutable handle will ensure that we do not have
-- multiple references to the same array of different lengths lying around and
-- potentially misused. In that case "snoc" need not return a new array (snoc
-- :: MutArray a -> a -> m ()), it will just modify the old reference.  The array
-- length will be mutable.  This means the length function would also be
-- monadic.  Mutable arrays would behave more like files that grow in that
-- case.

-- | Snoc using a 'Ptr'. Low level reusable function.
--
-- /Internal/
{-# INLINE snocNewEnd #-}
snocNewEnd :: (MonadIO m, Unbox a) => Int -> MutArray a -> a -> m (MutArray a)
snocNewEnd :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m (MutArray a)
snocNewEnd Int
newEnd arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a
x = IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ do
    Bool -> IO () -> IO ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrBound) (() -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
arrEnd MutByteArray
arrContents a
x
    MutArray a -> IO (MutArray a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> IO (MutArray a)) -> MutArray a -> IO (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutArray a
arr {arrEnd :: Int
arrEnd = Int
newEnd}

-- | Really really unsafe, appends the element into the first array, may
-- cause silent data corruption or if you are lucky a segfault if the first
-- array does not have enough space to append the element.
--
-- /Internal/
{-# INLINE unsafeSnoc #-}
snocUnsafe, unsafeSnoc :: forall m a. (MonadIO m, Unbox a) =>
    MutArray a -> a -> m (MutArray a)
unsafeSnoc :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
unsafeSnoc arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = Int -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m (MutArray a)
snocNewEnd (INDEX_NEXT(arrEnd,a)) arr

-- | Like 'snoc' but does not reallocate when pre-allocated array capacity
-- becomes full.
--
-- /Internal/
{-# INLINE snocMay #-}
snocMay :: forall m a. (MonadIO m, Unbox a) =>
    MutArray a -> a -> m (Maybe (MutArray a))
snocMay :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (Maybe (MutArray a))
snocMay arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a
x = do
    let newEnd :: Int
newEnd = INDEX_NEXT(arrEnd,a)
    if Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrBound
    then MutArray a -> Maybe (MutArray a)
forall a. a -> Maybe a
Just (MutArray a -> Maybe (MutArray a))
-> m (MutArray a) -> m (Maybe (MutArray a))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m (MutArray a)
snocNewEnd Int
newEnd MutArray a
arr a
x
    else Maybe (MutArray a) -> m (Maybe (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (MutArray a)
forall a. Maybe a
Nothing

-- | Increments the capacity such that there is at least one unused slot even
-- if the sizer returns a size less than or equal to current size.

-- NOINLINE to move it out of the way and not pollute the instruction cache.
{-# NOINLINE snocWithRealloc #-}
snocWithRealloc :: forall m a. (MonadIO m, Unbox a) =>
       (Int -> Int)
    -> MutArray a
    -> a
    -> m (MutArray a)
snocWithRealloc :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray a -> a -> m (MutArray a)
snocWithRealloc Int -> Int
sizer MutArray a
arr a
x = do
    MutArray a
arr1 <- [Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
reallocBytesWith [Char]
"snocWith" Int -> Int
sizer (SIZE_OF(a)) arr
    MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
unsafeSnoc MutArray a
arr1 a
x

-- XXX sizer should use elements instead of bytes? That may increase the cost
-- but sizing is not a frequent operation.

-- | @snocWith sizer arr elem@ mutates @arr@ to append @elem@. The used length
-- of the array increases by 1.
--
-- If there is no reserved space available in @arr@ it is reallocated to a size
-- in bytes determined by the @sizer oldSizeBytes@ function, where
-- @oldSizeBytes@ is the original size of the array in bytes. The sizer
-- function should return a capacity more than or equal to the current used
-- size. If the capacity returned is less than or equal to the current used
-- size, the array is still grown by one element.
--
-- If the new array size is more than 'largeObjectThreshold' then it is rounded
-- up to 'blockSize'.
--
-- Note that the returned array may be a mutated version of the original array.
--
-- /Pre-release/
{-# INLINE snocWith #-}
snocWith :: forall m a. (MonadIO m, Unbox a) =>
       (Int -> Int)
    -> MutArray a
    -> a
    -> m (MutArray a)
snocWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray a -> a -> m (MutArray a)
snocWith Int -> Int
sizer MutArray a
arr a
x = do
    let newEnd :: Int
newEnd = INDEX_NEXT(arrEnd arr,a)
    if Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= MutArray a -> Int
forall a. MutArray a -> Int
arrBound MutArray a
arr
    then Int -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m (MutArray a)
snocNewEnd Int
newEnd MutArray a
arr a
x
    else (Int -> Int) -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray a -> a -> m (MutArray a)
snocWithRealloc Int -> Int
sizer MutArray a
arr a
x

-- | The array is mutated to append an additional element to it. If there
-- is no reserved space available in the array then it is reallocated to grow
-- it by 'arrayChunkBytes' rounded up to 'blockSize' when the size becomes more
-- than 'largeObjectThreshold'.
--
-- Note that the returned array may be a mutated version of the original array.
--
-- Performs O(n^2) copies to grow but is thrifty on memory.
--
-- /Pre-release/
{-# INLINE snocLinear #-}
snocLinear :: forall m a. (MonadIO m, Unbox a) => MutArray a -> a -> m (MutArray a)
snocLinear :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
snocLinear = (Int -> Int) -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray a -> a -> m (MutArray a)
snocWith (Int -> Int -> Int
forall a. Num a => a -> a -> a
+ a -> Int -> Int
forall a. Unbox a => a -> Int -> Int
allocBytesToBytes (a
forall a. (?callStack::CallStack) => a
undefined :: a) Int
arrayChunkBytes)

-- | The array is mutated to append an additional element to it.
--
-- If there is no reserved space available in the array then it is reallocated
-- to grow it by adding space for the requested number of elements, the new
-- size is rounded up to 'blockSize' when the size becomes more than
-- 'largeObjectThreshold'. If the size specified is <= 0 then the array is
-- grown by one element.
--
-- Note that the returned array may be a mutated version of the original array.
--
-- Performs O(n^2) copies to grow but is thrifty on memory compared to 'snoc'.
--
-- /Pre-release/
{-# INLINE snocGrowBy #-}
snocGrowBy :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> a -> m (MutArray a)
snocGrowBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m (MutArray a)
snocGrowBy Int
n = (Int -> Int) -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray a -> a -> m (MutArray a)
snocWith (Int -> Int -> Int
forall a. Num a => a -> a -> a
+ (Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)))

-- | The array is mutated to append an additional element to it. If there is no
-- reserved space available in the array then it is reallocated to double the
-- original size and aligned to a power of 2.
--
-- This is useful to reduce allocations when appending unknown number of
-- elements.
--
-- Note that the returned array may be a mutated version of the original array.
--
-- Performs only O(n * log n) copies to grow, but is liberal with memory
-- allocation compared to 'snocGrowBy'.
--
{-# INLINE snoc #-}
snoc :: forall m a. (MonadIO m, Unbox a) => MutArray a -> a -> m (MutArray a)
snoc :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
snoc = (Int -> Int) -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray a -> a -> m (MutArray a)
snocWith Int -> Int
f

    where

    f :: Int -> Int
f Int
oldSize =
        if Int -> Bool
isPower2 Int
oldSize
        then Int
oldSize Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
2
        else Int -> Int
roundUpToPower2 Int
oldSize Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
2

-------------------------------------------------------------------------------
-- Serialization/Deserialization using Unbox
-------------------------------------------------------------------------------

{-# INLINE pokeNewEnd #-}
pokeNewEnd :: (MonadIO m, Unbox a) =>
    Int -> MutArray Word8 -> a -> m (MutArray Word8)
pokeNewEnd :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray Word8 -> a -> m (MutArray Word8)
pokeNewEnd Int
newEnd arr :: MutArray Word8
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a
x = IO (MutArray Word8) -> m (MutArray Word8)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray Word8) -> m (MutArray Word8))
-> IO (MutArray Word8) -> m (MutArray Word8)
forall a b. (a -> b) -> a -> b
$ do
    Bool -> IO () -> IO ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrBound) (() -> IO ()
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    IO () -> IO ()
forall a. IO a -> IO a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> IO ()) -> IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
arrEnd MutByteArray
arrContents a
x
    MutArray Word8 -> IO (MutArray Word8)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray Word8 -> IO (MutArray Word8))
-> MutArray Word8 -> IO (MutArray Word8)
forall a b. (a -> b) -> a -> b
$ MutArray Word8
arr {arrEnd :: Int
arrEnd = Int
newEnd}

-- | Really really unsafe, unboxes a Haskell type and appends the resulting
-- bytes to the byte array, may cause silent data corruption or if you are
-- lucky a segfault if the array does not have enough space to append the
-- element.
--
-- /Internal/
{-# INLINE unsafePoke #-}
unsafePoke :: forall m a. (MonadIO m, Unbox a) =>
    MutArray Word8 -> a -> m (MutArray Word8)
unsafePoke :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray Word8 -> a -> m (MutArray Word8)
unsafePoke arr :: MutArray Word8
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = Int -> MutArray Word8 -> a -> m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray Word8 -> a -> m (MutArray Word8)
pokeNewEnd (Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
+ SIZE_OF(a)) arr

-- | Skip the specified number of bytes in the array. The data in the skipped
-- region remains uninitialzed.
{-# INLINE unsafePokeSkip #-}
pokeSkipUnsafe, unsafePokeSkip :: Int -> MutArray Word8 -> MutArray Word8
unsafePokeSkip :: Int -> MutArray Word8 -> MutArray Word8
unsafePokeSkip Int
n arr :: MutArray Word8
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =  do
    let newEnd :: Int
newEnd = Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
n
     in Bool -> MutArray Word8 -> MutArray Word8
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrBound) (MutArray Word8
arr {arrEnd :: Int
arrEnd = Int
newEnd})

-- | Like 'poke' but does not grow the array when pre-allocated array
-- capacity becomes full.
--
-- /Internal/
{-# INLINE pokeMay #-}
pokeAppendMay, pokeMay :: forall m a. (MonadIO m, Unbox a) =>
    MutArray Word8 -> a -> m (Maybe (MutArray Word8))
pokeMay :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray Word8 -> a -> m (Maybe (MutArray Word8))
pokeMay arr :: MutArray Word8
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} a
x = IO (Maybe (MutArray Word8)) -> m (Maybe (MutArray Word8))
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Maybe (MutArray Word8)) -> m (Maybe (MutArray Word8)))
-> IO (Maybe (MutArray Word8)) -> m (Maybe (MutArray Word8))
forall a b. (a -> b) -> a -> b
$ do
    let newEnd :: Int
newEnd = Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
+ SIZE_OF(a)
    if Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrBound
    then MutArray Word8 -> Maybe (MutArray Word8)
forall a. a -> Maybe a
Just (MutArray Word8 -> Maybe (MutArray Word8))
-> IO (MutArray Word8) -> IO (Maybe (MutArray Word8))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> MutArray Word8 -> a -> IO (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray Word8 -> a -> m (MutArray Word8)
pokeNewEnd Int
newEnd MutArray Word8
arr a
x
    else Maybe (MutArray Word8) -> IO (Maybe (MutArray Word8))
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (MutArray Word8)
forall a. Maybe a
Nothing

{-# NOINLINE pokeWithRealloc #-}
pokeWithRealloc :: forall m a. (MonadIO m, Unbox a) =>
       (Int -> Int)
    -> MutArray Word8
    -> a
    -> m (MutArray Word8)
pokeWithRealloc :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray Word8 -> a -> m (MutArray Word8)
pokeWithRealloc Int -> Int
sizer MutArray Word8
arr a
x = do
    MutArray Word8
arr1 <- IO (MutArray Word8) -> m (MutArray Word8)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray Word8) -> m (MutArray Word8))
-> IO (MutArray Word8) -> m (MutArray Word8)
forall a b. (a -> b) -> a -> b
$ [Char]
-> (Int -> Int) -> Int -> MutArray Word8 -> IO (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
reallocBytesWith [Char]
"pokeWithRealloc" Int -> Int
sizer (SIZE_OF(a)) arr
    MutArray Word8 -> a -> m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray Word8 -> a -> m (MutArray Word8)
unsafePoke MutArray Word8
arr1 a
x

{-# INLINE pokeWith #-}
pokeWith :: forall m a. (MonadIO m, Unbox a) =>
       (Int -> Int)
    -> MutArray Word8
    -> a
    -> m (MutArray Word8)
pokeWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray Word8 -> a -> m (MutArray Word8)
pokeWith Int -> Int
allocSize MutArray Word8
arr a
x = IO (MutArray Word8) -> m (MutArray Word8)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray Word8) -> m (MutArray Word8))
-> IO (MutArray Word8) -> m (MutArray Word8)
forall a b. (a -> b) -> a -> b
$ do
    let newEnd :: Int
newEnd = MutArray Word8 -> Int
forall a. MutArray a -> Int
arrEnd MutArray Word8
arr Int -> Int -> Int
forall a. Num a => a -> a -> a
+ SIZE_OF(a)
    if Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= MutArray Word8 -> Int
forall a. MutArray a -> Int
arrBound MutArray Word8
arr
    then Int -> MutArray Word8 -> a -> IO (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray Word8 -> a -> m (MutArray Word8)
pokeNewEnd Int
newEnd MutArray Word8
arr a
x
    else (Int -> Int) -> MutArray Word8 -> a -> IO (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray Word8 -> a -> m (MutArray Word8)
pokeWithRealloc Int -> Int
allocSize MutArray Word8
arr a
x

-- | Unbox a Haskell type and append the resulting bytes to a mutable byte
-- array. The array is grown exponentially when more space is needed.
--
-- Like 'snoc' except that the value is unboxed to the byte array.
--
-- Note: If you are serializing a large number of small fields, and the types
-- are statically known, then it may be more efficient to declare a record of
-- those fields and derive an 'Unbox' instance of the entire record.
--
{-# INLINE poke #-}
pokeAppend, poke :: forall m a. (MonadIO m, Unbox a) =>
    MutArray Word8 -> a -> m (MutArray Word8)
poke :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray Word8 -> a -> m (MutArray Word8)
poke = (Int -> Int) -> MutArray Word8 -> a -> m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray Word8 -> a -> m (MutArray Word8)
pokeWith Int -> Int
f

    where

    f :: Int -> Int
f Int
oldSize =
        if Int -> Bool
isPower2 Int
oldSize
        then Int
oldSize Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
2
        else Int -> Int
roundUpToPower2 Int
oldSize Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
2

-- | Really really unsafe, create a Haskell value from an unboxed byte array,
-- does not check if the array is big enough, may return garbage or if you are
-- lucky may cause a segfault.
--
-- /Internal/
{-# INLINE unsafePeek #-}
peekUnconsUnsafe, unsafePeek :: forall m a. (MonadIO m, Unbox a) =>
    MutArray Word8 -> m (a, MutArray Word8)
unsafePeek :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray Word8 -> m (a, MutArray Word8)
unsafePeek MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let start1 :: Int
start1 = Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ SIZE_OF(a)
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
start1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrEnd) (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    IO (a, MutArray Word8) -> m (a, MutArray Word8)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (a, MutArray Word8) -> m (a, MutArray Word8))
-> IO (a, MutArray Word8) -> m (a, MutArray Word8)
forall a b. (a -> b) -> a -> b
$ do
        a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
arrStart MutByteArray
arrContents
        (a, MutArray Word8) -> IO (a, MutArray Word8)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (a
r, MutByteArray -> Int -> Int -> Int -> MutArray Word8
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
arrContents Int
start1 Int
arrEnd Int
arrBound)

-- | Discard the specified number of bytes at the beginning of the array.
{-# INLINE unsafePeekSkip #-}
peekSkipUnsafe, unsafePeekSkip :: Int -> MutArray Word8 -> MutArray Word8
unsafePeekSkip :: Int -> MutArray Word8 -> MutArray Word8
unsafePeekSkip Int
n MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    let start1 :: Int
start1 = Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
n
     in Bool -> MutArray Word8 -> MutArray Word8
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
start1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrEnd) (MutByteArray -> Int -> Int -> Int -> MutArray Word8
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
arrContents Int
start1 Int
arrEnd Int
arrBound)

-- | Create a Haskell value from its unboxed representation from the head of a
-- byte array, return the value and the remaining array.
--
-- Like 'uncons' except that the value is deserialized from the byte array.
--
-- Note: If you are deserializing a large number of small fields, and the types
-- are statically known, then it may be more efficient to declare a record of
-- those fields and derive an 'Unbox' instance of the entire record.
{-# INLINE peek #-}
peekUncons, peek :: forall m a. (MonadIO m, Unbox a) =>
    MutArray Word8 -> m (Maybe a, MutArray Word8)
peek :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray Word8 -> m (Maybe a, MutArray Word8)
peek arr :: MutArray Word8
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let start1 :: Int
start1 = Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ SIZE_OF(a)
    if Int
start1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
arrEnd
    then (Maybe a, MutArray Word8) -> m (Maybe a, MutArray Word8)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe a
forall a. Maybe a
Nothing, MutArray Word8
arr)
    else IO (Maybe a, MutArray Word8) -> m (Maybe a, MutArray Word8)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Maybe a, MutArray Word8) -> m (Maybe a, MutArray Word8))
-> IO (Maybe a, MutArray Word8) -> m (Maybe a, MutArray Word8)
forall a b. (a -> b) -> a -> b
$ do
        a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
arrStart MutByteArray
arrContents
        (Maybe a, MutArray Word8) -> IO (Maybe a, MutArray Word8)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Maybe a
forall a. a -> Maybe a
Just a
r, MutByteArray -> Int -> Int -> Int -> MutArray Word8
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
arrContents Int
start1 Int
arrEnd Int
arrBound)

-------------------------------------------------------------------------------
-- Random reads
-------------------------------------------------------------------------------

-- XXX Can this be deduplicated with array/foreign

-- | Return the element at the specified index without checking the bounds.
--
-- Unsafe because it does not check the bounds of the array.
{-# INLINE_NORMAL unsafeGetIndex #-}
getIndexUnsafe, unsafeGetIndex :: forall m a. (MonadIO m, Unbox a) => Int -> MutArray a -> m a
unsafeGetIndex :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m a
unsafeGetIndex Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let index :: Int
index = Int
INDEX_OF(arrStart,i,a)
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& INDEX_VALID(index,arrEnd,a)) (return ())
    IO a -> m a
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO a -> m a) -> IO a -> m a
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
index MutByteArray
arrContents

-- | /O(1)/ Lookup the element at the given index. Index starts from 0.
--
{-# INLINE getIndex #-}
getIndex :: forall m a. (MonadIO m, Unbox a) => Int -> MutArray a -> m (Maybe a)
getIndex :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (Maybe a)
getIndex Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let index :: Int
index = Int
INDEX_OF(arrStart,i,a)
    if Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& INDEX_VALID(index,arrEnd,a)
    then IO (Maybe a) -> m (Maybe a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Maybe a) -> m (Maybe a)) -> IO (Maybe a) -> m (Maybe a)
forall a b. (a -> b) -> a -> b
$ a -> Maybe a
forall a. a -> Maybe a
Just (a -> Maybe a) -> IO a -> IO (Maybe a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
index MutByteArray
arrContents
    else Maybe a -> m (Maybe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe a
forall a. Maybe a
Nothing

{-# INLINE_NORMAL unsafeGetIndexRev #-}
unsafeGetIndexRev :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> m a
unsafeGetIndexRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m a
unsafeGetIndexRev Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let index :: Int
index = RINDEX_OF(Proxy a -> Int
forall a. Unbox a => Proxy a -> Int
arrEnd,i,a)
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& INDEX_VALID(index,arrEnd,a)) (return ())
    IO a -> m a
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO a -> m a) -> IO a -> m a
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
index MutByteArray
arrContents

-- | /O(1)/ Lookup the element at the given index from the end of the array.
-- Index starts from 0.
--
-- Slightly faster than computing the forward index and using getIndex.
--
{-# INLINE getIndexRev #-}
getIndexRev :: forall m a. (MonadIO m, Unbox a) => Int -> MutArray a -> m a
getIndexRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m a
getIndexRev Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = do
    let index :: Int
index = RINDEX_OF(Proxy a -> Int
forall a. Unbox a => Proxy a -> Int
arrEnd,i,a)
    if Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& Int
index Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
arrStart
    then IO a -> m a
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO a -> m a) -> IO a -> m a
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
index MutByteArray
arrContents
    else [Char] -> Int -> m a
forall a. [Char] -> Int -> a
invalidIndex [Char]
"getIndexRev" Int
i

data GetIndicesState contents start end st =
    GetIndicesState contents start end st

{-# INLINE indexReaderWith #-}
indexReaderWith :: (Monad m, Unbox a) =>
    (forall b. IO b -> m b) -> D.Stream m Int -> Unfold m (MutArray a) a
indexReaderWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m Int -> Unfold m (MutArray a) a
indexReaderWith forall b. IO b -> m b
liftio (D.Stream State StreamK m Int -> s -> m (Step s Int)
stepi s
sti) = (GetIndicesState MutByteArray Int Int s
 -> m (Step (GetIndicesState MutByteArray Int Int s) a))
-> (MutArray a -> m (GetIndicesState MutByteArray Int Int s))
-> Unfold m (MutArray a) a
forall (m :: * -> *) a b s.
(s -> m (Step s b)) -> (a -> m s) -> Unfold m a b
Unfold GetIndicesState MutByteArray Int Int s
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
forall {a}.
Unbox a =>
GetIndicesState MutByteArray Int Int s
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
step MutArray a -> m (GetIndicesState MutByteArray Int Int s)
forall {m :: * -> *} {a}.
Monad m =>
MutArray a -> m (GetIndicesState MutByteArray Int Int s)
inject

    where

    inject :: MutArray a -> m (GetIndicesState MutByteArray Int Int s)
inject (MutArray MutByteArray
contents Int
start Int
end Int
_) =
        GetIndicesState MutByteArray Int Int s
-> m (GetIndicesState MutByteArray Int Int s)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (GetIndicesState MutByteArray Int Int s
 -> m (GetIndicesState MutByteArray Int Int s))
-> GetIndicesState MutByteArray Int Int s
-> m (GetIndicesState MutByteArray Int Int s)
forall a b. (a -> b) -> a -> b
$ MutByteArray
-> Int -> Int -> s -> GetIndicesState MutByteArray Int Int s
forall contents start end st.
contents
-> start -> end -> st -> GetIndicesState contents start end st
GetIndicesState MutByteArray
contents Int
start Int
end s
sti

    {-# INLINE_LATE step #-}
    step :: GetIndicesState MutByteArray Int Int s
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
step (GetIndicesState MutByteArray
contents Int
start Int
end s
st) = do
        Step s Int
r <- State StreamK m Int -> s -> m (Step s Int)
stepi State StreamK m Int
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState s
st
        case Step s Int
r of
            D.Yield Int
i s
s -> do
                Maybe a
x <- IO (Maybe a) -> m (Maybe a)
forall b. IO b -> m b
liftio (IO (Maybe a) -> m (Maybe a)) -> IO (Maybe a) -> m (Maybe a)
forall a b. (a -> b) -> a -> b
$ Int -> MutArray a -> IO (Maybe a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (Maybe a)
getIndex Int
i (MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
contents Int
start Int
end Int
forall a. (?callStack::CallStack) => a
undefined)
                case Maybe a
x of
                    Just a
v -> Step (GetIndicesState MutByteArray Int Int s) a
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (GetIndicesState MutByteArray Int Int s) a
 -> m (Step (GetIndicesState MutByteArray Int Int s) a))
-> Step (GetIndicesState MutByteArray Int Int s) a
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
forall a b. (a -> b) -> a -> b
$ a
-> GetIndicesState MutByteArray Int Int s
-> Step (GetIndicesState MutByteArray Int Int s) a
forall s a. a -> s -> Step s a
D.Yield a
v (MutByteArray
-> Int -> Int -> s -> GetIndicesState MutByteArray Int Int s
forall contents start end st.
contents
-> start -> end -> st -> GetIndicesState contents start end st
GetIndicesState MutByteArray
contents Int
start Int
end s
s)
                    Maybe a
Nothing -> [Char] -> m (Step (GetIndicesState MutByteArray Int Int s) a)
forall a. (?callStack::CallStack) => [Char] -> a
error [Char]
"Invalid Index"
            D.Skip s
s -> Step (GetIndicesState MutByteArray Int Int s) a
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (GetIndicesState MutByteArray Int Int s) a
 -> m (Step (GetIndicesState MutByteArray Int Int s) a))
-> Step (GetIndicesState MutByteArray Int Int s) a
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
forall a b. (a -> b) -> a -> b
$ GetIndicesState MutByteArray Int Int s
-> Step (GetIndicesState MutByteArray Int Int s) a
forall s a. s -> Step s a
D.Skip (MutByteArray
-> Int -> Int -> s -> GetIndicesState MutByteArray Int Int s
forall contents start end st.
contents
-> start -> end -> st -> GetIndicesState contents start end st
GetIndicesState MutByteArray
contents Int
start Int
end s
s)
            Step s Int
D.Stop -> Step (GetIndicesState MutByteArray Int Int s) a
-> m (Step (GetIndicesState MutByteArray Int Int s) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step (GetIndicesState MutByteArray Int Int s) a
forall s a. Step s a
D.Stop

{-# DEPRECATED getIndicesWith "Please use indexReaderWith instead." #-}
{-# INLINE getIndicesWith #-}
getIndicesWith :: (Monad m, Unbox a) =>
    (forall b. IO b -> m b) -> D.Stream m Int -> Unfold m (MutArray a) a
getIndicesWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m Int -> Unfold m (MutArray a) a
getIndicesWith = (forall b. IO b -> m b) -> Stream m Int -> Unfold m (MutArray a) a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m Int -> Unfold m (MutArray a) a
indexReaderWith

-- | Given an unfold that generates array indices, read the elements on those
-- indices from the supplied MutArray. An error is thrown if an index is out of
-- bounds.
--
-- /Pre-release/
{-# INLINE indexReader #-}
indexReader :: (MonadIO m, Unbox a) => Stream m Int -> Unfold m (MutArray a) a
indexReader :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m Int -> Unfold m (MutArray a) a
indexReader = (forall b. IO b -> m b) -> Stream m Int -> Unfold m (MutArray a) a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m Int -> Unfold m (MutArray a) a
indexReaderWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

-- XXX DO NOT REMOVE, change the signature to use Stream instead of unfold
{-# DEPRECATED getIndices "Please use indexReader instead." #-}
{-# INLINE getIndices #-}
getIndices :: (MonadIO m, Unbox a) => Stream m Int -> Unfold m (MutArray a) a
getIndices :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m Int -> Unfold m (MutArray a) a
getIndices = Stream m Int -> Unfold m (MutArray a) a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m Int -> Unfold m (MutArray a) a
indexReader

-------------------------------------------------------------------------------
-- Subarrays
-------------------------------------------------------------------------------

-- XXX We can also get immutable slices.
-- XXX sliceFromLen for a stream of slices starting from a given index

-- | /O(1)/ Slice an array in constant time.
--
-- Unsafe: The bounds of the slice are not checked.
--
-- /Unsafe/
--
-- /Pre-release/
{-# INLINE unsafeSliceOffLen #-}
unsafeSliceOffLen, getSliceUnsafe, unsafeGetSlice :: forall a. Unbox a
    => Int -- ^ from index
    -> Int -- ^ length of the slice
    -> MutArray a
    -> MutArray a
unsafeSliceOffLen :: forall a. Unbox a => Int -> Int -> MutArray a -> MutArray a
unsafeSliceOffLen Int
index Int
len (MutArray MutByteArray
contents Int
start Int
e Int
_) =
    let fp1 :: Int
fp1 = INDEX_OF(start,index,a)
        end :: Int
end = Int
fp1 Int -> Int -> Int
forall a. Num a => a -> a -> a
+ (Int
len Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a))
     in Bool -> MutArray a -> MutArray a
forall a. (?callStack::CallStack) => Bool -> a -> a
assert
            (Int
index Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& Int
end Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
e)
            -- Note: In a slice we always use bound = end so that the slice
            -- user cannot overwrite elements beyond the end of the slice.
            (MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
contents Int
fp1 Int
end Int
end)

-- | /O(1)/ Get a reference to a slice from a mutable array. Throws an error if
-- the slice extends out of the array bounds.
--
-- The capacity of the slice is the same as its length i.e. it does not have
-- any unused or reserved space at the end.
--
-- The slice shares the same underlying mutable array when created. However, if
-- the slice or the original array is reallocated by growing or shrinking then
-- it will be copied to new memory and they will no longer share the same
-- memory.
--
-- /Pre-release/
{-# INLINE sliceOffLen #-}
sliceOffLen, getSlice :: forall a. Unbox a =>
       Int -- ^ from index
    -> Int -- ^ length of the slice
    -> MutArray a
    -> MutArray a
sliceOffLen :: forall a. Unbox a => Int -> Int -> MutArray a -> MutArray a
sliceOffLen Int
index Int
len (MutArray MutByteArray
contents Int
start Int
e Int
_) =
    let fp1 :: Int
fp1 = INDEX_OF(start,index,a)
        end :: Int
end = Int
fp1 Int -> Int -> Int
forall a. Num a => a -> a -> a
+ (Int
len Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a))
     in if Int
index Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 Bool -> Bool -> Bool
&& Int
end Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
e
        -- Note: In a slice we always use bound = end so that the slice user
        -- cannot overwrite elements beyond the end of the slice.
        then MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
contents Int
fp1 Int
end Int
end
        else [Char] -> MutArray a
forall a. (?callStack::CallStack) => [Char] -> a
error
                ([Char] -> MutArray a) -> [Char] -> MutArray a
forall a b. (a -> b) -> a -> b
$ [Char]
"sliceOffLen: invalid slice, index "
                [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
index [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
" length " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
len

-------------------------------------------------------------------------------
-- In-place mutation algorithms
-------------------------------------------------------------------------------

-- XXX consider the bulk update/accumulation/permutation APIs from vector.

-- | You may not need to reverse an array because you can consume it in reverse
-- using 'readerRev'. To reverse large arrays you can read in reverse and write
-- to another array. However, in-place reverse can be useful to take adavantage
-- of cache locality and when you do not want to allocate additional memory.
--
{-# INLINE reverse #-}
reverse :: forall m a. (MonadIO m, Unbox a) => MutArray a -> m ()
reverse :: forall (m :: * -> *) a. (MonadIO m, Unbox a) => MutArray a -> m ()
reverse MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ do
    let l :: Int
l = Int
arrStart
        h :: Int
h = INDEX_PREV(arrEnd,a)
     in Int -> Int -> IO ()
swap Int
l Int
h

    where

    swap :: Int -> Int -> IO ()
swap Int
l Int
h = do
        Bool -> IO () -> IO ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
l Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
h) (IO () -> IO ()) -> IO () -> IO ()
forall a b. (a -> b) -> a -> b
$ do
            Proxy a -> MutByteArray -> Int -> Int -> IO ()
forall a. Unbox a => Proxy a -> MutByteArray -> Int -> Int -> IO ()
swapArrayByteIndices (Proxy a
forall {k} (t :: k). Proxy t
Proxy :: Proxy a) MutByteArray
arrContents Int
l Int
h
            Int -> Int -> IO ()
swap (INDEX_NEXT(l,a)) (INDEX_PREV(h,aInt
))

-- | Generate the next permutation of the sequence, returns False if this is
-- the last permutation.
--
-- /Unimplemented/
{-# INLINE permute #-}
permute :: MutArray a -> m Bool
permute :: forall a (m :: * -> *). MutArray a -> m Bool
permute = MutArray a -> m Bool
forall a. (?callStack::CallStack) => a
undefined

-- | Partition an array into two halves using a partitioning predicate. The
-- first half retains values where the predicate is 'False' and the second half
-- retains values where the predicate is 'True'.
--
-- /Pre-release/
{-# INLINE partitionBy #-}
partitionBy :: forall m a. (MonadIO m, Unbox a)
    => (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
partitionBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
partitionBy a -> Bool
f arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO (MutArray a, MutArray a) -> m (MutArray a, MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a, MutArray a) -> m (MutArray a, MutArray a))
-> IO (MutArray a, MutArray a) -> m (MutArray a, MutArray a)
forall a b. (a -> b) -> a -> b
$ do
    if Int
arrStart Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
arrEnd
    then (MutArray a, MutArray a) -> IO (MutArray a, MutArray a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a
arr, MutArray a
arr)
    else do
        Int
ptr <- Int -> Int -> IO Int
go Int
arrStart (INDEX_PREV(arrEnd,a))
        let pl :: MutArray a
pl = MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
arrContents Int
arrStart Int
ptr Int
ptr
            pr :: MutArray a
pr = MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
arrContents Int
ptr Int
arrEnd Int
arrEnd
        (MutArray a, MutArray a) -> IO (MutArray a, MutArray a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a
forall a. MutArray a
pl, MutArray a
forall a. MutArray a
pr)

    where

    -- Invariant low < high on entry, and on return as well
    moveHigh :: Int -> Int -> IO (Maybe (Int, a))
moveHigh Int
low Int
high = do
        a
h <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
high MutByteArray
arrContents
        if a -> Bool
f a
h
        then
            -- Correctly classified, continue the loop
            let high1 :: Int
high1 = INDEX_PREV(high,a)
             in if Int
low Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
high1
                then Maybe (Int, a) -> IO (Maybe (Int, a))
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (Int, a)
forall a. Maybe a
Nothing
                else Int -> Int -> IO (Maybe (Int, a))
moveHigh Int
low Int
high1
        else Maybe (Int, a) -> IO (Maybe (Int, a))
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ((Int, a) -> Maybe (Int, a)
forall a. a -> Maybe a
Just (Int
high, a
h)) -- incorrectly classified

    -- Keep a low pointer starting at the start of the array (first partition)
    -- and a high pointer starting at the end of the array (second partition).
    -- Keep incrementing the low ptr and decrementing the high ptr until both
    -- are wrongly classified, at that point swap the two and continue until
    -- the two pointer cross each other.
    --
    -- Invariants when entering this loop:
    -- low <= high
    -- Both low and high are valid locations within the array
    go :: Int -> Int -> IO Int
go Int
low Int
high = do
        a
l <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
low MutByteArray
arrContents
        if a -> Bool
f a
l
        then
            -- low is wrongly classified
            if Int
low Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
high
            then Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
low
            else do -- low < high
                Maybe (Int, a)
r <- Int -> Int -> IO (Maybe (Int, a))
moveHigh Int
low Int
high
                case Maybe (Int, a)
r of
                    Maybe (Int, a)
Nothing -> Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
low
                    Just (Int
high1, a
h) -> do -- low < high1
                        Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
low MutByteArray
arrContents a
h
                        Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
high1 MutByteArray
arrContents a
l
                        let low1 :: Int
low1 = INDEX_NEXT(low,a)
                            high2 :: Int
high2 = INDEX_PREV(high1,a)
                        if Int
low1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
high2
                        then Int -> Int -> IO Int
go Int
low1 Int
high2
                        else Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
low1 -- low1 > high2

        else do
            -- low is correctly classified
            let low1 :: Int
low1 = INDEX_NEXT(low,a)
            if Int
low Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
high
            then Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
low1
            else Int -> Int -> IO Int
go Int
low1 Int
high

-- | Shuffle corresponding elements from two arrays using a shuffle function.
-- If the shuffle function returns 'False' then do nothing otherwise swap the
-- elements. This can be used in a bottom up fold to shuffle or reorder the
-- elements.
--
-- /Unimplemented/
{-# INLINE shuffleBy #-}
shuffleBy :: (a -> a -> m Bool) -> MutArray a -> MutArray a -> m ()
shuffleBy :: forall a (m :: * -> *).
(a -> a -> m Bool) -> MutArray a -> MutArray a -> m ()
shuffleBy = (a -> a -> m Bool) -> MutArray a -> MutArray a -> m ()
forall a. (?callStack::CallStack) => a
undefined

-- XXX we can also make the folds partial by stopping at a certain level.
--
-- | @divideBy level partition array@  performs a top down hierarchical
-- recursive partitioning fold of items in the container using the given
-- function as the partition function.  Level indicates the level in the tree
-- where the fold would stop.
--
-- This performs a quick sort if the partition function is
-- 'partitionBy (< pivot)'.
--
-- /Unimplemented/
{-# INLINABLE divideBy #-}
divideBy ::
    Int -> (MutArray a -> m (MutArray a, MutArray a)) -> MutArray a -> m ()
divideBy :: forall a (m :: * -> *).
Int
-> (MutArray a -> m (MutArray a, MutArray a)) -> MutArray a -> m ()
divideBy = Int
-> (MutArray a -> m (MutArray a, MutArray a)) -> MutArray a -> m ()
forall a. (?callStack::CallStack) => a
undefined

-- | @mergeBy level merge array@ performs a pairwise bottom up fold recursively
-- merging the pairs using the supplied merge function. Level indicates the
-- level in the tree where the fold would stop.
--
-- This performs a random shuffle if the merge function is random.  If we
-- stop at level 0 and repeatedly apply the function then we can do a bubble
-- sort.
--
-- /Unimplemented/
mergeBy :: Int -> (MutArray a -> MutArray a -> m ()) -> MutArray a -> m ()
mergeBy :: forall a (m :: * -> *).
Int -> (MutArray a -> MutArray a -> m ()) -> MutArray a -> m ()
mergeBy = Int -> (MutArray a -> MutArray a -> m ()) -> MutArray a -> m ()
forall a. (?callStack::CallStack) => a
undefined

-- XXX Use vector instructions in arrays to find min/max/range faster

-- XXX If we can mutate the array then we can do pairwise processing to keep
-- min in the first slot and max in the second. Then compare adjacent mins and
-- keep the min of those in the first slot, and similarly for max. Thus
-- reducing the comparisons in binary fashion.
--
-- Or we can use mergeBy as defined above.
--
-- If we cannot mutate the array then we can (1) copy it and use the above
-- algo, or (2) stream the array and use pairwise concat.

-- | Find the minimum and maximum elements in the array using the provided
-- comparison function.
rangeBy :: (a -> a -> Ordering) -> MutArray a -> IO (Maybe (a, a))
rangeBy :: forall a. (a -> a -> Ordering) -> MutArray a -> IO (Maybe (a, a))
rangeBy = (a -> a -> Ordering) -> MutArray a -> IO (Maybe (a, a))
forall a. (?callStack::CallStack) => a
undefined

-------------------------------------------------------------------------------
-- Size
-------------------------------------------------------------------------------

-- | /O(1)/ Get the byte length of the array.
--
{-# INLINE byteLength #-}
byteLength :: MutArray a -> Int
byteLength :: forall a. MutArray a -> Int
byteLength MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    let len :: Int
len = Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart
    in Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0) Int
len

-- Note: try to avoid the use of length in performance sensitive internal
-- routines as it involves a costly 'div' operation. Instead use the end ptr
-- in the array to check the bounds etc.

-- | /O(1)/ Get the used length of the array i.e. the number of elements in the
-- array.
--
-- Note that 'byteLength' is less expensive than this operation, as 'length'
-- involves a costly division operation.
--
{-# INLINE length #-}
length :: forall a. Unbox a => MutArray a -> Int
length :: forall a. Unbox a => MutArray a -> Int
length MutArray a
arr =
    let elemSize :: Int
elemSize = SIZE_OF(a)
        blen :: Int
blen = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
     in Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
blen Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` Int
elemSize Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0) (Int
blen Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
elemSize)

-- | Get the total capacity of an array. An array may have space reserved
-- beyond the current used length of the array.
--
-- /Pre-release/
{-# INLINE byteCapacity #-}
byteCapacity :: MutArray a -> Int
byteCapacity :: forall a. MutArray a -> Int
byteCapacity MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    let len :: Int
len = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart
    in Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0) Int
len

-- | The remaining capacity in the array for appending more elements without
-- reallocation.
--
-- /Pre-release/
{-# INLINE bytesFree #-}
bytesFree :: MutArray a -> Int
bytesFree :: forall a. MutArray a -> Int
bytesFree MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    let n :: Int
n = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrEnd
    in Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
n Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0) Int
n

{-# INLINE capacity #-}
capacity :: forall a. Unbox a => MutArray a -> Int
capacity :: forall a. Unbox a => MutArray a -> Int
capacity MutArray a
arr =
    let elemSize :: Int
elemSize = SIZE_OF(a)
        bcap :: Int
bcap = MutArray a -> Int
forall a. MutArray a -> Int
byteCapacity MutArray a
arr
     in Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
bcap Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` Int
elemSize Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0) (Int
bcap Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
elemSize)

{-# INLINE free #-}
free :: forall a. Unbox a => MutArray a -> Int
free :: forall a. Unbox a => MutArray a -> Int
free MutArray a
arr =
    let elemSize :: Int
elemSize = SIZE_OF(a)
        bfree :: Int
bfree = MutArray a -> Int
forall a. MutArray a -> Int
bytesFree MutArray a
arr
     in Bool -> Int -> Int
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
bfree Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` Int
elemSize Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0) (Int
bfree Int -> Int -> Int
forall a. Integral a => a -> a -> a
`div` Int
elemSize)

-------------------------------------------------------------------------------
-- Streams of arrays - Creation
-------------------------------------------------------------------------------

data GroupState s contents start end bound
    = GroupStart s
    | GroupBuffer s contents start end bound
    | GroupYield
        contents start end bound (GroupState s contents start end bound)
    | GroupFinish

{-# INLINE_NORMAL chunksOfAs #-}
chunksOfAs :: forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> D.Stream m a -> D.Stream m (MutArray a)
chunksOfAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Stream m a -> Stream m (MutArray a)
chunksOfAs PinnedState
ps Int
n (D.Stream State StreamK m a -> s -> m (Step s a)
step s
state) =
    (State StreamK m (MutArray a)
 -> GroupState s MutByteArray Int Int Int
 -> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a)))
-> GroupState s MutByteArray Int Int Int -> Stream m (MutArray a)
forall (m :: * -> *) a s.
(State StreamK m a -> s -> m (Step s a)) -> s -> Stream m a
D.Stream State StreamK m (MutArray a)
-> GroupState s MutByteArray Int Int Int
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall {m :: * -> *} {a} {a}.
State StreamK m a
-> GroupState s MutByteArray Int Int Int
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
step' (s -> GroupState s MutByteArray Int Int Int
forall s contents start end bound.
s -> GroupState s contents start end bound
GroupStart s
state)

    where

    {-# INLINE_LATE step' #-}
    step' :: State StreamK m a
-> GroupState s MutByteArray Int Int Int
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
step' State StreamK m a
_ (GroupStart s
st) = do
        Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
n Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
0) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
            -- XXX we can pass the module string from the higher level API
            [Char] -> m ()
forall a. (?callStack::CallStack) => [Char] -> a
error ([Char] -> m ()) -> [Char] -> m ()
forall a b. (a -> b) -> a -> b
$ [Char]
"Streamly.Internal.Data.MutArray.Mut.Type.chunksOf: "
                    [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"the size of arrays [" [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
n
                    [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"] must be a natural number"
        (MutArray MutByteArray
contents Int
start Int
end Int
bound :: MutArray a) <- PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
ps Int
n
        Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (GroupState s MutByteArray Int Int Int) (MutArray a)
 -> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a)))
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a b. (a -> b) -> a -> b
$ GroupState s MutByteArray Int Int Int
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
forall s a. s -> Step s a
D.Skip (s
-> MutByteArray
-> Int
-> Int
-> Int
-> GroupState s MutByteArray Int Int Int
forall s contents start end bound.
s
-> contents
-> start
-> end
-> bound
-> GroupState s contents start end bound
GroupBuffer s
st MutByteArray
contents Int
start Int
end Int
bound)

    step' State StreamK m a
gst (GroupBuffer s
st MutByteArray
contents Int
start Int
end Int
bound) = do
        Step s a
r <- State StreamK m a -> s -> m (Step s a)
step (State StreamK m a -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m a
gst) s
st
        case Step s a
r of
            D.Yield a
x s
s -> do
                IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
end MutByteArray
contents  a
x
                let end1 :: Int
end1 = INDEX_NEXT(end,a)
                Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (GroupState s MutByteArray Int Int Int) (MutArray a)
 -> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a)))
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a b. (a -> b) -> a -> b
$
                    if Int
end1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
bound
                    then GroupState s MutByteArray Int Int Int
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
forall s a. s -> Step s a
D.Skip
                            (MutByteArray
-> Int
-> Int
-> Int
-> GroupState s MutByteArray Int Int Int
-> GroupState s MutByteArray Int Int Int
forall s contents start end bound.
contents
-> start
-> end
-> bound
-> GroupState s contents start end bound
-> GroupState s contents start end bound
GroupYield
                                MutByteArray
contents Int
start Int
end1 Int
bound (s -> GroupState s MutByteArray Int Int Int
forall s contents start end bound.
s -> GroupState s contents start end bound
GroupStart s
s))
                    else GroupState s MutByteArray Int Int Int
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
forall s a. s -> Step s a
D.Skip (s
-> MutByteArray
-> Int
-> Int
-> Int
-> GroupState s MutByteArray Int Int Int
forall s contents start end bound.
s
-> contents
-> start
-> end
-> bound
-> GroupState s contents start end bound
GroupBuffer s
s MutByteArray
contents Int
start Int
end1 Int
bound)
            D.Skip s
s ->
                Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (GroupState s MutByteArray Int Int Int) (MutArray a)
 -> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a)))
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a b. (a -> b) -> a -> b
$ GroupState s MutByteArray Int Int Int
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
forall s a. s -> Step s a
D.Skip (s
-> MutByteArray
-> Int
-> Int
-> Int
-> GroupState s MutByteArray Int Int Int
forall s contents start end bound.
s
-> contents
-> start
-> end
-> bound
-> GroupState s contents start end bound
GroupBuffer s
s MutByteArray
contents Int
start Int
end Int
bound)
            Step s a
D.Stop ->
                Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
                    (Step (GroupState s MutByteArray Int Int Int) (MutArray a)
 -> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a)))
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a b. (a -> b) -> a -> b
$ GroupState s MutByteArray Int Int Int
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
forall s a. s -> Step s a
D.Skip (MutByteArray
-> Int
-> Int
-> Int
-> GroupState s MutByteArray Int Int Int
-> GroupState s MutByteArray Int Int Int
forall s contents start end bound.
contents
-> start
-> end
-> bound
-> GroupState s contents start end bound
-> GroupState s contents start end bound
GroupYield MutByteArray
contents Int
start Int
end Int
bound GroupState s MutByteArray Int Int Int
forall s contents start end bound.
GroupState s contents start end bound
GroupFinish)

    step' State StreamK m a
_ (GroupYield MutByteArray
contents Int
start Int
end Int
bound GroupState s MutByteArray Int Int Int
next) =
        Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (GroupState s MutByteArray Int Int Int) (MutArray a)
 -> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a)))
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a b. (a -> b) -> a -> b
$ MutArray a
-> GroupState s MutByteArray Int Int Int
-> Step (GroupState s MutByteArray Int Int Int) (MutArray a)
forall s a. a -> s -> Step s a
D.Yield (MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
contents Int
start Int
end Int
bound) GroupState s MutByteArray Int Int Int
next

    step' State StreamK m a
_ GroupState s MutByteArray Int Int Int
GroupFinish = Step (GroupState s MutByteArray Int Int Int) (MutArray a)
-> m (Step (GroupState s MutByteArray Int Int Int) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step (GroupState s MutByteArray Int Int Int) (MutArray a)
forall s a. Step s a
D.Stop

-- | @chunksOf n stream@ groups the elements in the input stream into arrays of
-- @n@ elements each.
--
-- Same as the following but may be more efficient:
--
-- >>> chunksOf n = Stream.foldMany (MutArray.createOf n)
--
-- /Pre-release/
{-# INLINE_NORMAL chunksOf #-}
chunksOf :: forall m a. (MonadIO m, Unbox a)
    => Int -> D.Stream m a -> D.Stream m (MutArray a)
-- XXX the idiomatic implementation leads to large regression in the D.reverse'
-- benchmark. It seems it has difficulty producing optimized code when
-- converting to StreamK. Investigate GHC optimizations.
-- chunksOf n = D.foldMany (createOf n)
chunksOf :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream m a -> Stream m (MutArray a)
chunksOf = PinnedState -> Int -> Stream m a -> Stream m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Stream m a -> Stream m (MutArray a)
chunksOfAs PinnedState
Unpinned

-- | Like 'chunksOf' but creates pinned arrays.
{-# INLINE_NORMAL chunksOf' #-}
pinnedChunksOf, chunksOf' :: forall m a. (MonadIO m, Unbox a)
    => Int -> D.Stream m a -> D.Stream m (MutArray a)
-- chunksOf' n = D.foldMany (createOf' n)
chunksOf' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream m a -> Stream m (MutArray a)
chunksOf' = PinnedState -> Int -> Stream m a -> Stream m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Stream m a -> Stream m (MutArray a)
chunksOfAs PinnedState
Pinned
RENAME_PRIME(pinnedChunksOf,chunksOf)

-- | Create arrays from the input stream using a predicate to find the end of
-- the chunk. When the predicate matches, the chunk ends, the matching element
-- is included in the chunk.
--
--  Definition:
--
-- >>> chunksEndBy p = Stream.foldMany (Fold.takeEndBy p MutArray.create)
--
{-# INLINE chunksEndBy #-}
chunksEndBy :: forall m a. (MonadIO m, Unbox a)
    => (a -> Bool) -> D.Stream m a -> D.Stream m (MutArray a)
chunksEndBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> Stream m a -> Stream m (MutArray a)
chunksEndBy a -> Bool
p = Fold m a (MutArray a) -> Stream m a -> Stream m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> Stream m b
D.foldMany ((a -> Bool) -> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
(a -> Bool) -> Fold m a b -> Fold m a b
FL.takeEndBy a -> Bool
p Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
create)

-- | Like 'chunksEndBy' but creates pinned arrays.
--
{-# INLINE chunksEndBy' #-}
chunksEndBy' :: forall m a. (MonadIO m, Unbox a)
    => (a -> Bool) -> D.Stream m a -> D.Stream m (MutArray a)
chunksEndBy' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> Stream m a -> Stream m (MutArray a)
chunksEndBy' a -> Bool
p = Fold m a (MutArray a) -> Stream m a -> Stream m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> Stream m b
D.foldMany ((a -> Bool) -> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
(a -> Bool) -> Fold m a b -> Fold m a b
FL.takeEndBy a -> Bool
p Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
create')

-- | Create chunks using newline as the separator, including it.
{-# INLINE chunksEndByLn #-}
chunksEndByLn :: (MonadIO m)
    => D.Stream m Word8 -> D.Stream m (MutArray Word8)
chunksEndByLn :: forall (m :: * -> *).
MonadIO m =>
Stream m Word8 -> Stream m (MutArray Word8)
chunksEndByLn = (Word8 -> Bool) -> Stream m Word8 -> Stream m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> Stream m a -> Stream m (MutArray a)
chunksEndBy (Word8 -> Word8 -> Bool
forall a. Eq a => a -> a -> Bool
== Int -> Word8
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Char -> Int
ord Char
'\n'))

-- | Like 'chunksEndByLn' but creates pinned arrays.
{-# INLINE chunksEndByLn' #-}
chunksEndByLn' :: (MonadIO m)
    => D.Stream m Word8 -> D.Stream m (MutArray Word8)
chunksEndByLn' :: forall (m :: * -> *).
MonadIO m =>
Stream m Word8 -> Stream m (MutArray Word8)
chunksEndByLn' = (Word8 -> Bool) -> Stream m Word8 -> Stream m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> Stream m a -> Stream m (MutArray a)
chunksEndBy' (Word8 -> Word8 -> Bool
forall a. Eq a => a -> a -> Bool
== Int -> Word8
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Char -> Int
ord Char
'\n'))

-- | When we are buffering a stream of unknown size into an array we do not
-- know how much space to pre-allocate. So we start with the min size and emit
-- the array then keep on doubling the size every time. Thus we do not need to
-- guess the optimum chunk size.
--
-- We can incorporate this in chunksOfAs if the additional size parameter does
-- not impact perf.
--
{-# INLINE _chunksOfRange #-}
_chunksOfRange :: -- (MonadIO m, Unbox a) =>
    PinnedState -> Int -> Int -> D.Stream m a -> D.Stream m (MutArray a)
_chunksOfRange :: forall (m :: * -> *) a.
PinnedState -> Int -> Int -> Stream m a -> Stream m (MutArray a)
_chunksOfRange PinnedState
_ps Int
_low Int
_hi = Stream m a -> Stream m (MutArray a)
forall a. (?callStack::CallStack) => a
undefined

-- XXX buffer to a list instead?
-- | Buffer the stream into arrays in memory.
{-# INLINE arrayStreamKFromStreamDAs #-}
arrayStreamKFromStreamDAs :: forall m a. (MonadIO m, Unbox a) =>
    PinnedState -> D.Stream m a -> m (StreamK m (MutArray a))
arrayStreamKFromStreamDAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Stream m a -> m (StreamK m (MutArray a))
arrayStreamKFromStreamDAs PinnedState
ps =
    let n :: Int
n = a -> Int -> Int
forall a. Unbox a => a -> Int -> Int
allocBytesToElemCount (a
forall a. (?callStack::CallStack) => a
undefined :: a) Int
defaultChunkSize
     in (MutArray a -> StreamK m (MutArray a) -> StreamK m (MutArray a))
-> StreamK m (MutArray a)
-> Stream m (MutArray a)
-> m (StreamK m (MutArray a))
forall (m :: * -> *) a b.
Monad m =>
(a -> b -> b) -> b -> Stream m a -> m b
D.foldr MutArray a -> StreamK m (MutArray a) -> StreamK m (MutArray a)
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
K.cons StreamK m (MutArray a)
forall (m :: * -> *) a. StreamK m a
K.nil (Stream m (MutArray a) -> m (StreamK m (MutArray a)))
-> (Stream m a -> Stream m (MutArray a))
-> Stream m a
-> m (StreamK m (MutArray a))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. PinnedState -> Int -> Stream m a -> Stream m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Stream m a -> Stream m (MutArray a)
chunksOfAs PinnedState
ps Int
n

-------------------------------------------------------------------------------
-- Streams of arrays - Flattening
-------------------------------------------------------------------------------

data FlattenState s contents a =
      OuterLoop s
    | InnerLoop s contents !Int !Int

{-# INLINE_NORMAL concatWith #-}
concatWith :: forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> D.Stream m (MutArray a) -> D.Stream m a
concatWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m (MutArray a) -> Stream m a
concatWith forall b. IO b -> m b
liftio (D.Stream State StreamK m (MutArray a) -> s -> m (Step s (MutArray a))
step s
state) = (State StreamK m a
 -> FlattenState s MutByteArray Any
 -> m (Step (FlattenState s MutByteArray Any) a))
-> FlattenState s MutByteArray Any -> Stream m a
forall (m :: * -> *) a s.
(State StreamK m a -> s -> m (Step s a)) -> s -> Stream m a
D.Stream State StreamK m a
-> FlattenState s MutByteArray Any
-> m (Step (FlattenState s MutByteArray Any) a)
forall {a} {m :: * -> *} {a} {a} {a}.
Unbox a =>
State StreamK m a
-> FlattenState s MutByteArray a
-> m (Step (FlattenState s MutByteArray a) a)
step' (s -> FlattenState s MutByteArray Any
forall s contents a. s -> FlattenState s contents a
OuterLoop s
state)

    where

    {-# INLINE_LATE step' #-}
    step' :: State StreamK m a
-> FlattenState s MutByteArray a
-> m (Step (FlattenState s MutByteArray a) a)
step' State StreamK m a
gst (OuterLoop s
st) = do
        Step s (MutArray a)
r <- State StreamK m (MutArray a) -> s -> m (Step s (MutArray a))
step (State StreamK m a -> State StreamK m (MutArray a)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m a
gst) s
st
        Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (FlattenState s MutByteArray a) a
 -> m (Step (FlattenState s MutByteArray a) a))
-> Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a b. (a -> b) -> a -> b
$ case Step s (MutArray a)
r of
            D.Yield MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} s
s ->
                FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. s -> Step s a
D.Skip (s -> MutByteArray -> Int -> Int -> FlattenState s MutByteArray a
forall s contents a.
s -> contents -> Int -> Int -> FlattenState s contents a
InnerLoop s
s MutByteArray
arrContents Int
arrStart Int
arrEnd)
            D.Skip s
s -> FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. s -> Step s a
D.Skip (s -> FlattenState s MutByteArray a
forall s contents a. s -> FlattenState s contents a
OuterLoop s
s)
            Step s (MutArray a)
D.Stop -> Step (FlattenState s MutByteArray a) a
forall s a. Step s a
D.Stop

    step' State StreamK m a
_ (InnerLoop s
st MutByteArray
_ Int
p Int
end) | Bool -> Bool -> Bool
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
end) (Int
p Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
end) =
        Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (FlattenState s MutByteArray a) a
 -> m (Step (FlattenState s MutByteArray a) a))
-> Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a b. (a -> b) -> a -> b
$ FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. s -> Step s a
D.Skip (FlattenState s MutByteArray a
 -> Step (FlattenState s MutByteArray a) a)
-> FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall a b. (a -> b) -> a -> b
$ s -> FlattenState s MutByteArray a
forall s contents a. s -> FlattenState s contents a
OuterLoop s
st

    step' State StreamK m a
_ (InnerLoop s
st MutByteArray
contents Int
p Int
end) = do
        !a
x <- IO a -> m a
forall b. IO b -> m b
liftio (IO a -> m a) -> IO a -> m a
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
contents
        Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (FlattenState s MutByteArray a) a
 -> m (Step (FlattenState s MutByteArray a) a))
-> Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a b. (a -> b) -> a -> b
$ a
-> FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. a -> s -> Step s a
D.Yield a
x (s -> MutByteArray -> Int -> Int -> FlattenState s MutByteArray a
forall s contents a.
s -> contents -> Int -> Int -> FlattenState s contents a
InnerLoop s
st MutByteArray
contents (INDEX_NEXT(p,a)) end)

-- | Use the "reader" unfold instead.
--
-- @concat = unfoldMany reader@
--
-- We can try this if there are any fusion issues in the unfold.
--
{-# INLINE_NORMAL concat #-}
concat :: forall m a. (MonadIO m, Unbox a)
    => D.Stream m (MutArray a) -> D.Stream m a
concat :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m (MutArray a) -> Stream m a
concat = (forall b. IO b -> m b) -> Stream m (MutArray a) -> Stream m a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m (MutArray a) -> Stream m a
concatWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

{-# DEPRECATED flattenArrays "Please use \"unfoldMany reader\" instead." #-}
{-# INLINE flattenArrays #-}
flattenArrays :: forall m a. (MonadIO m, Unbox a)
    => D.Stream m (MutArray a) -> D.Stream m a
flattenArrays :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m (MutArray a) -> Stream m a
flattenArrays = Stream m (MutArray a) -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m (MutArray a) -> Stream m a
concat

{-# INLINE_NORMAL concatRevWith #-}
concatRevWith :: forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> D.Stream m (MutArray a) -> D.Stream m a
concatRevWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m (MutArray a) -> Stream m a
concatRevWith forall b. IO b -> m b
liftio (D.Stream State StreamK m (MutArray a) -> s -> m (Step s (MutArray a))
step s
state) = (State StreamK m a
 -> FlattenState s MutByteArray Any
 -> m (Step (FlattenState s MutByteArray Any) a))
-> FlattenState s MutByteArray Any -> Stream m a
forall (m :: * -> *) a s.
(State StreamK m a -> s -> m (Step s a)) -> s -> Stream m a
D.Stream State StreamK m a
-> FlattenState s MutByteArray Any
-> m (Step (FlattenState s MutByteArray Any) a)
forall {a} {m :: * -> *} {a} {a} {a}.
Unbox a =>
State StreamK m a
-> FlattenState s MutByteArray a
-> m (Step (FlattenState s MutByteArray a) a)
step' (s -> FlattenState s MutByteArray Any
forall s contents a. s -> FlattenState s contents a
OuterLoop s
state)

    where

    {-# INLINE_LATE step' #-}
    step' :: State StreamK m a
-> FlattenState s MutByteArray a
-> m (Step (FlattenState s MutByteArray a) a)
step' State StreamK m a
gst (OuterLoop s
st) = do
        Step s (MutArray a)
r <- State StreamK m (MutArray a) -> s -> m (Step s (MutArray a))
step (State StreamK m a -> State StreamK m (MutArray a)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m a
gst) s
st
        Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (FlattenState s MutByteArray a) a
 -> m (Step (FlattenState s MutByteArray a) a))
-> Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a b. (a -> b) -> a -> b
$ case Step s (MutArray a)
r of
            D.Yield MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} s
s ->
                let p :: Int
p = INDEX_PREV(arrEnd,a)
                 in FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. s -> Step s a
D.Skip (s -> MutByteArray -> Int -> Int -> FlattenState s MutByteArray a
forall s contents a.
s -> contents -> Int -> Int -> FlattenState s contents a
InnerLoop s
s MutByteArray
arrContents Int
p Int
arrStart)
            D.Skip s
s -> FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. s -> Step s a
D.Skip (s -> FlattenState s MutByteArray a
forall s contents a. s -> FlattenState s contents a
OuterLoop s
s)
            Step s (MutArray a)
D.Stop -> Step (FlattenState s MutByteArray a) a
forall s a. Step s a
D.Stop

    step' State StreamK m a
_ (InnerLoop s
st MutByteArray
_ Int
p Int
start) | Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
start =
        Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (FlattenState s MutByteArray a) a
 -> m (Step (FlattenState s MutByteArray a) a))
-> Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a b. (a -> b) -> a -> b
$ FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. s -> Step s a
D.Skip (FlattenState s MutByteArray a
 -> Step (FlattenState s MutByteArray a) a)
-> FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall a b. (a -> b) -> a -> b
$ s -> FlattenState s MutByteArray a
forall s contents a. s -> FlattenState s contents a
OuterLoop s
st

    step' State StreamK m a
_ (InnerLoop s
st MutByteArray
contents Int
p Int
start) = do
        !a
x <- IO a -> m a
forall b. IO b -> m b
liftio (IO a -> m a) -> IO a -> m a
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
contents
        let cur :: Int
cur = INDEX_PREV(p,a)
        Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (FlattenState s MutByteArray a) a
 -> m (Step (FlattenState s MutByteArray a) a))
-> Step (FlattenState s MutByteArray a) a
-> m (Step (FlattenState s MutByteArray a) a)
forall a b. (a -> b) -> a -> b
$ a
-> FlattenState s MutByteArray a
-> Step (FlattenState s MutByteArray a) a
forall s a. a -> s -> Step s a
D.Yield a
x (s -> MutByteArray -> Int -> Int -> FlattenState s MutByteArray a
forall s contents a.
s -> contents -> Int -> Int -> FlattenState s contents a
InnerLoop s
st MutByteArray
contents Int
cur Int
start)

-- | Use the "readerRev" unfold instead.
--
-- @concat = unfoldMany readerRev@
--
-- We can try this if there are any fusion issues in the unfold.
--
{-# INLINE_NORMAL concatRev #-}
concatRev :: forall m a. (MonadIO m, Unbox a)
    => D.Stream m (MutArray a) -> D.Stream m a
concatRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m (MutArray a) -> Stream m a
concatRev = (forall b. IO b -> m b) -> Stream m (MutArray a) -> Stream m a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Stream m (MutArray a) -> Stream m a
concatRevWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

{-# DEPRECATED flattenArraysRev "Please use \"unfoldMany readerRev\" instead." #-}
{-# INLINE flattenArraysRev #-}
flattenArraysRev :: forall m a. (MonadIO m, Unbox a)
    => D.Stream m (MutArray a) -> D.Stream m a
flattenArraysRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m (MutArray a) -> Stream m a
flattenArraysRev = Stream m (MutArray a) -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m (MutArray a) -> Stream m a
concatRev

-------------------------------------------------------------------------------
-- Unfolds
-------------------------------------------------------------------------------

data ArrayUnsafe a = ArrayUnsafe
    {-# UNPACK #-} !MutByteArray   -- contents
    {-# UNPACK #-} !Int                -- index 1
    {-# UNPACK #-} !Int                -- index 2

toArrayUnsafe :: MutArray a -> ArrayUnsafe a
toArrayUnsafe :: forall a. MutArray a -> ArrayUnsafe a
toArrayUnsafe (MutArray MutByteArray
contents Int
start Int
end Int
_) = MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
start Int
end

fromArrayUnsafe ::
#ifdef DEVBUILD
    Unbox a =>
#endif
    ArrayUnsafe a -> MutArray a
fromArrayUnsafe :: forall a. ArrayUnsafe a -> MutArray a
fromArrayUnsafe (ArrayUnsafe MutByteArray
contents Int
start Int
end) =
         MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
contents Int
start Int
end Int
end

{-# INLINE_NORMAL producerWith #-}
producerWith ::
       forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> Producer m (MutArray a) a
producerWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Producer m (MutArray a) a
producerWith forall b. IO b -> m b
liftio = (ArrayUnsafe a -> m (Step (ArrayUnsafe a) a))
-> (MutArray a -> m (ArrayUnsafe a))
-> (ArrayUnsafe a -> m (MutArray a))
-> Producer m (MutArray a) a
forall (m :: * -> *) a b s.
(s -> m (Step s b)) -> (a -> m s) -> (s -> m a) -> Producer m a b
Producer ArrayUnsafe a -> m (Step (ArrayUnsafe a) a)
forall {a} {a} {a}.
Unbox a =>
ArrayUnsafe a -> m (Step (ArrayUnsafe a) a)
step (ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ArrayUnsafe a -> m (ArrayUnsafe a))
-> (MutArray a -> ArrayUnsafe a) -> MutArray a -> m (ArrayUnsafe a)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. MutArray a -> ArrayUnsafe a
forall a. MutArray a -> ArrayUnsafe a
toArrayUnsafe) ArrayUnsafe a -> m (MutArray a)
forall {a}. ArrayUnsafe a -> m (MutArray a)
extract
    where

    {-# INLINE_LATE step #-}
    step :: ArrayUnsafe a -> m (Step (ArrayUnsafe a) a)
step (ArrayUnsafe MutByteArray
_ Int
cur Int
end)
        | Bool -> Bool -> Bool
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
cur Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
end) (Int
cur Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
end) = Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step (ArrayUnsafe a) a
forall s a. Step s a
D.Stop
    step (ArrayUnsafe MutByteArray
contents Int
cur Int
end) = do
            -- When we use a purely lazy Monad like Identity, we need to force a
            -- few actions for correctness and execution order sanity. We want
            -- the peek to occur right here and not lazily at some later point
            -- because we want the peek to be ordered with respect to the touch.
            !a
x <- IO a -> m a
forall b. IO b -> m b
liftio (IO a -> m a) -> IO a -> m a
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
cur MutByteArray
contents
            Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a))
-> Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a)
forall a b. (a -> b) -> a -> b
$ a -> ArrayUnsafe a -> Step (ArrayUnsafe a) a
forall s a. a -> s -> Step s a
D.Yield a
x (MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents (INDEX_NEXT(cur,a)) end)

    extract :: ArrayUnsafe a -> m (MutArray a)
extract = MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> m (MutArray a))
-> (ArrayUnsafe a -> MutArray a) -> ArrayUnsafe a -> m (MutArray a)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ArrayUnsafe a -> MutArray a
forall a. ArrayUnsafe a -> MutArray a
fromArrayUnsafe

-- | Resumable unfold of an array.
--
{-# INLINE_NORMAL producer #-}
producer :: forall m a. (MonadIO m, Unbox a) => Producer m (MutArray a) a
producer :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Producer m (MutArray a) a
producer = (forall b. IO b -> m b) -> Producer m (MutArray a) a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Producer m (MutArray a) a
producerWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

-- | Unfold an array into a stream.
--
{-# INLINE_NORMAL reader #-}
reader :: forall m a. (MonadIO m, Unbox a) => Unfold m (MutArray a) a
reader :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Unfold m (MutArray a) a
reader = Producer m (MutArray a) a -> Unfold m (MutArray a) a
forall (m :: * -> *) a b. Producer m a b -> Unfold m a b
Producer.simplify Producer m (MutArray a) a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Producer m (MutArray a) a
producer

{-# INLINE_NORMAL readerRevWith #-}
readerRevWith ::
       forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> Unfold m (MutArray a) a
readerRevWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Unfold m (MutArray a) a
readerRevWith forall b. IO b -> m b
liftio = (ArrayUnsafe Any -> m (Step (ArrayUnsafe Any) a))
-> (MutArray a -> m (ArrayUnsafe Any)) -> Unfold m (MutArray a) a
forall (m :: * -> *) a b s.
(s -> m (Step s b)) -> (a -> m s) -> Unfold m a b
Unfold ArrayUnsafe Any -> m (Step (ArrayUnsafe Any) a)
forall {a} {a} {a}.
Unbox a =>
ArrayUnsafe a -> m (Step (ArrayUnsafe a) a)
step MutArray a -> m (ArrayUnsafe Any)
forall {m :: * -> *} {a} {a}.
Monad m =>
MutArray a -> m (ArrayUnsafe a)
inject
    where

    inject :: MutArray a -> m (ArrayUnsafe a)
inject (MutArray MutByteArray
contents Int
start Int
end Int
_) =
        let p :: Int
p = INDEX_PREV(end,a)
         in ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ArrayUnsafe a -> m (ArrayUnsafe a))
-> ArrayUnsafe a -> m (ArrayUnsafe a)
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
start Int
p

    {-# INLINE_LATE step #-}
    step :: ArrayUnsafe a -> m (Step (ArrayUnsafe a) a)
step (ArrayUnsafe MutByteArray
_ Int
start Int
p) | Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
start = Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step (ArrayUnsafe a) a
forall s a. Step s a
D.Stop
    step (ArrayUnsafe MutByteArray
contents Int
start Int
p) = do
        !a
x <- IO a -> m a
forall b. IO b -> m b
liftio (IO a -> m a) -> IO a -> m a
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
contents
        Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a))
-> Step (ArrayUnsafe a) a -> m (Step (ArrayUnsafe a) a)
forall a b. (a -> b) -> a -> b
$ a -> ArrayUnsafe a -> Step (ArrayUnsafe a) a
forall s a. a -> s -> Step s a
D.Yield a
x (MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
start (INDEX_PREV(p,a)))

-- | Unfold an array into a stream in reverse order.
--
{-# INLINE_NORMAL readerRev #-}
readerRev :: forall m a. (MonadIO m, Unbox a) => Unfold m (MutArray a) a
readerRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Unfold m (MutArray a) a
readerRev = (forall b. IO b -> m b) -> Unfold m (MutArray a) a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> Unfold m (MutArray a) a
readerRevWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

-------------------------------------------------------------------------------
-- to Lists and streams
-------------------------------------------------------------------------------

{-
-- Use foldr/build fusion to fuse with list consumers
-- This can be useful when using the IsList instance
{-# INLINE_LATE toListFB #-}
toListFB :: forall a b. Unbox a => (a -> b -> b) -> b -> MutArray a -> b
toListFB c n MutArray{..} = go arrStart
    where

    go p | assert (p <= arrEnd) (p == arrEnd) = n
    go p =
        -- unsafeInlineIO allows us to run this in Identity monad for pure
        -- toList/foldr case which makes them much faster due to not
        -- accumulating the list and fusing better with the pure consumers.
        --
        -- This should be safe as the array contents are guaranteed to be
        -- evaluated/written to before we peek at them.
        -- XXX
        let !x = unsafeInlineIO $ do
                    r <- peekAt arrContents p
                    return r
        in c x (go (PTR_NEXT(p,a)))
-}

-- XXX Monadic foldr/build fusion?
-- Reference: https://www.researchgate.net/publication/220676509_Monadic_augment_and_generalised_short_cut_fusion

-- | Convert a 'MutArray' into a list.
--
{-# INLINE toList #-}
toList :: forall m a. (MonadIO m, Unbox a) => MutArray a -> m [a]
toList :: forall (m :: * -> *) a. (MonadIO m, Unbox a) => MutArray a -> m [a]
toList MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO [a] -> m [a]
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO [a] -> m [a]) -> IO [a] -> m [a]
forall a b. (a -> b) -> a -> b
$ Int -> IO [a]
forall {a}. Unbox a => Int -> IO [a]
go Int
arrStart
    where

    go :: Int -> IO [a]
go Int
p | Bool -> Bool -> Bool
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrEnd) (Int
p Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
arrEnd) = [a] -> IO [a]
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return []
    go Int
p = do
        a
x <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
arrContents
        (:) a
x ([a] -> [a]) -> IO [a] -> IO [a]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> IO [a]
go (INDEX_NEXT(p,a))

{-# INLINE_NORMAL toStreamWith #-}
toStreamWith ::
       forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> MutArray a -> D.Stream m a
toStreamWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> Stream m a
toStreamWith forall b. IO b -> m b
liftio MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = (State StreamK m a -> Int -> m (Step Int a)) -> Int -> Stream m a
forall (m :: * -> *) a s.
(State StreamK m a -> s -> m (Step s a)) -> s -> Stream m a
D.Stream State StreamK m a -> Int -> m (Step Int a)
forall {a} {p}. Unbox a => p -> Int -> m (Step Int a)
step Int
arrStart

    where

    {-# INLINE_LATE step #-}
    step :: p -> Int -> m (Step Int a)
step p
_ Int
p | Bool -> Bool -> Bool
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrEnd) (Int
p Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
arrEnd) = Step Int a -> m (Step Int a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step Int a
forall s a. Step s a
D.Stop
    step p
_ Int
p = IO (Step Int a) -> m (Step Int a)
forall b. IO b -> m b
liftio (IO (Step Int a) -> m (Step Int a))
-> IO (Step Int a) -> m (Step Int a)
forall a b. (a -> b) -> a -> b
$ do
        a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
arrContents
        Step Int a -> IO (Step Int a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step Int a -> IO (Step Int a)) -> Step Int a -> IO (Step Int a)
forall a b. (a -> b) -> a -> b
$ a -> Int -> Step Int a
forall s a. a -> s -> Step s a
D.Yield a
r (INDEX_NEXT(p,a))

-- | Convert a 'MutArray' into a stream.
--
-- >>> read = Stream.unfold MutArray.reader
--
{-# INLINE_NORMAL read #-}
read :: forall m a. (MonadIO m, Unbox a) => MutArray a -> D.Stream m a
read :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
read = (forall b. IO b -> m b) -> MutArray a -> Stream m a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> Stream m a
toStreamWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

{-# INLINE toStreamKWith #-}
toStreamKWith ::
       forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> MutArray a -> StreamK m a
toStreamKWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> StreamK m a
toStreamKWith forall b. IO b -> m b
liftio MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = Int -> StreamK m a
forall {a}. Unbox a => Int -> StreamK m a
go Int
arrStart

    where

    go :: Int -> StreamK m a
go Int
p | Bool -> Bool -> Bool
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrEnd) (Int
p Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
arrEnd) = StreamK m a
forall (m :: * -> *) a. StreamK m a
K.nil
         | Bool
otherwise =
        let elemM :: IO a
elemM = Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
arrContents
        in IO a -> m a
forall b. IO b -> m b
liftio IO a
elemM m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
`K.consM` Int -> StreamK m a
go (INDEX_NEXT(p,a))

{-# INLINE toStreamK #-}
toStreamK :: forall m a. (MonadIO m, Unbox a) => MutArray a -> StreamK m a
toStreamK :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> StreamK m a
toStreamK = (forall b. IO b -> m b) -> MutArray a -> StreamK m a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> StreamK m a
toStreamKWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

{-# INLINE_NORMAL toStreamRevWith #-}
toStreamRevWith ::
       forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> MutArray a -> D.Stream m a
toStreamRevWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> Stream m a
toStreamRevWith forall b. IO b -> m b
liftio MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    let p :: Int
p = INDEX_PREV(arrEnd,a)
    in (State StreamK m a -> Int -> m (Step Int a)) -> Int -> Stream m a
forall (m :: * -> *) a s.
(State StreamK m a -> s -> m (Step s a)) -> s -> Stream m a
D.Stream State StreamK m a -> Int -> m (Step Int a)
forall {a} {p}. Unbox a => p -> Int -> m (Step Int a)
step Int
p

    where

    {-# INLINE_LATE step #-}
    step :: p -> Int -> m (Step Int a)
step p
_ Int
p | Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
arrStart = Step Int a -> m (Step Int a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step Int a
forall s a. Step s a
D.Stop
    step p
_ Int
p = IO (Step Int a) -> m (Step Int a)
forall b. IO b -> m b
liftio (IO (Step Int a) -> m (Step Int a))
-> IO (Step Int a) -> m (Step Int a)
forall a b. (a -> b) -> a -> b
$ do
        a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
arrContents
        Step Int a -> IO (Step Int a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step Int a -> IO (Step Int a)) -> Step Int a -> IO (Step Int a)
forall a b. (a -> b) -> a -> b
$ a -> Int -> Step Int a
forall s a. a -> s -> Step s a
D.Yield a
r (INDEX_PREV(p,a))

-- | Convert a 'MutArray' into a stream in reverse order.
--
-- >>> readRev = Stream.unfold MutArray.readerRev
--
{-# INLINE_NORMAL readRev #-}
readRev :: forall m a. (MonadIO m, Unbox a) => MutArray a -> D.Stream m a
readRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
readRev = (forall b. IO b -> m b) -> MutArray a -> Stream m a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> Stream m a
toStreamRevWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

{-# INLINE toStreamKRevWith #-}
toStreamKRevWith ::
       forall m a. (Monad m, Unbox a)
    => (forall b. IO b -> m b) -> MutArray a -> StreamK m a
toStreamKRevWith :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> StreamK m a
toStreamKRevWith forall b. IO b -> m b
liftio MutArray {Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    let p :: Int
p = INDEX_PREV(arrEnd,a)
    in Int -> StreamK m a
forall {a}. Unbox a => Int -> StreamK m a
go Int
p

    where

    go :: Int -> StreamK m a
go Int
p | Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
arrStart = StreamK m a
forall (m :: * -> *) a. StreamK m a
K.nil
         | Bool
otherwise =
        let elemM :: IO a
elemM = Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
p MutByteArray
arrContents
        in IO a -> m a
forall b. IO b -> m b
liftio IO a
elemM m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
`K.consM` Int -> StreamK m a
go (INDEX_PREV(p,a))

{-# INLINE toStreamKRev #-}
toStreamKRev :: forall m a. (MonadIO m, Unbox a) => MutArray a -> StreamK m a
toStreamKRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> StreamK m a
toStreamKRev = (forall b. IO b -> m b) -> MutArray a -> StreamK m a
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
(forall b. IO b -> m b) -> MutArray a -> StreamK m a
toStreamKRevWith IO b -> m b
forall b. IO b -> m b
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO

-------------------------------------------------------------------------------
-- Folding
-------------------------------------------------------------------------------

-- XXX Need something like "MutArray m a" enforcing monadic action to avoid the
-- possibility of such APIs.
--
-- | Strict left fold of an array.
{-# INLINE_NORMAL foldl' #-}
foldl' :: (MonadIO m, Unbox a) => (b -> a -> b) -> b -> MutArray a -> m b
foldl' :: forall (m :: * -> *) a b.
(MonadIO m, Unbox a) =>
(b -> a -> b) -> b -> MutArray a -> m b
foldl' b -> a -> b
f b
z MutArray a
arr = (b -> a -> b) -> b -> Stream m a -> m b
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> b) -> b -> Stream m a -> m b
D.foldl' b -> a -> b
f b
z (Stream m a -> m b) -> Stream m a -> m b
forall a b. (a -> b) -> a -> b
$ MutArray a -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
read MutArray a
arr

-- | Right fold of an array.
{-# INLINE_NORMAL foldr #-}
foldr :: (MonadIO m, Unbox a) => (a -> b -> b) -> b -> MutArray a -> m b
foldr :: forall (m :: * -> *) a b.
(MonadIO m, Unbox a) =>
(a -> b -> b) -> b -> MutArray a -> m b
foldr a -> b -> b
f b
z MutArray a
arr = (a -> b -> b) -> b -> Stream m a -> m b
forall (m :: * -> *) a b.
Monad m =>
(a -> b -> b) -> b -> Stream m a -> m b
D.foldr a -> b -> b
f b
z (Stream m a -> m b) -> Stream m a -> m b
forall a b. (a -> b) -> a -> b
$ MutArray a -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
read MutArray a
arr

-- | Fold an array using a 'Fold'.
--
-- For example:
--
-- >>> findIndex eq = MutArray.fold (Fold.findIndex eq)
--
-- /Pre-release/
{-# INLINE fold #-}
fold :: (MonadIO m, Unbox a) => Fold m a b -> MutArray a -> m b
fold :: forall (m :: * -> *) a b.
(MonadIO m, Unbox a) =>
Fold m a b -> MutArray a -> m b
fold Fold m a b
f MutArray a
arr = Fold m a b -> Stream m a -> m b
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold Fold m a b
f (MutArray a -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
read MutArray a
arr)

-- | Fold an arary starting from end up to beginning.
--
-- For example:
--
-- >>> findIndexRev eq = MutArray.foldRev (Fold.findIndex eq)
--
foldRev :: (MonadIO m, Unbox a) => Fold m a b -> MutArray a -> m b
foldRev :: forall (m :: * -> *) a b.
(MonadIO m, Unbox a) =>
Fold m a b -> MutArray a -> m b
foldRev Fold m a b
f MutArray a
arr = Fold m a b -> Stream m a -> m b
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold Fold m a b
f (MutArray a -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
readRev MutArray a
arr)

-------------------------------------------------------------------------------
-- Folds for appending
-------------------------------------------------------------------------------

-- Note: Arrays may be allocated with a specific alignment at the beginning of
-- the array. If you need to maintain that alignment on reallocations then you
-- can resize the array manually before append, using an aligned resize
-- operation.

-- XXX Keep the bound intact to not lose any free space? Perf impact?

-- | @unsafeAppendN n arr@ appends up to @n@ input items to the supplied
-- array.
--
-- Unsafe: Do not drive the fold beyond @n@ elements, it will lead to memory
-- corruption or segfault.
--
-- Any free space left in the array after appending @n@ elements is lost.
--
-- /Internal/
{-# INLINE_NORMAL unsafeAppendN #-}
unsafeAppendN :: forall m a. (MonadIO m, Unbox a) =>
       Int
    -> m (MutArray a)
    -> Fold m a (MutArray a)
unsafeAppendN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a) -> Fold m a (MutArray a)
unsafeAppendN Int
n m (MutArray a)
action = (ArrayUnsafe a -> MutArray a)
-> Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a)
forall a b. (a -> b) -> Fold m a a -> Fold m a b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ArrayUnsafe a -> MutArray a
forall a. ArrayUnsafe a -> MutArray a
fromArrayUnsafe (Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a))
-> Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a)
forall a b. (a -> b) -> a -> b
$ (ArrayUnsafe a -> a -> m (ArrayUnsafe a))
-> m (ArrayUnsafe a) -> Fold m a (ArrayUnsafe a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' ArrayUnsafe a -> a -> m (ArrayUnsafe a)
forall {m :: * -> *} {a} {a} {a}.
(MonadIO m, Unbox a) =>
ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step m (ArrayUnsafe a)
initial

    where

    initial :: m (ArrayUnsafe a)
initial = do
        Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
n Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0) (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
        arr :: MutArray a
arr@(MutArray MutByteArray
_ Int
_ Int
end Int
bound) <- m (MutArray a)
action
        let free_ :: Int
free_ = Int
bound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
end
            needed :: Int
needed = Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)
        -- XXX We can also reallocate if the array has too much free space,
        -- otherwise we lose that space.
        MutArray a
arr1 <-
            if Int
free_ Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
needed
            then ([Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a))
-> [Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
forall a. a -> a
noinline [Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
reallocBytesWith [Char]
"unsafeAppendN" (Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
needed) Int
needed MutArray a
arr
            else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
arr
        ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ArrayUnsafe a -> m (ArrayUnsafe a))
-> ArrayUnsafe a -> m (ArrayUnsafe a)
forall a b. (a -> b) -> a -> b
$ MutArray a -> ArrayUnsafe a
forall a. MutArray a -> ArrayUnsafe a
toArrayUnsafe MutArray a
arr1

    step :: ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step (ArrayUnsafe MutByteArray
contents Int
start Int
end) a
x = do
        IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
end MutByteArray
contents a
x
        -- We are using end as the bound, so no reserved space left.
        ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ArrayUnsafe a -> m (ArrayUnsafe a))
-> ArrayUnsafe a -> m (ArrayUnsafe a)
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
start (INDEX_NEXT(end,a))

-- | @unsafeAppendMax n arr@ appends up to @n@ input items to the supplied
-- array.
--
-- Unsafe: Do not drive the fold beyond @n@ elements, it will lead to memory
-- corruption or segfault.
--
-- /Internal/
{-# INLINE_NORMAL unsafeAppendMax #-}
unsafeAppendMax :: forall m a. (MonadIO m, Unbox a) =>
       Int
    -> MutArray a
    -> Fold m a (MutArray a)
unsafeAppendMax :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> Fold m a (MutArray a)
unsafeAppendMax Int
n arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    (ArrayUnsafe a -> MutArray a)
-> Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a)
forall a b. (a -> b) -> Fold m a a -> Fold m a b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ArrayUnsafe a -> MutArray a
forall {a} {a}. ArrayUnsafe a -> MutArray a
final (Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a))
-> Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a)
forall a b. (a -> b) -> a -> b
$ (ArrayUnsafe a -> a -> m (ArrayUnsafe a))
-> m (ArrayUnsafe a) -> Fold m a (ArrayUnsafe a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' ArrayUnsafe a -> a -> m (ArrayUnsafe a)
forall {m :: * -> *} {a} {a} {a}.
(MonadIO m, Unbox a) =>
ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step m (ArrayUnsafe a)
initial

    where

    free_ :: Int
free_ = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrEnd
    needed :: Int
needed = Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)
    bound :: Int
bound = Int
arrBound Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
needed Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
free_

    initial :: m (ArrayUnsafe a)
initial = do
        Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
n Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0) (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
        MutArray a
arr1 <-
            if Int
free_ Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
needed
            then ([Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a))
-> [Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
forall a. a -> a
noinline
                    [Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[Char] -> (Int -> Int) -> Int -> MutArray a -> m (MutArray a)
reallocBytesWith [Char]
"unsafeAppendMax" (Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
needed) Int
needed MutArray a
arr
            else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
arr
        ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ArrayUnsafe a -> m (ArrayUnsafe a))
-> ArrayUnsafe a -> m (ArrayUnsafe a)
forall a b. (a -> b) -> a -> b
$ MutArray a -> ArrayUnsafe a
forall a. MutArray a -> ArrayUnsafe a
toArrayUnsafe MutArray a
arr1

    step :: ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step (ArrayUnsafe MutByteArray
contents Int
start Int
end) a
x = do
        IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
end MutByteArray
contents a
x
        ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (ArrayUnsafe a -> m (ArrayUnsafe a))
-> ArrayUnsafe a -> m (ArrayUnsafe a)
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
start (INDEX_NEXT(end,a))

    final :: ArrayUnsafe a -> MutArray a
final (ArrayUnsafe MutByteArray
contents Int
start Int
end) =
        MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
contents Int
start Int
end Int
bound

{-# DEPRECATED writeAppendNUnsafe "Please use unsafeAppendN instead." #-}
{-# INLINE writeAppendNUnsafe #-}
writeAppendNUnsafe :: forall m a. (MonadIO m, Unbox a) =>
       Int
    -> m (MutArray a)
    -> Fold m a (MutArray a)
writeAppendNUnsafe :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a) -> Fold m a (MutArray a)
writeAppendNUnsafe = Int -> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a) -> Fold m a (MutArray a)
unsafeAppendN

-- | Append @n@ elements to an existing array. Any free space left in the array
-- after appending @n@ elements is lost.
--
-- >>> appendN n initial = Fold.take n (MutArray.unsafeAppendN n initial)
--
{-# INLINE_NORMAL appendN #-}
appendN :: forall m a. (MonadIO m, Unbox a) =>
    Int -> m (MutArray a) -> Fold m a (MutArray a)
appendN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a) -> Fold m a (MutArray a)
appendN Int
n m (MutArray a)
initial = Int -> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Int -> Fold m a b -> Fold m a b
FL.take Int
n (Int -> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a) -> Fold m a (MutArray a)
unsafeAppendN Int
n m (MutArray a)
initial)

-- | Allocates space for n additional elements. The fold terminates after
-- appending n elements. If less than n elements are supplied then the space
-- for the remaining elements is guaranteed to be reserved.
--
-- >>> appendMax n arr = Fold.take n (MutArray.unsafeAppendMax n arr)
--
{-# INLINE_NORMAL appendMax #-}
appendMax :: forall m a. (MonadIO m, Unbox a) =>
    Int -> MutArray a -> Fold m a (MutArray a)
appendMax :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> Fold m a (MutArray a)
appendMax Int
n MutArray a
initial = Int -> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Int -> Fold m a b -> Fold m a b
FL.take Int
n (Int -> MutArray a -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> Fold m a (MutArray a)
unsafeAppendMax Int
n MutArray a
initial)

{-# DEPRECATED writeAppendN "Please use appendN instead." #-}
{-# INLINE writeAppendN #-}
writeAppendN :: forall m a. (MonadIO m, Unbox a) =>
    Int -> m (MutArray a) -> Fold m a (MutArray a)
writeAppendN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a) -> Fold m a (MutArray a)
writeAppendN = Int -> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a) -> Fold m a (MutArray a)
appendN

-- | @appendWith sizer action@ mutates the array generated by @action@ to
-- append the input stream. If there is no reserved space available in the
-- array it is reallocated to a size in bytes determined by @sizer oldSize@,
-- where @oldSize@ is the current size of the array in bytes. If the sizer
-- returns less than or equal to the current size then the size is incremented
-- by one element.
--
-- Note that the returned array may be a mutated version of original array.
--
-- >>> appendWith sizer = Fold.foldlM' (MutArray.snocWith sizer)
--
-- /Pre-release/
{-# INLINE appendWith #-}
appendWith :: forall m a. (MonadIO m, Unbox a) =>
    (Int -> Int) -> m (MutArray a) -> Fold m a (MutArray a)
appendWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> m (MutArray a) -> Fold m a (MutArray a)
appendWith Int -> Int
sizer = (MutArray a -> a -> m (MutArray a))
-> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' ((Int -> Int) -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> MutArray a -> a -> m (MutArray a)
snocWith Int -> Int
sizer)

{-# DEPRECATED writeAppendWith "Please use appendWith instead." #-}
{-# INLINE writeAppendWith #-}
writeAppendWith :: forall m a. (MonadIO m, Unbox a) =>
    (Int -> Int) -> m (MutArray a) -> Fold m a (MutArray a)
writeAppendWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> m (MutArray a) -> Fold m a (MutArray a)
writeAppendWith = (Int -> Int) -> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int) -> m (MutArray a) -> Fold m a (MutArray a)
appendWith

-- | @append action@ mutates the array generated by @action@ to append the
-- input stream. If there is no reserved space available in the array it is
-- reallocated to double the size and aligned to power of 2.
--
-- Note that the returned array may be a mutated version of original array.
--
-- >>> append = Fold.foldlM' MutArray.snoc
--
{-# INLINE append #-}
append :: forall m a. (MonadIO m, Unbox a) =>
    m (MutArray a) -> Fold m a (MutArray a)
-- append = appendWith (* 2)
append :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
m (MutArray a) -> Fold m a (MutArray a)
append = (MutArray a -> a -> m (MutArray a))
-> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
snoc

-- | Fold @append2 arr@ mutates the array arr to append the input stream. If
-- there is no reserved space available in the array it is reallocated to
-- double the size and aligned to power of 2.
--
-- Note that the returned array may be a mutated version of original array.
--
-- >>> append2 arr = Fold.foldlM' MutArray.snoc (pure arr)
--
{-# INLINE append2 #-}
append2 :: (MonadIO m, Unbox a) => MutArray a -> Fold m a (MutArray a)
append2 :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Fold m a (MutArray a)
append2 MutArray a
arr = (MutArray a -> a -> m (MutArray a))
-> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
snoc (MutArray a -> m (MutArray a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
arr)

{-# DEPRECATED writeAppend "Please use append instead." #-}
{-# INLINE writeAppend #-}
writeAppend :: forall m a. (MonadIO m, Unbox a) =>
    m (MutArray a) -> Fold m a (MutArray a)
writeAppend :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
m (MutArray a) -> Fold m a (MutArray a)
writeAppend = m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
m (MutArray a) -> Fold m a (MutArray a)
append

-- | @appendGrowBy arr@ mutates the array arr to append the input stream. If
-- there is no reserved space available in the array it is reallocated to add
-- space for the min number of elements supplied and align to block size if the
-- array becomes larger than 'largeObjectThreshold'.
--
-- Note that the returned array may be a mutated version of original array.
--
-- >>> appendGrowBy n arr = Fold.foldlM' (MutArray.snocGrowBy n) (pure arr)
--
{-# INLINE appendGrowBy #-}
appendGrowBy :: (MonadIO m, Unbox a) =>
    Int -> MutArray a -> Fold m a (MutArray a)
appendGrowBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> Fold m a (MutArray a)
appendGrowBy Int
n MutArray a
arr = (MutArray a -> a -> m (MutArray a))
-> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' (Int -> MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m (MutArray a)
snocGrowBy Int
n) (MutArray a -> m (MutArray a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
arr)

-------------------------------------------------------------------------------
-- Actions for Appending streams
-------------------------------------------------------------------------------

-- |
-- >>> appendStream arr = Stream.fold (MutArray.append (pure arr))
--
{-# INLINE appendStream #-}
appendStream :: (MonadIO m, Unbox a) =>
    MutArray a -> Stream m a -> m (MutArray a)
appendStream :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a -> m (MutArray a)
appendStream MutArray a
arr = Fold m a (MutArray a) -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold (m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
m (MutArray a) -> Fold m a (MutArray a)
append (MutArray a -> m (MutArray a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
arr))

-- |
-- >>> appendStreamN n arr = Stream.fold (MutArray.appendMax n arr)
--
{-# INLINE appendStreamN #-}
appendStreamN :: (MonadIO m, Unbox a) =>
    Int -> MutArray a -> Stream m a -> m (MutArray a)
appendStreamN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> Stream m a -> m (MutArray a)
appendStreamN Int
n MutArray a
arr = Fold m a (MutArray a) -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold (Int -> MutArray a -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> Fold m a (MutArray a)
appendMax Int
n MutArray a
arr)

-- | The array is grown only by the required amount of space.
{-# INLINE appendCString# #-}
appendCString# :: MonadIO m => MutArray Word8 -> Addr# -> m (MutArray Word8)
appendCString# :: forall (m :: * -> *).
MonadIO m =>
MutArray Word8 -> Addr# -> m (MutArray Word8)
appendCString# MutArray Word8
arr Addr#
addr = do
    CSize
len <- IO CSize -> m CSize
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO CSize -> m CSize) -> IO CSize -> m CSize
forall a b. (a -> b) -> a -> b
$ Addr# -> IO CSize
c_strlen_pinned Addr#
addr
    MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
forall (m :: * -> *).
MonadIO m =>
MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
appendPtrN MutArray Word8
arr (Addr# -> Ptr Word8
forall a. Addr# -> Ptr a
Ptr Addr#
addr) (CSize -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral CSize
len)

-- Note: in hsc code # is treated in a special way, so it is difficult to use
-- appendCString#
{-# INLINE appendCString #-}
appendCString :: MonadIO m => MutArray Word8 -> Ptr a -> m (MutArray Word8)
appendCString :: forall (m :: * -> *) a.
MonadIO m =>
MutArray Word8 -> Ptr a -> m (MutArray Word8)
appendCString MutArray Word8
arr (Ptr Addr#
addr) = MutArray Word8 -> Addr# -> m (MutArray Word8)
forall (m :: * -> *).
MonadIO m =>
MutArray Word8 -> Addr# -> m (MutArray Word8)
appendCString# MutArray Word8
arr Addr#
addr

-------------------------------------------------------------------------------
-- Folds for creating
-------------------------------------------------------------------------------

-- XXX Use "IO" instead of "m" in the alloc function

-- XXX We can carry bound as well in the state to make sure we do not lose the
-- remaining capacity. Need to check perf impact.

-- | Like 'unsafeCreateOf' but takes a new array allocator @alloc size@
-- function as argument.
--
-- >>> unsafeCreateOfWith alloc n = MutArray.unsafeAppendN (alloc n) n
--
-- /Pre-release/
{-# INLINE_NORMAL unsafeCreateOfWith #-}
unsafeCreateOfWith :: forall m a. (MonadIO m, Unbox a)
    => (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
unsafeCreateOfWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
unsafeCreateOfWith Int -> m (MutArray a)
alloc Int
n = ArrayUnsafe a -> MutArray a
forall a. ArrayUnsafe a -> MutArray a
fromArrayUnsafe (ArrayUnsafe a -> MutArray a)
-> Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (ArrayUnsafe a -> a -> m (ArrayUnsafe a))
-> m (ArrayUnsafe a) -> Fold m a (ArrayUnsafe a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' ArrayUnsafe a -> a -> m (ArrayUnsafe a)
forall {m :: * -> *} {a} {a} {a}.
(MonadIO m, Unbox a) =>
ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step m (ArrayUnsafe a)
initial

    where

    initial :: m (ArrayUnsafe a)
initial = MutArray a -> ArrayUnsafe a
forall a. MutArray a -> ArrayUnsafe a
toArrayUnsafe (MutArray a -> ArrayUnsafe a)
-> m (MutArray a) -> m (ArrayUnsafe a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> m (MutArray a)
alloc (Int -> Int -> Int
forall a. Ord a => a -> a -> a
max Int
n Int
0)

    step :: ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step (ArrayUnsafe MutByteArray
contents Int
start Int
end) a
x = do
        IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
end MutByteArray
contents a
x
        ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
          (ArrayUnsafe a -> m (ArrayUnsafe a))
-> ArrayUnsafe a -> m (ArrayUnsafe a)
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
start (INDEX_NEXT(end,a))

{-# DEPRECATED writeNWithUnsafe "Please use unsafeCreateOfWith instead." #-}
{-# INLINE writeNWithUnsafe #-}
writeNWithUnsafe :: forall m a. (MonadIO m, Unbox a)
    => (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeNWithUnsafe :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeNWithUnsafe = (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
unsafeCreateOfWith

{-# INLINE_NORMAL writeNUnsafeAs #-}
writeNUnsafeAs :: forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> Fold m a (MutArray a)
writeNUnsafeAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeNUnsafeAs PinnedState
ps = (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
unsafeCreateOfWith (PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
ps)

-- | Like 'createOf' but does not check the array bounds when writing. The fold
-- driver must not call the step function more than 'n' times otherwise it will
-- corrupt the memory and crash. This function exists mainly because any
-- conditional in the step function blocks fusion causing 10x performance
-- slowdown.
--
-- >>> unsafeCreateOf = MutArray.unsafeCreateOfWith MutArray.emptyOf
--
{-# INLINE_NORMAL unsafeCreateOf #-}
unsafeCreateOf :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m a (MutArray a)
unsafeCreateOf :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
unsafeCreateOf = PinnedState -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeNUnsafeAs PinnedState
Unpinned

{-# DEPRECATED writeNUnsafe "Please use unsafeCreateOf instead." #-}
{-# INLINE writeNUnsafe #-}
writeNUnsafe :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m a (MutArray a)
writeNUnsafe :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
writeNUnsafe = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
unsafeCreateOf

-- | Like 'unsafeCreateOf' but creates a pinned array.
{-# INLINE_NORMAL unsafeCreateOf' #-}
unsafePinnedCreateOf, unsafeCreateOf' :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m a (MutArray a)
unsafeCreateOf' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
unsafeCreateOf' = PinnedState -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeNUnsafeAs PinnedState
Pinned
RENAME_PRIME(unsafePinnedCreateOf,unsafeCreateOf)

{-# DEPRECATED pinnedWriteNUnsafe "Please use unsafeCreateOf' instead." #-}
{-# INLINE pinnedWriteNUnsafe #-}
pinnedWriteNUnsafe :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m a (MutArray a)
pinnedWriteNUnsafe :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
pinnedWriteNUnsafe = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
unsafeCreateOf'

-- XXX Use "IO" instead of "m" in the alloc function

-- | @createWithOf alloc n@ folds a maximum of @n@ elements into an array
-- allocated using the @alloc@ function.
--
-- The array capacity is guranteed to be at least @n@.
--
-- >>> createWithOf alloc n = Fold.take n (MutArray.unsafeCreateOfWith alloc n)
-- >>> createWithOf alloc n = MutArray.appendN (alloc n) n
--
{-# INLINE_NORMAL createWithOf #-}
createOfWith, createWithOf :: forall m a. (MonadIO m, Unbox a)
    => (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
createWithOf :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
createWithOf Int -> m (MutArray a)
alloc Int
n = Int -> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Int -> Fold m a b -> Fold m a b
FL.take Int
n ((Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
unsafeCreateOfWith Int -> m (MutArray a)
alloc Int
n)

{-# DEPRECATED writeNWith "Please use createWithOf instead." #-}
{-# INLINE writeNWith #-}
writeNWith :: forall m a. (MonadIO m, Unbox a)
    => (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeNWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeNWith = (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
createWithOf

{-# INLINE_NORMAL writeNAs #-}
writeNAs ::
       forall m a. (MonadIO m, Unbox a)
    => PinnedState
    -> Int
    -> Fold m a (MutArray a)
writeNAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeNAs PinnedState
ps = (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
createWithOf (PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
ps)

-- | @createOf n@ folds a maximum of @n@ elements from the input stream to an
-- 'MutArray'.
--
-- The array capacity is guranteed to be at least @n@.
--
-- >>> createOf = MutArray.createWithOf MutArray.emptyOf
-- >>> createOf n = Fold.take n (MutArray.unsafeCreateOf n)
-- >>> createOf n = MutArray.appendMax n MutArray.empty
--
{-# INLINE_NORMAL createOf #-}
createOf :: forall m a. (MonadIO m, Unbox a) => Int -> Fold m a (MutArray a)
createOf :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createOf = PinnedState -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeNAs PinnedState
Unpinned

{-# DEPRECATED writeN "Please use createOf instead." #-}
{-# INLINE writeN #-}
writeN :: forall m a. (MonadIO m, Unbox a) => Int -> Fold m a (MutArray a)
writeN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
writeN = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createOf

-- | Like 'createOf' but creates a pinned array.
{-# INLINE_NORMAL createOf' #-}
pinnedCreateOf, createOf' ::
       forall m a. (MonadIO m, Unbox a)
    => Int
    -> Fold m a (MutArray a)
createOf' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createOf' = PinnedState -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeNAs PinnedState
Pinned
RENAME_PRIME(pinnedCreateOf,createOf)

{-# DEPRECATED pinnedWriteN "Please use createOf' instead." #-}
{-# INLINE pinnedWriteN #-}
pinnedWriteN ::
       forall m a. (MonadIO m, Unbox a)
    => Int
    -> Fold m a (MutArray a)
pinnedWriteN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
pinnedWriteN = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createOf'

-- | Like unsafeCreateOfWith but writes the array in reverse order.
--
-- /Internal/
{-# INLINE_NORMAL writeRevNWithUnsafe #-}
writeRevNWithUnsafe :: forall m a. (MonadIO m, Unbox a)
    => (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeRevNWithUnsafe :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeRevNWithUnsafe Int -> m (MutArray a)
alloc Int
n = ArrayUnsafe a -> MutArray a
forall a. ArrayUnsafe a -> MutArray a
fromArrayUnsafe (ArrayUnsafe a -> MutArray a)
-> Fold m a (ArrayUnsafe a) -> Fold m a (MutArray a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (ArrayUnsafe a -> a -> m (ArrayUnsafe a))
-> m (ArrayUnsafe a) -> Fold m a (ArrayUnsafe a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' ArrayUnsafe a -> a -> m (ArrayUnsafe a)
forall {m :: * -> *} {a} {a} {a}.
(MonadIO m, Unbox a) =>
ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step m (ArrayUnsafe a)
forall {a}. m (ArrayUnsafe a)
initial

    where

    toArrayUnsafeRev :: MutArray a -> ArrayUnsafe a
toArrayUnsafeRev (MutArray MutByteArray
contents Int
_ Int
_ Int
bound) =
         MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
bound Int
bound

    initial :: m (ArrayUnsafe a)
initial = MutArray a -> ArrayUnsafe a
forall {a} {a}. MutArray a -> ArrayUnsafe a
toArrayUnsafeRev (MutArray a -> ArrayUnsafe a)
-> m (MutArray a) -> m (ArrayUnsafe a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int -> m (MutArray a)
alloc (Int -> Int -> Int
forall a. Ord a => a -> a -> a
max Int
n Int
0)

    step :: ArrayUnsafe a -> a -> m (ArrayUnsafe a)
step (ArrayUnsafe MutByteArray
contents Int
start Int
end) a
x = do
        let ptr :: Int
ptr = INDEX_PREV(start,a)
        IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
ptr MutByteArray
contents a
x
        ArrayUnsafe a -> m (ArrayUnsafe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
          (ArrayUnsafe a -> m (ArrayUnsafe a))
-> ArrayUnsafe a -> m (ArrayUnsafe a)
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> Int -> ArrayUnsafe a
forall a. MutByteArray -> Int -> Int -> ArrayUnsafe a
ArrayUnsafe MutByteArray
contents Int
ptr Int
end

-- | Like createWithOf but writes the array in reverse order.
--
-- /Internal/
{-# INLINE_NORMAL writeRevNWith #-}
writeRevNWith :: forall m a. (MonadIO m, Unbox a)
    => (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeRevNWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeRevNWith Int -> m (MutArray a)
alloc Int
n = Int -> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Int -> Fold m a b -> Fold m a b
FL.take Int
n ((Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeRevNWithUnsafe Int -> m (MutArray a)
alloc Int
n)

-- | Like 'createOf' but writes the array in reverse order.
--
-- /Pre-release/
{-# INLINE_NORMAL revCreateOf #-}
revCreateOf :: forall m a. (MonadIO m, Unbox a) => Int -> Fold m a (MutArray a)
revCreateOf :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
revCreateOf = (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
writeRevNWith Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
new

{-# DEPRECATED writeRevN "Please use revCreateOf instead." #-}
{-# INLINE writeRevN #-}
writeRevN :: forall m a. (MonadIO m, Unbox a) => Int -> Fold m a (MutArray a)
writeRevN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
writeRevN = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
revCreateOf

-- | @pinnedWriteNAligned align n@ folds a maximum of @n@ elements from the
-- input stream to a 'MutArray' aligned to the given size.
--
-- /Pre-release/
--
{-# INLINE_NORMAL pinnedWriteNAligned #-}
pinnedWriteNAligned :: forall m a. (MonadIO m, Unbox a)
    => Int -> Int -> Fold m a (MutArray a)
pinnedWriteNAligned :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Int -> Fold m a (MutArray a)
pinnedWriteNAligned Int
align = (Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> m (MutArray a)) -> Int -> Fold m a (MutArray a)
createWithOf (Int -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Int -> m (MutArray a)
pinnedNewAligned Int
align)

-- XXX Buffer to a list instead?

-- | Buffer a stream into a stream of arrays.
--
-- >>> buildChunks n = Fold.many (MutArray.createOf n) Fold.toStreamK
--
-- Breaking an array into an array stream  can be useful to consume a large
-- array sequentially such that memory of the array is released incrementatlly.
--
-- See also: 'arrayStreamKFromStreamD'.
--
-- /Unimplemented/
--
{-# INLINE_NORMAL buildChunks #-}
buildChunks :: (MonadIO m, Unbox a) =>
    Int -> Fold m a (StreamK n (MutArray a))
buildChunks :: forall (m :: * -> *) a (n :: * -> *).
(MonadIO m, Unbox a) =>
Int -> Fold m a (StreamK n (MutArray a))
buildChunks Int
n = Fold m a (MutArray a)
-> Fold m (MutArray a) (StreamK n (MutArray a))
-> Fold m a (StreamK n (MutArray a))
forall (m :: * -> *) a b c.
Monad m =>
Fold m a b -> Fold m b c -> Fold m a c
FL.many (Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createOf Int
n) Fold m (MutArray a) (StreamK n (MutArray a))
forall (m :: * -> *) a (n :: * -> *).
Monad m =>
Fold m a (StreamK n a)
FL.toStreamK

{-# DEPRECATED writeChunks "Please use buildChunks instead." #-}
{-# INLINE writeChunks #-}
writeChunks :: (MonadIO m, Unbox a) =>
    Int -> Fold m a (StreamK n (MutArray a))
writeChunks :: forall (m :: * -> *) a (n :: * -> *).
(MonadIO m, Unbox a) =>
Int -> Fold m a (StreamK n (MutArray a))
writeChunks = Int -> Fold m a (StreamK n (MutArray a))
forall (m :: * -> *) a (n :: * -> *).
(MonadIO m, Unbox a) =>
Int -> Fold m a (StreamK n (MutArray a))
buildChunks

-- | Grows by doubling
{-# INLINE_NORMAL writeWithAs #-}
writeWithAs :: forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> Fold m a (MutArray a)
-- writeWithAs ps n = FL.rmapM rightSize $ appendWith (* 2) (newAs ps n)
writeWithAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeWithAs PinnedState
ps Int
elemCount =
    (MutArray a -> m (MutArray a))
-> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) b c a.
Monad m =>
(b -> m c) -> Fold m a b -> Fold m a c
FL.rmapM MutArray a -> m (MutArray a)
extract (Fold m a (MutArray a) -> Fold m a (MutArray a))
-> Fold m a (MutArray a) -> Fold m a (MutArray a)
forall a b. (a -> b) -> a -> b
$ (MutArray a -> a -> m (MutArray a))
-> m (MutArray a) -> Fold m a (MutArray a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Fold m a b
FL.foldlM' MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
step m (MutArray a)
initial

    where

    -- XXX create an empty Array if the count is <= 0?
    initial :: m (MutArray a)
initial = do
        Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
elemCount Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
0) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ [Char] -> m ()
forall a. (?callStack::CallStack) => [Char] -> a
error [Char]
"createWith: elemCount is negative"
        PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
ps Int
elemCount

    step :: MutArray a -> a -> m (MutArray a)
step arr :: MutArray a
arr@(MutArray MutByteArray
_ Int
start Int
end Int
bound) a
x
        | INDEX_NEXT(end,a) > bound = do
        let oldSize = end - start
            newSize = max (oldSize * 2) 1
        arr1 <- liftIO $ reallocExplicitAs ps (SIZE_OF(a)) newSize arr
        unsafeSnoc arr1 x
    step MutArray a
arr a
x = MutArray a -> a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> a -> m (MutArray a)
unsafeSnoc MutArray a
arr a
x

    extract :: MutArray a -> m (MutArray a)
extract = IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> (MutArray a -> IO (MutArray a)) -> MutArray a -> m (MutArray a)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. MutArray a -> IO (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> m (MutArray a)
rightSize

-- XXX Compare createWith with fromStreamD which uses an array of streams
-- implementation. We can write this using buildChunks above if that is faster.
-- If createWith is faster then we should use that to implement
-- fromStreamD.
--
-- XXX The realloc based implementation needs to make one extra copy if we use
-- shrinkToFit.  On the other hand, the stream of arrays implementation may
-- buffer the array chunk pointers in memory but it does not have to shrink as
-- we know the exact size in the end. However, memory copying does not seem to
-- be as expensive as the allocations. Therefore, we need to reduce the number
-- of allocations instead. Also, the size of allocations matters, right sizing
-- an allocation even at the cost of copying seems to help.  Should be measured
-- on a big stream with heavy calls to toArray to see the effect.
--
-- XXX check if GHC's memory allocator is efficient enough. We can try the C
-- malloc to compare against.

-- | @createMinOf count@ folds the whole input to a single array. The array
-- starts at a size big enough to hold minCount elements, the size is doubled
-- every time the array needs to be grown.
--
-- The array capacity is guaranteed to be at least count.
--
-- /Caution! Do not use this on infinite streams./
--
-- >>> f n = MutArray.appendWith (* 2) (MutArray.emptyOf n)
-- >>> createWith n = Fold.rmapM MutArray.rightSize (f n)
-- >>> createWith n = Fold.rmapM MutArray.fromChunksK (MutArray.buildChunks n)
--
-- /Pre-release/
{-# INLINE_NORMAL createMinOf #-}
createMinOf, createWith :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m a (MutArray a)
-- createWith n = FL.rmapM rightSize $ appendWith (* 2) (emptyOf n)
createMinOf :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createMinOf = PinnedState -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeWithAs PinnedState
Unpinned

createWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createWith = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createMinOf

{-# DEPRECATED writeWith "Please use createMinOf instead." #-}
{-# INLINE writeWith #-}
writeWith :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m a (MutArray a)
writeWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
writeWith = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createMinOf

-- | Fold the whole input to a single array.
--
-- Same as 'createMinOf using an initial array size of 'arrayChunkBytes' bytes
-- rounded up to the element size. If the array is expected to be smaller than
-- 'arrayChunkBytes' then use 'createMinOf' to avoid wasting memory.
--
-- /Caution! Do not use this on infinite streams./
--
{-# INLINE create #-}
create :: forall m a. (MonadIO m, Unbox a) => Fold m a (MutArray a)
create :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
create = Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createMinOf (a -> Int -> Int
forall a. Unbox a => a -> Int -> Int
allocBytesToElemCount (a
forall a. (?callStack::CallStack) => a
undefined :: a) Int
arrayChunkBytes)

{-# DEPRECATED write "Please use create instead." #-}
{-# INLINE write #-}
write :: forall m a. (MonadIO m, Unbox a) => Fold m a (MutArray a)
write :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
write = Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
create

-- | Like 'create' but creates a pinned array.
{-# INLINE create' #-}
pinnedCreate, create' :: forall m a. (MonadIO m, Unbox a) => Fold m a (MutArray a)
create' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
create' =
    PinnedState -> Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m a (MutArray a)
writeWithAs PinnedState
Pinned (a -> Int -> Int
forall a. Unbox a => a -> Int -> Int
allocBytesToElemCount (a
forall a. (?callStack::CallStack) => a
undefined :: a) Int
arrayChunkBytes)
RENAME_PRIME(pinnedCreate,create)

{-# DEPRECATED pinnedWrite "Please use create' instead." #-}
{-# INLINE pinnedWrite #-}
pinnedWrite :: forall m a. (MonadIO m, Unbox a) => Fold m a (MutArray a)
pinnedWrite :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
pinnedWrite = Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
create'

-------------------------------------------------------------------------------
-- construct from streams, known size
-------------------------------------------------------------------------------

{-# INLINE_NORMAL fromStreamDNAs #-}
fromStreamDNAs :: forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> D.Stream m a -> m (MutArray a)
fromStreamDNAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Stream m a -> m (MutArray a)
fromStreamDNAs PinnedState
ps Int
limit Stream m a
str = do
    (MutArray a
arr :: MutArray a) <- PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
ps Int
limit
    Int
end <- (Int -> a -> m Int) -> m Int -> Stream m a -> m Int
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Stream m a -> m b
D.foldlM'
            (MutByteArray -> Int -> a -> m Int
forall {m :: * -> *} {a}.
(MonadIO m, Unbox a) =>
MutByteArray -> Int -> a -> m Int
fwrite (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
arr))
            (Int -> m Int
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Int -> m Int) -> Int -> m Int
forall a b. (a -> b) -> a -> b
$ MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
arr)
            (Stream m a -> m Int) -> Stream m a -> m Int
forall a b. (a -> b) -> a -> b
$ Int -> Stream m a -> Stream m a
forall (m :: * -> *) a.
Applicative m =>
Int -> Stream m a -> Stream m a
D.take Int
limit Stream m a
str
    MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> m (MutArray a)) -> MutArray a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutArray a
arr {arrEnd :: Int
arrEnd = Int
end}

    where

    fwrite :: MutByteArray -> Int -> a -> m Int
fwrite MutByteArray
arrContents Int
ptr a
x = do
        IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Int -> MutByteArray -> a -> IO ()
forall a. Unbox a => Int -> MutByteArray -> a -> IO ()
pokeAt Int
ptr MutByteArray
arrContents  a
x
        Int -> m Int
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Int -> m Int) -> Int -> m Int
forall a b. (a -> b) -> a -> b
$ INDEX_NEXT(ptr,a)

-- | Create a MutArray of given size from a stream.
--
-- >>> fromStreamN n = Stream.fold (MutArray.createOf n)
--
{-# INLINE_NORMAL fromStreamN #-}
fromStreamN :: forall m a. (MonadIO m, Unbox a)
    => Int -> D.Stream m a -> m (MutArray a)
-- fromStreamDN n = D.fold (createOf n)
fromStreamN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream m a -> m (MutArray a)
fromStreamN = PinnedState -> Int -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Stream m a -> m (MutArray a)
fromStreamDNAs PinnedState
Unpinned

{-# DEPRECATED fromStreamDN "Please use fromStreamN instead." #-}
{-# INLINE fromStreamDN #-}
fromStreamDN :: forall m a. (MonadIO m, Unbox a)
    => Int -> D.Stream m a -> m (MutArray a)
fromStreamDN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream m a -> m (MutArray a)
fromStreamDN = Int -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream m a -> m (MutArray a)
fromStreamN

-- | Create a 'MutArray' from the first N elements of a list. The array is
-- allocated to size N, if the list terminates before N elements then the
-- array may hold less than N elements.
--
{-# INLINABLE fromListN #-}
fromListN :: (MonadIO m, Unbox a) => Int -> [a] -> m (MutArray a)
fromListN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> [a] -> m (MutArray a)
fromListN Int
n [a]
xs = Int -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream m a -> m (MutArray a)
fromStreamN Int
n (Stream m a -> m (MutArray a)) -> Stream m a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ [a] -> Stream m a
forall (m :: * -> *) a. Applicative m => [a] -> Stream m a
D.fromList [a]
xs

-- | Like 'fromListN' but creates a pinned array.
{-# INLINABLE fromListN' #-}
pinnedFromListN, fromListN' :: (MonadIO m, Unbox a) => Int -> [a] -> m (MutArray a)
fromListN' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> [a] -> m (MutArray a)
fromListN' Int
n [a]
xs = PinnedState -> Int -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Stream m a -> m (MutArray a)
fromStreamDNAs PinnedState
Pinned Int
n (Stream m a -> m (MutArray a)) -> Stream m a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ [a] -> Stream m a
forall (m :: * -> *) a. Applicative m => [a] -> Stream m a
D.fromList [a]
xs
RENAME_PRIME(pinnedFromListN,fromListN)

-- | Like fromListN but writes the array in reverse order.
--
-- /Pre-release/
{-# INLINE fromListRevN #-}
fromListRevN :: (MonadIO m, Unbox a) => Int -> [a] -> m (MutArray a)
fromListRevN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> [a] -> m (MutArray a)
fromListRevN Int
n [a]
xs = Fold m a (MutArray a) -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold (Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
revCreateOf Int
n) (Stream m a -> m (MutArray a)) -> Stream m a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ [a] -> Stream m a
forall (m :: * -> *) a. Applicative m => [a] -> Stream m a
D.fromList [a]
xs

-- | Convert a pure stream in Identity monad to a mutable array.
{-# INLINABLE fromPureStreamN #-}
fromPureStreamN :: (MonadIO m, Unbox a) =>
    Int -> Stream Identity a -> m (MutArray a)
fromPureStreamN :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream Identity a -> m (MutArray a)
fromPureStreamN Int
n = Fold m a (MutArray a) -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold (Int -> Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m a (MutArray a)
createOf Int
n) (Stream m a -> m (MutArray a))
-> (Stream Identity a -> Stream m a)
-> Stream Identity a
-> m (MutArray a)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Stream Identity a -> Stream m a
forall (m :: * -> *) a. Monad m => Stream Identity a -> Stream m a
D.generalizeInner

-- | Convert a pure stream in Identity monad to a mutable array.
{-# INLINABLE fromPureStream #-}
fromPureStream :: (MonadIO m, Unbox a) => Stream Identity a -> m (MutArray a)
fromPureStream :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream Identity a -> m (MutArray a)
fromPureStream = Fold m a (MutArray a) -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold Fold m a (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Fold m a (MutArray a)
create (Stream m a -> m (MutArray a))
-> (Stream Identity a -> Stream m a)
-> Stream Identity a
-> m (MutArray a)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Stream Identity a -> Stream m a
forall (m :: * -> *) a. Monad m => Stream Identity a -> Stream m a
D.generalizeInner

-- | @fromPtrN len addr@ copies @len@ bytes from @addr@ into an array.
--
-- /Unsafe:/
--
-- The caller has to ensure that:
--
-- 1. the pointer is pinned and alive during the call.
-- 2. the pointer passed is valid up to the given length.
--
{-# INLINABLE fromPtrN #-}
fromPtrN :: MonadIO m => Int -> Ptr Word8 -> m (MutArray Word8)
fromPtrN :: forall (m :: * -> *).
MonadIO m =>
Int -> Ptr Word8 -> m (MutArray Word8)
fromPtrN Int
len Ptr Word8
addr = do
    -- memcpy is better than stream copy when the size is known.
    -- XXX We can implement a stream copy in a similar way by streaming Word64
    -- first and then remaining Word8.
    (MutArray Word8
arr :: MutArray Word8) <- Int -> m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
emptyOf Int
len
    let mbarr :: MutableByteArray# RealWorld
mbarr = MutByteArray -> MutableByteArray# RealWorld
getMutByteArray# (MutArray Word8 -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray Word8
arr)
    Ptr Word8
_ <- IO (Ptr Word8) -> m (Ptr Word8)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Ptr Word8) -> m (Ptr Word8))
-> IO (Ptr Word8) -> m (Ptr Word8)
forall a b. (a -> b) -> a -> b
$ MutableByteArray# RealWorld -> Ptr Word8 -> CSize -> IO (Ptr Word8)
c_memcpy_pinned_src MutableByteArray# RealWorld
mbarr Ptr Word8
addr (Int -> CSize
forall a b. (Integral a, Num b) => a -> b
fromIntegral Int
len)
    MutArray Word8 -> m (MutArray Word8)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (MutArray Word8
arr { arrEnd :: Int
arrEnd = Int
len })

-- | @fromCString# addr@ copies a C string consisting of bytes and
-- terminated by a null byte, into a Word8 array. The null byte is not copied.
--
-- >>> MutArray.fromCString# "hello"#
--
-- /Unsafe:/
--
-- The caller has to ensure that:
--
-- 1. the @addr@ is pinned and alive during the call.
-- 2. the pointer passed is valid up to the point where null byte is found.
--
{-# INLINABLE fromCString# #-}
fromCString# :: MonadIO m => Addr# -> m (MutArray Word8)
fromCString# :: forall (m :: * -> *). MonadIO m => Addr# -> m (MutArray Word8)
fromCString# Addr#
addr = do
    -- It is better to count the size first and allocate exact space.
    -- Also, memcpy is better than stream copy when the size is known.
    -- C strlen compares 4 bytes at a time, so is better than the stream
    -- version. https://github.com/bminor/glibc/blob/master/string/strlen.c
    -- XXX We can possibly use a stream of Word64 to do the same.
    -- fromByteStr# addr = fromPureStream (D.fromByteStr# addr)
    CSize
len <- IO CSize -> m CSize
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO CSize -> m CSize) -> IO CSize -> m CSize
forall a b. (a -> b) -> a -> b
$ Addr# -> IO CSize
c_strlen_pinned Addr#
addr
    Int -> Ptr Word8 -> m (MutArray Word8)
forall (m :: * -> *).
MonadIO m =>
Int -> Ptr Word8 -> m (MutArray Word8)
fromPtrN (CSize -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral CSize
len) (Addr# -> Ptr Word8
forall a. Addr# -> Ptr a
Ptr Addr#
addr)

{-# DEPRECATED fromByteStr# "Please fromCString# instead." #-}
{-# INLINABLE fromByteStr# #-}
fromByteStr# :: MonadIO m => Addr# -> m (MutArray Word8)
fromByteStr# :: forall (m :: * -> *). MonadIO m => Addr# -> m (MutArray Word8)
fromByteStr# = Addr# -> m (MutArray Word8)
forall (m :: * -> *). MonadIO m => Addr# -> m (MutArray Word8)
fromCString#

-- | @fromW16CString# addr@ copies a C string consisting of 16-bit wide chars
-- and terminated by a 16-bit null char, into a Word16 array. The null
-- character is not copied.
--
-- Useful for copying UTF16 strings on Windows.
--
-- /Unsafe:/
--
-- The caller has to ensure that:
--
-- 1. the @addr@ is pinned and alive during the call.
-- 2. the pointer passed is valid up to the point where null Word16 is found.
--
{-# INLINABLE fromW16CString# #-}
fromW16CString# :: MonadIO m => Addr# -> m (MutArray Word16)
fromW16CString# :: forall (m :: * -> *). MonadIO m => Addr# -> m (MutArray Word16)
fromW16CString# Addr#
addr = do
    -- XXX this can be done faster if we process one Word64 at a time
    Int
w16len <- Fold m Word16 Int -> Stream m Word16 -> m Int
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> m b
D.fold Fold m Word16 Int
forall (m :: * -> *) a. Monad m => Fold m a Int
FL.length (Stream m Word16 -> m Int) -> Stream m Word16 -> m Int
forall a b. (a -> b) -> a -> b
$ Addr# -> Stream m Word16
forall (m :: * -> *). Monad m => Addr# -> Stream m Word16
D.fromW16CString# Addr#
addr
    let bytes :: Int
bytes = Int
w16len Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
2
    MutArray Word8
arr <- Int -> Ptr Word8 -> m (MutArray Word8)
forall (m :: * -> *).
MonadIO m =>
Int -> Ptr Word8 -> m (MutArray Word8)
fromPtrN Int
bytes (Addr# -> Ptr Word8
forall a. Addr# -> Ptr a
Ptr Addr#
addr)
    MutArray Word16 -> m (MutArray Word16)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (MutArray Word16 -> m (MutArray Word16))
-> MutArray Word16 -> m (MutArray Word16)
forall a b. (a -> b) -> a -> b
$ MutArray Word8 -> MutArray Word16
forall a b. MutArray a -> MutArray b
unsafeCast MutArray Word8
arr

-------------------------------------------------------------------------------
-- convert a stream of arrays to a single array by reallocating and copying
-------------------------------------------------------------------------------

-- XXX Both of these implementations of splicing seem to perform equally well.
-- We need to perform benchmarks over a range of sizes though.

-- | Also see 'fromChunksK'.
{-# INLINE fromChunksRealloced #-}
fromChunksRealloced :: forall m a. (MonadIO m, Unbox a)
    => Stream m (MutArray a) -> m (MutArray a)
fromChunksRealloced :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m (MutArray a) -> m (MutArray a)
fromChunksRealloced Stream m (MutArray a)
s = do
    Maybe (MutArray a, Stream m (MutArray a))
res <- Stream m (MutArray a)
-> m (Maybe (MutArray a, Stream m (MutArray a)))
forall (m :: * -> *) a.
Monad m =>
Stream m a -> m (Maybe (a, Stream m a))
D.uncons Stream m (MutArray a)
s
    case Maybe (MutArray a, Stream m (MutArray a))
res of
        Just (MutArray a
a, Stream m (MutArray a)
strm) -> do
            MutArray a
arr <- (MutArray a -> MutArray a -> m (MutArray a))
-> m (MutArray a) -> Stream m (MutArray a) -> m (MutArray a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> Stream m a -> m b
D.foldlM' MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> MutArray a -> m (MutArray a)
spliceExp (MutArray a -> m (MutArray a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
a) Stream m (MutArray a)
strm
            -- Reallocation is exponential so there may be 50% empty space in
            -- worst case. One more reallocation to reclaim the space.
            MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> m (MutArray a)
rightSize MutArray a
arr
        Maybe (MutArray a, Stream m (MutArray a))
Nothing -> MutArray a -> m (MutArray a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
forall a. MutArray a
nil

-------------------------------------------------------------------------------
-- convert a stream of arrays to a single array by buffering arrays first
-------------------------------------------------------------------------------

{-# INLINE arrayStreamKLength #-}
arrayStreamKLength :: (Monad m, Unbox a) => StreamK m (MutArray a) -> m Int
arrayStreamKLength :: forall (m :: * -> *) a.
(Monad m, Unbox a) =>
StreamK m (MutArray a) -> m Int
arrayStreamKLength StreamK m (MutArray a)
as = (Int -> Int -> Int) -> Int -> StreamK m Int -> m Int
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> b) -> b -> StreamK m a -> m b
K.foldl' Int -> Int -> Int
forall a. Num a => a -> a -> a
(+) Int
0 ((MutArray a -> Int) -> StreamK m (MutArray a) -> StreamK m Int
forall a b (m :: * -> *). (a -> b) -> StreamK m a -> StreamK m b
K.map MutArray a -> Int
forall a. Unbox a => MutArray a -> Int
length StreamK m (MutArray a)
as)

-- | Convert an array stream to an array. Note that this requires peak memory
-- that is double the size of the array stream.
--
{-# INLINE fromChunkskAs #-}
fromChunkskAs :: (Unbox a, MonadIO m) =>
    PinnedState -> StreamK m (MutArray a) -> m (MutArray a)
fromChunkskAs :: forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
PinnedState -> StreamK m (MutArray a) -> m (MutArray a)
fromChunkskAs PinnedState
ps StreamK m (MutArray a)
as = do
    Int
len <- StreamK m (MutArray a) -> m Int
forall (m :: * -> *) a.
(Monad m, Unbox a) =>
StreamK m (MutArray a) -> m Int
arrayStreamKLength StreamK m (MutArray a)
as
    MutArray a
arr <- PinnedState -> Int -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> m (MutArray a)
newAs PinnedState
ps Int
len
    -- XXX is StreamK fold faster or StreamD fold?
    (MutArray a -> MutArray a -> m (MutArray a))
-> m (MutArray a) -> StreamK m (MutArray a) -> m (MutArray a)
forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> StreamK m a -> m b
K.foldlM' MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice (MutArray a -> m (MutArray a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure MutArray a
arr) StreamK m (MutArray a)
as
    -- fromStreamDN len $ D.unfoldMany reader $ D.fromStreamK as

-- XXX Need to compare this with fromChunks and fromChunkList and keep the
-- fastest or simplest one if all are equally fast.

-- | Convert an array stream to an array. Note that this requires peak memory
-- that is double the size of the array stream.
--
-- Also see 'fromChunksRealloced'.
--
{-# INLINE fromChunksK #-}
fromChunksK :: (Unbox a, MonadIO m) =>
    StreamK m (MutArray a) -> m (MutArray a)
fromChunksK :: forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
StreamK m (MutArray a) -> m (MutArray a)
fromChunksK = PinnedState -> StreamK m (MutArray a) -> m (MutArray a)
forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
PinnedState -> StreamK m (MutArray a) -> m (MutArray a)
fromChunkskAs PinnedState
Unpinned

{-# DEPRECATED fromArrayStreamK "Please use fromChunksK instead." #-}
{-# INLINE fromArrayStreamK #-}
fromArrayStreamK :: (Unbox a, MonadIO m) =>
    StreamK m (MutArray a) -> m (MutArray a)
fromArrayStreamK :: forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
StreamK m (MutArray a) -> m (MutArray a)
fromArrayStreamK = StreamK m (MutArray a) -> m (MutArray a)
forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
StreamK m (MutArray a) -> m (MutArray a)
fromChunksK

{-# INLINE fromStreamDAs #-}
fromStreamDAs ::
       (MonadIO m, Unbox a) => PinnedState -> D.Stream m a -> m (MutArray a)
fromStreamDAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Stream m a -> m (MutArray a)
fromStreamDAs PinnedState
ps Stream m a
m =
    PinnedState -> Stream m a -> m (StreamK m (MutArray a))
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Stream m a -> m (StreamK m (MutArray a))
arrayStreamKFromStreamDAs PinnedState
Unpinned Stream m a
m m (StreamK m (MutArray a))
-> (StreamK m (MutArray a) -> m (MutArray a)) -> m (MutArray a)
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= PinnedState -> StreamK m (MutArray a) -> m (MutArray a)
forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
PinnedState -> StreamK m (MutArray a) -> m (MutArray a)
fromChunkskAs PinnedState
ps

-- | Create an 'Array' from a stream. This is useful when we want to create a
-- single array from a stream of unknown size. 'createOf' is at least twice
-- as efficient when the size is already known.
--
-- Note that if the input stream is too large memory allocation for the array
-- may fail.  When the stream size is not known, `chunksOf` followed by
-- processing of indvidual arrays in the resulting stream should be preferred.
--
-- /Pre-release/
{-# INLINE fromStream #-}
fromStream :: (MonadIO m, Unbox a) => Stream m a -> m (MutArray a)
fromStream :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m a -> m (MutArray a)
fromStream = PinnedState -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Stream m a -> m (MutArray a)
fromStreamDAs PinnedState
Unpinned

-- fromStream (Stream m) = P.fold create m
-- CAUTION: a very large number (millions) of arrays can degrade performance
-- due to GC overhead because we need to buffer the arrays before we flatten
-- all the arrays.
--
-- XXX Compare if this is faster or "fold create".
--
-- | We could take the approach of doubling the memory allocation on each
-- overflow. This would result in more or less the same amount of copying as in
-- the chunking approach. However, if we have to shrink in the end then it may
-- result in an extra copy of the entire data.
--
-- >>> fromStreamD = StreamD.fold MutArray.create
--
{-# INLINE fromStreamD #-}
{-# DEPRECATED fromStreamD "Please use fromStream instead." #-}
fromStreamD :: (MonadIO m, Unbox a) => D.Stream m a -> m (MutArray a)
fromStreamD :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m a -> m (MutArray a)
fromStreamD = Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m a -> m (MutArray a)
fromStream

-- | Create a 'MutArray' from a list. The list must be of finite size.
--
{-# INLINE fromList #-}
fromList :: (MonadIO m, Unbox a) => [a] -> m (MutArray a)
fromList :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[a] -> m (MutArray a)
fromList [a]
xs = Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Stream m a -> m (MutArray a)
fromStreamD (Stream m a -> m (MutArray a)) -> Stream m a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ [a] -> Stream m a
forall (m :: * -> *) a. Applicative m => [a] -> Stream m a
D.fromList [a]
xs

-- | Like 'fromList' but creates a pinned array.
{-# INLINE fromList' #-}
pinnedFromList, fromList' :: (MonadIO m, Unbox a) => [a] -> m (MutArray a)
fromList' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[a] -> m (MutArray a)
fromList' [a]
xs = PinnedState -> Stream m a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Stream m a -> m (MutArray a)
fromStreamDAs PinnedState
Pinned (Stream m a -> m (MutArray a)) -> Stream m a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ [a] -> Stream m a
forall (m :: * -> *) a. Applicative m => [a] -> Stream m a
D.fromList [a]
xs
RENAME_PRIME(pinnedFromList,fromList)

-- XXX We are materializing the whole list first for getting the length. Check
-- if the 'fromList' like chunked implementation would fare better.

-- | Like 'fromList' but writes the contents of the list in reverse order.
{-# INLINE fromListRev #-}
fromListRev :: (MonadIO m, Unbox a) => [a] -> m (MutArray a)
fromListRev :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
[a] -> m (MutArray a)
fromListRev [a]
xs = Int -> [a] -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> [a] -> m (MutArray a)
fromListRevN ([a] -> Int
forall a. [a] -> Int
forall (t :: * -> *) a. Foldable t => t a -> Int
Prelude.length [a]
xs) [a]
xs

-------------------------------------------------------------------------------
-- Cloning
-------------------------------------------------------------------------------

-- Arrays are aligned on 64-bit boundaries. The fastest way to copy an array is
-- to unsafeCast it to Word64, read it, write it to Word64 array and unsafeCast
-- it again. We can use SIMD read/write as well.

{-# INLINE cloneAs #-}
cloneAs ::
    ( MonadIO m
#ifdef DEVBUILD
    , Unbox a
#endif
    )
    => PinnedState -> MutArray a -> m (MutArray a)
cloneAs :: forall (m :: * -> *) a.
MonadIO m =>
PinnedState -> MutArray a -> m (MutArray a)
cloneAs PinnedState
ps MutArray a
src =
    do
        let startSrc :: Int
startSrc = MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
src
            srcLen :: Int
srcLen = MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
src Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
startSrc
        MutByteArray
newArrContents <-
            PinnedState -> Int -> Int -> MutByteArray -> m MutByteArray
forall (m :: * -> *).
MonadIO m =>
PinnedState -> Int -> Int -> MutByteArray -> m MutByteArray
Unboxed.unsafeCloneSliceAs PinnedState
ps Int
startSrc Int
srcLen (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
src)
        MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> m (MutArray a)) -> MutArray a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
newArrContents Int
0 Int
srcLen Int
srcLen

-- | Clone the elements of a MutArray. Does not clone the reserve capacity.
--
-- To clone a slice of "MutArray" you can create a slice with "unsafeSliceOffLen"
-- and then use "clone".
--
-- The new "MutArray" is unpinned in nature. Use "clone'" to clone the
-- MutArray in pinned memory.
{-# INLINE clone #-}
clone ::
    ( MonadIO m
#ifdef DEVBUILD
    , Unbox a
#endif
    )
    => MutArray a -> m (MutArray a)
clone :: forall (m :: * -> *) a. MonadIO m => MutArray a -> m (MutArray a)
clone = PinnedState -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
PinnedState -> MutArray a -> m (MutArray a)
cloneAs PinnedState
Unpinned

-- Similar to "clone" but uses pinned memory.
{-# INLINE clone' #-}
pinnedClone, clone' ::
    ( MonadIO m
#ifdef DEVBUILD
    , Unbox a
#endif
    )
    => MutArray a -> m (MutArray a)
clone' :: forall (m :: * -> *) a. MonadIO m => MutArray a -> m (MutArray a)
clone' = PinnedState -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
PinnedState -> MutArray a -> m (MutArray a)
cloneAs PinnedState
Pinned
RENAME_PRIME(pinnedClone,clone)

-------------------------------------------------------------------------------
-- Combining
-------------------------------------------------------------------------------

-- | Copy two arrays into a newly allocated array. If the first array is pinned
-- the spliced array is also pinned.
--
-- Note: If you freeze and splice it will create a new array.
{-# INLINE spliceCopy #-}
spliceCopy :: forall m a. MonadIO m =>
#ifdef DEVBUILD
    Unbox a =>
#endif
    MutArray a -> MutArray a -> m (MutArray a)
spliceCopy :: forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
spliceCopy MutArray a
arr1 MutArray a
arr2 = do
    let start1 :: Int
start1 = MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
arr1
        start2 :: Int
start2 = MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
arr2
        len1 :: Int
len1 = MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
arr1 Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
start1
        len2 :: Int
len2 = MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
arr2 Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
start2
    let len :: Int
len = Int
len1 Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
len2
    MutByteArray
newArrContents <-
        if MutByteArray -> Bool
Unboxed.isPinned (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
arr1)
        then IO MutByteArray -> m MutByteArray
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO MutByteArray -> m MutByteArray)
-> IO MutByteArray -> m MutByteArray
forall a b. (a -> b) -> a -> b
$ Int -> IO MutByteArray
Unboxed.new' Int
len
        else IO MutByteArray -> m MutByteArray
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO MutByteArray -> m MutByteArray)
-> IO MutByteArray -> m MutByteArray
forall a b. (a -> b) -> a -> b
$ Int -> IO MutByteArray
Unboxed.new Int
len
    MutByteArray -> Int -> MutByteArray -> Int -> Int -> m ()
forall (m :: * -> *).
MonadIO m =>
MutByteArray -> Int -> MutByteArray -> Int -> Int -> m ()
unsafePutSlice (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
arr1) Int
start1 MutByteArray
newArrContents Int
0 Int
len1
    MutByteArray -> Int -> MutByteArray -> Int -> Int -> m ()
forall (m :: * -> *).
MonadIO m =>
MutByteArray -> Int -> MutByteArray -> Int -> Int -> m ()
unsafePutSlice (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
arr2) Int
start2 MutByteArray
newArrContents Int
len1 Int
len2
    MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> m (MutArray a)) -> MutArray a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> Int -> Int -> MutArray a
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
newArrContents Int
0 Int
len Int
len

-- | Really really unsafe, appends the second array into the first array. If
-- the first array does not have enough space it may cause silent data
-- corruption or if you are lucky a segfault.
{-# INLINE unsafeSplice #-}
spliceUnsafe, unsafeSplice :: MonadIO m =>
    MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice :: forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice MutArray a
dst MutArray a
src = do
     let startSrc :: Int
startSrc = MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
src
         srcLen :: Int
srcLen = MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
src Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
startSrc
         endDst :: Int
endDst = MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
dst
     assertM(Int
endDst Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
srcLen Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= MutArray a -> Int
forall a. MutArray a -> Int
arrBound MutArray a
dst)
     MutByteArray -> Int -> MutByteArray -> Int -> Int -> m ()
forall (m :: * -> *).
MonadIO m =>
MutByteArray -> Int -> MutByteArray -> Int -> Int -> m ()
unsafePutSlice
         (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
src) Int
startSrc (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
dst) Int
endDst Int
srcLen
     MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> m (MutArray a)) -> MutArray a -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ MutArray a
dst {arrEnd :: Int
arrEnd = Int
endDst Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
srcLen}

-- | Append specified number of bytes from a given pointer to the MutArray.
--
-- /Unsafe:/
--
-- The caller has to ensure that:
--
-- 1. the MutArray is valid up to the given length.
-- 2. the source pointer is pinned and alive during the call.
-- 3. the pointer passed is valid up to the given length.
--
{-# INLINE unsafeAppendPtrN #-}
unsafeAppendPtrN :: MonadIO m =>
    MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
unsafeAppendPtrN :: forall (m :: * -> *).
MonadIO m =>
MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
unsafeAppendPtrN MutArray Word8
arr Ptr Word8
ptr Int
ptrLen = do
    let newEnd :: Int
newEnd = MutArray Word8 -> Int
forall a. MutArray a -> Int
arrEnd MutArray Word8
arr Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
ptrLen
    assertM(Int
newEnd Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= MutArray Word8 -> Int
forall a. MutArray a -> Int
arrBound MutArray Word8
arr)
    Ptr Word8 -> MutByteArray -> Int -> Int -> m ()
forall (m :: * -> *).
MonadIO m =>
Ptr Word8 -> MutByteArray -> Int -> Int -> m ()
Unboxed.unsafePutPtrN Ptr Word8
ptr (MutArray Word8 -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray Word8
arr) (MutArray Word8 -> Int
forall a. MutArray a -> Int
arrEnd MutArray Word8
arr) Int
ptrLen
    MutArray Word8 -> m (MutArray Word8)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray Word8 -> m (MutArray Word8))
-> MutArray Word8 -> m (MutArray Word8)
forall a b. (a -> b) -> a -> b
$ MutArray Word8
arr {arrEnd :: Int
arrEnd = Int
newEnd}

{-# INLINE appendPtrN #-}
appendPtrN :: MonadIO m =>
    MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
appendPtrN :: forall (m :: * -> *).
MonadIO m =>
MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
appendPtrN MutArray Word8
arr Ptr Word8
ptr Int
ptrLen = do
    MutArray Word8
arr1 <- Int -> MutArray Word8 -> m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
growBy Int
ptrLen MutArray Word8
arr
    MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
forall (m :: * -> *).
MonadIO m =>
MutArray Word8 -> Ptr Word8 -> Int -> m (MutArray Word8)
unsafeAppendPtrN MutArray Word8
arr1 Ptr Word8
ptr Int
ptrLen

-- | @spliceWith sizer dst src@ mutates @dst@ to append @src@. If there is no
-- reserved space available in @dst@ it is reallocated to a size determined by
-- the @sizer dstBytes srcBytes@ function, where @dstBytes@ is the size of the
-- first array and @srcBytes@ is the size of the second array, in bytes.
--
-- Note that the returned array may be a mutated version of first array.
--
-- /Pre-release/
{-# INLINE spliceWith #-}
spliceWith :: forall m a. (MonadIO m, Unbox a) =>
    (Int -> Int -> Int) -> MutArray a -> MutArray a -> m (MutArray a)
spliceWith :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int -> Int) -> MutArray a -> MutArray a -> m (MutArray a)
spliceWith Int -> Int -> Int
sizer dst :: MutArray a
dst@(MutArray MutByteArray
_ Int
start Int
end Int
bound) MutArray a
src = do
{-
    let f = appendWith (`sizer` byteLength src) (return dst)
     in D.fold f (toStreamD src)
-}
    Bool -> m () -> m ()
forall a. (?callStack::CallStack) => Bool -> a -> a
assert (Int
end Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
bound) (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
    let srcBytes :: Int
srcBytes = MutArray a -> Int
forall a. MutArray a -> Int
arrEnd MutArray a
src Int -> Int -> Int
forall a. Num a => a -> a -> a
- MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
src

    MutArray a
dst1 <-
        if Int
end Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
srcBytes Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
bound
        then do
            let dstBytes :: Int
dstBytes = Int
end Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
start
                newSizeInBytes :: Int
newSizeInBytes = Int -> Int -> Int
sizer Int
dstBytes Int
srcBytes
            Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
newSizeInBytes Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
dstBytes Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
srcBytes)
                (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ [Char] -> m ()
forall a. (?callStack::CallStack) => [Char] -> a
error
                    ([Char] -> m ()) -> [Char] -> m ()
forall a b. (a -> b) -> a -> b
$ [Char]
"splice: newSize is less than the total size "
                    [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"of arrays being appended. Please check the "
                    [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"sizer function passed."
            Int -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m (MutArray a)
realloc Int
newSizeInBytes MutArray a
dst
        else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
dst
    MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice MutArray a
dst1 MutArray a
src

-- | The first array is extended in-place to append the second array. If there is no
-- reserved space available in the first array then a new allocation of exact
-- required size is done.
--
-- Note that the returned array may be an extended version of first array,
-- referring to the same memory as the original array.
--
-- >>> splice = MutArray.spliceWith (+)
--
-- If the original array is pinned the spliced array is also pinned.
--
-- /Pre-release/
{-# INLINE splice #-}
splice :: (MonadIO m, Unbox a) => MutArray a -> MutArray a -> m (MutArray a)
splice :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> MutArray a -> m (MutArray a)
splice = (Int -> Int -> Int) -> MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int -> Int) -> MutArray a -> MutArray a -> m (MutArray a)
spliceWith Int -> Int -> Int
forall a. Num a => a -> a -> a
(+)

-- | Like 'append' but the growth of the array is exponential. Whenever a new
-- allocation is required the previous array size is at least doubled.
--
-- This is useful to reduce allocations when folding many arrays together.
--
-- Note that the returned array may be a mutated version of first array.
--
-- >>> spliceExp = MutArray.spliceWith (\l1 l2 -> max (l1 * 2) (l1 + l2))
--
-- /Pre-release/
{-# INLINE spliceExp #-}
spliceExp :: (MonadIO m, Unbox a) => MutArray a -> MutArray a -> m (MutArray a)
spliceExp :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> MutArray a -> m (MutArray a)
spliceExp = (Int -> Int -> Int) -> MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(Int -> Int -> Int) -> MutArray a -> MutArray a -> m (MutArray a)
spliceWith (\Int
l1 Int
l2 -> Int -> Int -> Int
forall a. Ord a => a -> a -> a
max (Int
l1 Int -> Int -> Int
forall a. Num a => a -> a -> a
* Int
2) (Int
l1 Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
l2))

-------------------------------------------------------------------------------
-- Splitting
-------------------------------------------------------------------------------

{-# INLINE splitUsing #-}
splitUsing :: (MonadIO m, Unbox a) =>
    ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
    -> (a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitUsing :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitUsing (a -> Bool) -> Stream m a -> Stream m (Int, Int)
f a -> Bool
predicate MutArray a
arr =
    ((Int, Int) -> MutArray a)
-> Stream m (Int, Int) -> Stream m (MutArray a)
forall a b. (a -> b) -> Stream m a -> Stream m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(Int
i, Int
len) -> Int -> Int -> MutArray a -> MutArray a
forall a. Unbox a => Int -> Int -> MutArray a -> MutArray a
unsafeSliceOffLen Int
i Int
len MutArray a
arr)
        (Stream m (Int, Int) -> Stream m (MutArray a))
-> Stream m (Int, Int) -> Stream m (MutArray a)
forall a b. (a -> b) -> a -> b
$ (a -> Bool) -> Stream m a -> Stream m (Int, Int)
f a -> Bool
predicate (MutArray a -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
read MutArray a
arr)

-- | Generate a stream of array slices using a predicate. The array element
-- matching the predicate is dropped.
--
-- /Pre-release/
{-# INLINE splitEndBy_ #-}
splitEndBy_, sliceEndBy_, splitOn :: (MonadIO m, Unbox a) =>
    (a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitEndBy_ :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitEndBy_ = ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool) -> MutArray a -> Stream m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitUsing (a -> Bool) -> Stream m a -> Stream m (Int, Int)
forall (m :: * -> *) a.
Monad m =>
(a -> Bool) -> Stream m a -> Stream m (Int, Int)
D.indexEndBy_

RENAME(splitOn,splitEndBy_)
RENAME(sliceEndBy_,splitEndBy_)

-- | Generate a stream of array slices using a predicate. The array element
-- matching the predicate is included.
--
-- /Pre-release/
{-# INLINE splitEndBy #-}
splitEndBy :: (MonadIO m, Unbox a) =>
    (a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitEndBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitEndBy = ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool) -> MutArray a -> Stream m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool) -> MutArray a -> Stream m (MutArray a)
splitUsing (a -> Bool) -> Stream m a -> Stream m (Int, Int)
forall (m :: * -> *) a.
Monad m =>
(a -> Bool) -> Stream m a -> Stream m (Int, Int)
D.indexEndBy

-- XXX See advanceStartTill for a potential performance issue with this type of
-- code which needed to be investigated. Measure the perf of this and use
-- advanceStartTill if that turns out to be better.

{-# INLINE breakUsing #-}
breakUsing :: (MonadIO m, Unbox a) =>
    Int -> ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
    -> (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
breakUsing :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int
-> ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool)
-> MutArray a
-> m (MutArray a, MutArray a)
breakUsing Int
adj (a -> Bool) -> Stream m a -> Stream m (Int, Int)
indexer a -> Bool
predicate MutArray a
arr = do
    -- XXX Use MutArray.fold Fold.findIndex instead.
    Maybe (Int, Int)
r <- Stream m (Int, Int) -> m (Maybe (Int, Int))
forall (m :: * -> *) a. Monad m => Stream m a -> m (Maybe a)
D.head (Stream m (Int, Int) -> m (Maybe (Int, Int)))
-> Stream m (Int, Int) -> m (Maybe (Int, Int))
forall a b. (a -> b) -> a -> b
$ (a -> Bool) -> Stream m a -> Stream m (Int, Int)
indexer a -> Bool
predicate (MutArray a -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
read MutArray a
arr)
    case Maybe (Int, Int)
r of
        Just (Int
i, Int
len) ->
            -- assert (i == 0)
            -- XXX avoid using length (div operation)
            let arrLen :: Int
arrLen = MutArray a -> Int
forall a. Unbox a => MutArray a -> Int
length MutArray a
arr
                i1 :: Int
i1 = Int
len Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
adj
                arr1 :: MutArray a
arr1 =
                    if Int
i1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
arrLen
                    then MutArray a
forall a. MutArray a
empty
                    else Int -> Int -> MutArray a -> MutArray a
forall a. Unbox a => Int -> Int -> MutArray a -> MutArray a
unsafeSliceOffLen Int
i1 (Int
arrLen Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
i1) MutArray a
arr
             in (MutArray a, MutArray a) -> m (MutArray a, MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Int -> Int -> MutArray a -> MutArray a
forall a. Unbox a => Int -> Int -> MutArray a -> MutArray a
unsafeSliceOffLen Int
i Int
len MutArray a
arr, MutArray a
arr1)
        Maybe (Int, Int)
Nothing -> (MutArray a, MutArray a) -> m (MutArray a, MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a
arr, MutArray a
forall a. MutArray a
empty)

{-# INLINE revBreakUsing #-}
revBreakUsing :: (MonadIO m, Unbox a) =>
    Bool -> (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakUsing :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Bool -> (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakUsing Bool
withSep a -> Bool
predicate MutArray a
arr = do
    let indexer :: (a -> Bool) -> Stream m a -> Stream m (Int, Int)
indexer = if Bool
withSep then (a -> Bool) -> Stream m a -> Stream m (Int, Int)
forall (m :: * -> *) a.
Monad m =>
(a -> Bool) -> Stream m a -> Stream m (Int, Int)
D.indexEndBy else (a -> Bool) -> Stream m a -> Stream m (Int, Int)
forall (m :: * -> *) a.
Monad m =>
(a -> Bool) -> Stream m a -> Stream m (Int, Int)
D.indexEndBy_
        adj :: Int
adj = if Bool
withSep then Int
0 else Int
1
    -- XXX Use MutArray.foldRev Fold.findIndex instead.
    Maybe (Int, Int)
r <- Stream m (Int, Int) -> m (Maybe (Int, Int))
forall (m :: * -> *) a. Monad m => Stream m a -> m (Maybe a)
D.head (Stream m (Int, Int) -> m (Maybe (Int, Int)))
-> Stream m (Int, Int) -> m (Maybe (Int, Int))
forall a b. (a -> b) -> a -> b
$ (a -> Bool) -> Stream m a -> Stream m (Int, Int)
forall {a}. (a -> Bool) -> Stream m a -> Stream m (Int, Int)
indexer a -> Bool
predicate (MutArray a -> Stream m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
MutArray a -> Stream m a
readRev MutArray a
arr)
    case Maybe (Int, Int)
r of
        Just (Int
_, Int
len) ->
            -- assert (i == 0)
            -- XXX avoid using length (div operation)
            let arrLen :: Int
arrLen = MutArray a -> Int
forall a. Unbox a => MutArray a -> Int
length MutArray a
arr
                len1 :: Int
len1 = Int
len Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
adj
                arr0 :: MutArray a
arr0 =
                    if Int
len1 Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
arrLen
                    then MutArray a
forall a. MutArray a
empty
                    else Int -> Int -> MutArray a -> MutArray a
forall a. Unbox a => Int -> Int -> MutArray a -> MutArray a
unsafeSliceOffLen Int
0 (Int
arrLen Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
len1) MutArray a
arr
                arr1 :: MutArray a
arr1 = Int -> Int -> MutArray a -> MutArray a
forall a. Unbox a => Int -> Int -> MutArray a -> MutArray a
unsafeSliceOffLen (Int
arrLen Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
len) Int
len MutArray a
arr
             in (MutArray a, MutArray a) -> m (MutArray a, MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a
arr0, MutArray a
arr1)
        Maybe (Int, Int)
Nothing -> (MutArray a, MutArray a) -> m (MutArray a, MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a
arr, MutArray a
forall a. MutArray a
empty)

-- |
-- >>> arr <- MutArray.fromList "hello world"
-- >>> (a,b) <- MutArray.breakEndBy (== ' ') arr
-- >>> MutArray.toList a
-- "hello "
-- >>> MutArray.toList b
-- "world"
--
{-# INLINE breakEndBy #-}
breakEndBy :: (MonadIO m, Unbox a) =>
    (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
breakEndBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
breakEndBy = Int
-> ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool)
-> MutArray a
-> m (MutArray a, MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int
-> ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool)
-> MutArray a
-> m (MutArray a, MutArray a)
breakUsing Int
0 (a -> Bool) -> Stream m a -> Stream m (Int, Int)
forall (m :: * -> *) a.
Monad m =>
(a -> Bool) -> Stream m a -> Stream m (Int, Int)
D.indexEndBy

-- | Break the array into two slices when the predicate succeeds. The array
-- element matching the predicate is dropped. If the predicate never succeeds
-- the second array is empty.
--
-- >>> arr <- MutArray.fromList "hello world"
-- >>> (a,b) <- MutArray.breakEndBy_ (== ' ') arr
-- >>> MutArray.toList a
-- "hello"
-- >>> MutArray.toList b
-- "world"
--
-- /Pre-release/
{-# INLINE breakEndBy_ #-}
breakEndBy_ :: (MonadIO m, Unbox a) =>
    (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
breakEndBy_ :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
breakEndBy_ = Int
-> ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool)
-> MutArray a
-> m (MutArray a, MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int
-> ((a -> Bool) -> Stream m a -> Stream m (Int, Int))
-> (a -> Bool)
-> MutArray a
-> m (MutArray a, MutArray a)
breakUsing Int
1 (a -> Bool) -> Stream m a -> Stream m (Int, Int)
forall (m :: * -> *) a.
Monad m =>
(a -> Bool) -> Stream m a -> Stream m (Int, Int)
D.indexEndBy_

-- |
--
-- >>> arr <- MutArray.fromList "hello world"
-- >>> (a,b) <- MutArray.revBreakEndBy (== ' ') arr
-- >>> MutArray.toList a
-- "hello"
-- >>> MutArray.toList b
-- " world"
--
{-# INLINE revBreakEndBy #-}
revBreakEndBy :: (MonadIO m, Unbox a) =>
    (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakEndBy :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakEndBy = Bool -> (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Bool -> (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakUsing Bool
True

-- |
--
-- >>> arr <- MutArray.fromList "hello world"
-- >>> (a,b) <- MutArray.revBreakEndBy_ (== ' ') arr
-- >>> MutArray.toList a
-- "hello"
-- >>> MutArray.toList b
-- "world"
--
{-# INLINE revBreakEndBy_ #-}
revBreakEndBy_ :: (MonadIO m, Unbox a) =>
    (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakEndBy_ :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakEndBy_ = Bool -> (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Bool -> (a -> Bool) -> MutArray a -> m (MutArray a, MutArray a)
revBreakUsing Bool
False

-- Note: We could return empty array instead of Nothing. But then we cannot
-- distinguish if the separator was found in the end or was not found at all.
-- XXX Do we need to distinguish that?

-- | Drops the separator byte
{-# INLINE breakEndByWord8_ #-}
breakEndByWord8_, breakOn :: MonadIO m
    => Word8 -> MutArray Word8 -> m (MutArray Word8, Maybe (MutArray Word8))
breakEndByWord8_ :: forall (m :: * -> *).
MonadIO m =>
Word8
-> MutArray Word8 -> m (MutArray Word8, Maybe (MutArray Word8))
breakEndByWord8_ Word8
sep arr :: MutArray Word8
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO (MutArray Word8, Maybe (MutArray Word8))
-> m (MutArray Word8, Maybe (MutArray Word8))
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray Word8, Maybe (MutArray Word8))
 -> m (MutArray Word8, Maybe (MutArray Word8)))
-> IO (MutArray Word8, Maybe (MutArray Word8))
-> m (MutArray Word8, Maybe (MutArray Word8))
forall a b. (a -> b) -> a -> b
$ do
    -- XXX We do not need memchr here, we can use a Haskell equivalent.
    -- Need efficient stream based primitives that work on Word64.
    let marr :: MutableByteArray# RealWorld
marr = MutByteArray -> MutableByteArray# RealWorld
getMutByteArray# MutByteArray
arrContents
        len :: CSize
len = Int -> CSize
forall a b. (Integral a, Num b) => a -> b
fromIntegral (Int
arrEnd Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
arrStart)
    CSize
sepIndex <- MutableByteArray# RealWorld -> CSize -> Word8 -> CSize -> IO CSize
c_memchr_index MutableByteArray# RealWorld
marr (Int -> CSize
forall a b. (Integral a, Num b) => a -> b
fromIntegral Int
arrStart) Word8
sep CSize
len
    let intIndex :: Int
intIndex = CSize -> Int
forall a b. (Integral a, Num b) => a -> b
fromIntegral CSize
sepIndex
    (MutArray Word8, Maybe (MutArray Word8))
-> IO (MutArray Word8, Maybe (MutArray Word8))
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return ((MutArray Word8, Maybe (MutArray Word8))
 -> IO (MutArray Word8, Maybe (MutArray Word8)))
-> (MutArray Word8, Maybe (MutArray Word8))
-> IO (MutArray Word8, Maybe (MutArray Word8))
forall a b. (a -> b) -> a -> b
$
        if CSize
sepIndex CSize -> CSize -> Bool
forall a. Ord a => a -> a -> Bool
>= CSize
len
        then (MutArray Word8
arr, Maybe (MutArray Word8)
forall a. Maybe a
Nothing)
        else
            ( MutArray
                { arrContents :: MutByteArray
arrContents = MutByteArray
arrContents
                , arrStart :: Int
arrStart = Int
arrStart
                , arrEnd :: Int
arrEnd = Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
intIndex -- exclude the separator
                , arrBound :: Int
arrBound = Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
intIndex
                }
            , MutArray Word8 -> Maybe (MutArray Word8)
forall a. a -> Maybe a
Just (MutArray Word8 -> Maybe (MutArray Word8))
-> MutArray Word8 -> Maybe (MutArray Word8)
forall a b. (a -> b) -> a -> b
$ MutArray
                    { arrContents :: MutByteArray
arrContents = MutByteArray
arrContents
                    , arrStart :: Int
arrStart = Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ (Int
intIndex Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1)
                    , arrEnd :: Int
arrEnd = Int
arrEnd
                    , arrBound :: Int
arrBound = Int
arrBound
                    }
            )
RENAME(breakOn,breakEndByWord8_)

-- | Like 'breakAt' but does not check whether the index is valid.
--
-- >>> unsafeBreakAt i arr = (MutArray.unsafeSliceOffLen 0 i arr, MutArray.unsafeSliceOffLen i (MutArray.length arr - i) arr)
--
{-# INLINE unsafeBreakAt #-}
unsafeBreakAt, unsafeSplitAt :: forall a. Unbox a =>
    Int -> MutArray a -> (MutArray a, MutArray a)
unsafeBreakAt :: forall a. Unbox a => Int -> MutArray a -> (MutArray a, MutArray a)
unsafeBreakAt Int
i MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} =
    -- (unsafeSliceOffLen 0 i arr, unsafeSliceOffLen i (length arr - i) arr)
    let off :: Int
off = Int
i Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)
        p :: Int
p = Int
arrStart Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
off
     in ( MutArray
         { arrContents :: MutByteArray
arrContents = MutByteArray
arrContents
         , arrStart :: Int
arrStart = Int
arrStart
         , arrEnd :: Int
arrEnd = Int
p
         , arrBound :: Int
arrBound = Int
p
         }
        , MutArray
          { arrContents :: MutByteArray
arrContents = MutByteArray
arrContents
          , arrStart :: Int
arrStart = Int
p
          , arrEnd :: Int
arrEnd = Int
arrEnd
          , arrBound :: Int
arrBound = Int
arrBound
          }
        )
RENAME(unsafeSplitAt,unsafeBreakAt)

-- | Create two slices of an array without copying the original array. The
-- specified index @i@ is the first index of the second slice.
--
{-# INLINE breakAt #-}
breakAt, splitAt
    :: forall a. Unbox a => Int -> MutArray a -> (MutArray a, MutArray a)
breakAt :: forall a. Unbox a => Int -> MutArray a -> (MutArray a, MutArray a)
breakAt Int
i MutArray a
arr =
    let maxIndex :: Int
maxIndex = MutArray a -> Int
forall a. Unbox a => MutArray a -> Int
length MutArray a
arr Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1
    in  if Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
0
        then [Char] -> (MutArray a, MutArray a)
forall a. (?callStack::CallStack) => [Char] -> a
error [Char]
"sliceAt: negative array index"
        else if Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
maxIndex
             then [Char] -> (MutArray a, MutArray a)
forall a. (?callStack::CallStack) => [Char] -> a
error ([Char] -> (MutArray a, MutArray a))
-> [Char] -> (MutArray a, MutArray a)
forall a b. (a -> b) -> a -> b
$ [Char]
"sliceAt: specified array index " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
i
                        [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
" is beyond the maximum index " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
maxIndex
             else Int -> MutArray a -> (MutArray a, MutArray a)
forall a. Unbox a => Int -> MutArray a -> (MutArray a, MutArray a)
unsafeBreakAt Int
i MutArray a
arr
RENAME(splitAt,breakAt)

-------------------------------------------------------------------------------
-- Casting
-------------------------------------------------------------------------------

-- | Cast an array having elements of type @a@ into an array having elements of
-- type @b@. The array size must be a multiple of the size of type @b@
-- otherwise accessing the last element of the array may result into a crash or
-- a random value.
--
-- /Pre-release/
--
castUnsafe, unsafeCast ::
#ifdef DEVBUILD
    Unbox b =>
#endif
    MutArray a -> MutArray b
unsafeCast :: forall a b. MutArray a -> MutArray b
unsafeCast (MutArray MutByteArray
contents Int
start Int
end Int
bound) =
    MutByteArray -> Int -> Int -> Int -> MutArray b
forall a. MutByteArray -> Int -> Int -> Int -> MutArray a
MutArray MutByteArray
contents Int
start Int
end Int
bound

-- | Cast an @MutArray a@ into an @MutArray Word8@.
--
asBytes :: MutArray a -> MutArray Word8
asBytes :: forall a. MutArray a -> MutArray Word8
asBytes = MutArray a -> MutArray Word8
forall a b. MutArray a -> MutArray b
unsafeCast

-- | Cast an array having elements of type @a@ into an array having elements of
-- type @b@. The length of the array should be a multiple of the size of the
-- target element otherwise 'Nothing' is returned.
--
cast :: forall a b. Unbox b => MutArray a -> Maybe (MutArray b)
cast :: forall a b. Unbox b => MutArray a -> Maybe (MutArray b)
cast MutArray a
arr =
    let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
        r :: Int
r = Int
len Int -> Int -> Int
forall a. Integral a => a -> a -> a
`mod` SIZE_OF(b)
     in if Int
r Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
/= Int
0
        then Maybe (MutArray b)
forall a. Maybe a
Nothing
        else MutArray b -> Maybe (MutArray b)
forall a. a -> Maybe a
Just (MutArray b -> Maybe (MutArray b))
-> MutArray b -> Maybe (MutArray b)
forall a b. (a -> b) -> a -> b
$ MutArray a -> MutArray b
forall a b. MutArray a -> MutArray b
unsafeCast MutArray a
arr

-- XXX Should we just name it asPtr, the unsafety is implicit for any pointer
-- operations. And we are safe from Haskell perspective because we will be
-- pinning the memory.

-- | NOTE: this is deprecated because it can lead to accidental problems if the
-- user tries to use it to mutate the array because it does not return the new
-- array after pinning.
{-# DEPRECATED unsafePinnedAsPtr "Pin the array and then use unsafeAsPtr." #-}
{-# INLINE unsafePinnedAsPtr #-}
unsafePinnedAsPtr :: MonadIO m => MutArray a -> (Ptr a -> Int -> m b) -> m b
unsafePinnedAsPtr :: forall (m :: * -> *) a b.
MonadIO m =>
MutArray a -> (Ptr a -> Int -> m b) -> m b
unsafePinnedAsPtr MutArray a
mutarr Ptr a -> Int -> m b
f = do
    let arr0 :: MutByteArray
arr0 = MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
mutarr
    MutByteArray
arr <- IO MutByteArray -> m MutByteArray
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO MutByteArray -> m MutByteArray)
-> IO MutByteArray -> m MutByteArray
forall a b. (a -> b) -> a -> b
$ MutByteArray -> IO MutByteArray
Unboxed.pin MutByteArray
arr0
    let !ptr :: Ptr a
ptr = Addr# -> Ptr a
forall a. Addr# -> Ptr a
Ptr (ByteArray# -> Addr#
byteArrayContents#
                     (MutableByteArray# RealWorld -> ByteArray#
forall a b. a -> b
unsafeCoerce# (MutByteArray -> MutableByteArray# RealWorld
getMutByteArray# MutByteArray
arr)))
    b
r <- Ptr a -> Int -> m b
f (Ptr Any
forall {a}. Ptr a
ptr Ptr Any -> Int -> Ptr a
forall a b. Ptr a -> Int -> Ptr b
`plusPtr` MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
mutarr) (MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
mutarr)
    IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ MutByteArray -> IO ()
Unboxed.touch MutByteArray
arr
    b -> m b
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return b
r

{-# DEPRECATED asPtrUnsafe "Pin the array and then use unsafeAsPtr." #-}
{-# INLINE asPtrUnsafe #-}
asPtrUnsafe :: MonadIO m => MutArray a -> (Ptr a -> m b) -> m b
asPtrUnsafe :: forall (m :: * -> *) a b.
MonadIO m =>
MutArray a -> (Ptr a -> m b) -> m b
asPtrUnsafe MutArray a
a Ptr a -> m b
f = MutArray a -> (Ptr a -> Int -> m b) -> m b
forall (m :: * -> *) a b.
MonadIO m =>
MutArray a -> (Ptr a -> Int -> m b) -> m b
unsafePinnedAsPtr MutArray a
a (\Ptr a
p Int
_ -> Ptr a -> m b
f Ptr a
p)

-- | @unsafeAsPtr arr f@, f is a function used as @f ptr len@ where @ptr@ is a
-- pointer to the beginning of array and @len@ is the byte-length of the array.
--
-- /Unsafe/ WARNING:
--
-- 1. The array must be pinned, otherwise it will lead to memory corruption.
-- 2. The user must not use the pointer beyond the supplied length.
--
-- /Pre-release/
--
{-# INLINE unsafeAsPtr #-}
unsafeAsPtr :: MonadIO m => MutArray a -> (Ptr a -> Int -> IO b) -> m b
unsafeAsPtr :: forall (m :: * -> *) a b.
MonadIO m =>
MutArray a -> (Ptr a -> Int -> IO b) -> m b
unsafeAsPtr MutArray a
arr Ptr a -> Int -> IO b
f =
    MutByteArray -> (Ptr Any -> IO b) -> m b
forall (m :: * -> *) a b.
MonadIO m =>
MutByteArray -> (Ptr a -> IO b) -> m b
Unboxed.unsafeAsPtr
        (MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
arr)
        (\Ptr Any
ptr -> Ptr a -> Int -> IO b
f (Ptr Any
ptr Ptr Any -> Int -> Ptr a
forall a b. Ptr a -> Int -> Ptr b
`plusPtr` MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
arr) (MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr))

-- | @unsafeCreateWithPtr' capacity populator@ creates a pinned array of
-- @capacity@ bytes and invokes the @populator@ function to populate it.
-- @populator ptr len@ gets the pointer to the array and MUST return the amount
-- of the capacity populated in bytes.
--
-- /Unsafe/ because the populator is allowed to use the pointer only up to
-- specified length. In other words, bytes populated MUST be less than or equal
-- to the total capacity.
{-# INLINE unsafeCreateWithPtr' #-}
unsafeCreateWithPtr'
    :: MonadIO m => Int -> (Ptr Word8 -> IO Int) -> m (MutArray Word8)
unsafeCreateWithPtr' :: forall (m :: * -> *).
MonadIO m =>
Int -> (Ptr Word8 -> IO Int) -> m (MutArray Word8)
unsafeCreateWithPtr' Int
cap Ptr Word8 -> IO Int
pop = do
    (MutArray Word8
arr :: MutArray Word8) <- Int -> m (MutArray Word8)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> m (MutArray a)
emptyOf' Int
cap
    Int
len <- MutByteArray -> (Ptr Word8 -> IO Int) -> m Int
forall (m :: * -> *) a b.
MonadIO m =>
MutByteArray -> (Ptr a -> IO b) -> m b
Unboxed.unsafeAsPtr (MutArray Word8 -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray Word8
arr) Ptr Word8 -> IO Int
pop
    Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
cap) ([Char] -> m ()
forall a. (?callStack::CallStack) => [Char] -> a
error (Int -> [Char]
forall a. Show a => a -> [Char]
errMsg Int
len))
    -- arrStart == 0
    MutArray Word8 -> m (MutArray Word8)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (MutArray Word8
arr { arrEnd :: Int
arrEnd = Int
len })


    where

    errMsg :: a -> [Char]
errMsg a
len =
        [Char]
"unsafeCreateWithPtr': length > capacity, "
             [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"length = " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ a -> [Char]
forall a. Show a => a -> [Char]
show a
len [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
", "
             [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"capacity = " [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
cap

-------------------------------------------------------------------------------
-- Equality
-------------------------------------------------------------------------------

-- | Byte compare two arrays. Compare the length of the arrays. If the length
-- is equal, compare the lexicographical ordering of two underlying byte arrays
-- otherwise return the result of length comparison.
--
-- /Unsafe/: Note that the 'Unbox' instance of sum types with constructors of
-- different sizes may leave some memory uninitialized which can make byte
-- comparison unreliable.
--
-- /Pre-release/
{-# INLINE byteCmp #-}
byteCmp :: MonadIO m => MutArray a -> MutArray a -> m Ordering
byteCmp :: forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m Ordering
byteCmp MutArray a
arr1 MutArray a
arr2 = do
    let !marr1 :: MutByteArray
marr1 = MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
arr1
        !marr2 :: MutByteArray
marr2 = MutArray a -> MutByteArray
forall a. MutArray a -> MutByteArray
arrContents MutArray a
arr2
        !len1 :: Int
len1 = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr1
        !len2 :: Int
len2 = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr2
        !st1 :: Int
st1 = MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
arr1
        !st2 :: Int
st2 = MutArray a -> Int
forall a. MutArray a -> Int
arrStart MutArray a
arr2
    case Int -> Int -> Ordering
forall a. Ord a => a -> a -> Ordering
compare Int
len1 Int
len2 of
        Ordering
EQ -> do
            Int
r <- IO Int -> m Int
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO Int -> m Int) -> IO Int -> m Int
forall a b. (a -> b) -> a -> b
$ MutByteArray -> Int -> MutByteArray -> Int -> Int -> IO Int
unsafeByteCmp MutByteArray
marr1 Int
st1 MutByteArray
marr2 Int
st2 Int
len1
            Ordering -> m Ordering
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Ordering -> m Ordering) -> Ordering -> m Ordering
forall a b. (a -> b) -> a -> b
$ Int -> Int -> Ordering
forall a. Ord a => a -> a -> Ordering
compare Int
r Int
0
        Ordering
x -> Ordering -> m Ordering
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Ordering
x

{-# INLINE cmp #-}
{-# DEPRECATED cmp "Please use byteCmp instead." #-}
cmp :: MonadIO m => MutArray a -> MutArray a -> m Ordering
cmp :: forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m Ordering
cmp = MutArray a -> MutArray a -> m Ordering
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m Ordering
byteCmp

-- | Byte equality of two arrays.
--
-- >>> byteEq arr1 arr2 = (==) EQ <$> MutArray.byteCmp arr1 arr2
--
-- /Unsafe/: See 'byteCmp'.
{-# INLINE byteEq #-}
byteEq :: MonadIO m => MutArray a -> MutArray a -> m Bool
byteEq :: forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m Bool
byteEq MutArray a
arr1 MutArray a
arr2 = (Ordering -> Bool) -> m Ordering -> m Bool
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Ordering
EQ Ordering -> Ordering -> Bool
forall a. Eq a => a -> a -> Bool
==) (m Ordering -> m Bool) -> m Ordering -> m Bool
forall a b. (a -> b) -> a -> b
$ MutArray a -> MutArray a -> m Ordering
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m Ordering
byteCmp MutArray a
arr1 MutArray a
arr2

-------------------------------------------------------------------------------
-- Compact
-------------------------------------------------------------------------------

-- Note: LE versions avoid an extra copy compared to GE. LE parser trades
-- backtracking one array in lieu of avoiding a copy. However, LE and GE both
-- can leave some memory unused. They may split the last array to fit it
-- exactly in the space.

{-# INLINE_NORMAL pCompactLeAs #-}
pCompactLeAs ::
       forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> Parser (MutArray a) m (MutArray a)
pCompactLeAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Parser (MutArray a) m (MutArray a)
pCompactLeAs PinnedState
ps Int
maxElems = (Maybe (MutArray a)
 -> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> m (Initial (Maybe (MutArray a)) (MutArray a))
-> (Maybe (MutArray a)
    -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> Parser (MutArray a) m (MutArray a)
forall a (m :: * -> *) b s.
(s -> a -> m (Step s b))
-> m (Initial s b) -> (s -> m (Step s b)) -> Parser a m b
Parser Maybe (MutArray a)
-> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a))
forall {m :: * -> *} {a}.
MonadIO m =>
Maybe (MutArray a)
-> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a))
step m (Initial (Maybe (MutArray a)) (MutArray a))
forall {a} {b}. m (Initial (Maybe a) b)
initial Maybe (MutArray a) -> m (Step (Maybe (MutArray a)) (MutArray a))
forall {m :: * -> *} {a} {s}.
Monad m =>
Maybe (MutArray a) -> m (Step s (MutArray a))
extract

    where

    maxBytes :: Int
maxBytes = Int
maxElems Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)

    functionName :: [Char]
functionName = [Char]
"Streamly.Internal.Data.MutArray.pCompactLE"

    initial :: m (Initial (Maybe a) b)
initial =
        Initial (Maybe a) b -> m (Initial (Maybe a) b)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
            (Initial (Maybe a) b -> m (Initial (Maybe a) b))
-> Initial (Maybe a) b -> m (Initial (Maybe a) b)
forall a b. (a -> b) -> a -> b
$ if Int
maxElems Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
0
              then [Char] -> Initial (Maybe a) b
forall a. (?callStack::CallStack) => [Char] -> a
error
                       ([Char] -> Initial (Maybe a) b) -> [Char] -> Initial (Maybe a) b
forall a b. (a -> b) -> a -> b
$ [Char]
functionName
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
": the size of arrays ["
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
maxElems [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"] must be a natural number"
              else Maybe a -> Initial (Maybe a) b
forall s b. s -> Initial s b
Parser.IPartial Maybe a
forall a. Maybe a
Nothing

    step :: Maybe (MutArray a)
-> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a))
step Maybe (MutArray a)
Nothing MutArray a
arr =
        Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
            (Step (Maybe (MutArray a)) (MutArray a)
 -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
               in if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
maxBytes
                  then Int -> MutArray a -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. Int -> b -> Step s b
Parser.Done Int
0 MutArray a
arr
                  else Int -> Maybe (MutArray a) -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. Int -> s -> Step s b
Parser.Partial Int
0 (MutArray a -> Maybe (MutArray a)
forall a. a -> Maybe a
Just MutArray a
arr)
    -- XXX Split the last array to use the space more compactly.
    step (Just MutArray a
buf) MutArray a
arr =
        let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
buf Int -> Int -> Int
forall a. Num a => a -> a -> a
+ MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
         in if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
maxBytes
            then Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Maybe (MutArray a)) (MutArray a)
 -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ Int -> MutArray a -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. Int -> b -> Step s b
Parser.Done Int
1 MutArray a
buf
            else do
                MutArray a
buf1 <-
                    if MutArray a -> Int
forall a. MutArray a -> Int
byteCapacity MutArray a
buf Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
maxBytes
                    then IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
forall a.
PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs
                            PinnedState
ps (SIZE_OF(a)) maxBytes buf
                    else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
buf
                MutArray a
buf2 <- MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice MutArray a
buf1 MutArray a
arr
                Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Maybe (MutArray a)) (MutArray a)
 -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ Int -> Maybe (MutArray a) -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. Int -> s -> Step s b
Parser.Partial Int
0 (MutArray a -> Maybe (MutArray a)
forall a. a -> Maybe a
Just MutArray a
buf2)

    extract :: Maybe (MutArray a) -> m (Step s (MutArray a))
extract Maybe (MutArray a)
Nothing = Step s (MutArray a) -> m (Step s (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step s (MutArray a) -> m (Step s (MutArray a)))
-> Step s (MutArray a) -> m (Step s (MutArray a))
forall a b. (a -> b) -> a -> b
$ Int -> MutArray a -> Step s (MutArray a)
forall s b. Int -> b -> Step s b
Parser.Done Int
0 MutArray a
forall a. MutArray a
nil
    extract (Just MutArray a
buf) = Step s (MutArray a) -> m (Step s (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step s (MutArray a) -> m (Step s (MutArray a)))
-> Step s (MutArray a) -> m (Step s (MutArray a))
forall a b. (a -> b) -> a -> b
$ Int -> MutArray a -> Step s (MutArray a)
forall s b. Int -> b -> Step s b
Parser.Done Int
0 MutArray a
buf

-- | Parser @createCompactMax maxElems@ coalesces adjacent arrays in the
-- input stream only if the combined size would be less than or equal to
-- @maxElems@ elements. Note that it won't split an array if the original array
-- is already larger than maxElems.
--
-- @maxElems@ must be greater than 0.
--
-- Generates unpinned arrays irrespective of the pinning status of input
-- arrays.
--
-- Note that a fold compacting to less than or equal to a given size is not
-- possible, as folds cannot backtrack.
--
-- /Internal/
{-# INLINE createCompactMax #-}
createCompactMax, pCompactLE ::
       forall m a. (MonadIO m, Unbox a)
    => Int -> Parser (MutArray a) m (MutArray a)
createCompactMax :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Parser (MutArray a) m (MutArray a)
createCompactMax = PinnedState -> Int -> Parser (MutArray a) m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Parser (MutArray a) m (MutArray a)
pCompactLeAs PinnedState
Unpinned

RENAME(pCompactLE,createCompactMax)

-- | Pinned version of 'createCompactMax'.
{-# INLINE createCompactMax' #-}
createCompactMax', pPinnedCompactLE ::
       forall m a. (MonadIO m, Unbox a)
    => Int -> Parser (MutArray a) m (MutArray a)
createCompactMax' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Parser (MutArray a) m (MutArray a)
createCompactMax' = PinnedState -> Int -> Parser (MutArray a) m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Parser (MutArray a) m (MutArray a)
pCompactLeAs PinnedState
Pinned

{-# DEPRECATED pPinnedCompactLE "Please use createCompactMax' instead." #-}
{-# INLINE pPinnedCompactLE #-}
pPinnedCompactLE :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Parser (MutArray a) m (MutArray a)
pPinnedCompactLE = Int -> Parser (MutArray a) m (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Parser (MutArray a) m (MutArray a)
createCompactMax'

data SpliceState s arr
    = SpliceInitial s
    | SpliceBuffering s arr
    | SpliceYielding arr (SpliceState s arr)
    | SpliceFinish

-- | This mutates the first array (if it has space) to append values from the
-- second one. This would work for immutable arrays as well because an
-- immutable array never has additional space so a new array is allocated
-- instead of mutating it.
{-# INLINE_NORMAL compactLeAs #-}
compactLeAs :: forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> D.Stream m (MutArray a) -> D.Stream m (MutArray a)
compactLeAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState
-> Int -> Stream m (MutArray a) -> Stream m (MutArray a)
compactLeAs PinnedState
ps Int
maxElems (D.Stream State StreamK m (MutArray a) -> s -> m (Step s (MutArray a))
step s
state) =
    (State StreamK m (MutArray a)
 -> SpliceState s (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> SpliceState s (MutArray a) -> Stream m (MutArray a)
forall (m :: * -> *) a s.
(State StreamK m a -> s -> m (Step s a)) -> s -> Stream m a
D.Stream State StreamK m (MutArray a)
-> SpliceState s (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
step' (s -> SpliceState s (MutArray a)
forall s arr. s -> SpliceState s arr
SpliceInitial s
state)

    where

    maxBytes :: Int
maxBytes = Int
maxElems Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)

    functionName :: [Char]
functionName = [Char]
"Streamly.Internal.Data.MutArray.rCompactLE"

    {-# INLINE_LATE step' #-}
    step' :: State StreamK m (MutArray a)
-> SpliceState s (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
step' State StreamK m (MutArray a)
gst (SpliceInitial s
st) = do
        Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
maxElems Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
0) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
            -- XXX we can pass the module string from the higher level API
            [Char] -> m ()
forall a. (?callStack::CallStack) => [Char] -> a
error ([Char] -> m ()) -> [Char] -> m ()
forall a b. (a -> b) -> a -> b
$ [Char]
functionName [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
": the size of arrays [" [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
maxElems
                [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"] must be a natural number"
        Step s (MutArray a)
r <- State StreamK m (MutArray a) -> s -> m (Step s (MutArray a))
step State StreamK m (MutArray a)
gst s
st
        case Step s (MutArray a)
r of
            D.Yield MutArray a
arr s
s -> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (SpliceState s (MutArray a)) (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$
                let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
                 in if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
maxBytes
                    then SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. s -> Step s a
D.Skip (MutArray a
-> SpliceState s (MutArray a) -> SpliceState s (MutArray a)
forall s arr. arr -> SpliceState s arr -> SpliceState s arr
SpliceYielding MutArray a
arr (s -> SpliceState s (MutArray a)
forall s arr. s -> SpliceState s arr
SpliceInitial s
s))
                    else SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. s -> Step s a
D.Skip (s -> MutArray a -> SpliceState s (MutArray a)
forall s arr. s -> arr -> SpliceState s arr
SpliceBuffering s
s MutArray a
arr)
            D.Skip s
s -> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (SpliceState s (MutArray a)) (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. s -> Step s a
D.Skip (s -> SpliceState s (MutArray a)
forall s arr. s -> SpliceState s arr
SpliceInitial s
s)
            Step s (MutArray a)
D.Stop -> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. Step s a
D.Stop

    -- XXX Split the last array to use the space more compactly.
    step' State StreamK m (MutArray a)
gst (SpliceBuffering s
st MutArray a
buf) = do
        Step s (MutArray a)
r <- State StreamK m (MutArray a) -> s -> m (Step s (MutArray a))
step State StreamK m (MutArray a)
gst s
st
        case Step s (MutArray a)
r of
            D.Yield MutArray a
arr s
s -> do
                let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
buf Int -> Int -> Int
forall a. Num a => a -> a -> a
+ MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
                if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
maxBytes
                then Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (SpliceState s (MutArray a)) (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$
                    SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. s -> Step s a
D.Skip (MutArray a
-> SpliceState s (MutArray a) -> SpliceState s (MutArray a)
forall s arr. arr -> SpliceState s arr -> SpliceState s arr
SpliceYielding MutArray a
buf (s -> MutArray a -> SpliceState s (MutArray a)
forall s arr. s -> arr -> SpliceState s arr
SpliceBuffering s
s MutArray a
arr))
                else do
                    MutArray a
buf1 <- if MutArray a -> Int
forall a. MutArray a -> Int
byteCapacity MutArray a
buf Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
maxBytes
                            then IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
forall a.
PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs
                                    PinnedState
ps (SIZE_OF(a)) maxBytes buf
                            else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
buf
                    MutArray a
buf2 <- MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice MutArray a
buf1 MutArray a
arr
                    Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (SpliceState s (MutArray a)) (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. s -> Step s a
D.Skip (s -> MutArray a -> SpliceState s (MutArray a)
forall s arr. s -> arr -> SpliceState s arr
SpliceBuffering s
s MutArray a
buf2)
            D.Skip s
s -> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (SpliceState s (MutArray a)) (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. s -> Step s a
D.Skip (s -> MutArray a -> SpliceState s (MutArray a)
forall s arr. s -> arr -> SpliceState s arr
SpliceBuffering s
s MutArray a
buf)
            Step s (MutArray a)
D.Stop -> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (SpliceState s (MutArray a)) (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. s -> Step s a
D.Skip (MutArray a
-> SpliceState s (MutArray a) -> SpliceState s (MutArray a)
forall s arr. arr -> SpliceState s arr -> SpliceState s arr
SpliceYielding MutArray a
buf SpliceState s (MutArray a)
forall s arr. SpliceState s arr
SpliceFinish)

    step' State StreamK m (MutArray a)
_ SpliceState s (MutArray a)
SpliceFinish = Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. Step s a
D.Stop

    step' State StreamK m (MutArray a)
_ (SpliceYielding MutArray a
arr SpliceState s (MutArray a)
next) = Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (SpliceState s (MutArray a)) (MutArray a)
 -> m (Step (SpliceState s (MutArray a)) (MutArray a)))
-> Step (SpliceState s (MutArray a)) (MutArray a)
-> m (Step (SpliceState s (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ MutArray a
-> SpliceState s (MutArray a)
-> Step (SpliceState s (MutArray a)) (MutArray a)
forall s a. a -> s -> Step s a
D.Yield MutArray a
arr SpliceState s (MutArray a)
next


{-# INLINE_NORMAL fCompactGeAs #-}
fCompactGeAs ::
       forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> FL.Fold m (MutArray a) (MutArray a)
fCompactGeAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m (MutArray a) (MutArray a)
fCompactGeAs PinnedState
ps Int
minElems = (Maybe (MutArray a)
 -> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> m (Step (Maybe (MutArray a)) (MutArray a))
-> (Maybe (MutArray a) -> m (MutArray a))
-> (Maybe (MutArray a) -> m (MutArray a))
-> Fold m (MutArray a) (MutArray a)
forall (m :: * -> *) a b s.
(s -> a -> m (Step s b))
-> m (Step s b) -> (s -> m b) -> (s -> m b) -> Fold m a b
Fold Maybe (MutArray a)
-> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a))
forall {m :: * -> *} {a}.
MonadIO m =>
Maybe (MutArray a)
-> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a))
step m (Step (Maybe (MutArray a)) (MutArray a))
forall {a} {b}. m (Step (Maybe a) b)
initial Maybe (MutArray a) -> m (MutArray a)
forall {m :: * -> *} {a}.
Monad m =>
Maybe (MutArray a) -> m (MutArray a)
extract Maybe (MutArray a) -> m (MutArray a)
forall {m :: * -> *} {a}.
Monad m =>
Maybe (MutArray a) -> m (MutArray a)
extract

    where

    minBytes :: Int
minBytes = Int
minElems Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)

    functionName :: [Char]
functionName = [Char]
"Streamly.Internal.Data.MutArray.fCompactGE"

    initial :: m (Step (Maybe a) b)
initial =
        Step (Maybe a) b -> m (Step (Maybe a) b)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
            (Step (Maybe a) b -> m (Step (Maybe a) b))
-> Step (Maybe a) b -> m (Step (Maybe a) b)
forall a b. (a -> b) -> a -> b
$ if Int
minElems Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
0
              then [Char] -> Step (Maybe a) b
forall a. (?callStack::CallStack) => [Char] -> a
error
                       ([Char] -> Step (Maybe a) b) -> [Char] -> Step (Maybe a) b
forall a b. (a -> b) -> a -> b
$ [Char]
functionName
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
": the size of arrays ["
                       [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
minElems [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"] must be a natural number"
              else Maybe a -> Step (Maybe a) b
forall s b. s -> Step s b
FL.Partial Maybe a
forall a. Maybe a
Nothing

    step :: Maybe (MutArray a)
-> MutArray a -> m (Step (Maybe (MutArray a)) (MutArray a))
step Maybe (MutArray a)
Nothing MutArray a
arr =
        Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return
            (Step (Maybe (MutArray a)) (MutArray a)
 -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
               in if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
minBytes
                  then MutArray a -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. b -> Step s b
FL.Done MutArray a
arr
                  else Maybe (MutArray a) -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. s -> Step s b
FL.Partial (MutArray a -> Maybe (MutArray a)
forall a. a -> Maybe a
Just MutArray a
arr)
    -- XXX Buffer arrays as a list to avoid copy and reallocations
    step (Just MutArray a
buf) MutArray a
arr = do
        let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
buf Int -> Int -> Int
forall a. Num a => a -> a -> a
+ MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
        MutArray a
buf1 <-
            if MutArray a -> Int
forall a. MutArray a -> Int
byteCapacity MutArray a
buf Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
len
            then IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
forall a.
PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs
                    PinnedState
ps (SIZE_OF(a)) (max minBytes len) buf
            else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
buf
        MutArray a
buf2 <- MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice MutArray a
buf1 MutArray a
arr
        if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
minBytes
        then Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Maybe (MutArray a)) (MutArray a)
 -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ MutArray a -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. b -> Step s b
FL.Done MutArray a
buf2
        else Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Maybe (MutArray a)) (MutArray a)
 -> m (Step (Maybe (MutArray a)) (MutArray a)))
-> Step (Maybe (MutArray a)) (MutArray a)
-> m (Step (Maybe (MutArray a)) (MutArray a))
forall a b. (a -> b) -> a -> b
$ Maybe (MutArray a) -> Step (Maybe (MutArray a)) (MutArray a)
forall s b. s -> Step s b
FL.Partial (MutArray a -> Maybe (MutArray a)
forall a. a -> Maybe a
Just MutArray a
buf2)

    extract :: Maybe (MutArray a) -> m (MutArray a)
extract Maybe (MutArray a)
Nothing = MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
forall a. MutArray a
nil
    extract (Just MutArray a
buf) = MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
buf

-- | Fold @createCompactMin minElems@ coalesces adjacent arrays in the
-- input stream until the size becomes greater than or equal to @minElems@.
--
-- Generates unpinned arrays irrespective of the pinning status of input
-- arrays.
{-# INLINE createCompactMin #-}
createCompactMin, fCompactGE ::
       forall m a. (MonadIO m, Unbox a)
    => Int -> FL.Fold m (MutArray a) (MutArray a)
createCompactMin :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m (MutArray a) (MutArray a)
createCompactMin = PinnedState -> Int -> Fold m (MutArray a) (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m (MutArray a) (MutArray a)
fCompactGeAs PinnedState
Unpinned

RENAME(fCompactGE,createCompactMin)

-- | Pinned version of 'createCompactMin'.
{-# INLINE createCompactMin' #-}
createCompactMin', fPinnedCompactGE ::
       forall m a. (MonadIO m, Unbox a)
    => Int -> FL.Fold m (MutArray a) (MutArray a)
createCompactMin' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m (MutArray a) (MutArray a)
createCompactMin' = PinnedState -> Int -> Fold m (MutArray a) (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Fold m (MutArray a) (MutArray a)
fCompactGeAs PinnedState
Pinned

{-# DEPRECATED fPinnedCompactGE "Please use createCompactMin' instead." #-}
{-# INLINE fPinnedCompactGE #-}
fPinnedCompactGE :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m (MutArray a) (MutArray a)
fPinnedCompactGE = Int -> Fold m (MutArray a) (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m (MutArray a) (MutArray a)
createCompactMin'

{-# INLINE_NORMAL lCompactGeAs #-}
lCompactGeAs :: forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
-- The fold version turns out to be a little bit slower.
-- lCompactGeAs ps n = FL.many (fCompactGeAs ps n)
lCompactGeAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState
-> Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
lCompactGeAs PinnedState
ps Int
minElems (Fold s -> MutArray a -> m (Step s ())
step1 m (Step s ())
initial1 s -> m ()
_ s -> m ()
final1) =
    (Tuple' (Maybe (MutArray a)) s
 -> MutArray a -> m (Step (Tuple' (Maybe (MutArray a)) s) ()))
-> m (Step (Tuple' (Maybe (MutArray a)) s) ())
-> (Tuple' (Maybe (MutArray a)) s -> m ())
-> (Tuple' (Maybe (MutArray a)) s -> m ())
-> Fold m (MutArray a) ()
forall (m :: * -> *) a b s.
(s -> a -> m (Step s b))
-> m (Step s b) -> (s -> m b) -> (s -> m b) -> Fold m a b
Fold Tuple' (Maybe (MutArray a)) s
-> MutArray a -> m (Step (Tuple' (Maybe (MutArray a)) s) ())
step m (Step (Tuple' (Maybe (MutArray a)) s) ())
forall {a}. m (Step (Tuple' (Maybe a) s) ())
initial Tuple' (Maybe (MutArray a)) s -> m ()
forall {p} {a}. p -> a
extract Tuple' (Maybe (MutArray a)) s -> m ()
final

    where

    minBytes :: Int
minBytes = Int
minElems Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)

    functionName :: [Char]
functionName = [Char]
"Streamly.Internal.Data.MutArray.lCompactGE"

    initial :: m (Step (Tuple' (Maybe a) s) ())
initial = do
        Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
minElems Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
0) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
            -- XXX we can pass the module string from the higher level API
            [Char] -> m ()
forall a. (?callStack::CallStack) => [Char] -> a
error ([Char] -> m ()) -> [Char] -> m ()
forall a b. (a -> b) -> a -> b
$ [Char]
functionName [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
": the size of arrays ["
                [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
minElems [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"] must be a natural number"

        Step s ()
r <- m (Step s ())
initial1
        Step (Tuple' (Maybe a) s) () -> m (Step (Tuple' (Maybe a) s) ())
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Tuple' (Maybe a) s) () -> m (Step (Tuple' (Maybe a) s) ()))
-> Step (Tuple' (Maybe a) s) () -> m (Step (Tuple' (Maybe a) s) ())
forall a b. (a -> b) -> a -> b
$ (s -> Tuple' (Maybe a) s)
-> Step s () -> Step (Tuple' (Maybe a) s) ()
forall a b c. (a -> b) -> Step a c -> Step b c
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first (Maybe a -> s -> Tuple' (Maybe a) s
forall a b. a -> b -> Tuple' a b
Tuple' Maybe a
forall a. Maybe a
Nothing) Step s ()
r

    {-# INLINE runInner #-}
    runInner :: Int
-> s -> MutArray a -> m (Step (Tuple' (Maybe (MutArray a)) s) ())
runInner Int
len s
acc MutArray a
buf =
            if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
minBytes
            then do
                Step s ()
r <- s -> MutArray a -> m (Step s ())
step1 s
acc MutArray a
buf
                case Step s ()
r of
                    FL.Done ()
_ -> Step (Tuple' (Maybe (MutArray a)) s) ()
-> m (Step (Tuple' (Maybe (MutArray a)) s) ())
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Tuple' (Maybe (MutArray a)) s) ()
 -> m (Step (Tuple' (Maybe (MutArray a)) s) ()))
-> Step (Tuple' (Maybe (MutArray a)) s) ()
-> m (Step (Tuple' (Maybe (MutArray a)) s) ())
forall a b. (a -> b) -> a -> b
$ () -> Step (Tuple' (Maybe (MutArray a)) s) ()
forall s b. b -> Step s b
FL.Done ()
                    FL.Partial s
s -> do
                        ()
_ <- s -> m ()
final1 s
s
                        Step s ()
res <- m (Step s ())
initial1
                        Step (Tuple' (Maybe (MutArray a)) s) ()
-> m (Step (Tuple' (Maybe (MutArray a)) s) ())
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Tuple' (Maybe (MutArray a)) s) ()
 -> m (Step (Tuple' (Maybe (MutArray a)) s) ()))
-> Step (Tuple' (Maybe (MutArray a)) s) ()
-> m (Step (Tuple' (Maybe (MutArray a)) s) ())
forall a b. (a -> b) -> a -> b
$ (s -> Tuple' (Maybe (MutArray a)) s)
-> Step s () -> Step (Tuple' (Maybe (MutArray a)) s) ()
forall a b c. (a -> b) -> Step a c -> Step b c
forall (p :: * -> * -> *) a b c.
Bifunctor p =>
(a -> b) -> p a c -> p b c
first (Maybe (MutArray a) -> s -> Tuple' (Maybe (MutArray a)) s
forall a b. a -> b -> Tuple' a b
Tuple' Maybe (MutArray a)
forall a. Maybe a
Nothing) Step s ()
res
            else Step (Tuple' (Maybe (MutArray a)) s) ()
-> m (Step (Tuple' (Maybe (MutArray a)) s) ())
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (Tuple' (Maybe (MutArray a)) s) ()
 -> m (Step (Tuple' (Maybe (MutArray a)) s) ()))
-> Step (Tuple' (Maybe (MutArray a)) s) ()
-> m (Step (Tuple' (Maybe (MutArray a)) s) ())
forall a b. (a -> b) -> a -> b
$ Tuple' (Maybe (MutArray a)) s
-> Step (Tuple' (Maybe (MutArray a)) s) ()
forall s b. s -> Step s b
FL.Partial (Tuple' (Maybe (MutArray a)) s
 -> Step (Tuple' (Maybe (MutArray a)) s) ())
-> Tuple' (Maybe (MutArray a)) s
-> Step (Tuple' (Maybe (MutArray a)) s) ()
forall a b. (a -> b) -> a -> b
$ Maybe (MutArray a) -> s -> Tuple' (Maybe (MutArray a)) s
forall a b. a -> b -> Tuple' a b
Tuple' (MutArray a -> Maybe (MutArray a)
forall a. a -> Maybe a
Just MutArray a
buf) s
acc

    step :: Tuple' (Maybe (MutArray a)) s
-> MutArray a -> m (Step (Tuple' (Maybe (MutArray a)) s) ())
step (Tuple' Maybe (MutArray a)
Nothing s
r1) MutArray a
arr =
         Int
-> s -> MutArray a -> m (Step (Tuple' (Maybe (MutArray a)) s) ())
runInner (MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr) s
r1 MutArray a
arr

    -- XXX Buffer arrays as a list to avoid copy and reallocations
    step (Tuple' (Just MutArray a
buf) s
r1) MutArray a
arr = do
        let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
buf Int -> Int -> Int
forall a. Num a => a -> a -> a
+ MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
        MutArray a
buf1 <- if MutArray a -> Int
forall a. MutArray a -> Int
byteCapacity MutArray a
buf Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
len
                then IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
forall a.
PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs
                        PinnedState
ps (SIZE_OF(a)) (max minBytes len) buf
                else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
buf
        MutArray a
buf2 <- MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice MutArray a
buf1 MutArray a
arr
        Int
-> s -> MutArray a -> m (Step (Tuple' (Maybe (MutArray a)) s) ())
runInner Int
len s
r1 MutArray a
buf2

    -- XXX Several folds do extract >=> final, therefore, we need to make final
    -- return "m b" rather than using extract post it if we want extract to be
    -- partial.
    --
    -- extract forces the pending buffer to be sent to the fold which is not
    -- what we want.
    extract :: p -> a
extract p
_ = [Char] -> a
forall a. (?callStack::CallStack) => [Char] -> a
error [Char]
"lCompactGE: not designed for scanning"

    final :: Tuple' (Maybe (MutArray a)) s -> m ()
final (Tuple' Maybe (MutArray a)
Nothing s
r1) = s -> m ()
final1 s
r1
    final (Tuple' (Just MutArray a
buf) s
r1) = do
        Step s ()
r <- s -> MutArray a -> m (Step s ())
step1 s
r1 MutArray a
buf
        case Step s ()
r of
            FL.Partial s
rr -> s -> m ()
final1 s
rr
            FL.Done ()
_ -> () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()

-- | Like 'compactGE' but for transforming folds instead of stream.
--
-- >> lCompactGE n = Fold.many (MutArray.fCompactGE n)
--
-- Generates unpinned arrays irrespective of the pinning status of input
-- arrays.
{-# DEPRECATED lCompactGE "Please use scanCompactMin instead." #-}
{-# INLINE lCompactGE #-}
lCompactGE :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
lCompactGE :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
lCompactGE = PinnedState
-> Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState
-> Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
lCompactGeAs PinnedState
Unpinned

-- | Pinned version of 'lCompactGE'.
{-# DEPRECATED lPinnedCompactGE "Please use scanCompactMin' instead." #-}
{-# INLINE lPinnedCompactGE #-}
lPinnedCompactGE :: forall m a. (MonadIO m, Unbox a)
    => Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
lPinnedCompactGE :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
lPinnedCompactGE = PinnedState
-> Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState
-> Int -> Fold m (MutArray a) () -> Fold m (MutArray a) ()
lCompactGeAs PinnedState
Pinned

data CompactMinState arr =
    CompactMinInit | CompactMinIncomplete arr | CompactMinComplete arr

{-# INLINE_NORMAL scanCompactMinAs #-}
scanCompactMinAs :: forall m a. (MonadIO m, Unbox a)
    => PinnedState -> Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMinAs :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMinAs PinnedState
ps Int
minElems =
    (CompactMinState (MutArray a)
 -> MutArray a
 -> m (Step (CompactMinState (MutArray a)) (Maybe (MutArray a))))
-> m (Step (CompactMinState (MutArray a)) (Maybe (MutArray a)))
-> (CompactMinState (MutArray a) -> m (Maybe (MutArray a)))
-> (CompactMinState (MutArray a) -> m (Maybe (MutArray a)))
-> Scanl m (MutArray a) (Maybe (MutArray a))
forall (m :: * -> *) a b s.
(s -> a -> m (Step s b))
-> m (Step s b) -> (s -> m b) -> (s -> m b) -> Scanl m a b
Scanl CompactMinState (MutArray a)
-> MutArray a
-> m (Step (CompactMinState (MutArray a)) (Maybe (MutArray a)))
forall {m :: * -> *} {a} {b}.
MonadIO m =>
CompactMinState (MutArray a)
-> MutArray a -> m (Step (CompactMinState (MutArray a)) b)
step m (Step (CompactMinState (MutArray a)) (Maybe (MutArray a)))
forall {arr} {b}. m (Step (CompactMinState arr) b)
initial CompactMinState (MutArray a) -> m (Maybe (MutArray a))
forall {m :: * -> *} {a}.
Monad m =>
CompactMinState a -> m (Maybe a)
extract CompactMinState (MutArray a) -> m (Maybe (MutArray a))
forall {m :: * -> *} {a}.
Monad m =>
CompactMinState a -> m (Maybe a)
final

    where

    minBytes :: Int
minBytes = Int
minElems Int -> Int -> Int
forall a. Num a => a -> a -> a
* SIZE_OF(a)

    functionName :: [Char]
functionName = [Char]
"Streamly.Internal.Data.MutArray.scanCompactMin"

    initial :: m (Step (CompactMinState arr) b)
initial = do
        Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
minElems Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
0) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$
            -- XXX we can pass the module string from the higher level API
            [Char] -> m ()
forall a. (?callStack::CallStack) => [Char] -> a
error ([Char] -> m ()) -> [Char] -> m ()
forall a b. (a -> b) -> a -> b
$ [Char]
functionName [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
": the size of arrays ["
                [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ Int -> [Char]
forall a. Show a => a -> [Char]
show Int
minElems [Char] -> [Char] -> [Char]
forall a. [a] -> [a] -> [a]
++ [Char]
"] must be a natural number"

        Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b))
-> Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b)
forall a b. (a -> b) -> a -> b
$ CompactMinState arr -> Step (CompactMinState arr) b
forall s b. s -> Step s b
FL.Partial CompactMinState arr
forall arr. CompactMinState arr
CompactMinInit

    {-# INLINE runInner #-}
    runInner :: Int -> arr -> m (Step (CompactMinState arr) b)
runInner Int
len arr
buf =
            if Int
len Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
minBytes
            then do
                Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b))
-> Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b)
forall a b. (a -> b) -> a -> b
$ CompactMinState arr -> Step (CompactMinState arr) b
forall s b. s -> Step s b
FL.Partial (CompactMinState arr -> Step (CompactMinState arr) b)
-> CompactMinState arr -> Step (CompactMinState arr) b
forall a b. (a -> b) -> a -> b
$ arr -> CompactMinState arr
forall arr. arr -> CompactMinState arr
CompactMinComplete arr
buf
            else Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b))
-> Step (CompactMinState arr) b -> m (Step (CompactMinState arr) b)
forall a b. (a -> b) -> a -> b
$ CompactMinState arr -> Step (CompactMinState arr) b
forall s b. s -> Step s b
FL.Partial (CompactMinState arr -> Step (CompactMinState arr) b)
-> CompactMinState arr -> Step (CompactMinState arr) b
forall a b. (a -> b) -> a -> b
$ arr -> CompactMinState arr
forall arr. arr -> CompactMinState arr
CompactMinIncomplete arr
buf

    step :: CompactMinState (MutArray a)
-> MutArray a -> m (Step (CompactMinState (MutArray a)) b)
step CompactMinState (MutArray a)
CompactMinInit MutArray a
arr =
         Int -> MutArray a -> m (Step (CompactMinState (MutArray a)) b)
forall {m :: * -> *} {arr} {b}.
Monad m =>
Int -> arr -> m (Step (CompactMinState arr) b)
runInner (MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr) MutArray a
arr

    step (CompactMinComplete MutArray a
_) MutArray a
arr =
         Int -> MutArray a -> m (Step (CompactMinState (MutArray a)) b)
forall {m :: * -> *} {arr} {b}.
Monad m =>
Int -> arr -> m (Step (CompactMinState arr) b)
runInner (MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr) MutArray a
arr

    -- XXX Buffer arrays as a list to avoid copy and reallocations
    step (CompactMinIncomplete MutArray a
buf) MutArray a
arr = do
        let len :: Int
len = MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
buf Int -> Int -> Int
forall a. Num a => a -> a -> a
+ MutArray a -> Int
forall a. MutArray a -> Int
byteLength MutArray a
arr
        MutArray a
buf1 <- if MutArray a -> Int
forall a. MutArray a -> Int
byteCapacity MutArray a
buf Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
len
                then IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
forall a.
PinnedState -> Int -> Int -> MutArray a -> IO (MutArray a)
reallocExplicitAs
                        PinnedState
ps (SIZE_OF(a)) (max minBytes len) buf
                else MutArray a -> m (MutArray a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return MutArray a
buf
        MutArray a
buf2 <- MutArray a -> MutArray a -> m (MutArray a)
forall (m :: * -> *) a.
MonadIO m =>
MutArray a -> MutArray a -> m (MutArray a)
unsafeSplice MutArray a
buf1 MutArray a
arr
        Int -> MutArray a -> m (Step (CompactMinState (MutArray a)) b)
forall {m :: * -> *} {arr} {b}.
Monad m =>
Int -> arr -> m (Step (CompactMinState arr) b)
runInner Int
len MutArray a
buf2

    extract :: CompactMinState a -> m (Maybe a)
extract CompactMinState a
CompactMinInit = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe a
forall a. Maybe a
Nothing
    extract (CompactMinComplete a
arr) = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Maybe a
forall a. a -> Maybe a
Just a
arr)
    extract (CompactMinIncomplete a
_) = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe a
forall a. Maybe a
Nothing

    final :: CompactMinState a -> m (Maybe a)
final CompactMinState a
CompactMinInit = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe a
forall a. Maybe a
Nothing
    final (CompactMinComplete a
arr) = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Maybe a
forall a. a -> Maybe a
Just a
arr)
    final (CompactMinIncomplete a
arr) = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Maybe a
forall a. a -> Maybe a
Just a
arr)

-- | Like 'compactMin' but a scan.
{-# INLINE scanCompactMin #-}
scanCompactMin :: forall m a. (MonadIO m, Unbox a)
    => Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMin :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMin = PinnedState -> Int -> Scanl m (MutArray a) (Maybe (MutArray a))
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMinAs PinnedState
Unpinned

-- | Like 'compactMin'' but a scan.
{-# INLINE scanCompactMin' #-}
scanCompactMin' :: forall m a. (MonadIO m, Unbox a)
    => Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMin' :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMin' = PinnedState -> Int -> Scanl m (MutArray a) (Maybe (MutArray a))
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
PinnedState -> Int -> Scanl m (MutArray a) (Maybe (MutArray a))
scanCompactMinAs PinnedState
Pinned

-- | @compactMin n stream@ coalesces adjacent arrays in the @stream@ until
-- the compacted array size becomes greater than or equal to @n@.
--
-- >>> compactMin n = Stream.foldMany (MutArray.createCompactMin n)
--
{-# INLINE compactMin #-}
compactMin, compactGE ::
       (MonadIO m, Unbox a)
    => Int -> Stream m (MutArray a) -> Stream m (MutArray a)
compactMin :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Stream m (MutArray a) -> Stream m (MutArray a)
compactMin Int
n = Fold m (MutArray a) (MutArray a)
-> Stream m (MutArray a) -> Stream m (MutArray a)
forall (m :: * -> *) a b.
Monad m =>
Fold m a b -> Stream m a -> Stream m b
D.foldMany (Int -> Fold m (MutArray a) (MutArray a)
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> Fold m (MutArray a) (MutArray a)
createCompactMin Int
n)

RENAME(compactGE,compactMin)

-- | 'compactExact n' coalesces adajacent arrays in the input stream to
-- arrays of exact size @n@.
--
-- /Unimplemented/
{-# INLINE compactExact #-}
compactExact :: -- (MonadIO m, Unbox a) =>
    Int -> Stream m (MutArray a) -> Stream m (MutArray a)
compactExact :: forall (m :: * -> *) a.
Int -> Stream m (MutArray a) -> Stream m (MutArray a)
compactExact Int
_n = Stream m (MutArray a) -> Stream m (MutArray a)
forall a. (?callStack::CallStack) => a
undefined -- D.parseManyD (pCompactEQ n)

-------------------------------------------------------------------------------
-- In-place mutation algorithms
-------------------------------------------------------------------------------

-- XXX Can use SIMD
-- XXX findIndex can be implemented using this if fold perf is not good enough.

{-# INLINE advanceStartTill #-}
advanceStartTill :: forall a. (Unbox a) => (a -> Bool) -> MutArray a -> IO Int
advanceStartTill :: forall a. Unbox a => (a -> Bool) -> MutArray a -> IO Int
advanceStartTill a -> Bool
eq MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = Int -> IO Int
go Int
arrStart

    where

    {-
    -- XXX This should have the same perf but it does not, investigate.
    getStart = do
        r <- liftIO $ D.head $ D.findIndices (not . eq) $ toStreamD arr
        pure $
            case r of
                Nothing -> arrEnd
                Just i -> PTR_INDEX(arrStart,i,a)
    -}

    go :: Int -> IO Int
go Int
cur =
        if Int
cur Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
< Int
arrEnd
        then do
            a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
cur MutByteArray
arrContents
            if a -> Bool
eq a
r
            then Int -> IO Int
go (INDEX_NEXT(cur,a))
            else Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
cur
        else Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
cur

{-# INLINE retractEndTill #-}
retractEndTill :: forall a. (Unbox a) => (a -> Bool) -> MutArray a -> IO Int
retractEndTill :: forall a. Unbox a => (a -> Bool) -> MutArray a -> IO Int
retractEndTill a -> Bool
eq MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = Int -> IO Int
go Int
arrEnd

    where

    go :: Int -> IO Int
go Int
cur = do
        if Int
cur Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
arrStart
        then do
            let prev :: Int
prev = INDEX_PREV(cur,a)
            a
r <- Int -> MutByteArray -> IO a
forall a. Unbox a => Int -> MutByteArray -> IO a
peekAt Int
prev MutByteArray
arrContents
            if a -> Bool
eq a
r
            then Int -> IO Int
go Int
prev
            else Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
cur
        else Int -> IO Int
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return Int
cur

-- | Strip elements which match the predicate, from the start of the array.
--
-- >>> arr <- MutArray.fromList "    hello world"
-- >>> a <- MutArray.dropWhile (== ' ') arr
-- >>> MutArray.toList a
-- "hello world"
--
-- /Pre-release/
{-# INLINE dropWhile #-}
dropWhile, stripStart :: forall a m. (Unbox a, MonadIO m) =>
    (a -> Bool) -> MutArray a -> m (MutArray a)
dropWhile :: forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
(a -> Bool) -> MutArray a -> m (MutArray a)
dropWhile a -> Bool
eq arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ do
    Int
st <- (a -> Bool) -> MutArray a -> IO Int
forall a. Unbox a => (a -> Bool) -> MutArray a -> IO Int
advanceStartTill a -> Bool
eq MutArray a
arr
    -- return arr{arrStart = st}
    MutArray a -> IO (MutArray a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> IO (MutArray a)) -> MutArray a -> IO (MutArray a)
forall a b. (a -> b) -> a -> b
$
        if Int
st Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
arrEnd
        then MutArray a
forall a. MutArray a
empty
        else MutArray a
arr{arrStart :: Int
arrStart = Int
st}
RENAME(stripStart,dropWhile)

-- | Strip elements which match the predicate, from the end of the array.
--
-- >>> arr <- MutArray.fromList "hello world    "
-- >>> a <- MutArray.revDropWhile (== ' ') arr
-- >>> MutArray.toList a
-- "hello world"
--
-- /Pre-release/
{-# INLINE revDropWhile #-}
revDropWhile, stripEnd :: forall a m. (Unbox a, MonadIO m) =>
    (a -> Bool) -> MutArray a -> m (MutArray a)
revDropWhile :: forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
(a -> Bool) -> MutArray a -> m (MutArray a)
revDropWhile a -> Bool
eq arr :: MutArray a
arr@MutArray{Int
MutByteArray
arrContents :: forall a. MutArray a -> MutByteArray
arrStart :: forall a. MutArray a -> Int
arrEnd :: forall a. MutArray a -> Int
arrBound :: forall a. MutArray a -> Int
arrContents :: MutByteArray
arrStart :: Int
arrEnd :: Int
arrBound :: Int
..} = IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ do
    Int
end <- (a -> Bool) -> MutArray a -> IO Int
forall a. Unbox a => (a -> Bool) -> MutArray a -> IO Int
retractEndTill a -> Bool
eq MutArray a
arr
    -- return arr {arrEnd = end}
    MutArray a -> IO (MutArray a)
forall a. a -> IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (MutArray a -> IO (MutArray a)) -> MutArray a -> IO (MutArray a)
forall a b. (a -> b) -> a -> b
$
        if Int
end Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
<= Int
arrStart
        then MutArray a
forall a. MutArray a
empty
        else MutArray a
arr{arrEnd :: Int
arrEnd = Int
end}
RENAME(stripEnd,revDropWhile)

-- | Strip elements which match the predicate, from both ends.
--
-- >>> arr <- MutArray.fromList "   hello world    "
-- >>> a <- MutArray.dropAround (== ' ') arr
-- >>> MutArray.toList a
-- "hello world"
--
-- /Pre-release/
{-# INLINE dropAround #-}
dropAround, strip :: forall a m. (Unbox a, MonadIO m) =>
    (a -> Bool) -> MutArray a -> m (MutArray a)
dropAround :: forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
(a -> Bool) -> MutArray a -> m (MutArray a)
dropAround a -> Bool
eq MutArray a
arr = IO (MutArray a) -> m (MutArray a)
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (MutArray a) -> m (MutArray a))
-> IO (MutArray a) -> m (MutArray a)
forall a b. (a -> b) -> a -> b
$ (a -> Bool) -> MutArray a -> IO (MutArray a)
forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
(a -> Bool) -> MutArray a -> m (MutArray a)
dropWhile a -> Bool
eq MutArray a
arr IO (MutArray a)
-> (MutArray a -> IO (MutArray a)) -> IO (MutArray a)
forall a b. IO a -> (a -> IO b) -> IO b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= (a -> Bool) -> MutArray a -> IO (MutArray a)
forall a (m :: * -> *).
(Unbox a, MonadIO m) =>
(a -> Bool) -> MutArray a -> m (MutArray a)
revDropWhile a -> Bool
eq
RENAME(strip,dropAround)

-- | Given an array sorted in ascending order except the last element being out
-- of order, use bubble sort to place the last element at the right place such
-- that the array remains sorted in ascending order.
--
-- /Pre-release/
{-# INLINE bubble #-}
bubble :: (MonadIO m, Unbox a) => (a -> a -> Ordering) -> MutArray a -> m ()
bubble :: forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
(a -> a -> Ordering) -> MutArray a -> m ()
bubble a -> a -> Ordering
cmp0 MutArray a
arr =
    Bool -> m () -> m ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (Int
l Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
1) (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ do
        a
x <- Int -> MutArray a -> m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m a
unsafeGetIndex (Int
l Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1) MutArray a
arr
        a -> Int -> m ()
forall {m :: * -> *}. MonadIO m => a -> Int -> m ()
go a
x (Int
l Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
2)

        where

        l :: Int
l = MutArray a -> Int
forall a. Unbox a => MutArray a -> Int
length MutArray a
arr

        go :: a -> Int -> m ()
go a
x Int
i =
            if Int
i Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0
            then do
                a
x1 <- Int -> MutArray a -> m a
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> m a
unsafeGetIndex Int
i MutArray a
arr
                case a
x a -> a -> Ordering
`cmp0` a
x1 of
                    Ordering
LT -> do
                        Int -> MutArray a -> a -> m ()
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m ()
unsafePutIndex (Int
i Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1) MutArray a
arr a
x1
                        a -> Int -> m ()
go a
x (Int
i Int -> Int -> Int
forall a. Num a => a -> a -> a
- Int
1)
                    Ordering
_ -> Int -> MutArray a -> a -> m ()
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m ()
unsafePutIndex (Int
i Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1) MutArray a
arr a
x
            else Int -> MutArray a -> a -> m ()
forall (m :: * -> *) a.
(MonadIO m, Unbox a) =>
Int -> MutArray a -> a -> m ()
unsafePutIndex (Int
i Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1) MutArray a
arr a
x

--------------------------------------------------------------------------------
-- Renaming
--------------------------------------------------------------------------------

RENAME(realloc,reallocBytes)
RENAME(castUnsafe,unsafeCast)
RENAME(newArrayWith,emptyWithAligned)
RENAME(getSliceUnsafe,unsafeSliceOffLen)
RENAME(unsafeGetSlice,unsafeSliceOffLen)
RENAME(getSlice,sliceOffLen)
RENAME(putIndexUnsafe,unsafePutIndex)
RENAME(modifyIndexUnsafe,unsafeModifyIndex)
RENAME(getIndexUnsafe,unsafeGetIndex)
RENAME(snocUnsafe,unsafeSnoc)
RENAME(spliceUnsafe,unsafeSplice)
RENAME(pokeSkipUnsafe,unsafePokeSkip)
RENAME(peekSkipUnsafe,unsafePeekSkip)
RENAME(peekUncons,peek)
RENAME(peekUnconsUnsafe,unsafePeek)
RENAME(pokeAppend,poke)
RENAME(pokeAppendMay,pokeMay)

-- This renaming can be done directly without deprecations. But I'm keeping this
-- intentionally. Packdiff should be able to point out such APIs that we can
-- just remove.
RENAME(createOfWith,createWithOf)