{-# LANGUAGE CPP #-}
{-# LANGUAGE TypeFamilies #-}
-- Must come after TypeFamilies, otherwise it is re-enabled.
-- MonoLocalBinds enabled by TypeFamilies causes perf regressions in general.
{-# LANGUAGE NoMonoLocalBinds #-}
{-# LANGUAGE UndecidableInstances #-}
-- |
-- Module      : Streamly.Internal.Data.StreamK.Type
-- Copyright   : (c) 2017 Composewell Technologies
--
-- License     : BSD3
-- Maintainer  : streamly@composewell.com
-- Stability   : experimental
-- Portability : GHC
--
--
-- Continuation passing style (CPS) stream implementation. The symbol 'K' below
-- denotes a function as well as a Kontinuation.
--
module Streamly.Internal.Data.StreamK.Type
    (
    -- * StreamK type
      Stream
    , StreamK (..)

    -- * Nested type wrapper
    , Nested(..)
    , FairNested(..) -- experimental, do not release, associativity issues

    -- * foldr/build Fusion
    , mkStream
    , foldStream
    , foldStreamShared
    , foldrM
    , foldrS
    , foldrSShared
    , foldrSM
    , build
    , buildS
    , buildM
    , buildSM
    , augmentS
    , augmentSM
    , unShare

    -- * Construction
    -- ** Primitives
    , fromStopK
    , fromYieldK
    , consK
    , cons
    , (.:)
    , consM
    , consMBy
    , nil
    , nilM

    -- ** Unfolding
    , unfoldr
    , unfoldrMWith
    , unfoldrM

    -- ** From Values
    , fromEffect
    , fromPure
    , repeat
    , repeatMWith
    , replicateMWith

    -- ** From Indices
    , fromIndicesMWith

    -- ** Iteration
    , iterateMWith

    -- ** From Containers
    , fromFoldable
    , fromFoldableM
    , Streamly.Internal.Data.StreamK.Type.fromList

    -- ** Cyclic
    , mfix

    -- * Elimination
    -- ** Primitives
    , uncons

    -- ** Strict Left Folds
    , Streamly.Internal.Data.StreamK.Type.foldl'
    , foldlx'
    , foldlMx'
    , foldlM'

    -- ** Lazy Right Folds
    , Streamly.Internal.Data.StreamK.Type.foldr

    -- ** Specific Folds
    , drain
    , null
    , headNonEmpty
    , tail
    , tailNonEmpty
    , init
    , initNonEmpty

    -- * Mapping
    , map
    , mapMWith
    , mapMSerial
    , mapMAccum

    -- * Combining Two Streams
    -- ** Appending
    , conjoin
    , append

    -- ** Interleave
    , interleave
    , interleaveEndBy'
    , interleaveSepBy

    -- ** Cross Product
    , crossApplyWith
    , crossApply
    , crossApplySnd
    , crossApplyFst
    , crossWith
    , cross

    -- * Concat

    -- ** Concat Effects
    , before
    , concatEffect
    , concatMapEffect

    -- ** ConcatMap
    , concatMapWith
    , concatMap
    , bfsConcatMap
    , fairConcatMap
    , concatMapMAccum

    -- ** concatFor (bind)
    , concatFor
    , bfsConcatFor
    , fairConcatFor
    , concatForWith

    -- ** concatForM
    , concatForM
    , bfsConcatForM
    , fairConcatForM
    , concatForWithM

    -- ** Iterated concat
    , concatIterateWith
    , concatIterateLeftsWith
    , concatIterateScanWith

    -- * Merge
    , mergeMapWith
    , mergeIterateWith

    -- * Buffered Operations
    , foldlS
    , reverse

    -- * Deprecated
    , interleaveFst
    , interleaveMin
    , CrossStreamK
    , mkCross
    , unCross
    , bindWith
    )
where

#include "inline.hs"
#include "deprecation.h"

import Control.Applicative (Alternative(..))
import Control.Monad ((>=>), ap, MonadPlus(..))
import Control.Monad.Catch (MonadThrow, throwM)
import Control.Monad.Trans.Class (MonadTrans(lift))
#if !MIN_VERSION_base(4,18,0)
import Control.Applicative (liftA2)
#endif
import Control.Monad.IO.Class (MonadIO(..))
import Data.Foldable (Foldable(foldl'), fold, foldr)
import Data.Function (fix)
import Data.Functor.Identity (Identity(..))
#if __GLASGOW_HASKELL__ >= 810
import Data.Kind (Type)
#endif
import Data.Maybe (fromMaybe)
import Data.Semigroup (Endo(..))
import GHC.Exts (IsList(..), IsString(..), oneShot, inline)
import Streamly.Internal.BaseCompat ((#.))
import Streamly.Internal.Data.Maybe.Strict (Maybe'(..), toMaybe)
import Streamly.Internal.Data.SVar.Type (State, adaptState, defState)
import Text.Read
       ( Lexeme(Ident), lexP, parens, prec, readPrec, readListPrec
       , readListPrecDefault)

import qualified Control.Monad.Fail as Fail
import qualified Prelude

import Prelude hiding
    (map, mapM, concatMap, foldr, repeat, null, reverse, tail, init)

#include "DocTestDataStreamK.hs"

------------------------------------------------------------------------------
-- Basic stream type
------------------------------------------------------------------------------

-- It uses stop, singleton and yield continuations equivalent to the following
-- direct style type:
--
-- @
-- data StreamK m a = Stop | Singleton a | Yield a (StreamK m a)
-- @
--
-- To facilitate parallel composition we maintain a local state in an 'SVar'
-- that is shared across and is used for synchronization of the streams being
-- composed.
--
-- The singleton case can be expressed in terms of stop and yield but we have
-- it as a separate case to optimize composition operations for streams with
-- single element.  We build singleton streams in the implementation of 'pure'
-- for Applicative and Monad, and in 'lift' for MonadTrans.

-- XXX can we replace it with a direct style type? With foldr/build fusion.
-- StreamK (m (Maybe (a, StreamK m a)))
-- XXX remove the State param.

-- | Continuation Passing Style (CPS) version of "Streamly.Data.Stream.Stream".
-- Unlike "Streamly.Data.Stream.Stream", 'StreamK' can be composed recursively
-- without affecting performance.
--
-- Semigroup instance appends two streams:
--
-- >>> (<>) = Stream.append
--
{-# DEPRECATED Stream "Please use StreamK instead." #-}
type Stream = StreamK

newtype StreamK m a =
    MkStream (forall r.
               State StreamK m a         -- state
            -> (a -> StreamK m a -> m r) -- yield
            -> (a -> m r)               -- singleton
            -> m r                      -- stop
            -> m r
            )

mkStream
    :: (forall r. State StreamK m a
        -> (a -> StreamK m a -> m r)
        -> (a -> m r)
        -> m r
        -> m r)
    -> StreamK m a
mkStream :: forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
MkStream

-- | A terminal function that has no continuation to follow.
#if __GLASGOW_HASKELL__ >= 810
type StopK :: (Type -> Type) -> Type
#endif
type StopK m = forall r. m r -> m r

-- | A monadic continuation, it is a function that yields a value of type "a"
-- and calls the argument (a -> m r) as a continuation with that value. We can
-- also think of it as a callback with a handler (a -> m r).  Category
-- theorists call it a codensity type, a special type of right kan extension.
#if __GLASGOW_HASKELL__ >= 810
type YieldK :: (Type -> Type) -> Type -> Type
#endif
type YieldK m a = forall r. (a -> m r) -> m r

_wrapM :: Monad m => m a -> YieldK m a
_wrapM :: forall (m :: * -> *) a. Monad m => m a -> YieldK m a
_wrapM m a
m = (m a
m m a -> (a -> m r) -> m r
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>=)

-- | Make an empty stream from a stop function.
fromStopK :: StopK m -> StreamK m a
fromStopK :: forall (m :: * -> *) a. StopK m -> StreamK m a
fromStopK StopK m
k = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
_ a -> m r
_ m r
stp -> m r -> m r
StopK m
k m r
stp

-- | Make a singleton stream from a callback function. The callback function
-- calls the one-shot yield continuation to yield an element.
fromYieldK :: YieldK m a -> StreamK m a
fromYieldK :: forall (m :: * -> *) a. YieldK m a -> StreamK m a
fromYieldK YieldK m a
k = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
_ a -> m r
sng m r
_ -> (a -> m r) -> m r
YieldK m a
k a -> m r
sng

-- | Add a yield function at the head of the stream.
consK :: YieldK m a -> StreamK m a -> StreamK m a
consK :: forall (m :: * -> *) a. YieldK m a -> StreamK m a -> StreamK m a
consK YieldK m a
k StreamK m a
r = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
yld a -> m r
_ m r
_ -> (a -> m r) -> m r
YieldK m a
k (a -> StreamK m a -> m r
`yld` StreamK m a
r)

-- XXX Build a stream from a repeating callback function.

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

infixr 5 `cons`

-- faster than consM because there is no bind.

-- | A right associative prepend operation to add a pure value at the head of
-- an existing stream:
--
-- >>> s = 1 `StreamK.cons` 2 `StreamK.cons` 3 `StreamK.cons` StreamK.nil
-- >>> Stream.fold Fold.toList (StreamK.toStream s)
-- [1,2,3]
--
-- Unlike "Streamly.Data.Stream" cons StreamK cons can be used
-- recursively:
--
-- >>> repeat x = let xs = StreamK.cons x xs in xs
-- >>> fromFoldable = Prelude.foldr StreamK.cons StreamK.nil
--
-- cons is same as the following but more efficient:
--
-- >>> cons x xs = return x `StreamK.consM` xs
--
{-# INLINE_NORMAL cons #-}
cons :: a -> StreamK m a -> StreamK m a
cons :: forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons a
a StreamK m a
r = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
yield a -> m r
_ m r
_ -> a -> StreamK m a -> m r
yield a
a StreamK m a
r

infixr 5 .:

-- | Operator equivalent of 'cons'.
--
-- @
-- > toList $ 1 .: 2 .: 3 .: nil
-- [1,2,3]
-- @
--
{-# INLINE (.:) #-}
(.:) :: a -> StreamK m a -> StreamK m a
.: :: forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
(.:) = a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons

-- | A stream that terminates without producing any output or side effect.
--
-- >>> Stream.fold Fold.toList (StreamK.toStream StreamK.nil)
-- []
--
{-# INLINE_NORMAL nil #-}
nil :: StreamK m a
nil :: forall (m :: * -> *) a. StreamK m a
nil = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
_ a -> m r
_ m r
stp -> m r
stp

-- | A stream that terminates without producing any output, but produces a side
-- effect.
--
-- >>> Stream.fold Fold.toList (StreamK.toStream (StreamK.nilM (print "nil")))
-- "nil"
-- []
--
-- /Pre-release/
{-# INLINE_NORMAL nilM #-}
nilM :: Applicative m => m b -> StreamK m a
nilM :: forall (m :: * -> *) b a. Applicative m => m b -> StreamK m a
nilM m b
m = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
_ a -> m r
_ m r
stp -> m b
m m b -> m r -> m r
forall a b. m a -> m b -> m b
forall (f :: * -> *) a b. Applicative f => f a -> f b -> f b
*> m r
stp

-- Create a singleton stream from a pure value.
--
-- >>> fromPure a = a `StreamK.cons` StreamK.nil
-- >>> fromPure = pure
-- >>> fromPure = StreamK.fromEffect . pure
--
{-# INLINE_NORMAL fromPure #-}
fromPure :: a -> StreamK m a
fromPure :: forall a (m :: * -> *). a -> StreamK m a
fromPure a
a = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
_ a -> m r
single m r
_ -> a -> m r
single a
a

-- Create a singleton stream from a monadic action.
--
-- >>> fromEffect m = m `StreamK.consM` StreamK.nil
--
-- >>> Stream.fold Fold.drain $ StreamK.toStream $ StreamK.fromEffect (putStrLn "hello")
-- hello
--
{-# INLINE_NORMAL fromEffect #-}
fromEffect :: Monad m => m a -> StreamK m a
fromEffect :: forall (m :: * -> *) a. Monad m => m a -> StreamK m a
fromEffect m a
m = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
_ a -> m r
single m r
_ -> m a
m m a -> (a -> m r) -> m r
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= a -> m r
single

infixr 5 `consM`

-- NOTE: specializing the function outside the instance definition seems to
-- improve performance quite a bit at times, even if we have the same
-- SPECIALIZE in the instance definition.

-- | A right associative prepend operation to add an effectful value at the
-- head of an existing stream::
--
-- >>> s = putStrLn "hello" `StreamK.consM` putStrLn "world" `StreamK.consM` StreamK.nil
-- >>> Stream.fold Fold.drain (StreamK.toStream s)
-- hello
-- world
--
-- It can be used efficiently with 'Prelude.foldr':
--
-- >>> fromFoldableM = Prelude.foldr StreamK.consM StreamK.nil
--
-- Same as the following but more efficient:
--
-- >>> consM x xs = StreamK.fromEffect x `StreamK.append` xs
--
{-# INLINE consM #-}
{-# SPECIALIZE consM :: IO a -> StreamK IO a -> StreamK IO a #-}
consM :: Monad m => m a -> StreamK m a -> StreamK m a
consM :: forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
consM m a
m StreamK m a
r = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
MkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
_ a -> StreamK m a -> m r
yld a -> m r
_ m r
_ -> m a
m m a -> (a -> m r) -> m r
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= (a -> StreamK m a -> m r
`yld` StreamK m a
r)

-- XXX specialize to IO?
{-# INLINE consMBy #-}
consMBy :: Monad m =>
    (StreamK m a -> StreamK m a -> StreamK m a) -> m a -> StreamK m a -> StreamK m a
consMBy :: forall (m :: * -> *) a.
Monad m =>
(StreamK m a -> StreamK m a -> StreamK m a)
-> m a -> StreamK m a -> StreamK m a
consMBy StreamK m a -> StreamK m a -> StreamK m a
f m a
m StreamK m a
r = m a -> StreamK m a
forall (m :: * -> *) a. Monad m => m a -> StreamK m a
fromEffect m a
m StreamK m a -> StreamK m a -> StreamK m a
`f` StreamK m a
r

------------------------------------------------------------------------------
-- Folding a stream
------------------------------------------------------------------------------

-- | Fold a stream by providing an SVar, a stop continuation, a singleton
-- continuation and a yield continuation. The stream would share the current
-- SVar passed via the State.
{-# INLINE_EARLY foldStreamShared #-}
foldStreamShared
    :: State StreamK m a
    -> (a -> StreamK m a -> m r)
    -> (a -> m r)
    -> m r
    -> StreamK m a
    -> m r
foldStreamShared :: forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
s a -> StreamK m a -> m r
yield a -> m r
single m r
stop (MkStream forall r.
State StreamK m a
-> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
k) = State StreamK m a
-> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
forall r.
State StreamK m a
-> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
k State StreamK m a
s a -> StreamK m a -> m r
yield a -> m r
single m r
stop

-- | Fold a stream by providing a State, stop continuation, a singleton
-- continuation and a yield continuation. The stream will not use the SVar
-- passed via State.
{-# INLINE foldStream #-}
foldStream
    :: State StreamK m a
    -> (a -> StreamK m a -> m r)
    -> (a -> m r)
    -> m r
    -> StreamK m a
    -> m r
foldStream :: forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
s a -> StreamK m a -> m r
yield a -> m r
single m r
stop (MkStream forall r.
State StreamK m a
-> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
k) =
    State StreamK m a
-> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
forall r.
State StreamK m a
-> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
k (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
s) a -> StreamK m a -> m r
yield a -> m r
single m r
stop

-------------------------------------------------------------------------------
-- foldr/build fusion
-------------------------------------------------------------------------------

-- XXX perhaps we can just use foldrSM/buildM everywhere as they are more
-- general and cover foldrS/buildS as well.

-- | The function 'f' decides how to reconstruct the stream. We could
-- reconstruct using a shared state (SVar) or without sharing the state.
--
{-# INLINE foldrSWith #-}
foldrSWith ::
    (forall r. State StreamK m b
        -> (b -> StreamK m b -> m r)
        -> (b -> m r)
        -> m r
        -> StreamK m b
        -> m r)
    -> (a -> StreamK m b -> StreamK m b)
    -> StreamK m b
    -> StreamK m a
    -> StreamK m b
foldrSWith :: forall (m :: * -> *) b a.
(forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
foldrSWith forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
f a -> StreamK m b -> StreamK m b
step StreamK m b
final StreamK m a
m = StreamK m a -> StreamK m b
go StreamK m a
m
    where
    go :: StreamK m a -> StreamK m b
go StreamK m a
m1 = (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
        let run :: StreamK m b -> m r
run StreamK m b
x = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
f State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp StreamK m b
x
            stop :: m r
stop = StreamK m b -> m r
run StreamK m b
final
            single :: a -> m r
single a
a = StreamK m b -> m r
run (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ a -> StreamK m b -> StreamK m b
step a
a StreamK m b
final
            yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m b -> m r
run (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ a -> StreamK m b -> StreamK m b
step a
a (StreamK m a -> StreamK m b
go StreamK m a
r)
         -- XXX if type a and b are the same we do not need adaptState, can we
         -- save some perf with that?
         -- XXX since we are using adaptState anyway here we can use
         -- foldStreamShared instead, will that save some perf?
         in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m1

-- XXX we can use rewrite rules just for foldrSWith, if the function f is the
-- same we can rewrite it.

-- | Fold sharing the SVar state within the reconstructed stream
{-# INLINE_NORMAL foldrSShared #-}
foldrSShared ::
       (a -> StreamK m b -> StreamK m b)
    -> StreamK m b
    -> StreamK m a
    -> StreamK m b
foldrSShared :: forall a (m :: * -> *) b.
(a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrSShared = (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
forall (m :: * -> *) b a.
(forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
foldrSWith State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared

-- XXX consM is a typeclass method, therefore rewritten already. Instead maybe
-- we can make consM polymorphic using rewrite rules.
-- {-# RULES "foldrSShared/id"     foldrSShared consM nil = \x -> x #-}
{-# RULES "foldrSShared/nil"
    forall k z. foldrSShared k z nil = z #-}
{-# RULES "foldrSShared/single"
    forall k z x. foldrSShared k z (fromPure x) = k x z #-}
-- {-# RULES "foldrSShared/app" [1]
--     forall ys. foldrSShared consM ys = \xs -> xs `conjoin` ys #-}

-- | Right fold to a streaming monad.
--
-- > foldrS StreamK.cons StreamK.nil === id
--
-- 'foldrS' can be used to perform stateless stream to stream transformations
-- like map and filter in general. It can be coupled with a scan to perform
-- stateful transformations. However, note that the custom map and filter
-- routines can be much more efficient than this due to better stream fusion.
--
-- >>> input = StreamK.fromStream $ Stream.fromList [1..5]
-- >>> Stream.fold Fold.toList $ StreamK.toStream $ StreamK.foldrS StreamK.cons StreamK.nil input
-- [1,2,3,4,5]
--
-- Find if any element in the stream is 'True':
--
-- >>> step x xs = if odd x then StreamK.fromPure True else xs
-- >>> input = StreamK.fromStream (Stream.fromList (2:4:5:undefined)) :: StreamK IO Int
-- >>> Stream.fold Fold.toList $ StreamK.toStream $ StreamK.foldrS step (StreamK.fromPure False) input
-- [True]
--
-- Map (+2) on odd elements and filter out the even elements:
--
-- >>> step x xs = if odd x then (x + 2) `StreamK.cons` xs else xs
-- >>> input = StreamK.fromStream (Stream.fromList [1..5]) :: StreamK IO Int
-- >>> Stream.fold Fold.toList $ StreamK.toStream $ StreamK.foldrS step StreamK.nil input
-- [3,5,7]
--
-- /Pre-release/
{-# INLINE_NORMAL foldrS #-}
foldrS ::
       (a -> StreamK m b -> StreamK m b)
    -> StreamK m b
    -> StreamK m a
    -> StreamK m b
foldrS :: forall a (m :: * -> *) b.
(a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrS = (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
forall (m :: * -> *) b a.
(forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
foldrSWith State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream

{-# RULES "foldrS/id"     foldrS cons nil = \x -> x #-}
{-# RULES "foldrS/nil"    forall k z.   foldrS k z nil  = z #-}
-- See notes in GHC.Base about this rule
-- {-# RULES "foldr/cons"
--  forall k z x xs. foldrS k z (x `cons` xs) = k x (foldrS k z xs) #-}
{-# RULES "foldrS/single" forall k z x. foldrS k z (fromPure x) = k x z #-}
-- {-# RULES "foldrS/app" [1]
--  forall ys. foldrS cons ys = \xs -> xs `conjoin` ys #-}

-------------------------------------------------------------------------------
-- foldrS with monadic cons i.e. consM
-------------------------------------------------------------------------------

{-# INLINE foldrSMWith #-}
foldrSMWith :: Monad m
    => (forall r. State StreamK m b
        -> (b -> StreamK m b -> m r)
        -> (b -> m r)
        -> m r
        -> StreamK m b
        -> m r)
    -> (m a -> StreamK m b -> StreamK m b)
    -> StreamK m b
    -> StreamK m a
    -> StreamK m b
foldrSMWith :: forall (m :: * -> *) b a.
Monad m =>
(forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (m a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
foldrSMWith forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
f m a -> StreamK m b -> StreamK m b
step StreamK m b
final StreamK m a
m = StreamK m a -> StreamK m b
go StreamK m a
m
    where
    go :: StreamK m a -> StreamK m b
go StreamK m a
m1 = (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
        let run :: StreamK m b -> m r
run StreamK m b
x = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
f State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp StreamK m b
x
            stop :: m r
stop = StreamK m b -> m r
run StreamK m b
final
            single :: a -> m r
single a
a = StreamK m b -> m r
run (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ m a -> StreamK m b -> StreamK m b
step (a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return a
a) StreamK m b
final
            yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m b -> m r
run (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ m a -> StreamK m b -> StreamK m b
step (a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return a
a) (StreamK m a -> StreamK m b
go StreamK m a
r)
         in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m1

{-# INLINE_NORMAL foldrSM #-}
foldrSM :: Monad m
    => (m a -> StreamK m b -> StreamK m b)
    -> StreamK m b
    -> StreamK m a
    -> StreamK m b
foldrSM :: forall (m :: * -> *) a b.
Monad m =>
(m a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrSM = (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (m a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
forall (m :: * -> *) b a.
Monad m =>
(forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (m a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
foldrSMWith State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream

-- {-# RULES "foldrSM/id"     foldrSM consM nil = \x -> x #-}
{-# RULES "foldrSM/nil"    forall k z.   foldrSM k z nil  = z #-}
{-# RULES "foldrSM/single" forall k z x. foldrSM k z (fromEffect x) = k x z #-}
-- {-# RULES "foldrSM/app" [1]
--  forall ys. foldrSM consM ys = \xs -> xs `conjoin` ys #-}

-- Like foldrSM but sharing the SVar state within the recostructed stream.
{-# INLINE_NORMAL foldrSMShared #-}
foldrSMShared :: Monad m
    => (m a -> StreamK m b -> StreamK m b)
    -> StreamK m b
    -> StreamK m a
    -> StreamK m b
foldrSMShared :: forall (m :: * -> *) a b.
Monad m =>
(m a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrSMShared = (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (m a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
forall (m :: * -> *) b a.
Monad m =>
(forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r)
 -> (b -> m r)
 -> m r
 -> StreamK m b
 -> m r)
-> (m a -> StreamK m b -> StreamK m b)
-> StreamK m b
-> StreamK m a
-> StreamK m b
foldrSMWith State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall r.
State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared

-- {-# RULES "foldrSM/id"     foldrSM consM nil = \x -> x #-}
{-# RULES "foldrSMShared/nil"
    forall k z. foldrSMShared k z nil = z #-}
{-# RULES "foldrSMShared/single"
    forall k z x. foldrSMShared k z (fromEffect x) = k x z #-}
-- {-# RULES "foldrSM/app" [1]
--  forall ys. foldrSM consM ys = \xs -> xs `conjoin` ys #-}

-------------------------------------------------------------------------------
-- build
-------------------------------------------------------------------------------

{-# INLINE_NORMAL build #-}
build :: forall m a. (forall b. (a -> b -> b) -> b -> b) -> StreamK m a
build :: forall (m :: * -> *) a.
(forall b. (a -> b -> b) -> b -> b) -> StreamK m a
build forall b. (a -> b -> b) -> b -> b
g = (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
forall b. (a -> b -> b) -> b -> b
g a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons StreamK m a
forall (m :: * -> *) a. StreamK m a
nil

{-# RULES "foldrM/build"
    forall k z (g :: forall b. (a -> b -> b) -> b -> b).
    foldrM k z (build g) = g k z #-}

{-# RULES "foldrS/build"
      forall k z (g :: forall b. (a -> b -> b) -> b -> b).
      foldrS k z (build g) = g k z #-}

{-# RULES "foldrS/cons/build"
      forall k z x (g :: forall b. (a -> b -> b) -> b -> b).
      foldrS k z (x `cons` build g) = k x (g k z) #-}

{-# RULES "foldrSShared/build"
      forall k z (g :: forall b. (a -> b -> b) -> b -> b).
      foldrSShared k z (build g) = g k z #-}

{-# RULES "foldrSShared/cons/build"
      forall k z x (g :: forall b. (a -> b -> b) -> b -> b).
      foldrSShared k z (x `cons` build g) = k x (g k z) #-}

-- build a stream by applying cons and nil to a build function
{-# INLINE_NORMAL buildS #-}
buildS ::
       ((a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
    -> StreamK m a
buildS :: forall a (m :: * -> *).
((a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a
buildS (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g = (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons StreamK m a
forall (m :: * -> *) a. StreamK m a
nil

{-# RULES "foldrS/buildS"
      forall k z
        (g :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
      foldrS k z (buildS g) = g k z #-}

{-# RULES "foldrS/cons/buildS"
      forall k z x
        (g :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
      foldrS k z (x `cons` buildS g) = k x (g k z) #-}

{-# RULES "foldrSShared/buildS"
      forall k z
        (g :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
      foldrSShared k z (buildS g) = g k z #-}

{-# RULES "foldrSShared/cons/buildS"
      forall k z x
        (g :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
      foldrSShared k z (x `cons` buildS g) = k x (g k z) #-}

-- build a stream by applying consM and nil to a build function
{-# INLINE_NORMAL buildSM #-}
buildSM :: Monad m
    => ((m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
    -> StreamK m a
buildSM :: forall (m :: * -> *) a.
Monad m =>
((m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a
buildSM (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g = (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
consM StreamK m a
forall (m :: * -> *) a. StreamK m a
nil

{-# RULES "foldrSM/buildSM"
     forall k z
        (g :: (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
     foldrSM k z (buildSM g) = g k z #-}

{-# RULES "foldrSMShared/buildSM"
     forall k z
        (g :: (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
     foldrSMShared k z (buildSM g) = g k z #-}

-- Disabled because this may not fire as consM is a class Op
{-
{-# RULES "foldrS/consM/buildSM"
      forall k z x (g :: (m a -> t m a -> t m a) -> t m a -> t m a)
    . foldrSM k z (x `consM` buildSM g)
    = k x (g k z)
#-}
-}

-- Build using monadic build functions (continuations) instead of
-- reconstructing a stream.
{-# INLINE_NORMAL buildM #-}
buildM :: Monad m
    => (forall r. (a -> StreamK m a -> m r)
        -> (a -> m r)
        -> m r
        -> m r
       )
    -> StreamK m a
buildM :: forall (m :: * -> *) a.
Monad m =>
(forall r. (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
buildM forall r. (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
g = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
    (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
forall r. (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
g (\a
a StreamK m a
r -> State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return a
a m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
`consM` StreamK m a
r)) a -> m r
sng m r
stp

-- | Like 'buildM' but shares the SVar state across computations.
{-# INLINE_NORMAL sharedMWith #-}
sharedMWith :: Monad m
    => (m a -> StreamK m a -> StreamK m a)
    -> (forall r. (a -> StreamK m a -> m r)
        -> (a -> m r)
        -> m r
        -> m r
       )
    -> StreamK m a
sharedMWith :: forall (m :: * -> *) a.
Monad m =>
(m a -> StreamK m a -> StreamK m a)
-> (forall r.
    (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
sharedMWith m a -> StreamK m a -> StreamK m a
cns forall r. (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
g = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
    (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
forall r. (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r
g (\a
a StreamK m a
r -> State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return a
a m a -> StreamK m a -> StreamK m a
`cns` StreamK m a
r)) a -> m r
sng m r
stp

-------------------------------------------------------------------------------
-- augment
-------------------------------------------------------------------------------

{-# INLINE_NORMAL augmentS #-}
augmentS ::
       ((a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
    -> StreamK m a
    -> StreamK m a
augmentS :: forall a (m :: * -> *).
((a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a -> StreamK m a
augmentS (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g StreamK m a
xs = (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons StreamK m a
xs

{-# RULES "augmentS/nil"
    forall (g :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
    augmentS g nil = buildS g
    #-}

{-# RULES "foldrS/augmentS"
    forall k z xs
        (g :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
    foldrS k z (augmentS g xs) = g k (foldrS k z xs)
    #-}

{-# RULES "augmentS/buildS"
    forall (g :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
           (h :: (a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
    augmentS g (buildS h) = buildS (\c n -> g c (h c n))
    #-}

{-# INLINE_NORMAL augmentSM #-}
augmentSM :: Monad m =>
       ((m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
    -> StreamK m a -> StreamK m a
augmentSM :: forall (m :: * -> *) a.
Monad m =>
((m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a -> StreamK m a
augmentSM (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g StreamK m a
xs = (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a
g m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
consM StreamK m a
xs

{-# RULES "augmentSM/nil"
    forall
        (g :: (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
    augmentSM g nil = buildSM g
    #-}

{-# RULES "foldrSM/augmentSM"
    forall k z xs
        (g :: (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
    foldrSM k z (augmentSM g xs) = g k (foldrSM k z xs)
    #-}

{-# RULES "augmentSM/buildSM"
    forall
        (g :: (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
        (h :: (m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a).
    augmentSM g (buildSM h) = buildSM (\c n -> g c (h c n))
    #-}

-------------------------------------------------------------------------------
-- Experimental foldrM/buildM
-------------------------------------------------------------------------------

-- | Lazy right fold with a monadic step function.
{-# INLINE_NORMAL foldrM #-}
foldrM :: (a -> m b -> m b) -> m b -> StreamK m a -> m b
foldrM :: forall a (m :: * -> *) b.
(a -> m b -> m b) -> m b -> StreamK m a -> m b
foldrM a -> m b -> m b
step m b
acc StreamK m a
m = StreamK m a -> m b
go StreamK m a
m
    where
    go :: StreamK m a -> m b
go StreamK m a
m1 =
        let stop :: m b
stop = m b
acc
            single :: a -> m b
single a
a = a -> m b -> m b
step a
a m b
acc
            yieldk :: a -> StreamK m a -> m b
yieldk a
a StreamK m a
r = a -> m b -> m b
step a
a (StreamK m a -> m b
go StreamK m a
r)
        in State StreamK m a
-> (a -> StreamK m a -> m b)
-> (a -> m b)
-> m b
-> StreamK m a
-> m b
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState a -> StreamK m a -> m b
yieldk a -> m b
single m b
stop StreamK m a
m1

{-# INLINE_NORMAL foldrMKWith #-}
foldrMKWith
    :: (State StreamK m a
        -> (a -> StreamK m a -> m b)
        -> (a -> m b)
        -> m b
        -> StreamK m a
        -> m b)
    -> (a -> m b -> m b)
    -> m b
    -> ((a -> StreamK m a -> m b) -> (a -> m b) -> m b -> m b)
    -> m b
foldrMKWith :: forall (m :: * -> *) a b.
(State StreamK m a
 -> (a -> StreamK m a -> m b)
 -> (a -> m b)
 -> m b
 -> StreamK m a
 -> m b)
-> (a -> m b -> m b)
-> m b
-> ((a -> StreamK m a -> m b) -> (a -> m b) -> m b -> m b)
-> m b
foldrMKWith State StreamK m a
-> (a -> StreamK m a -> m b)
-> (a -> m b)
-> m b
-> StreamK m a
-> m b
f a -> m b -> m b
step m b
acc = ((a -> StreamK m a -> m b) -> (a -> m b) -> m b -> m b) -> m b
go
    where
    go :: ((a -> StreamK m a -> m b) -> (a -> m b) -> m b -> m b) -> m b
go (a -> StreamK m a -> m b) -> (a -> m b) -> m b -> m b
k =
        let stop :: m b
stop = m b
acc
            single :: a -> m b
single a
a = a -> m b -> m b
step a
a m b
acc
            yieldk :: a -> StreamK m a -> m b
yieldk a
a StreamK m a
r = a -> m b -> m b
step a
a (((a -> StreamK m a -> m b) -> (a -> m b) -> m b -> m b) -> m b
go (\a -> StreamK m a -> m b
yld a -> m b
sng m b
stp -> State StreamK m a
-> (a -> StreamK m a -> m b)
-> (a -> m b)
-> m b
-> StreamK m a
-> m b
f State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState a -> StreamK m a -> m b
yld a -> m b
sng m b
stp StreamK m a
r))
        in (a -> StreamK m a -> m b) -> (a -> m b) -> m b -> m b
k a -> StreamK m a -> m b
yieldk a -> m b
single m b
stop

{-
{-# RULES "foldrM/buildS"
      forall k z (g :: (a -> t m a -> t m a) -> t m a -> t m a)
    . foldrM k z (buildS g)
    = g k z
#-}
-}
-- XXX in which case will foldrM/buildM fusion be useful?
{-# RULES "foldrM/buildM"
    forall step acc (g :: (forall r.
           (a -> StreamK m a -> m r)
        -> (a -> m r)
        -> m r
        -> m r
       )).
    foldrM step acc (buildM g) = foldrMKWith foldStream step acc g
    #-}

{-
{-# RULES "foldrM/sharedM"
    forall step acc (g :: (forall r.
           (a -> StreamK m a -> m r)
        -> (a -> m r)
        -> m r
        -> m r
       )).
    foldrM step acc (sharedM g) = foldrMKWith foldStreamShared step acc g
    #-}
-}

------------------------------------------------------------------------------
-- Left fold
------------------------------------------------------------------------------

-- | Strict left fold with an extraction function. Like the standard strict
-- left fold, but applies a user supplied extraction function (the third
-- argument) to the folded value at the end. This is designed to work with the
-- @foldl@ library. The suffix @x@ is a mnemonic for extraction.
--
-- Note that the accumulator is always evaluated including the initial value.
{-# INLINE foldlx' #-}
foldlx' :: forall m a b x. Monad m
    => (x -> a -> x) -> x -> (x -> b) -> StreamK m a -> m b
foldlx' :: forall (m :: * -> *) a b x.
Monad m =>
(x -> a -> x) -> x -> (x -> b) -> StreamK m a -> m b
foldlx' x -> a -> x
step x
begin x -> b
done =
    (x -> a -> m x) -> m x -> (x -> m b) -> StreamK m a -> m b
forall (m :: * -> *) x a b.
Monad m =>
(x -> a -> m x) -> m x -> (x -> m b) -> StreamK m a -> m b
foldlMx' (\x
x a
a -> x -> m x
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (x -> a -> x
step x
x a
a)) (x -> m x
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return x
begin) (b -> m b
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (b -> m b) -> (x -> b) -> x -> m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. x -> b
done)

-- | Strict left associative fold.
{-# INLINE foldl' #-}
foldl' :: Monad m => (b -> a -> b) -> b -> StreamK m a -> m b
foldl' :: forall (m :: * -> *) b a.
Monad m =>
(b -> a -> b) -> b -> StreamK m a -> m b
foldl' b -> a -> b
step b
begin = (b -> a -> b) -> b -> (b -> b) -> StreamK m a -> m b
forall (m :: * -> *) a b x.
Monad m =>
(x -> a -> x) -> x -> (x -> b) -> StreamK m a -> m b
foldlx' b -> a -> b
step b
begin b -> b
forall a. a -> a
id

-- | Like 'foldx', but with a monadic step function.
{-# INLINE foldlMx' #-}
foldlMx' :: Monad m
    => (x -> a -> m x) -> m x -> (x -> m b) -> StreamK m a -> m b
foldlMx' :: forall (m :: * -> *) x a b.
Monad m =>
(x -> a -> m x) -> m x -> (x -> m b) -> StreamK m a -> m b
foldlMx' x -> a -> m x
step m x
begin x -> m b
done StreamK m a
stream =
    -- Note: Unrolling improves the last benchmark significantly.
    let stop :: m b
stop = m x
begin m x -> (x -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= x -> m b
done
        single :: a -> m b
single a
a = m x
begin m x -> (x -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \x
x -> x -> a -> m x
step x
x a
a m x -> (x -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= x -> m b
done
        yieldk :: a -> StreamK m a -> m b
yieldk a
a StreamK m a
r = m x
begin m x -> (x -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \x
x -> x -> a -> m x
step x
x a
a m x -> (x -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= StreamK m a -> x -> m b
go StreamK m a
r
     in State StreamK m a
-> (a -> StreamK m a -> m b)
-> (a -> m b)
-> m b
-> StreamK m a
-> m b
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState a -> StreamK m a -> m b
yieldk a -> m b
single m b
stop StreamK m a
stream

    where

    go :: StreamK m a -> x -> m b
go StreamK m a
m1 !x
acc =
        let stop :: m b
stop = x -> m b
done (x -> m b) -> x -> m b
forall a b. (a -> b) -> a -> b
$! x
acc
            single :: a -> m b
single a
a = x -> a -> m x
step x
acc a
a m x -> (x -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= x -> m b
done
            yieldk :: a -> StreamK m a -> m b
yieldk a
a StreamK m a
r = x -> a -> m x
step x
acc a
a m x -> (x -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= StreamK m a -> x -> m b
go StreamK m a
r
         in State StreamK m a
-> (a -> StreamK m a -> m b)
-> (a -> m b)
-> m b
-> StreamK m a
-> m b
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState a -> StreamK m a -> m b
yieldk a -> m b
single m b
stop StreamK m a
m1

-- | Like 'foldl'' but with a monadic step function.
{-# INLINE foldlM' #-}
foldlM' :: Monad m => (b -> a -> m b) -> m b -> StreamK m a -> m b
foldlM' :: forall (m :: * -> *) b a.
Monad m =>
(b -> a -> m b) -> m b -> StreamK m a -> m b
foldlM' b -> a -> m b
step m b
begin = (b -> a -> m b) -> m b -> (b -> m b) -> StreamK m a -> m b
forall (m :: * -> *) x a b.
Monad m =>
(x -> a -> m x) -> m x -> (x -> m b) -> StreamK m a -> m b
foldlMx' b -> a -> m b
step m b
begin b -> m b
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return

------------------------------------------------------------------------------
-- Specialized folds
------------------------------------------------------------------------------

-- XXX use foldrM to implement folds where possible
-- XXX This (commented) definition of drain and mapM_ perform much better on
-- some benchmarks but worse on others. Need to investigate why, maybe there is
-- an optimization opportunity that we can exploit.
-- drain = foldrM (\_ xs -> return () >> xs) (return ())

--
-- > drain = foldl' (\_ _ -> ()) ()
-- > drain = mapM_ (\_ -> return ())
{-# INLINE drain #-}
drain :: Monad m => StreamK m a -> m ()
drain :: forall (m :: * -> *) a. Monad m => StreamK m a -> m ()
drain = (a -> m () -> m ()) -> m () -> StreamK m a -> m ()
forall a (m :: * -> *) b.
(a -> m b -> m b) -> m b -> StreamK m a -> m b
foldrM (\a
_ m ()
xs -> m ()
xs) (() -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ())
{-
drain = go
    where
    go m1 =
        let stop = return ()
            single _ = return ()
            yieldk _ r = go r
         in foldStream defState yieldk single stop m1
-}

{-# INLINE null #-}
null :: Monad m => StreamK m a -> m Bool
-- null = foldrM (\_ _ -> return True) (return False)
null :: forall (m :: * -> *) a. Monad m => StreamK m a -> m Bool
null StreamK m a
m =
    let stop :: m Bool
stop      = Bool -> m Bool
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Bool
True
        single :: p -> m Bool
single p
_  = Bool -> m Bool
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Bool
False
        yieldk :: p -> p -> m Bool
yieldk p
_ p
_ = Bool -> m Bool
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Bool
False
    in State StreamK m a
-> (a -> StreamK m a -> m Bool)
-> (a -> m Bool)
-> m Bool
-> StreamK m a
-> m Bool
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState a -> StreamK m a -> m Bool
forall {m :: * -> *} {p} {p}. Monad m => p -> p -> m Bool
yieldk a -> m Bool
forall {m :: * -> *} {p}. Monad m => p -> m Bool
single m Bool
stop StreamK m a
m

------------------------------------------------------------------------------
-- Semigroup
------------------------------------------------------------------------------

infixr 6 `append`

-- | Unlike the fused "Streamly.Data.Stream" append, StreamK append can be used
-- at scale, recursively, with linear performance:
--
-- >>> cycle xs = let ys = xs `StreamK.append` ys in ys
--
-- 'concatMapWith' 'append' (same as concatMap) flattens a stream of streams in a
-- depth-first manner i.e. it yields each stream fully and then the next and so
-- on. Given a stream of three streams:
--
-- @
-- 1. [1,2,3]
-- 2. [4,5,6]
-- 3. [7,8,9]
-- @
--
-- The resulting stream will be @[1,2,3,4,5,6,7,8,9]@.
--
-- Best used in a right associative manner.
--
{-# INLINE append #-}
append :: StreamK m a -> StreamK m a -> StreamK m a
-- XXX This doubles the time of toNullAp benchmark, may not be fusing properly
-- serial xs ys = augmentS (\c n -> foldrS c n xs) ys
append :: forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
append StreamK m a
m1 StreamK m a
m2 =
    (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
        let stop :: m r
stop       = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp StreamK m a
m2
            single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a StreamK m a
m2
            yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (StreamK m a -> StreamK m a
go StreamK m a
r)
        in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m1

    where

    go :: StreamK m a -> StreamK m a
go StreamK m a
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let stop :: m r
stop       = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp StreamK m a
m2
                single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a StreamK m a
m2
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (StreamK m a -> StreamK m a
go StreamK m a
r)
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m

-- join/merge/append streams depending on consM
{-# INLINE conjoin #-}
conjoin :: Monad m => StreamK m a -> StreamK m a -> StreamK m a
conjoin :: forall (m :: * -> *) a.
Monad m =>
StreamK m a -> StreamK m a -> StreamK m a
conjoin StreamK m a
xs = ((m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
((m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a -> StreamK m a
augmentSM (\m a -> StreamK m a -> StreamK m a
c StreamK m a
n -> (m a -> StreamK m a -> StreamK m a)
-> StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a b.
Monad m =>
(m a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrSM m a -> StreamK m a -> StreamK m a
c StreamK m a
n StreamK m a
xs)

instance Semigroup (StreamK m a) where
    <> :: StreamK m a -> StreamK m a -> StreamK m a
(<>) = StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
append

------------------------------------------------------------------------------
-- Monoid
------------------------------------------------------------------------------

instance Monoid (StreamK m a) where
    mempty :: StreamK m a
mempty = StreamK m a
forall (m :: * -> *) a. StreamK m a
nil
    mappend :: StreamK m a -> StreamK m a -> StreamK m a
mappend = StreamK m a -> StreamK m a -> StreamK m a
forall a. Semigroup a => a -> a -> a
(<>)

-------------------------------------------------------------------------------
-- Functor
-------------------------------------------------------------------------------

-- IMPORTANT: This is eta expanded on purpose. This should not be eta
-- reduced. This will cause a lot of regressions, probably because of some
-- rewrite rules. Ideally don't run hlint on this file.
{-# INLINE_LATE mapFB #-}
mapFB :: forall b m a.
       (b -> StreamK m b -> StreamK m b)
    -> (a -> b)
    -> a
    -> StreamK m b
    -> StreamK m b
mapFB :: forall b (m :: * -> *) a.
(b -> StreamK m b -> StreamK m b)
-> (a -> b) -> a -> StreamK m b -> StreamK m b
mapFB b -> StreamK m b -> StreamK m b
c a -> b
f = \a
x StreamK m b
ys -> b -> StreamK m b -> StreamK m b
c (a -> b
f a
x) StreamK m b
ys

{-# RULES
"mapFB/mapFB" forall c f g. mapFB (mapFB c f) g = mapFB c (f . g)
"mapFB/id"    forall c.     mapFB c (\x -> x)   = c
    #-}

{-# INLINE map #-}
map :: (a -> b) -> StreamK m a -> StreamK m b
map :: forall a b (m :: * -> *). (a -> b) -> StreamK m a -> StreamK m b
map a -> b
f StreamK m a
xs = ((b -> StreamK m b -> StreamK m b) -> StreamK m b -> StreamK m b)
-> StreamK m b
forall a (m :: * -> *).
((a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a
buildS (\b -> StreamK m b -> StreamK m b
c StreamK m b
n -> (a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
forall a (m :: * -> *) b.
(a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrS ((b -> StreamK m b -> StreamK m b)
-> (a -> b) -> a -> StreamK m b -> StreamK m b
forall b (m :: * -> *) a.
(b -> StreamK m b -> StreamK m b)
-> (a -> b) -> a -> StreamK m b -> StreamK m b
mapFB b -> StreamK m b -> StreamK m b
c a -> b
f) StreamK m b
n StreamK m a
xs)

-- XXX This definition might potentially be more efficient, but the cost in the
-- benchmark is dominated by unfoldrM cost so we cannot correctly determine
-- differences in the mapping cost. We should perhaps deduct the cost of
-- unfoldrM from the benchmarks and then compare.
{-
map f m = go m
    where
        go m1 =
            mkStream $ \st yld sng stp ->
            let single     = sng . f
                yieldk a r = yld (f a) (go r)
            in foldStream (adaptState st) yieldk single stp m1
-}

{-# INLINE_LATE mapMFB #-}
mapMFB :: Monad m => (m b -> t m b -> t m b) -> (a -> m b) -> m a -> t m b -> t m b
mapMFB :: forall (m :: * -> *) b (t :: (* -> *) -> * -> *) a.
Monad m =>
(m b -> t m b -> t m b) -> (a -> m b) -> m a -> t m b -> t m b
mapMFB m b -> t m b -> t m b
c a -> m b
f m a
x = m b -> t m b -> t m b
c (m a
x m a -> (a -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= a -> m b
f)

{-# RULES
    "mapMFB/mapMFB" forall c f g. mapMFB (mapMFB c f) g = mapMFB c (f >=> g)
    #-}
-- XXX These rules may never fire because pure/return type class rules will
-- fire first.
{-
"mapMFB/pure"    forall c.     mapMFB c (\x -> pure x)   = c
"mapMFB/return"  forall c.     mapMFB c (\x -> return x) = c
-}

-- This is experimental serial version supporting fusion.
--
-- XXX what if we do not want to fuse two concurrent mapMs?
-- XXX we can combine two concurrent mapM only if the SVar is of the same type
-- So for now we use it only for serial streams.
-- XXX fusion would be easier for monomoprhic stream types.
-- {-# RULES "mapM serial" mapM = mapMSerial #-}
{-# INLINE mapMSerial #-}
mapMSerial :: Monad m => (a -> m b) -> StreamK m a -> StreamK m b
mapMSerial :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> StreamK m a -> StreamK m b
mapMSerial a -> m b
f StreamK m a
xs = ((m b -> StreamK m b -> StreamK m b) -> StreamK m b -> StreamK m b)
-> StreamK m b
forall (m :: * -> *) a.
Monad m =>
((m a -> StreamK m a -> StreamK m a) -> StreamK m a -> StreamK m a)
-> StreamK m a
buildSM (\m b -> StreamK m b -> StreamK m b
c StreamK m b
n -> (m a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
forall (m :: * -> *) a b.
Monad m =>
(m a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrSMShared ((m b -> StreamK m b -> StreamK m b)
-> (a -> m b) -> m a -> StreamK m b -> StreamK m b
forall (m :: * -> *) b (t :: (* -> *) -> * -> *) a.
Monad m =>
(m b -> t m b -> t m b) -> (a -> m b) -> m a -> t m b -> t m b
mapMFB m b -> StreamK m b -> StreamK m b
c a -> m b
f) StreamK m b
n StreamK m a
xs)

{-# INLINE mapMWith #-}
mapMWith ::
       (m b -> StreamK m b -> StreamK m b)
    -> (a -> m b)
    -> StreamK m a
    -> StreamK m b
mapMWith :: forall (m :: * -> *) b a.
(m b -> StreamK m b -> StreamK m b)
-> (a -> m b) -> StreamK m a -> StreamK m b
mapMWith m b -> StreamK m b -> StreamK m b
cns a -> m b
f = (a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
forall a (m :: * -> *) b.
(a -> StreamK m b -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldrSShared (\a
x StreamK m b
xs -> a -> m b
f a
x m b -> StreamK m b -> StreamK m b
`cns` StreamK m b
xs) StreamK m b
forall (m :: * -> *) a. StreamK m a
nil

{-
-- See note under map definition above.
mapMWith cns f = go
    where
    go m1 = mkStream $ \st yld sng stp ->
        let single a  = f a >>= sng
            yieldk a r = foldStreamShared st yld sng stp $ f a `cns` go r
         in foldStream (adaptState st) yieldk single stp m1
-}

-- XXX in fact use the Stream type everywhere and only use polymorphism in the
-- high level modules/prelude.
instance Monad m => Functor (StreamK m) where
    fmap :: forall a b. (a -> b) -> StreamK m a -> StreamK m b
fmap = (a -> b) -> StreamK m a -> StreamK m b
forall a b (m :: * -> *). (a -> b) -> StreamK m a -> StreamK m b
map

-- $smapM_Notes
--
-- The stateful step function can be simplified to @(s -> a -> m b)@ to provide
-- a read-only environment. However, that would just be 'mapM'.
--
-- The initial action could be @m (s, Maybe b)@, and we can also add a final
-- action @s -> m (Maybe b)@. This can be used to get pre/post scan like
-- functionality and also to flush the state in the end like scanlMAfter'.
-- We can also use it along with a fusible version of bracket to get
-- scanlMAfter' like functionality. See issue #677.
--
-- This can be further generalized to a type similar to Fold/Parser, giving it
-- filtering and parsing capability as well (this is in fact equivalent to
-- parseMany):
--
-- smapM :: (s -> a -> m (Step s b)) -> m s -> t m a -> t m b
--

-- | A stateful map aka scan but with a slight difference.
--
-- This is similar to a scan except that instead of emitting the state it emits
-- a separate result. This is also similar to mapAccumL but does not return the
-- final value of the state.
--
-- Separation of state from the output makes it easier to think in terms of a
-- shared state, and also makes it easier to keep the state fully strict and
-- the output lazy.
--
-- /Unimplemented/
--
{-# INLINE mapMAccum #-}
mapMAccum :: -- Monad m =>
       (s -> a -> m (s, b))
    -> m s
    -> StreamK m a
    -> StreamK m b
mapMAccum :: forall s a (m :: * -> *) b.
(s -> a -> m (s, b)) -> m s -> StreamK m a -> StreamK m b
mapMAccum s -> a -> m (s, b)
_step m s
_initial StreamK m a
_stream = StreamK m b
forall a. HasCallStack => a
undefined
{-
    -- XXX implement this directly instead of using scanlM'
    -- Once we have postscanlM' with monadic initial we can use this code
    -- let r = postscanlM'
    --              (\(s, _) a -> step s a)
    --              (fmap (,undefined) initial)
    --              stream
    let r = postscanlM'
                (\(s, _) a -> step s a)
                (fmap (,undefined) initial)
                stream
     in map snd r
-}

-- | Like 'concatMapWith' but carries a state which can be used to share
-- information across multiple steps of concat.
--
-- @
-- concatSmapMWith combine f initial = concatMapWith combine id . smapM f initial
-- @
--
-- /Unimplemented/
--
{-# INLINE concatMapMAccum #-}
concatMapMAccum :: -- (Monad m) =>
       (StreamK m b -> StreamK m b -> StreamK m b)
    -> (s -> a -> m (s, StreamK m b))
    -> m s
    -> StreamK m a
    -> StreamK m b
concatMapMAccum :: forall (m :: * -> *) b s a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (s -> a -> m (s, StreamK m b))
-> m s
-> StreamK m a
-> StreamK m b
concatMapMAccum StreamK m b -> StreamK m b -> StreamK m b
combine s -> a -> m (s, StreamK m b)
f m s
initial =
    (StreamK m b -> StreamK m b -> StreamK m b)
-> (StreamK m b -> StreamK m b)
-> StreamK m (StreamK m b)
-> StreamK m b
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
concatMapWith StreamK m b -> StreamK m b -> StreamK m b
combine StreamK m b -> StreamK m b
forall a. a -> a
id (StreamK m (StreamK m b) -> StreamK m b)
-> (StreamK m a -> StreamK m (StreamK m b))
-> StreamK m a
-> StreamK m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (s -> a -> m (s, StreamK m b))
-> m s -> StreamK m a -> StreamK m (StreamK m b)
forall s a (m :: * -> *) b.
(s -> a -> m (s, b)) -> m s -> StreamK m a -> StreamK m b
mapMAccum s -> a -> m (s, StreamK m b)
f m s
initial

------------------------------------------------------------------------------
-- Lists
------------------------------------------------------------------------------

-- Serial streams can act like regular lists using the Identity monad

-- XXX Show instance is 10x slower compared to read, we can do much better.
-- The list show instance itself is really slow.

-- XXX The default definitions of "<" in the Ord instance etc. do not perform
-- well, because they do not get inlined. Need to add INLINE in Ord class in
-- base?

instance IsList (StreamK Identity a) where
    type (Item (StreamK Identity a)) = a

    {-# INLINE fromList #-}
    fromList :: [Item (StreamK Identity a)] -> StreamK Identity a
fromList = [a] -> StreamK Identity a
[Item (StreamK Identity a)] -> StreamK Identity a
forall (f :: * -> *) a (m :: * -> *).
Foldable f =>
f a -> StreamK m a
fromFoldable

    {-# INLINE toList #-}
    toList :: StreamK Identity a -> [Item (StreamK Identity a)]
toList = (a -> [a] -> [a]) -> [a] -> StreamK Identity a -> [a]
forall a b. (a -> b -> b) -> b -> StreamK Identity a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
Data.Foldable.foldr (:) []

-- XXX Fix these
{-
instance Eq a => Eq (StreamK Identity a) where
    {-# INLINE (==) #-}
    (==) xs ys = runIdentity $ eqBy (==) xs ys

instance Ord a => Ord (StreamK Identity a) where
    {-# INLINE compare #-}
    compare xs ys = runIdentity $ cmpBy compare xs ys

    {-# INLINE (<) #-}
    x < y =
        case compare x y of
            LT -> True
            _ -> False

    {-# INLINE (<=) #-}
    x <= y =
        case compare x y of
            GT -> False
            _ -> True

    {-# INLINE (>) #-}
    x > y =
        case compare x y of
            GT -> True
            _ -> False

    {-# INLINE (>=) #-}
    x >= y =
        case compare x y of
            LT -> False
            _ -> True

    {-# INLINE max #-}
    max x y = if x <= y then y else x

    {-# INLINE min #-}
    min x y = if x <= y then x else y
-}

instance Show a => Show (StreamK Identity a) where
    showsPrec :: Int -> StreamK Identity a -> ShowS
showsPrec Int
p StreamK Identity a
dl = Bool -> ShowS -> ShowS
showParen (Int
p Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
> Int
10) (ShowS -> ShowS) -> ShowS -> ShowS
forall a b. (a -> b) -> a -> b
$
        String -> ShowS
showString String
"fromList " ShowS -> ShowS -> ShowS
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [a] -> ShowS
forall a. Show a => a -> ShowS
shows (StreamK Identity a -> [Item (StreamK Identity a)]
forall l. IsList l => l -> [Item l]
toList StreamK Identity a
dl)

instance Read a => Read (StreamK Identity a) where
    readPrec :: ReadPrec (StreamK Identity a)
readPrec = ReadPrec (StreamK Identity a) -> ReadPrec (StreamK Identity a)
forall a. ReadPrec a -> ReadPrec a
parens (ReadPrec (StreamK Identity a) -> ReadPrec (StreamK Identity a))
-> ReadPrec (StreamK Identity a) -> ReadPrec (StreamK Identity a)
forall a b. (a -> b) -> a -> b
$ Int
-> ReadPrec (StreamK Identity a) -> ReadPrec (StreamK Identity a)
forall a. Int -> ReadPrec a -> ReadPrec a
prec Int
10 (ReadPrec (StreamK Identity a) -> ReadPrec (StreamK Identity a))
-> ReadPrec (StreamK Identity a) -> ReadPrec (StreamK Identity a)
forall a b. (a -> b) -> a -> b
$ do
        Ident String
"fromList" <- ReadPrec Lexeme
lexP
        [a] -> StreamK Identity a
[Item (StreamK Identity a)] -> StreamK Identity a
forall l. IsList l => [Item l] -> l
GHC.Exts.fromList ([a] -> StreamK Identity a)
-> ReadPrec [a] -> ReadPrec (StreamK Identity a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ReadPrec [a]
forall a. Read a => ReadPrec a
readPrec

    readListPrec :: ReadPrec [StreamK Identity a]
readListPrec = ReadPrec [StreamK Identity a]
forall a. Read a => ReadPrec [a]
readListPrecDefault

instance (a ~ Char) => IsString (StreamK Identity a) where
    {-# INLINE fromString #-}
    fromString :: String -> StreamK Identity a
fromString = String -> StreamK Identity a
[Item (StreamK Identity a)] -> StreamK Identity a
forall l. IsList l => [Item l] -> l
GHC.Exts.fromList

-------------------------------------------------------------------------------
-- Foldable
-------------------------------------------------------------------------------

-- | Lazy right associative fold.
{-# INLINE foldr #-}
foldr :: Monad m => (a -> b -> b) -> b -> StreamK m a -> m b
foldr :: forall (m :: * -> *) a b.
Monad m =>
(a -> b -> b) -> b -> StreamK m a -> m b
foldr a -> b -> b
step b
acc = (a -> m b -> m b) -> m b -> StreamK m a -> m b
forall a (m :: * -> *) b.
(a -> m b -> m b) -> m b -> StreamK m a -> m b
foldrM (\a
x m b
xs -> m b
xs m b -> (b -> m b) -> m b
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= \b
b -> b -> m b
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> b -> b
step a
x b
b)) (b -> m b
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return b
acc)

-- The default Foldable instance has several issues:
-- 1) several definitions do not have INLINE on them, so we provide
--    re-implementations with INLINE pragmas.
-- 2) the definitions of sum/product/maximum/minimum are inefficient as they
--    use right folds, they cannot run in constant memory. We provide
--    implementations using strict left folds here.

instance (Foldable m, Monad m) => Foldable (StreamK m) where

    {-# INLINE foldMap #-}
    foldMap :: forall m a. Monoid m => (a -> m) -> StreamK m a -> m
foldMap a -> m
f =
          m m -> m
forall m. Monoid m => m m -> m
forall (t :: * -> *) m. (Foldable t, Monoid m) => t m -> m
fold
        (m m -> m) -> (StreamK m a -> m m) -> StreamK m a -> m
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (a -> m -> m) -> m -> StreamK m a -> m m
forall (m :: * -> *) a b.
Monad m =>
(a -> b -> b) -> b -> StreamK m a -> m b
Streamly.Internal.Data.StreamK.Type.foldr (m -> m -> m
forall a. Monoid a => a -> a -> a
mappend (m -> m -> m) -> (a -> m) -> a -> m -> m
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> m
f) m
forall a. Monoid a => a
mempty

    {-# INLINE foldr #-}
    foldr :: forall a b. (a -> b -> b) -> b -> StreamK m a -> b
foldr a -> b -> b
f b
z StreamK m a
t = Endo b -> b -> b
forall a. Endo a -> a -> a
appEndo ((a -> Endo b) -> StreamK m a -> Endo b
forall m a. Monoid m => (a -> m) -> StreamK m a -> m
forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap ((b -> b) -> Endo b
forall a. (a -> a) -> Endo a
Endo ((b -> b) -> Endo b) -> (a -> b -> b) -> a -> Endo b
forall b c a. Coercible b c => (b -> c) -> (a -> b) -> a -> c
#. a -> b -> b
f) StreamK m a
t) b
z

    {-# INLINE foldl' #-}
    foldl' :: forall b a. (b -> a -> b) -> b -> StreamK m a -> b
foldl' b -> a -> b
f b
z0 StreamK m a
xs = (a -> (b -> b) -> b -> b) -> (b -> b) -> StreamK m a -> b -> b
forall a b. (a -> b -> b) -> b -> StreamK m a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
Data.Foldable.foldr a -> (b -> b) -> b -> b
forall {b}. a -> (b -> b) -> b -> b
f' b -> b
forall a. a -> a
id StreamK m a
xs b
z0
        where f' :: a -> (b -> b) -> b -> b
f' a
x b -> b
k = (b -> b) -> b -> b
forall a b. (a -> b) -> a -> b
oneShot ((b -> b) -> b -> b) -> (b -> b) -> b -> b
forall a b. (a -> b) -> a -> b
$ \b
z -> b -> b
k (b -> b) -> b -> b
forall a b. (a -> b) -> a -> b
$! b -> a -> b
f b
z a
x

    {-# INLINE length #-}
    length :: forall a. StreamK m a -> Int
length = (Int -> a -> Int) -> Int -> StreamK m a -> Int
forall b a. (b -> a -> b) -> b -> StreamK m a -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Data.Foldable.foldl' (\Int
n a
_ -> Int
n Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1) Int
0

    {-# INLINE elem #-}
    elem :: forall a. Eq a => a -> StreamK m a -> Bool
elem = (a -> Bool) -> StreamK m a -> Bool
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Bool
any ((a -> Bool) -> StreamK m a -> Bool)
-> (a -> a -> Bool) -> a -> StreamK m a -> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> a -> Bool
forall a. Eq a => a -> a -> Bool
(==)

    {-# INLINE maximum #-}
    maximum :: forall a. Ord a => StreamK m a -> a
maximum =
          a -> Maybe a -> a
forall a. a -> Maybe a -> a
fromMaybe (String -> a
forall a. String -> a
errorWithoutStackTrace String
"maximum: empty stream")
        (Maybe a -> a) -> (StreamK m a -> Maybe a) -> StreamK m a -> a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Maybe' a -> Maybe a
forall a. Maybe' a -> Maybe a
toMaybe
        (Maybe' a -> Maybe a)
-> (StreamK m a -> Maybe' a) -> StreamK m a -> Maybe a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Maybe' a -> a -> Maybe' a) -> Maybe' a -> StreamK m a -> Maybe' a
forall b a. (b -> a -> b) -> b -> StreamK m a -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Data.Foldable.foldl' Maybe' a -> a -> Maybe' a
forall {a}. Ord a => Maybe' a -> a -> Maybe' a
getMax Maybe' a
forall a. Maybe' a
Nothing'

        where

        getMax :: Maybe' a -> a -> Maybe' a
getMax Maybe' a
Nothing' a
x = a -> Maybe' a
forall a. a -> Maybe' a
Just' a
x
        getMax (Just' a
mx) a
x = a -> Maybe' a
forall a. a -> Maybe' a
Just' (a -> Maybe' a) -> a -> Maybe' a
forall a b. (a -> b) -> a -> b
$! a -> a -> a
forall a. Ord a => a -> a -> a
max a
mx a
x

    {-# INLINE minimum #-}
    minimum :: forall a. Ord a => StreamK m a -> a
minimum =
          a -> Maybe a -> a
forall a. a -> Maybe a -> a
fromMaybe (String -> a
forall a. String -> a
errorWithoutStackTrace String
"minimum: empty stream")
        (Maybe a -> a) -> (StreamK m a -> Maybe a) -> StreamK m a -> a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Maybe' a -> Maybe a
forall a. Maybe' a -> Maybe a
toMaybe
        (Maybe' a -> Maybe a)
-> (StreamK m a -> Maybe' a) -> StreamK m a -> Maybe a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Maybe' a -> a -> Maybe' a) -> Maybe' a -> StreamK m a -> Maybe' a
forall b a. (b -> a -> b) -> b -> StreamK m a -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Data.Foldable.foldl' Maybe' a -> a -> Maybe' a
forall {a}. Ord a => Maybe' a -> a -> Maybe' a
getMin Maybe' a
forall a. Maybe' a
Nothing'

        where

        getMin :: Maybe' a -> a -> Maybe' a
getMin Maybe' a
Nothing' a
x = a -> Maybe' a
forall a. a -> Maybe' a
Just' a
x
        getMin (Just' a
mn) a
x = a -> Maybe' a
forall a. a -> Maybe' a
Just' (a -> Maybe' a) -> a -> Maybe' a
forall a b. (a -> b) -> a -> b
$! a -> a -> a
forall a. Ord a => a -> a -> a
min a
mn a
x

    {-# INLINE sum #-}
    sum :: forall a. Num a => StreamK m a -> a
sum = (a -> a -> a) -> a -> StreamK m a -> a
forall b a. (b -> a -> b) -> b -> StreamK m a -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Data.Foldable.foldl' a -> a -> a
forall a. Num a => a -> a -> a
(+) a
0

    {-# INLINE product #-}
    product :: forall a. Num a => StreamK m a -> a
product = (a -> a -> a) -> a -> StreamK m a -> a
forall b a. (b -> a -> b) -> b -> StreamK m a -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
Data.Foldable.foldl' a -> a -> a
forall a. Num a => a -> a -> a
(*) a
1

-------------------------------------------------------------------------------
-- Traversable
-------------------------------------------------------------------------------

instance Traversable (StreamK Identity) where
    {-# INLINE traverse #-}
    traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> StreamK Identity a -> f (StreamK Identity b)
traverse a -> f b
f StreamK Identity a
xs =
        Identity (f (StreamK Identity b)) -> f (StreamK Identity b)
forall a. Identity a -> a
runIdentity
            (Identity (f (StreamK Identity b)) -> f (StreamK Identity b))
-> Identity (f (StreamK Identity b)) -> f (StreamK Identity b)
forall a b. (a -> b) -> a -> b
$ (a -> f (StreamK Identity b) -> f (StreamK Identity b))
-> f (StreamK Identity b)
-> StreamK Identity a
-> Identity (f (StreamK Identity b))
forall (m :: * -> *) a b.
Monad m =>
(a -> b -> b) -> b -> StreamK m a -> m b
Streamly.Internal.Data.StreamK.Type.foldr
                a -> f (StreamK Identity b) -> f (StreamK Identity b)
forall {m :: * -> *}. a -> f (StreamK m b) -> f (StreamK m b)
consA (StreamK Identity b -> f (StreamK Identity b)
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure StreamK Identity b
forall a. Monoid a => a
mempty) StreamK Identity a
xs

        where

        consA :: a -> f (StreamK m b) -> f (StreamK m b)
consA a
x f (StreamK m b)
ys = (b -> StreamK m b -> StreamK m b)
-> f b -> f (StreamK m b) -> f (StreamK m b)
forall a b c. (a -> b -> c) -> f a -> f b -> f c
forall (f :: * -> *) a b c.
Applicative f =>
(a -> b -> c) -> f a -> f b -> f c
liftA2 b -> StreamK m b -> StreamK m b
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons (a -> f b
f a
x) f (StreamK m b)
ys

-------------------------------------------------------------------------------
-- Nesting
-------------------------------------------------------------------------------

-- | Detach a stream from an SVar
{-# INLINE unShare #-}
unShare :: StreamK m a -> StreamK m a
unShare :: forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare StreamK m a
x = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
    State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp StreamK m a
x

-- XXX the function stream and value stream can run in parallel
{-# INLINE crossApplyWith #-}
crossApplyWith ::
       (StreamK m b -> StreamK m b -> StreamK m b)
    -> StreamK m (a -> b)
    -> StreamK m a
    -> StreamK m b
crossApplyWith :: forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m (a -> b) -> StreamK m a -> StreamK m b
crossApplyWith StreamK m b -> StreamK m b -> StreamK m b
par StreamK m (a -> b)
fstream StreamK m a
stream = StreamK m (a -> b) -> StreamK m b
go1 StreamK m (a -> b)
fstream

    where

    go1 :: StreamK m (a -> b) -> StreamK m b
go1 StreamK m (a -> b)
m =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                single :: (a -> b) -> m r
single a -> b
f   = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare ((a -> b) -> StreamK m a -> StreamK m b
forall a b (m :: * -> *). (a -> b) -> StreamK m a -> StreamK m b
go2 a -> b
f StreamK m a
stream)
                yieldk :: (a -> b) -> StreamK m (a -> b) -> m r
yieldk a -> b
f StreamK m (a -> b)
r = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare ((a -> b) -> StreamK m a -> StreamK m b
forall a b (m :: * -> *). (a -> b) -> StreamK m a -> StreamK m b
go2 a -> b
f StreamK m a
stream) StreamK m b -> StreamK m b -> StreamK m b
`par` StreamK m (a -> b) -> StreamK m b
go1 StreamK m (a -> b)
r
            in State StreamK m (a -> b)
-> ((a -> b) -> StreamK m (a -> b) -> m r)
-> ((a -> b) -> m r)
-> m r
-> StreamK m (a -> b)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m (a -> b)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) (a -> b) -> StreamK m (a -> b) -> m r
yieldk (a -> b) -> m r
single m r
stp StreamK m (a -> b)
m

    go2 :: (a -> a) -> StreamK m a -> StreamK m a
go2 a -> a
f StreamK m a
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let single :: a -> m r
single a
a   = a -> m r
sng (a -> a
f a
a)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld (a -> a
f a
a) ((a -> a) -> StreamK m a -> StreamK m a
go2 a -> a
f StreamK m a
r)
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (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
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stp StreamK m a
m

-- | Apply a stream of functions to a stream of values and flatten the results.
--
-- Note that the second stream is evaluated multiple times.
--
-- Definition:
--
-- >>> crossApply = StreamK.crossApplyWith StreamK.append
-- >>> crossApply = Stream.crossWith id
--
{-# INLINE crossApply #-}
crossApply ::
       StreamK m (a -> b)
    -> StreamK m a
    -> StreamK m b
crossApply :: forall (m :: * -> *) a b.
StreamK m (a -> b) -> StreamK m a -> StreamK m b
crossApply StreamK m (a -> b)
fstream StreamK m a
stream = StreamK m (a -> b) -> StreamK m b
forall {a}. StreamK m (a -> a) -> StreamK m a
go1 StreamK m (a -> b)
fstream

    where

    go1 :: StreamK m (a -> a) -> StreamK m a
go1 StreamK m (a -> a)
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let foldShared :: StreamK m a -> m r
foldShared = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp
                single :: (a -> a) -> m r
single a -> a
f   = StreamK m a -> m r
foldShared (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ (a -> a) -> StreamK m a -> StreamK m a
forall a b (m :: * -> *). (a -> b) -> StreamK m a -> StreamK m b
go3 a -> a
f StreamK m a
stream
                yieldk :: (a -> a) -> StreamK m (a -> a) -> m r
yieldk a -> a
f StreamK m (a -> a)
r = StreamK m a -> m r
foldShared (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ (a -> a) -> StreamK m (a -> a) -> StreamK m a -> StreamK m a
go2 a -> a
f StreamK m (a -> a)
r StreamK m a
stream
            in State StreamK m (a -> a)
-> ((a -> a) -> StreamK m (a -> a) -> m r)
-> ((a -> a) -> m r)
-> m r
-> StreamK m (a -> a)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m a -> State StreamK m (a -> a)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m a
st) (a -> a) -> StreamK m (a -> a) -> m r
yieldk (a -> a) -> m r
single m r
stp StreamK m (a -> a)
m

    go2 :: (a -> a) -> StreamK m (a -> a) -> StreamK m a -> StreamK m a
go2 a -> a
f StreamK m (a -> a)
r1 StreamK m a
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let foldShared :: StreamK m a -> m r
foldShared = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp
                stop :: m r
stop = StreamK m a -> m r
foldShared (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m (a -> a) -> StreamK m a
go1 StreamK m (a -> a)
r1
                single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld (a -> a
f a
a) (StreamK m (a -> a) -> StreamK m a
go1 StreamK m (a -> a)
r1)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld (a -> a
f a
a) ((a -> a) -> StreamK m (a -> a) -> StreamK m a -> StreamK m a
go2 a -> a
f StreamK m (a -> a)
r1 StreamK m a
r)
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (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
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m

    go3 :: (a -> a) -> StreamK m a -> StreamK m a
go3 a -> a
f StreamK m a
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let single :: a -> m r
single a
a   = a -> m r
sng (a -> a
f a
a)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld (a -> a
f a
a) ((a -> a) -> StreamK m a -> StreamK m a
go3 a -> a
f StreamK m a
r)
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (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
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stp StreamK m a
m

{-# INLINE crossApplySnd #-}
crossApplySnd ::
       StreamK m a
    -> StreamK m b
    -> StreamK m b
crossApplySnd :: forall (m :: * -> *) a b. StreamK m a -> StreamK m b -> StreamK m b
crossApplySnd StreamK m a
fstream StreamK m b
stream = StreamK m a -> StreamK m b
forall {a}. StreamK m a -> StreamK m b
go1 StreamK m a
fstream

    where

    go1 :: StreamK m a -> StreamK m b
go1 StreamK m a
m =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                single :: p -> m r
single p
_   = StreamK m b -> m r
foldShared StreamK m b
stream
                yieldk :: p -> StreamK m a -> m r
yieldk p
_ StreamK m a
r = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m a -> StreamK m b -> StreamK m b
go2 StreamK m a
r StreamK m b
stream
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) a -> StreamK m a -> m r
forall {p}. p -> StreamK m a -> m r
yieldk a -> m r
forall {p}. p -> m r
single m r
stp StreamK m a
m

    go2 :: StreamK m a -> StreamK m b -> StreamK m b
go2 StreamK m a
r1 StreamK m b
m =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                stop :: m r
stop = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m a -> StreamK m b
go1 StreamK m a
r1
                single :: b -> m r
single b
a   = b -> StreamK m b -> m r
yld b
a (StreamK m a -> StreamK m b
go1 StreamK m a
r1)
                yieldk :: b -> StreamK m b -> m r
yieldk b
a StreamK m b
r = b -> StreamK m b -> m r
yld b
a (StreamK m a -> StreamK m b -> StreamK m b
go2 StreamK m a
r1 StreamK m b
r)
            in State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yieldk b -> m r
single m r
stop StreamK m b
m

{-# INLINE crossApplyFst #-}
crossApplyFst ::
       StreamK m a
    -> StreamK m b
    -> StreamK m a
crossApplyFst :: forall (m :: * -> *) a b. StreamK m a -> StreamK m b -> StreamK m a
crossApplyFst StreamK m a
fstream StreamK m b
stream = StreamK m a -> StreamK m a
forall {a}. StreamK m a -> StreamK m a
go1 StreamK m a
fstream

    where

    go1 :: StreamK m a -> StreamK m a
go1 StreamK m a
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let foldShared :: StreamK m a -> m r
foldShared = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp
                single :: a -> m r
single a
f   = StreamK m a -> m r
foldShared (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ a -> StreamK m b -> StreamK m a
forall {a} {m :: * -> *} {a}. a -> StreamK m a -> StreamK m a
go3 a
f StreamK m b
stream
                yieldk :: a -> StreamK m a -> m r
yieldk a
f StreamK m a
r = StreamK m a -> m r
foldShared (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ a -> StreamK m a -> StreamK m b -> StreamK m a
go2 a
f StreamK m a
r StreamK m b
stream
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stp StreamK m a
m

    go2 :: a -> StreamK m a -> StreamK m b -> StreamK m a
go2 a
f StreamK m a
r1 StreamK m b
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let foldShared :: StreamK m a -> m r
foldShared = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp
                stop :: m r
stop = StreamK m a -> m r
foldShared (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m a -> StreamK m a
go1 StreamK m a
r1
                single :: p -> m r
single p
_   = a -> StreamK m a -> m r
yld a
f (StreamK m a -> StreamK m a
go1 StreamK m a
r1)
                yieldk :: p -> StreamK m b -> m r
yieldk p
_ StreamK m b
r = a -> StreamK m a -> m r
yld a
f (a -> StreamK m a -> StreamK m b -> StreamK m a
go2 a
f StreamK m a
r1 StreamK m b
r)
            in State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m a -> State StreamK m b
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m a
st) b -> StreamK m b -> m r
forall {p}. p -> StreamK m b -> m r
yieldk b -> m r
forall {p}. p -> m r
single m r
stop StreamK m b
m

    go3 :: a -> StreamK m a -> StreamK m a
go3 a
f StreamK m a
m =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
            let single :: p -> m r
single p
_   = a -> m r
sng a
f
                yieldk :: p -> StreamK m a -> m r
yieldk p
_ StreamK m a
r = a -> StreamK m a -> m r
yld a
f (a -> StreamK m a -> StreamK m a
go3 a
f StreamK m a
r)
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (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
st) a -> StreamK m a -> m r
forall {p}. p -> StreamK m a -> m r
yieldk a -> m r
forall {p}. p -> m r
single m r
stp StreamK m a
m

-- |
-- Definition:
--
-- >>> crossWith f m1 m2 = fmap f m1 `StreamK.crossApply` m2
--
-- Note that the second stream is evaluated multiple times.
--
{-# INLINE crossWith #-}
crossWith :: Monad m => (a -> b -> c) -> StreamK m a -> StreamK m b -> StreamK m c
crossWith :: forall (m :: * -> *) a b c.
Monad m =>
(a -> b -> c) -> StreamK m a -> StreamK m b -> StreamK m c
crossWith a -> b -> c
f StreamK m a
m1 StreamK m b
m2 = (a -> b -> c) -> StreamK m a -> StreamK m (b -> c)
forall a b. (a -> b) -> StreamK m a -> StreamK m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap a -> b -> c
f StreamK m a
m1 StreamK m (b -> c) -> StreamK m b -> StreamK m c
forall (m :: * -> *) a b.
StreamK m (a -> b) -> StreamK m a -> StreamK m b
`crossApply` StreamK m b
m2

-- | Given a @StreamK m a@ and @StreamK m b@ generate a stream with all possible
-- combinations of the tuple @(a, b)@.
--
-- Definition:
--
-- >>> cross = StreamK.crossWith (,)
--
-- The second stream is evaluated multiple times. If that is not desired it can
-- be cached in an 'Data.Array.Array' and then generated from the array before
-- calling this function. Caching may also improve performance if the stream is
-- expensive to evaluate.
--
-- See 'Streamly.Internal.Data.Unfold.cross' for a much faster fused
-- alternative.
--
-- Time: O(m x n)
--
-- /Pre-release/
{-# INLINE cross #-}
cross :: Monad m => StreamK m a -> StreamK m b -> StreamK m (a, b)
cross :: forall (m :: * -> *) a b.
Monad m =>
StreamK m a -> StreamK m b -> StreamK m (a, b)
cross = (a -> b -> (a, b))
-> StreamK m a -> StreamK m b -> StreamK m (a, b)
forall (m :: * -> *) a b c.
Monad m =>
(a -> b -> c) -> StreamK m a -> StreamK m b -> StreamK m c
crossWith (,)

-- XXX This is just concatMapWith with arguments flipped. We need to keep this
-- instead of using a concatMap style definition because the bind
-- implementation in Async and WAsync streams show significant perf degradation
-- if the argument order is changed.
{-# INLINE concatForWith #-}
bindWith, concatForWith ::
       (StreamK m b -> StreamK m b -> StreamK m b)
    -> StreamK m a
    -> (a -> StreamK m b)
    -> StreamK m b
concatForWith :: forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
concatForWith StreamK m b -> StreamK m b -> StreamK m b
combine StreamK m a
m1 a -> StreamK m b
f = StreamK m a -> StreamK m b
go StreamK m a
m1
{-
    -- There is a small improvement by unrolling the first iteration
    mkStream $ \st yld sng stp ->
        let foldShared = foldStreamShared st yld sng stp
            single a   = foldShared $ unShare (f a)
            yieldk a r = foldShared $ unShare (f a) `combine` go r
        in foldStreamShared (adaptState st) yieldk single stp m1
-}

    where

    go :: StreamK m a -> StreamK m b
go StreamK m a
m =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                single :: a -> m r
single a
a   = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare (a -> StreamK m b
f a
a)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare (a -> StreamK m b
f a
a) StreamK m b -> StreamK m b -> StreamK m b
`combine` StreamK m a -> StreamK m b
go StreamK m a
r
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared (State StreamK m b -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stp StreamK m a
m

RENAME(bindWith,concatForWith)

-- XXX express in terms of foldrS?
-- XXX can we use a different stream type for the generated stream being
-- falttened so that we can combine them differently and keep the resulting
-- stream different?
-- XXX do we need specialize to IO?
-- XXX can we optimize when c and a are same, by removing the forall using
-- rewrite rules with type applications?

-- | Perform a 'concatMap' using a specified concat strategy. The first
-- argument specifies a merge or concat function that is used to merge the
-- streams generated by the map function.
--
-- For example, interleaving n streams in a left biased manner:
--
-- >>> lists = mk [[1,5],[2,6],[3,7],[4,8]]
-- >>> un $ StreamK.concatMapWith StreamK.interleave mk lists
-- [1,2,5,3,6,4,7,8]
--
-- For a fair interleaving example see 'bfsConcatMap' and 'mergeMapWith'.
--
{-# INLINE concatMapWith #-}
concatMapWith
    ::
       (StreamK m b -> StreamK m b -> StreamK m b)
    -> (a -> StreamK m b)
    -> StreamK m a
    -> StreamK m b
concatMapWith :: forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
concatMapWith StreamK m b -> StreamK m b -> StreamK m b
par a -> StreamK m b
f StreamK m a
xs = (StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
concatForWith StreamK m b -> StreamK m b -> StreamK m b
par StreamK m a
xs a -> StreamK m b
f

-- | Like 'concatForWith' but maps an effectful function.
{-# INLINE concatForWithM #-}
concatForWithM :: Monad m =>
       (StreamK m b -> StreamK m b -> StreamK m b)
    -> StreamK m a
    -> (a -> m (StreamK m b))
    -> StreamK m b
concatForWithM :: forall (m :: * -> *) b a.
Monad m =>
(StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> m (StreamK m b)) -> StreamK m b
concatForWithM StreamK m b -> StreamK m b -> StreamK m b
combine StreamK m a
s a -> m (StreamK m b)
f = (StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
concatForWith StreamK m b -> StreamK m b -> StreamK m b
combine StreamK m a
s (m (StreamK m b) -> StreamK m b
forall (m :: * -> *) a. Monad m => m (StreamK m a) -> StreamK m a
concatEffect (m (StreamK m b) -> StreamK m b)
-> (a -> m (StreamK m b)) -> a -> StreamK m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> m (StreamK m b)
f)

-- |
-- If total iterations are kept the same, each increase in the nesting level
-- increases the cost by roughly 1.5 times.
--
{-# INLINE concatMap #-}
concatMap :: (a -> StreamK m b) -> StreamK m a -> StreamK m b
concatMap :: forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
concatMap = (StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
concatMapWith StreamK m b -> StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
append

{-
-- Fused version.
-- XXX This fuses but when the stream is nil this performs poorly.
-- The filterAllOut benchmark degrades. Need to investigate and fix that.
{-# INLINE concatMap #-}
concatMap :: IsStream t => (a -> t m b) -> t m a -> t m b
concatMap f xs = buildS
    (\c n -> foldrS (\x b -> foldrS c b (f x)) n xs)

-- Stream polymorphic concatMap implementation
-- XXX need to use buildSM/foldrSMShared for parallel behavior
-- XXX unShare seems to degrade the fused performance
{-# INLINE_EARLY concatMap_ #-}
concatMap_ :: IsStream t => (a -> t m b) -> t m a -> t m b
concatMap_ f xs = buildS
     (\c n -> foldrSShared (\x b -> foldrSShared c b (unShare $ f x)) n xs)
-}

-- | Map a stream generating function on each element of a stream and
-- concatenate the results. This is the same as the bind function of the monad
-- instance. It is just a flipped 'concatMap' but more convenient to use for
-- nested use case, feels like an imperative @for@ loop.
--
-- >>> concatFor = flip StreamK.concatMap
--
-- A concatenating @for@ loop:
--
-- >>> :{
-- un $
--     StreamK.concatFor (mk [1,2,3]) $ \x ->
--       StreamK.fromPure x
-- :}
-- [1,2,3]
--
-- Nested concatenating @for@ loops:
--
-- >>> :{
-- un $
--     StreamK.concatFor (mk [1,2,3]) $ \x ->
--      StreamK.concatFor (mk [4,5,6]) $ \y ->
--       StreamK.fromPure (x, y)
-- :}
-- [(1,4),(1,5),(1,6),(2,4),(2,5),(2,6),(3,4),(3,5),(3,6)]
--
{-# INLINE concatFor #-}
concatFor :: StreamK m a -> (a -> StreamK m b) -> StreamK m b
concatFor :: forall (m :: * -> *) a b.
StreamK m a -> (a -> StreamK m b) -> StreamK m b
concatFor = (StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
concatForWith StreamK m b -> StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
append

-- | Like 'concatFor' but maps an effectful function. It allows conveniently
-- mixing monadic effects with streams.
--
-- >>> import Control.Monad.IO.Class (liftIO)
-- >>> :{
-- un $
--     StreamK.concatForM (mk [1,2,3]) $ \x -> do
--       liftIO $ putStrLn (show x)
--       pure $ StreamK.fromPure x
-- :}
-- 1
-- 2
-- 3
-- [1,2,3]
--
-- Nested concatentating @for@ loops:
--
-- >>> :{
-- un $
--     StreamK.concatForM (mk [1,2,3]) $ \x -> do
--       liftIO $ putStrLn (show x)
--       pure $ StreamK.concatFor (mk [4,5,6]) $ \y ->
--         StreamK.fromPure (x, y)
-- :}
-- 1
-- 2
-- 3
-- [(1,4),(1,5),(1,6),(2,4),(2,5),(2,6),(3,4),(3,5),(3,6)]
--
{-# INLINE concatForM #-}
concatForM :: Monad m => StreamK m a -> (a -> m (StreamK m b)) -> StreamK m b
concatForM :: forall (m :: * -> *) a b.
Monad m =>
StreamK m a -> (a -> m (StreamK m b)) -> StreamK m b
concatForM StreamK m a
s a -> m (StreamK m b)
f =
    -- This should be better than implementing a custom concatForWithM because
    -- here we do not need to inline concatFor, "concatEffect . f" should get
    -- fused right here.
    StreamK m a -> (a -> StreamK m b) -> StreamK m b
forall (m :: * -> *) a b.
StreamK m a -> (a -> StreamK m b) -> StreamK m b
concatFor StreamK m a
s (m (StreamK m b) -> StreamK m b
forall (m :: * -> *) a. Monad m => m (StreamK m a) -> StreamK m a
concatEffect (m (StreamK m b) -> StreamK m b)
-> (a -> m (StreamK m b)) -> a -> StreamK m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> m (StreamK m b)
f)

-- XXX Instead of using "mergeMapWith interleave" we can implement an N-way
-- interleaving CPS combinator which behaves like unfoldEachInterleave. Instead
-- of pairing up the streams we just need to go yielding one element from each
-- stream and storing the remaining streams and then keep doing rounds through
-- those in a round robin fashion. This would be much like wAsync.

-- | Combine streams in pairs using a binary combinator, the resulting streams
-- are then combined again in pairs recursively until we get to a single
-- combined stream. The composition would thus form a binary tree.
--
-- For example, 'mergeMapWith interleave' gives the following result:
--
-- >>> lists = mk [[1,2,3],[4,5,6],[7,8,9],[10,11,12]]
-- >>> un $ StreamK.mergeMapWith StreamK.interleave mk lists
-- [1,7,4,10,2,8,5,11,3,9,6,12]
--
-- The above example is equivalent to the following pairings:
--
-- >>> pair1 = mk [1,2,3] `StreamK.interleave` mk [4,5,6]
-- >>> pair2 = mk [7,8,9] `StreamK.interleave` mk [10,11,12]
-- >>> un $ pair1 `StreamK.interleave` pair2
-- [1,7,4,10,2,8,5,11,3,9,6,12]
--
-- If the number of streams being combined is not a power of 2, the binary tree
-- composed by mergeMapWith is not balanced, therefore, the output may not look
-- fairly interleaved, it will be biased towards the unpaired streams:
--
-- >>> lists = mk [[1,2,3],[4,5,6],[7,8,9]]
-- >>> un $ StreamK.mergeMapWith StreamK.interleave mk lists
-- [1,7,4,8,2,9,5,3,6]
--
-- An efficient merge sort can be implemented by using 'mergeBy' as the
-- combining function:
--
-- >>> combine = StreamK.mergeBy compare
-- >>> un $ StreamK.mergeMapWith combine StreamK.fromPure (mk [5,1,7,9,2])
-- [1,2,5,7,9]
--
-- /Caution: the stream of streams must be finite/
--
-- /Pre-release/
--
{-# INLINE mergeMapWith #-}
mergeMapWith
    ::
       (StreamK m b -> StreamK m b -> StreamK m b)
    -> (a -> StreamK m b)
    -> StreamK m a
    -> StreamK m b
mergeMapWith :: forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
mergeMapWith StreamK m b -> StreamK m b -> StreamK m b
combine a -> StreamK m b
f StreamK m a
str = StreamK m (StreamK m b) -> StreamK m b
go (StreamK m a -> StreamK m (StreamK m b)
forall {m :: * -> *}. StreamK m a -> StreamK m (StreamK m b)
leafPairs StreamK m a
str)

    where

    go :: StreamK m (StreamK m b) -> StreamK m b
go StreamK m (StreamK m b)
stream =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                single :: StreamK m b -> m r
single StreamK m b
a   = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare StreamK m b
a
                yieldk :: StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b
a StreamK m (StreamK m b)
r = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m (StreamK m b) -> StreamK m b
go1 StreamK m b
a StreamK m (StreamK m b)
r
            in State StreamK m (StreamK m b)
-> (StreamK m b -> StreamK m (StreamK m b) -> m r)
-> (StreamK m b -> m r)
-> m r
-> StreamK m (StreamK m b)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m (StreamK m b)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b -> m r
single m r
stp StreamK m (StreamK m b)
stream

    go1 :: StreamK m b -> StreamK m (StreamK m b) -> StreamK m b
go1 StreamK m b
a1 StreamK m (StreamK m b)
stream =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                stop :: m r
stop = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare StreamK m b
a1
                single :: StreamK m b -> m r
single StreamK m b
a = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a
unShare StreamK m b
a1 StreamK m b -> StreamK m b -> StreamK m b
`combine` StreamK m b
a
                yieldk :: StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b
a StreamK m (StreamK m b)
r =
                    StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m (StreamK m b) -> StreamK m b
go (StreamK m (StreamK m b) -> StreamK m b)
-> StreamK m (StreamK m b) -> StreamK m b
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m b -> StreamK m b
combine StreamK m b
a1 StreamK m b
a StreamK m b -> StreamK m (StreamK m b) -> StreamK m (StreamK m b)
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
`cons` StreamK m (StreamK m b) -> StreamK m (StreamK m b)
forall {m :: * -> *}.
StreamK m (StreamK m b) -> StreamK m (StreamK m b)
nonLeafPairs StreamK m (StreamK m b)
r
            in State StreamK m (StreamK m b)
-> (StreamK m b -> StreamK m (StreamK m b) -> m r)
-> (StreamK m b -> m r)
-> m r
-> StreamK m (StreamK m b)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m (StreamK m b)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b -> m r
single m r
stop StreamK m (StreamK m b)
stream

    -- Exactly the same as "go" except that stop continuation extracts the
    -- stream.
    leafPairs :: StreamK m a -> StreamK m (StreamK m b)
leafPairs StreamK m a
stream =
        (forall r.
 State StreamK m (StreamK m b)
 -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
 -> (StreamK m b -> m r)
 -> m r
 -> m r)
-> StreamK m (StreamK m b)
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m (StreamK m b)
  -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
  -> (StreamK m b -> m r)
  -> m r
  -> m r)
 -> StreamK m (StreamK m b))
-> (forall r.
    State StreamK m (StreamK m b)
    -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
    -> (StreamK m b -> m r)
    -> m r
    -> m r)
-> StreamK m (StreamK m b)
forall a b. (a -> b) -> a -> b
$ \State StreamK m (StreamK m b)
st StreamK m b -> StreamK m (StreamK m b) -> m r
yld StreamK m b -> m r
sng m r
stp ->
            let foldShared :: StreamK m (StreamK m b) -> m r
foldShared = State StreamK m (StreamK m b)
-> (StreamK m b -> StreamK m (StreamK m b) -> m r)
-> (StreamK m b -> m r)
-> m r
-> StreamK m (StreamK m b)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m (StreamK m b)
st StreamK m b -> StreamK m (StreamK m b) -> m r
yld StreamK m b -> m r
sng m r
stp
                single :: a -> m r
single a
a   = StreamK m b -> m r
sng (a -> StreamK m b
f a
a)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m (StreamK m b) -> m r
foldShared (StreamK m (StreamK m b) -> m r) -> StreamK m (StreamK m b) -> m r
forall a b. (a -> b) -> a -> b
$ a -> StreamK m a -> StreamK m (StreamK m b)
leafPairs1 a
a StreamK m a
r
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m (StreamK m b) -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m (StreamK m b)
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stp StreamK m a
stream

    leafPairs1 :: a -> StreamK m a -> StreamK m (StreamK m b)
leafPairs1 a
a1 StreamK m a
stream =
        (forall r.
 State StreamK m (StreamK m b)
 -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
 -> (StreamK m b -> m r)
 -> m r
 -> m r)
-> StreamK m (StreamK m b)
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m (StreamK m b)
  -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
  -> (StreamK m b -> m r)
  -> m r
  -> m r)
 -> StreamK m (StreamK m b))
-> (forall r.
    State StreamK m (StreamK m b)
    -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
    -> (StreamK m b -> m r)
    -> m r
    -> m r)
-> StreamK m (StreamK m b)
forall a b. (a -> b) -> a -> b
$ \State StreamK m (StreamK m b)
st StreamK m b -> StreamK m (StreamK m b) -> m r
yld StreamK m b -> m r
sng m r
_ ->
            let stop :: m r
stop = StreamK m b -> m r
sng (a -> StreamK m b
f a
a1)
                single :: a -> m r
single a
a = StreamK m b -> m r
sng (a -> StreamK m b
f a
a1 StreamK m b -> StreamK m b -> StreamK m b
`combine` a -> StreamK m b
f a
a)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m b -> StreamK m (StreamK m b) -> m r
yld (a -> StreamK m b
f a
a1 StreamK m b -> StreamK m b -> StreamK m b
`combine` a -> StreamK m b
f a
a) (StreamK m (StreamK m b) -> m r) -> StreamK m (StreamK m b) -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m a -> StreamK m (StreamK m b)
leafPairs StreamK m a
r
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m (StreamK m b) -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m (StreamK m b)
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
stream

    -- Exactly the same as "leafPairs" except that it does not map "f"
    nonLeafPairs :: StreamK m (StreamK m b) -> StreamK m (StreamK m b)
nonLeafPairs StreamK m (StreamK m b)
stream =
        (forall r.
 State StreamK m (StreamK m b)
 -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
 -> (StreamK m b -> m r)
 -> m r
 -> m r)
-> StreamK m (StreamK m b)
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m (StreamK m b)
  -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
  -> (StreamK m b -> m r)
  -> m r
  -> m r)
 -> StreamK m (StreamK m b))
-> (forall r.
    State StreamK m (StreamK m b)
    -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
    -> (StreamK m b -> m r)
    -> m r
    -> m r)
-> StreamK m (StreamK m b)
forall a b. (a -> b) -> a -> b
$ \State StreamK m (StreamK m b)
st StreamK m b -> StreamK m (StreamK m b) -> m r
yld StreamK m b -> m r
sng m r
stp ->
            let foldShared :: StreamK m (StreamK m b) -> m r
foldShared = State StreamK m (StreamK m b)
-> (StreamK m b -> StreamK m (StreamK m b) -> m r)
-> (StreamK m b -> m r)
-> m r
-> StreamK m (StreamK m b)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m (StreamK m b)
st StreamK m b -> StreamK m (StreamK m b) -> m r
yld StreamK m b -> m r
sng m r
stp
                single :: StreamK m b -> m r
single StreamK m b
a   = StreamK m b -> m r
sng StreamK m b
a
                yieldk :: StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b
a StreamK m (StreamK m b)
r = StreamK m (StreamK m b) -> m r
foldShared (StreamK m (StreamK m b) -> m r) -> StreamK m (StreamK m b) -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m (StreamK m b) -> StreamK m (StreamK m b)
nonLeafPairs1 StreamK m b
a StreamK m (StreamK m b)
r
            in State StreamK m (StreamK m b)
-> (StreamK m b -> StreamK m (StreamK m b) -> m r)
-> (StreamK m b -> m r)
-> m r
-> StreamK m (StreamK m b)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m (StreamK m b) -> State StreamK m (StreamK m b)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m (StreamK m b)
st) StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b -> m r
single m r
stp StreamK m (StreamK m b)
stream

    nonLeafPairs1 :: StreamK m b -> StreamK m (StreamK m b) -> StreamK m (StreamK m b)
nonLeafPairs1 StreamK m b
a1 StreamK m (StreamK m b)
stream =
        (forall r.
 State StreamK m (StreamK m b)
 -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
 -> (StreamK m b -> m r)
 -> m r
 -> m r)
-> StreamK m (StreamK m b)
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m (StreamK m b)
  -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
  -> (StreamK m b -> m r)
  -> m r
  -> m r)
 -> StreamK m (StreamK m b))
-> (forall r.
    State StreamK m (StreamK m b)
    -> (StreamK m b -> StreamK m (StreamK m b) -> m r)
    -> (StreamK m b -> m r)
    -> m r
    -> m r)
-> StreamK m (StreamK m b)
forall a b. (a -> b) -> a -> b
$ \State StreamK m (StreamK m b)
st StreamK m b -> StreamK m (StreamK m b) -> m r
yld StreamK m b -> m r
sng m r
_ ->
            let stop :: m r
stop = StreamK m b -> m r
sng StreamK m b
a1
                single :: StreamK m b -> m r
single StreamK m b
a = StreamK m b -> m r
sng (StreamK m b
a1 StreamK m b -> StreamK m b -> StreamK m b
`combine` StreamK m b
a)
                yieldk :: StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b
a StreamK m (StreamK m b)
r = StreamK m b -> StreamK m (StreamK m b) -> m r
yld (StreamK m b
a1 StreamK m b -> StreamK m b -> StreamK m b
`combine` StreamK m b
a) (StreamK m (StreamK m b) -> m r) -> StreamK m (StreamK m b) -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m (StreamK m b) -> StreamK m (StreamK m b)
nonLeafPairs StreamK m (StreamK m b)
r
            in State StreamK m (StreamK m b)
-> (StreamK m b -> StreamK m (StreamK m b) -> m r)
-> (StreamK m b -> m r)
-> m r
-> StreamK m (StreamK m b)
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m (StreamK m b) -> State StreamK m (StreamK m b)
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m (StreamK m b)
st) StreamK m b -> StreamK m (StreamK m b) -> m r
yieldk StreamK m b -> m r
single m r
stop StreamK m (StreamK m b)
stream

-- | See 'bfsConcatFor' for detailed documentation.
--
-- >>> bfsConcatMap = flip StreamK.bfsConcatFor
--
{-# INLINE bfsConcatMap #-}
bfsConcatMap ::
       (a -> StreamK m b)
    -> StreamK m a
    -> StreamK m b
bfsConcatMap :: forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
bfsConcatMap a -> StreamK m b
f StreamK m a
m1 = ([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go [StreamK m b] -> [StreamK m b]
forall a. a -> a
id StreamK m a
m1

    where

    go :: ([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go [StreamK m b] -> [StreamK m b]
xs StreamK m a
m =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                stop :: m r
stop       = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp (([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
forall {m :: * -> *} {a}.
([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m b] -> [StreamK m b]
forall a. a -> a
id ([StreamK m b] -> [StreamK m b]
xs []))
                single :: a -> m r
single a
a   = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ ([StreamK m b] -> [StreamK m b]) -> StreamK m b -> StreamK m b
forall {m :: * -> *} {a}.
([StreamK m a] -> [StreamK m a]) -> StreamK m a -> StreamK m a
goLast [StreamK m b] -> [StreamK m b]
xs (a -> StreamK m b
f a
a)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ ([StreamK m b] -> [StreamK m b])
-> StreamK m a -> StreamK m b -> StreamK m b
goNext [StreamK m b] -> [StreamK m b]
xs StreamK m a
r (a -> StreamK m b
f a
a)
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m

    -- generate first element from cur stream, and then put it back in the
    -- queue xs.
    goNext :: ([StreamK m b] -> [StreamK m b])
-> StreamK m a -> StreamK m b -> StreamK m b
goNext [StreamK m b] -> [StreamK m b]
xs StreamK m a
m StreamK m b
cur =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp -> do
            let stop :: m r
stop       = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp (([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go [StreamK m b] -> [StreamK m b]
xs StreamK m a
m)
                single :: b -> m r
single b
a   = b -> StreamK m b -> m r
yld b
a (([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go [StreamK m b] -> [StreamK m b]
xs StreamK m a
m)
                yieldk :: b -> StreamK m b -> m r
yieldk b
a StreamK m b
r = b -> StreamK m b -> m r
yld b
a (([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go ([StreamK m b] -> [StreamK m b]
xs ([StreamK m b] -> [StreamK m b])
-> ([StreamK m b] -> [StreamK m b])
-> [StreamK m b]
-> [StreamK m b]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (StreamK m b
r StreamK m b -> [StreamK m b] -> [StreamK m b]
forall a. a -> [a] -> [a]
:)) StreamK m a
m)
            State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yieldk b -> m r
single m r
stop StreamK m b
cur

    goLast :: ([StreamK m a] -> [StreamK m a]) -> StreamK m a -> StreamK m a
goLast [StreamK m a] -> [StreamK m a]
xs StreamK m a
cur =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp -> do
            let stop :: m r
stop       = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
forall {m :: * -> *} {a}.
([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
forall a. a -> a
id ([StreamK m a] -> [StreamK m a]
xs []))
                single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
forall {m :: * -> *} {a}.
([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
forall a. a -> a
id ([StreamK m a] -> [StreamK m a]
xs []))
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
forall {m :: * -> *} {a}.
([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
forall a. a -> a
id (([StreamK m a] -> [StreamK m a]
xs ([StreamK m a] -> [StreamK m a])
-> ([StreamK m a] -> [StreamK m a])
-> [StreamK m a]
-> [StreamK m a]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (StreamK m a
r StreamK m a -> [StreamK m a] -> [StreamK m a]
forall a. a -> [a] -> [a]
:)) []))
            State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
cur

    -- Loop through all streams in the queue ys until they are over
    goLoop :: ([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
ys [] =
            -- We will do this optimization only after two iterations are
            -- over, if doing this earlier is helpful we can do it in
            -- goLast as well, before calling goLoop.
           let xs :: [StreamK m a]
xs = [StreamK m a] -> [StreamK m a]
ys []
            in case [StreamK m a]
xs of
                    [] -> StreamK m a
forall (m :: * -> *) a. StreamK m a
nil
                    (StreamK m a
z:[]) -> StreamK m a
z
                    (StreamK m a
z1:StreamK m a
z2:[]) -> StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleave StreamK m a
z1 StreamK m a
z2
                    [StreamK m a]
zs -> ([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
forall a. a -> a
id [StreamK m a]
zs
    goLoop [StreamK m a] -> [StreamK m a]
ys (StreamK m a
x:[StreamK m a]
xs) =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp -> do
            let stop :: m r
stop       = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
ys [StreamK m a]
xs)
                single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
ys [StreamK m a]
xs)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop ([StreamK m a] -> [StreamK m a]
ys ([StreamK m a] -> [StreamK m a])
-> ([StreamK m a] -> [StreamK m a])
-> [StreamK m a]
-> [StreamK m a]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (StreamK m a
r StreamK m a -> [StreamK m a] -> [StreamK m a]
forall a. a -> [a] -> [a]
:)) [StreamK m a]
xs)
            State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
x

-- | While 'concatFor' flattens a stream of streams in a depth first manner,
-- 'bfsConcatFor' flattens it in a breadth-first manner. It yields one item
-- from the first stream, then one item from the next stream and so on. In
-- nested loops it has the effect of prioritizing the new outer loop iteration
-- over the inner loops, thus inverting the looping.
-- Given a stream of three streams:
--
-- @
-- 1. [1,2,3]
-- 2. [4,5,6]
-- 3. [7,8,9]
-- @
--
-- The resulting stream is @(1,4,7),(2,5,8),(3,6,9)@. The parenthesis are added
-- to emphasize the iterations.
--
-- For example:
--
-- >>> stream = mk [[1,2,3],[4,5,6],[7,8,9]]
-- >>> :{
--  un $
--      StreamK.bfsConcatFor stream $ \x ->
--          StreamK.fromStream $ Stream.fromList x
-- :}
-- [1,4,7,2,5,8,3,6,9]
--
-- Compare with 'concatForWith' 'interleave' which explores the depth
-- exponentially more compared to the breadth, such that each stream yields
-- twice as many items compared to the next stream.
--
-- See also the equivalent fused version 'Data.Stream.unfoldEachInterleave'.
--
{-# INLINE bfsConcatFor #-}
bfsConcatFor :: StreamK m a -> (a -> StreamK m b) -> StreamK m b
bfsConcatFor :: forall (m :: * -> *) a b.
StreamK m a -> (a -> StreamK m b) -> StreamK m b
bfsConcatFor = ((a -> StreamK m b) -> StreamK m a -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
forall a b c. (a -> b -> c) -> b -> a -> c
flip (a -> StreamK m b) -> StreamK m a -> StreamK m b
forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
bfsConcatMap

-- | Like 'bfsConcatFor' but maps a monadic function.
{-# INLINE bfsConcatForM #-}
bfsConcatForM :: Monad m => StreamK m a -> (a -> m (StreamK m b)) -> StreamK m b
bfsConcatForM :: forall (m :: * -> *) a b.
Monad m =>
StreamK m a -> (a -> m (StreamK m b)) -> StreamK m b
bfsConcatForM StreamK m a
s a -> m (StreamK m b)
f = (a -> StreamK m b) -> StreamK m a -> StreamK m b
forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
bfsConcatMap (m (StreamK m b) -> StreamK m b
forall (m :: * -> *) a. Monad m => m (StreamK m a) -> StreamK m a
concatEffect (m (StreamK m b) -> StreamK m b)
-> (a -> m (StreamK m b)) -> a -> StreamK m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> m (StreamK m b)
f) StreamK m a
s

-- | See 'fairConcatFor' for detailed documentation.
--
-- >>> fairConcatMap = flip StreamK.fairConcatFor
--
{-# INLINE fairConcatMap #-}
fairConcatMap ::
       (a -> StreamK m b)
    -> StreamK m a
    -> StreamK m b
fairConcatMap :: forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
fairConcatMap a -> StreamK m b
f StreamK m a
m1 = ([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go [StreamK m b] -> [StreamK m b]
forall a. a -> a
id StreamK m a
m1

    where

    go :: ([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go [StreamK m b] -> [StreamK m b]
xs StreamK m a
m =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
            let foldShared :: StreamK m b -> m r
foldShared = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp
                stop :: m r
stop       = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp (([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
forall {m :: * -> *} {a}.
([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m b] -> [StreamK m b]
forall a. a -> a
id ([StreamK m b] -> [StreamK m b]
xs []))
                single :: a -> m r
single a
a   = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ ([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
forall {m :: * -> *} {a}.
([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m b] -> [StreamK m b]
forall a. a -> a
id ([StreamK m b] -> [StreamK m b]
xs [a -> StreamK m b
f a
a])
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m b -> m r
foldShared (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m a
-> ([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
goNext StreamK m a
r [StreamK m b] -> [StreamK m b]
forall a. a -> a
id ([StreamK m b] -> [StreamK m b]
xs [a -> StreamK m b
f a
a])
            in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m

    goNext :: StreamK m a
-> ([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
goNext StreamK m a
m [StreamK m b] -> [StreamK m b]
ys [] = ([StreamK m b] -> [StreamK m b]) -> StreamK m a -> StreamK m b
go [StreamK m b] -> [StreamK m b]
ys StreamK m a
m
    goNext StreamK m a
m [StreamK m b] -> [StreamK m b]
ys (StreamK m b
x:[StreamK m b]
xs) =
        (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp -> do
            let stop :: m r
stop       = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp (StreamK m a
-> ([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
goNext StreamK m a
m [StreamK m b] -> [StreamK m b]
ys [StreamK m b]
xs)
                single :: b -> m r
single b
a   = b -> StreamK m b -> m r
yld b
a (StreamK m a
-> ([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
goNext StreamK m a
m [StreamK m b] -> [StreamK m b]
ys [StreamK m b]
xs)
                yieldk :: b -> StreamK m b -> m r
yieldk b
a StreamK m b
r = b -> StreamK m b -> m r
yld b
a (StreamK m a
-> ([StreamK m b] -> [StreamK m b]) -> [StreamK m b] -> StreamK m b
goNext StreamK m a
m ([StreamK m b] -> [StreamK m b]
ys ([StreamK m b] -> [StreamK m b])
-> ([StreamK m b] -> [StreamK m b])
-> [StreamK m b]
-> [StreamK m b]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (StreamK m b
r StreamK m b -> [StreamK m b] -> [StreamK m b]
forall a. a -> [a] -> [a]
:)) [StreamK m b]
xs)
            State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yieldk b -> m r
single m r
stop StreamK m b
x

    -- Loop through all streams in the queue ys until they are over
    goLoop :: ([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
ys [] =
            -- We will do this optimization only after two iterations are
            -- over, if doing this earlier is helpful we can do it in
            -- goLast as well, before calling goLoop.
           let xs :: [StreamK m a]
xs = [StreamK m a] -> [StreamK m a]
ys []
            in case [StreamK m a]
xs of
                    [] -> StreamK m a
forall (m :: * -> *) a. StreamK m a
nil
                    (StreamK m a
z:[]) -> StreamK m a
z
                    (StreamK m a
z1:StreamK m a
z2:[]) -> StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleave StreamK m a
z1 StreamK m a
z2
                    [StreamK m a]
zs -> ([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
forall a. a -> a
id [StreamK m a]
zs
    goLoop [StreamK m a] -> [StreamK m a]
ys (StreamK m a
x:[StreamK m a]
xs) =
        (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp -> do
            let stop :: m r
stop       = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
ys [StreamK m a]
xs)
                single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop [StreamK m a] -> [StreamK m a]
ys [StreamK m a]
xs)
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (([StreamK m a] -> [StreamK m a]) -> [StreamK m a] -> StreamK m a
goLoop ([StreamK m a] -> [StreamK m a]
ys ([StreamK m a] -> [StreamK m a])
-> ([StreamK m a] -> [StreamK m a])
-> [StreamK m a]
-> [StreamK m a]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (StreamK m a
r StreamK m a -> [StreamK m a] -> [StreamK m a]
forall a. a -> [a] -> [a]
:)) [StreamK m a]
xs)
            State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
x

-- | 'fairConcatFor' is like 'concatFor' but traverses the depth and breadth of
-- nesting equally. Therefore, the outer and the inner loops in a nested loop
-- get equal priority. It can be used to nest infinite streams without starving
-- outer streams due to inner ones.
--
-- Given a stream of three streams:
--
-- @
-- 1. [1,2,3]
-- 2. [4,5,6]
-- 3. [7,8,9]
-- @
--
-- Here, outer loop is the stream of streams and the inner loops are the
-- individual streams. The traversal sweeps the diagonals in the above grid to
-- give equal chance to outer and inner loops. The resulting stream is
-- @(1),(2,4),(3,5,7),(6,8),(9)@, diagonals are parenthesized for emphasis.
--
-- == Looping
--
-- A single stream case is equivalent to 'concatFor':
--
-- >>> un $ StreamK.fairConcatFor (mk [1,2]) $ \x -> StreamK.fromPure x
-- [1,2]
--
-- == Fair Nested Looping
--
-- Multiple streams nest like @for@ loops. The result is a cross product of the
-- streams. However, the ordering of the results of the cross product is such
-- that each stream gets consumed equally. In other words, inner iterations of
-- a nested loop get the same priority as the outer iterations. Inner
-- iterations do not finish completely before the outer iterations start.
--
-- >>> :{
-- un $ do
--     StreamK.fairConcatFor (mk [1,2,3]) $ \x ->
--      StreamK.fairConcatFor (mk [4,5,6]) $ \y ->
--       StreamK.fromPure (x, y)
-- :}
-- [(1,4),(1,5),(2,4),(1,6),(2,5),(3,4),(2,6),(3,5),(3,6)]
--
-- == Nesting Infinite Streams
--
-- Example with infinite streams. Print all pairs in the cross product with sum
-- less than a specified number.
--
-- >>> :{
-- Stream.toList
--  $ Stream.takeWhile (\(x,y) -> x + y < 6)
--  $ StreamK.toStream
--  $ StreamK.fairConcatFor (mk [1..]) $ \x ->
--     StreamK.fairConcatFor (mk [1..]) $ \y ->
--      StreamK.fromPure (x, y)
-- :}
-- [(1,1),(1,2),(2,1),(1,3),(2,2),(3,1),(1,4),(2,3),(3,2),(4,1)]
--
-- == How the nesting works?
--
-- If we look at the cross product of [1,2,3], [4,5,6], the streams being
-- combined using 'fairConcatFor' are the following sequential loop iterations:
--
-- @
-- (1,4) (1,5) (1,6) -- first iteration of the outer loop
-- (2,4) (2,5) (2,6) -- second iteration of the outer loop
-- (3,4) (3,5) (3,6) -- third iteration of the outer loop
-- @
--
-- The result is a triangular or diagonal traversal of these iterations:
--
-- @
-- [(1,4),(1,5),(2,4),(1,6),(2,5),(3,4),(2,6),(3,5),(3,6)]
-- @
--
-- == Non-Termination Cases
--
-- If one of the two interleaved streams does not produce an output at all and
-- continues forever then the other stream will never get scheduled. This is
-- because a stream is unscheduled only after it produces an output. This can
-- lead to non-terminating programs, an example is provided below.
--
-- >>> :{
-- oddsIf x = mk (if x then [1,3..] else [2,4..])
-- filterEven x = if even x then StreamK.fromPure x else StreamK.nil
-- :}
--
-- >>> :{
-- evens =
--     StreamK.fairConcatFor (mk [True,False]) $ \r ->
--      StreamK.concatFor (oddsIf r) filterEven
-- :}
--
-- The @evens@ function does not terminate because, when r is True, the nested
-- 'concatFor' is a non-productive infinite loop, therefore, the outer loop
-- never gets a chance to generate the @False@ value.
--
-- But the following refactoring of the above code works as expected:
--
-- >>> :{
-- mixed =
--      StreamK.fairConcatFor (mk [True,False]) $ \r ->
--          StreamK.concatFor (oddsIf r) StreamK.fromPure
-- :}
--
-- >>> evens = StreamK.fairConcatFor mixed filterEven
-- >>> Stream.toList $ Stream.take 3 $ StreamK.toStream evens
-- [2,4,6]
--
-- This works because in @mixed@ both the streams being interleaved are
-- productive.
--
-- Care should be taken how you write your program, keep in mind the scheduling
-- implications. To avoid such scheduling problems in serial interleaving, you
-- can use concurrent interleaving instead i.e. parFairConcatFor. Due to
-- concurrent threads the other branch will make progress even if one is an
-- infinite loop producing nothing.
--
-- == Logic Programming
--
-- Streamly provides all operations for logic programming. It provides
-- functionality equivalent to 'LogicT' type from the 'logict' package.
-- The @MonadLogic@ operations can be implemented using the available stream
-- operations. For example, 'uncons' is @msplit@, 'interleave' corresponds to
-- the @interleave@ operation of MonadLogic, 'fairConcatFor' is the
-- fair bind (@>>-@) operation.
--
-- == Related Operations
--
-- 'concatForWith' 'interleave' is another way to interleave two serial
-- streams. In this case, the inner loop iterations get exponentially more
-- priority over the outer iterations of the nested loop. This is biased
-- towards the inner loops - this is exactly how the logic-t and list-t
-- implementation of fair bind works.
--
{-# INLINE fairConcatFor #-}
fairConcatFor :: StreamK m a -> (a -> StreamK m b) -> StreamK m b
fairConcatFor :: forall (m :: * -> *) a b.
StreamK m a -> (a -> StreamK m b) -> StreamK m b
fairConcatFor = ((a -> StreamK m b) -> StreamK m a -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
forall a b c. (a -> b -> c) -> b -> a -> c
flip (a -> StreamK m b) -> StreamK m a -> StreamK m b
forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
fairConcatMap

-- | Like 'fairConcatFor' but maps a monadic function.
{-# INLINE fairConcatForM #-}
fairConcatForM :: Monad m => StreamK m a -> (a -> m (StreamK m b)) -> StreamK m b
fairConcatForM :: forall (m :: * -> *) a b.
Monad m =>
StreamK m a -> (a -> m (StreamK m b)) -> StreamK m b
fairConcatForM StreamK m a
s a -> m (StreamK m b)
f = (a -> StreamK m b) -> StreamK m a -> StreamK m b
forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
fairConcatMap (m (StreamK m b) -> StreamK m b
forall (m :: * -> *) a. Monad m => m (StreamK m a) -> StreamK m a
concatEffect (m (StreamK m b) -> StreamK m b)
-> (a -> m (StreamK m b)) -> a -> StreamK m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> m (StreamK m b)
f) StreamK m a
s

{-
instance Monad m => Applicative (StreamK m) where
    {-# INLINE pure #-}
    pure = fromPure

    {-# INLINE (<*>) #-}
    (<*>) = crossApply

    {-# INLINE liftA2 #-}
    liftA2 f x = (<*>) (fmap f x)

    {-# INLINE (*>) #-}
    (*>) = crossApplySnd

    {-# INLINE (<*) #-}
    (<*) = crossApplyFst

-- NOTE: even though concatMap for StreamD is 3x faster compared to StreamK,
-- the monad instance of StreamD is slower than StreamK after foldr/build
-- fusion.
instance Monad m => Monad (StreamK m) where
    {-# INLINE return #-}
    return = pure

    {-# INLINE (>>=) #-}
    (>>=) = flip concatMap
-}

{-
-- Like concatMap but generates stream using an unfold function. Similar to
-- unfoldMany but for StreamK.
concatUnfoldr :: IsStream t
    => (b -> t m (Maybe (a, b))) -> t m b -> t m a
concatUnfoldr = undefined
-}

------------------------------------------------------------------------------
-- concatIterate - Map and flatten Trees of Streams
------------------------------------------------------------------------------

-- | Yield an input element in the output stream, map a stream generator on it
-- and repeat the process on the resulting stream. Resulting streams are
-- flattened using the 'concatMapWith' combinator. This can be used for a depth
-- first style (DFS) traversal of a tree like structure.
--
-- Example, list a directory tree using DFS:
--
-- >>> f = StreamK.fromStream . either (Dir.readEitherPaths id) (const Stream.nil)
-- >>> input = StreamK.fromEffect (Left <$> Path.fromString ".")
-- >>> ls = StreamK.concatIterateWith StreamK.append f input
--
-- Note that 'iterateM' is a special case of 'concatIterateWith':
--
-- >>> iterateM f = StreamK.concatIterateWith StreamK.append (StreamK.fromEffect . f) . StreamK.fromEffect
--
-- /Pre-release/
--
{-# INLINE concatIterateWith #-}
concatIterateWith ::
       (StreamK m a -> StreamK m a -> StreamK m a)
    -> (a -> StreamK m a)
    -> StreamK m a
    -> StreamK m a
concatIterateWith :: forall (m :: * -> *) a.
(StreamK m a -> StreamK m a -> StreamK m a)
-> (a -> StreamK m a) -> StreamK m a -> StreamK m a
concatIterateWith StreamK m a -> StreamK m a -> StreamK m a
combine a -> StreamK m a
f = StreamK m a -> StreamK m a
iterateStream

    where

    iterateStream :: StreamK m a -> StreamK m a
iterateStream = (StreamK m a -> StreamK m a -> StreamK m a)
-> (a -> StreamK m a) -> StreamK m a -> StreamK m a
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
concatMapWith StreamK m a -> StreamK m a -> StreamK m a
combine a -> StreamK m a
generate

    generate :: a -> StreamK m a
generate a
x = a
x a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
`cons` StreamK m a -> StreamK m a
iterateStream (a -> StreamK m a
f a
x)

-- | Like 'concatIterateWith' but uses the pairwise flattening combinator
-- 'mergeMapWith' for flattening the resulting streams. This can be used for a
-- balanced traversal of a tree like structure.
--
-- Example, list a directory tree using balanced traversal:
--
-- >>> f = StreamK.fromStream . either (Dir.readEitherPaths id) (const Stream.nil)
-- >>> input = StreamK.fromEffect (Left <$> Path.fromString ".")
-- >>> ls = StreamK.mergeIterateWith StreamK.interleave f input
--
-- /Pre-release/
--
{-# INLINE mergeIterateWith #-}
mergeIterateWith ::
       (StreamK m a -> StreamK m a -> StreamK m a)
    -> (a -> StreamK m a)
    -> StreamK m a
    -> StreamK m a
mergeIterateWith :: forall (m :: * -> *) a.
(StreamK m a -> StreamK m a -> StreamK m a)
-> (a -> StreamK m a) -> StreamK m a -> StreamK m a
mergeIterateWith StreamK m a -> StreamK m a -> StreamK m a
combine a -> StreamK m a
f = StreamK m a -> StreamK m a
iterateStream

    where

    iterateStream :: StreamK m a -> StreamK m a
iterateStream = (StreamK m a -> StreamK m a -> StreamK m a)
-> (a -> StreamK m a) -> StreamK m a -> StreamK m a
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
mergeMapWith StreamK m a -> StreamK m a -> StreamK m a
combine a -> StreamK m a
generate

    generate :: a -> StreamK m a
generate a
x = a
x a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
`cons` StreamK m a -> StreamK m a
iterateStream (a -> StreamK m a
f a
x)

------------------------------------------------------------------------------
-- Flattening Graphs
------------------------------------------------------------------------------

-- To traverse graphs we need a state to be carried around in the traversal.
-- For example, we can use a hashmap to store the visited status of nodes.

-- XXX rename to concateIterateAccum? Like mapMAccum

-- | Like 'iterateMap' but carries a state in the stream generation function.
-- This can be used to traverse graph like structures, we can remember the
-- visited nodes in the state to avoid cycles.
--
-- Note that a combination of 'iterateMap' and 'usingState' can also be used to
-- traverse graphs. However, this function provides a more localized state
-- instead of using a global state.
--
-- See also: 'mfix'
--
-- /Pre-release/
--
{-# INLINE concatIterateScanWith #-}
concatIterateScanWith
    :: Monad m
    => (StreamK m a -> StreamK m a -> StreamK m a)
    -> (b -> a -> m (b, StreamK m a))
    -> m b
    -> StreamK m a
    -> StreamK m a
concatIterateScanWith :: forall (m :: * -> *) a b.
Monad m =>
(StreamK m a -> StreamK m a -> StreamK m a)
-> (b -> a -> m (b, StreamK m a))
-> m b
-> StreamK m a
-> StreamK m a
concatIterateScanWith StreamK m a -> StreamK m a -> StreamK m a
combine b -> a -> m (b, StreamK m a)
f m b
initial StreamK m a
stream =
    m (StreamK m a) -> StreamK m a
forall (m :: * -> *) a. Monad m => m (StreamK m a) -> StreamK m a
concatEffect (m (StreamK m a) -> StreamK m a) -> m (StreamK m a) -> StreamK m a
forall a b. (a -> b) -> a -> b
$ do
        b
b <- m b
initial
        (b, StreamK m a) -> m (StreamK m a)
iterateStream (b
b, StreamK m a
stream)

    where

    iterateStream :: (b, StreamK m a) -> m (StreamK m a)
iterateStream (b
b, StreamK m a
s) = StreamK m a -> m (StreamK m a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (StreamK m a -> m (StreamK m a)) -> StreamK m a -> m (StreamK m a)
forall a b. (a -> b) -> a -> b
$ (StreamK m a -> StreamK m a -> StreamK m a)
-> (a -> StreamK m a) -> StreamK m a -> StreamK m a
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m a -> StreamK m b
concatMapWith StreamK m a -> StreamK m a -> StreamK m a
combine (b -> a -> StreamK m a
generate b
b) StreamK m a
s

    generate :: b -> a -> StreamK m a
generate b
b a
a = a
a a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
`cons` b -> a -> StreamK m a
feedback b
b a
a

    feedback :: b -> a -> StreamK m a
feedback b
b a
a = m (StreamK m a) -> StreamK m a
forall (m :: * -> *) a. Monad m => m (StreamK m a) -> StreamK m a
concatEffect (m (StreamK m a) -> StreamK m a) -> m (StreamK m a) -> StreamK m a
forall a b. (a -> b) -> a -> b
$ b -> a -> m (b, StreamK m a)
f b
b a
a m (b, StreamK m a)
-> ((b, StreamK m a) -> m (StreamK m a)) -> m (StreamK m a)
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= (b, StreamK m a) -> m (StreamK m a)
iterateStream

------------------------------------------------------------------------------
-- Either streams
------------------------------------------------------------------------------

-- Keep concating either streams as long as rights are generated, stop as soon
-- as a left is generated and concat the left stream.
--
-- See also: 'handle'
--
-- /Unimplemented/
--
{-
concatMapEitherWith
    :: (forall x. t m x -> t m x -> t m x)
    -> (a -> t m (Either (StreamK m b) b))
    -> StreamK m a
    -> StreamK m b
concatMapEitherWith = undefined
-}

-- XXX We should prefer using the Maybe stream returning signatures over this.
-- This API should perhaps be removed in favor of those.

-- | In an 'Either' stream iterate on 'Left's.  This is a special case of
-- 'concatIterateWith':
--
-- >>> concatIterateLeftsWith combine f = StreamK.concatIterateWith combine (either f (const StreamK.nil))
--
-- To traverse a directory tree:
--
-- >>> input = StreamK.fromEffect (Left <$> Path.fromString ".")
-- >>> ls = StreamK.concatIterateLeftsWith StreamK.append (StreamK.fromStream . Dir.readEither id) input
--
-- /Pre-release/
--
{-# INLINE concatIterateLeftsWith #-}
concatIterateLeftsWith
    :: (b ~ Either a c)
    => (StreamK m b -> StreamK m b -> StreamK m b)
    -> (a -> StreamK m b)
    -> StreamK m b
    -> StreamK m b
concatIterateLeftsWith :: forall b a c (m :: * -> *).
(b ~ Either a c) =>
(StreamK m b -> StreamK m b -> StreamK m b)
-> (a -> StreamK m b) -> StreamK m b -> StreamK m b
concatIterateLeftsWith StreamK m b -> StreamK m b -> StreamK m b
combine a -> StreamK m b
f =
    (StreamK m b -> StreamK m b -> StreamK m b)
-> (b -> StreamK m b) -> StreamK m b -> StreamK m b
forall (m :: * -> *) a.
(StreamK m a -> StreamK m a -> StreamK m a)
-> (a -> StreamK m a) -> StreamK m a -> StreamK m a
concatIterateWith StreamK m b -> StreamK m b -> StreamK m b
combine ((a -> StreamK m b)
-> (c -> StreamK m b) -> Either a c -> StreamK m b
forall a c b. (a -> c) -> (b -> c) -> Either a b -> c
either a -> StreamK m b
f (StreamK m b -> c -> StreamK m b
forall a b. a -> b -> a
const StreamK m b
forall (m :: * -> *) a. StreamK m a
nil))

------------------------------------------------------------------------------
-- Interleaving
------------------------------------------------------------------------------

infixr 6 `interleave`

-- We can have a variant of interleave where m elements yield from the first
-- stream and n elements yielding from the second stream. We can also have time
-- slicing variants of positional interleaving, e.g. run first stream for m
-- seconds and run the second stream for n seconds.
--
-- TBD; give an example to show second stream is half consumed.
--
-- a1,a2,a3,a4,a5,a6,a7,a8
-- b1,b2,b3,b4,b5,b6,b7,b8
-- c1,c2,c3,c4,c5,c6,c7,c8
-- d1,d2,d3,d4,d5,d6,d7,d8
-- e1,e2,e3,e4,e5,e6,e7,e8
-- f1,f2,f3,f4,f5,f6,f7,f8
-- g1,g2,g3,g4,g5,g6,g7,g8
-- h1,h2,h3,h4,h5,h6,h7,h8
--
-- Produces: (..)
--

-- | Interleave two streams fairly, yielding one item from each in a
-- round-robin fashion:
--
-- >>> un $ StreamK.interleave (mk [1,3,5]) (mk [2,4,6])
-- [1,2,3,4,5,6]
-- >>> un $ StreamK.interleave (mk [1,3]) (mk [2,4,6])
-- [1,2,3,4,6]
-- >>> un $ StreamK.interleave (mk []) (mk [2,4,6])
-- [2,4,6]
--
-- 'interleave' is right associative when used as an infix operator.
--
-- >>> un $ mk [1,2,3] `StreamK.interleave` mk [4,5,6] `StreamK.interleave` mk [7,8,9]
-- [1,4,2,7,3,5,8,6,9]
--
-- Because of right association, the first stream yields as many items as the
-- next two streams combined.
--
-- Be careful when refactoring code involving a chain of three or more
-- 'interleave' operations as it is not associative i.e. right associated code
-- may not produce the same result as left associated. This is a direct
-- consequence of the disbalance of scheduling in the previous example. If left
-- associated the above example would produce:
--
-- >>> un $ (mk [1,2,3] `StreamK.interleave` mk [4,5,6]) `StreamK.interleave` mk [7,8,9]
-- [1,7,4,8,2,9,5,3,6]
--
-- Note: Use concatMap based interleaving instead of the binary operator to
-- interleave more than two streams to avoid associativity issues.
--
-- 'concatMapWith' 'interleave' flattens a stream of streams using 'interleave'
-- in a right associative manner. Given a stream of three streams:
--
-- @
-- 1. [1,2,3]
-- 2. [4,5,6]
-- 3. [7,8,9]
-- @
--
-- The resulting sequence is @[1,4,2,7,3,5,8,6,9]@.
--
-- For this reason, the right associated flattening with 'interleave' can work
-- with infinite number of streams without opening too many streams at the same
-- time. Each stream is consumed twice as much as the next one; if we are
-- combining an infinite number of streams of size @n@ then at most @log n@
-- streams will be opened at any given time, because the first stream will
-- finish by the time the stream after @log n@ th stream is opened.
--
-- Compare with 'bfsConcatMap' and 'mergeMapWith' 'interleave'.
--
-- For interleaving many streams, the best way is to use 'bfsConcatMap'.
--
-- See also the fused version 'Streamly.Data.Stream.interleave'.
{-# INLINE interleave #-}
interleave :: StreamK m a -> StreamK m a -> StreamK m a
interleave :: forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleave StreamK m a
m1 StreamK m a
m2 = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp -> do
    let stop :: m r
stop       = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp StreamK m a
m2
        single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a StreamK m a
m2
        yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleave StreamK m a
m2 StreamK m a
r)
    State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m1

-- Examples:
--
-- >>> fromList = StreamK.fromStream . Stream.fromList
-- >>> toList = Stream.toList . StreamK.toStream
-- >>> f x y = toList $ StreamK.interleaveSepBy (fromList x) (fromList y)
--
-- -- This is broken.
-- >> f "..." "abc"
-- "a.b.c"

-- >>> f ".." "abc"
-- "a.b.c"
-- >>> f "." "abc"
-- "a.bc"
--
{-# INLINE interleaveSepBy #-}
interleaveSepBy :: StreamK m a -> StreamK m a -> StreamK m a
interleaveSepBy :: forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveSepBy StreamK m a
m2 StreamK m a
m1 = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp -> do
    let yieldFirst :: a -> StreamK m a -> m r
yieldFirst a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
yieldSecond StreamK m a
r StreamK m a
m2)
     in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldFirst a -> m r
sng m r
stp StreamK m a
m1

    where

    yieldSecond :: StreamK m a -> StreamK m a -> StreamK m a
yieldSecond StreamK m a
s1 StreamK m a
s2 = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp -> do
            let stop :: m r
stop       = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp StreamK m a
s1
                single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a StreamK m a
s1
                yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleave StreamK m a
s1 StreamK m a
r)
             in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
s2

infixr 6 `interleaveFst`

{-# DEPRECATED interleaveFst "Please use flip interleaveSepBy instead." #-}
{-# INLINE interleaveFst #-}
interleaveFst :: StreamK m a -> StreamK m a -> StreamK m a
interleaveFst :: forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveFst = (StreamK m a -> StreamK m a -> StreamK m a)
-> StreamK m a -> StreamK m a -> StreamK m a
forall a b c. (a -> b -> c) -> b -> a -> c
flip StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveSepBy

-- |
--
-- Examples:
--
-- >>> fromList = StreamK.fromStream . Stream.fromList
-- >>> toList = Stream.toList . StreamK.toStream
-- >>> f x y = toList $ StreamK.interleaveEndBy' (fromList x) (fromList y)
-- >>> f "..." "abc"
-- "a.b.c."
-- >>> f "..." "ab"
-- "a.b."
--
-- Currently broken, generates an additional element at the end::
--
-- >> f ".." "abc"
-- "a.b."
--
{-# INLINE interleaveEndBy' #-}
interleaveEndBy' :: StreamK m a -> StreamK m a -> StreamK m a
interleaveEndBy' :: forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveEndBy' StreamK m a
m2 StreamK m a
m1 = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
_ m r
stp -> do
    let stop :: m r
stop       = m r
stp
        -- "single a" is defined as "yld a (interleaveMin m2 nil)" instead of
        -- "sng a" to keep the behaviour consistent with the yield
        -- continuation.
        single :: a -> m r
single a
a   = a -> StreamK m a -> m r
yld a
a (StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveEndBy' StreamK m a
forall (m :: * -> *) a. StreamK m a
nil StreamK m a
m2)
        yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = a -> StreamK m a -> m r
yld a
a (StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveEndBy' StreamK m a
r StreamK m a
m2)
    State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
m1

infixr 6 `interleaveMin`

{-# DEPRECATED interleaveMin "Please use flip interleaveEndBy' instead." #-}
{-# INLINE interleaveMin #-}
interleaveMin :: StreamK m a -> StreamK m a -> StreamK m a
interleaveMin :: forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveMin = (StreamK m a -> StreamK m a -> StreamK m a)
-> StreamK m a -> StreamK m a -> StreamK m a
forall a b c. (a -> b -> c) -> b -> a -> c
flip StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
interleaveEndBy'

-------------------------------------------------------------------------------
-- Generation
-------------------------------------------------------------------------------

-- |
-- >>> :{
-- unfoldr step s =
--     case step s of
--         Nothing -> StreamK.nil
--         Just (a, b) -> a `StreamK.cons` unfoldr step b
-- :}
--
-- Build a stream by unfolding a /pure/ step function @step@ starting from a
-- seed @s@.  The step function returns the next element in the stream and the
-- next seed value. When it is done it returns 'Nothing' and the stream ends.
-- For example,
--
-- >>> :{
-- let f b =
--         if b > 2
--         then Nothing
--         else Just (b, b + 1)
-- in StreamK.toList $ StreamK.unfoldr f 0
-- :}
-- [0,1,2]
--
{-# INLINE unfoldr #-}
unfoldr :: (b -> Maybe (a, b)) -> b -> StreamK m a
unfoldr :: forall b a (m :: * -> *). (b -> Maybe (a, b)) -> b -> StreamK m a
unfoldr b -> Maybe (a, b)
next b
s0 = (forall b. (a -> b -> b) -> b -> b) -> StreamK m a
forall (m :: * -> *) a.
(forall b. (a -> b -> b) -> b -> b) -> StreamK m a
build ((forall b. (a -> b -> b) -> b -> b) -> StreamK m a)
-> (forall b. (a -> b -> b) -> b -> b) -> StreamK m a
forall a b. (a -> b) -> a -> b
$ \a -> b -> b
yld b
stp ->
    let go :: b -> b
go b
s =
            case b -> Maybe (a, b)
next b
s of
                Just (a
a, b
b) -> a -> b -> b
yld a
a (b -> b
go b
b)
                Maybe (a, b)
Nothing -> b
stp
    in b -> b
go b
s0

{-# INLINE unfoldrMWith #-}
unfoldrMWith :: Monad m =>
       (m a -> StreamK m a -> StreamK m a)
    -> (b -> m (Maybe (a, b)))
    -> b
    -> StreamK m a
unfoldrMWith :: forall (m :: * -> *) a b.
Monad m =>
(m a -> StreamK m a -> StreamK m a)
-> (b -> m (Maybe (a, b))) -> b -> StreamK m a
unfoldrMWith m a -> StreamK m a -> StreamK m a
cns b -> m (Maybe (a, b))
step = b -> StreamK m a
go

    where

    go :: b -> StreamK m a
go b
s = (m a -> StreamK m a -> StreamK m a)
-> (forall r.
    (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
Monad m =>
(m a -> StreamK m a -> StreamK m a)
-> (forall r.
    (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
sharedMWith m a -> StreamK m a -> StreamK m a
cns ((forall r. (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \a -> StreamK m a -> m r
yld a -> m r
_ m r
stp -> do
                Maybe (a, b)
r <- b -> m (Maybe (a, b))
step b
s
                case Maybe (a, b)
r of
                    Just (a
a, b
b) -> a -> StreamK m a -> m r
yld a
a (b -> StreamK m a
go b
b)
                    Maybe (a, b)
Nothing -> m r
stp

-- | Build a stream by unfolding a /monadic/ step function starting from a
-- seed.  The step function returns the next element in the stream and the next
-- seed value. When it is done it returns 'Nothing' and the stream ends. For
-- example,
--
-- >>> :{
-- let f b =
--         if b > 2
--         then return Nothing
--         else return (Just (b, b + 1))
-- in StreamK.toList $ StreamK.unfoldrM f 0
-- :}
-- [0,1,2]
--
{-# INLINE unfoldrM #-}
unfoldrM :: Monad m => (b -> m (Maybe (a, b))) -> b -> StreamK m a
unfoldrM :: forall (m :: * -> *) b a.
Monad m =>
(b -> m (Maybe (a, b))) -> b -> StreamK m a
unfoldrM = (m a -> StreamK m a -> StreamK m a)
-> (b -> m (Maybe (a, b))) -> b -> StreamK m a
forall (m :: * -> *) a b.
Monad m =>
(m a -> StreamK m a -> StreamK m a)
-> (b -> m (Maybe (a, b))) -> b -> StreamK m a
unfoldrMWith m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
consM

-- | Generate an infinite stream by repeating a pure value.
--
-- >>> repeat x = let xs = StreamK.cons x xs in xs
--
-- /Pre-release/
{-# INLINE repeat #-}
repeat :: a -> StreamK m a
repeat :: forall a (m :: * -> *). a -> StreamK m a
repeat a
x = let xs :: StreamK m a
xs = a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons a
x StreamK m a
xs in StreamK m a
forall {m :: * -> *}. StreamK m a
xs

-- | Like 'repeatM' but takes a stream 'cons' operation to combine the actions
-- in a stream specific manner. A serial cons would repeat the values serially
-- while an async cons would repeat concurrently.
--
-- /Pre-release/
repeatMWith :: (m a -> t m a -> t m a) -> m a -> t m a
repeatMWith :: forall (m :: * -> *) a (t :: (* -> *) -> * -> *).
(m a -> t m a -> t m a) -> m a -> t m a
repeatMWith m a -> t m a -> t m a
cns = m a -> t m a
go

    where

    go :: m a -> t m a
go m a
m = m a
m m a -> t m a -> t m a
`cns` m a -> t m a
go m a
m

{-# INLINE replicateMWith #-}
replicateMWith :: (m a -> StreamK m a -> StreamK m a) -> Int -> m a -> StreamK m a
replicateMWith :: forall (m :: * -> *) a.
(m a -> StreamK m a -> StreamK m a) -> Int -> m a -> StreamK m a
replicateMWith m a -> StreamK m a -> StreamK m a
cns Int
n m a
m = Int -> StreamK m a
forall {t}. (Ord t, Num t) => t -> StreamK m a
go Int
n

    where

    go :: t -> StreamK m a
go t
cnt = if t
cnt t -> t -> Bool
forall a. Ord a => a -> a -> Bool
<= t
0 then StreamK m a
forall (m :: * -> *) a. StreamK m a
nil else m a
m m a -> StreamK m a -> StreamK m a
`cns` t -> StreamK m a
go (t
cnt t -> t -> t
forall a. Num a => a -> a -> a
- t
1)

{-# INLINE fromIndicesMWith #-}
fromIndicesMWith ::
    (m a -> StreamK m a -> StreamK m a) -> (Int -> m a) -> StreamK m a
fromIndicesMWith :: forall (m :: * -> *) a.
(m a -> StreamK m a -> StreamK m a) -> (Int -> m a) -> StreamK m a
fromIndicesMWith m a -> StreamK m a -> StreamK m a
cns Int -> m a
gen = Int -> StreamK m a
go Int
0

    where

    go :: Int -> StreamK m a
go Int
i = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
stp a -> m r
sng m r
yld -> do
        State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
stp a -> m r
sng m r
yld (Int -> m a
gen Int
i m a -> StreamK m a -> StreamK m a
`cns` Int -> StreamK m a
go (Int
i Int -> Int -> Int
forall a. Num a => a -> a -> a
+ Int
1))

{-# INLINE iterateMWith #-}
iterateMWith :: Monad m =>
    (m a -> StreamK m a -> StreamK m a) -> (a -> m a) -> m a -> StreamK m a
iterateMWith :: forall (m :: * -> *) a.
Monad m =>
(m a -> StreamK m a -> StreamK m a)
-> (a -> m a) -> m a -> StreamK m a
iterateMWith m a -> StreamK m a -> StreamK m a
cns a -> m a
step = m a -> StreamK m a
go

    where

    go :: m a -> StreamK m a
go m a
s = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
stp a -> m r
sng m r
yld -> do
        !a
next <- m a
s
        State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
stp a -> m r
sng m r
yld (a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return a
next m a -> StreamK m a -> StreamK m a
`cns` m a -> StreamK m a
go (a -> m a
step a
next))

-- | head for non-empty streams, fails for empty stream case.
--
{-# INLINE headNonEmpty #-}
headNonEmpty :: Monad m => StreamK m a -> m a
headNonEmpty :: forall (m :: * -> *) a. Monad m => StreamK m a -> m a
headNonEmpty = (a -> m a -> m a) -> m a -> StreamK m a -> m a
forall a (m :: * -> *) b.
(a -> m b -> m b) -> m b -> StreamK m a -> m b
foldrM (\a
x m a
_ -> a -> m a
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return a
x) (String -> m a
forall a. HasCallStack => String -> a
error String
"headNonEmpty: empty stream")

-- | init for non-empty streams, fails for empty stream case.
--
-- See also 'init' for a non-partial version of this function..
{-# INLINE initNonEmpty #-}
initNonEmpty :: Stream m a -> Stream m a
initNonEmpty :: forall (m :: * -> *) a. StreamK m a -> StreamK m a
initNonEmpty = StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a
go0

    where

    go0 :: StreamK m a -> StreamK m a
go0 StreamK m a
m = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
        let stop :: a
stop = String -> a
forall a. HasCallStack => String -> a
error String
"initNonEmpty: Empty Stream."
            single :: p -> m r
single p
_ = m r
stp
            yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
go1 a
a StreamK m a
r)
         in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yieldk a -> m r
forall {p}. p -> m r
single m r
forall {a}. a
stop StreamK m a
m

    go1 :: t -> StreamK m t -> StreamK m t
go1 t
a StreamK m t
r = (forall r.
 State StreamK m t
 -> (t -> StreamK m t -> m r) -> (t -> m r) -> m r -> m r)
-> StreamK m t
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m t
  -> (t -> StreamK m t -> m r) -> (t -> m r) -> m r -> m r)
 -> StreamK m t)
-> (forall r.
    State StreamK m t
    -> (t -> StreamK m t -> m r) -> (t -> m r) -> m r -> m r)
-> StreamK m t
forall a b. (a -> b) -> a -> b
$ \State StreamK m t
st t -> StreamK m t -> m r
yld t -> m r
sng m r
stp ->
        let stop :: m r
stop = m r
stp
            single :: p -> m r
single p
_ = t -> m r
sng t
a
            yieldk :: t -> StreamK m t -> m r
yieldk t
a1 StreamK m t
r1 = t -> StreamK m t -> m r
yld t
a (t -> StreamK m t -> StreamK m t
go1 t
a1 StreamK m t
r1)
         in State StreamK m t
-> (t -> StreamK m t -> m r)
-> (t -> m r)
-> m r
-> StreamK m t
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m t
st t -> StreamK m t -> m r
yieldk t -> m r
forall {p}. p -> m r
single m r
stop StreamK m t
r

-- | tail for non-empty streams, fails for empty stream case.
--
-- See also 'tail' for a non-partial version of this function..
--
-- Note: this is same as "drop 1" with error on empty stream.
{-# INLINE tailNonEmpty #-}
tailNonEmpty :: StreamK m a -> StreamK m a
tailNonEmpty :: forall (m :: * -> *) a. StreamK m a -> StreamK m a
tailNonEmpty StreamK m a
m = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
    let stop :: a
stop      = String -> a
forall a. HasCallStack => String -> a
error String
"tailNonEmpty: empty stream"
        single :: p -> m r
single p
_  = m r
stp
        yieldk :: p -> StreamK m a -> m r
yieldk p
_ StreamK m a
r = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp StreamK m a
r
    in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
forall {p}. p -> StreamK m a -> m r
yieldk a -> m r
forall {p}. p -> m r
single m r
forall {a}. a
stop StreamK m a
m

-- | We can define cyclic structures using @let@:
--
-- >>> :set -fno-warn-unrecognised-warning-flags
-- >>> :set -fno-warn-x-partial
-- >>> let (a, b) = ([1, b], head a) in (a, b)
-- ([1,1],1)
--
-- The function @fix@ defined as:
--
-- >>> fix f = let x = f x in x
--
-- ensures that the argument of a function and its output refer to the same
-- lazy value @x@ i.e.  the same location in memory.  Thus @x@ can be defined
-- in terms of itself, creating structures with cyclic references.
--
-- >>> f ~(a, b) = ([1, b], head a)
-- >>> fix f
-- ([1,1],1)
--
-- 'Control.Monad.mfix' is essentially the same as @fix@ but for monadic
-- values.
--
-- Using 'mfix' for streams we can construct a stream in which each element of
-- the stream is defined in a cyclic fashion. The argument of the function
-- being fixed represents the current element of the stream which is being
-- returned by the stream monad. Thus, we can use the argument to construct
-- itself.
--
-- In the following example, the argument @action@ of the function @f@
-- represents the tuple @(x,y)@ returned by it in a given iteration. We define
-- the first element of the tuple in terms of the second.
--
-- >>> import System.IO.Unsafe (unsafeInterleaveIO)
--
-- >>> :{
-- main = Stream.fold (Fold.drainMapM print) $ StreamK.toStream $ StreamK.mfix f
--     where
--     f action = StreamK.unNested $ do
--         let incr n act = fmap ((+n) . snd) $ unsafeInterleaveIO act
--         x <- StreamK.Nested $ StreamK.fromStream $ Stream.sequence $ Stream.fromList [incr 1 action, incr 2 action]
--         y <- StreamK.Nested $ StreamK.fromStream $ Stream.fromList [4,5]
--         return (x, y)
-- :}
--
-- Note: you cannot achieve this by just changing the order of the monad
-- statements because that would change the order in which the stream elements
-- are generated.
--
-- Note that the function @f@ must be lazy in its argument, that's why we use
-- 'unsafeInterleaveIO' on @action@ because IO monad is strict.
--
-- /Pre-release/
{-# INLINE mfix #-}
mfix :: Monad m => (m a -> StreamK m a) -> StreamK m a
mfix :: forall (m :: * -> *) a.
Monad m =>
(m a -> StreamK m a) -> StreamK m a
mfix m a -> StreamK m a
f = (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
    let single :: a -> m r
single a
a  = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ a
a a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
`cons` StreamK m a
ys
        yieldk :: a -> p -> m r
yieldk a
a p
_ = State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (StreamK m a -> m r) -> StreamK m a -> m r
forall a b. (a -> b) -> a -> b
$ a
a a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
`cons` StreamK m a
ys
    in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
st a -> StreamK m a -> m r
forall {p}. a -> p -> m r
yieldk a -> m r
single m r
stp StreamK m a
xs

    where

    -- fix the head element of the stream
    xs :: StreamK m a
xs = (StreamK m a -> StreamK m a) -> StreamK m a
forall a. (a -> a) -> a
fix  (m a -> StreamK m a
f (m a -> StreamK m a)
-> (StreamK m a -> m a) -> StreamK m a -> StreamK m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. StreamK m a -> m a
forall (m :: * -> *) a. Monad m => StreamK m a -> m a
headNonEmpty)

    -- now fix the tail recursively
    ys :: StreamK m a
ys = (m a -> StreamK m a) -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
(m a -> StreamK m a) -> StreamK m a
mfix (StreamK m a -> StreamK m a
forall (m :: * -> *) a. StreamK m a -> StreamK m a
tailNonEmpty (StreamK m a -> StreamK m a)
-> (m a -> StreamK m a) -> m a -> StreamK m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. m a -> StreamK m a
f)

-------------------------------------------------------------------------------
-- Conversions
-------------------------------------------------------------------------------

-- |
-- >>> fromFoldable = Prelude.foldr StreamK.cons StreamK.nil
--
-- Construct a stream from a 'Foldable' containing pure values:
--
{-# INLINE fromFoldable #-}
fromFoldable :: Foldable f => f a -> StreamK m a
fromFoldable :: forall (f :: * -> *) a (m :: * -> *).
Foldable f =>
f a -> StreamK m a
fromFoldable = (a -> StreamK m a -> StreamK m a)
-> StreamK m a -> f a -> StreamK m a
forall a b. (a -> b -> b) -> b -> f a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
Prelude.foldr a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons StreamK m a
forall (m :: * -> *) a. StreamK m a
nil

{-# INLINE fromFoldableM #-}
fromFoldableM :: (Foldable f, Monad m) => f (m a) -> StreamK m a
fromFoldableM :: forall (f :: * -> *) (m :: * -> *) a.
(Foldable f, Monad m) =>
f (m a) -> StreamK m a
fromFoldableM = (m a -> StreamK m a -> StreamK m a)
-> StreamK m a -> f (m a) -> StreamK m a
forall a b. (a -> b -> b) -> b -> f a -> b
forall (t :: * -> *) a b.
Foldable t =>
(a -> b -> b) -> b -> t a -> b
Prelude.foldr m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) a.
Monad m =>
m a -> StreamK m a -> StreamK m a
consM StreamK m a
forall (m :: * -> *) a. StreamK m a
nil

{-# INLINE fromList #-}
fromList :: [a] -> StreamK m a
fromList :: forall a (m :: * -> *). [a] -> StreamK m a
fromList = [a] -> StreamK m a
forall (f :: * -> *) a (m :: * -> *).
Foldable f =>
f a -> StreamK m a
fromFoldable

-------------------------------------------------------------------------------
-- Deconstruction
-------------------------------------------------------------------------------

{-# INLINE uncons #-}
uncons :: Applicative m => StreamK m a -> m (Maybe (a, StreamK m a))
uncons :: forall (m :: * -> *) a.
Applicative m =>
StreamK m a -> m (Maybe (a, StreamK m a))
uncons StreamK m a
m =
    let stop :: m (Maybe a)
stop = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe a
forall a. Maybe a
Nothing
        single :: a -> f (Maybe (a, StreamK m a))
single a
a = Maybe (a, StreamK m a) -> f (Maybe (a, StreamK m a))
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ((a, StreamK m a) -> Maybe (a, StreamK m a)
forall a. a -> Maybe a
Just (a
a, StreamK m a
forall (m :: * -> *) a. StreamK m a
nil))
        yieldk :: a -> b -> f (Maybe (a, b))
yieldk a
a b
r = Maybe (a, b) -> f (Maybe (a, b))
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ((a, b) -> Maybe (a, b)
forall a. a -> Maybe a
Just (a
a, b
r))
    in State StreamK m a
-> (a -> StreamK m a -> m (Maybe (a, StreamK m a)))
-> (a -> m (Maybe (a, StreamK m a)))
-> m (Maybe (a, StreamK m a))
-> StreamK m a
-> m (Maybe (a, StreamK m a))
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState a -> StreamK m a -> m (Maybe (a, StreamK m a))
forall {f :: * -> *} {a} {b}.
Applicative f =>
a -> b -> f (Maybe (a, b))
yieldk a -> m (Maybe (a, StreamK m a))
forall {f :: * -> *} {a} {m :: * -> *} {a}.
Applicative f =>
a -> f (Maybe (a, StreamK m a))
single m (Maybe (a, StreamK m a))
forall {a}. m (Maybe a)
stop StreamK m a
m

-- Note that this is not a StreamK -> StreamK because then we cannot handle the
-- empty stream case without making this a partial function.
--
-- See tailNonEmpty as well above.

-- | Same as:
--
-- >>> tail = fmap (fmap snd) . StreamK.uncons
--
{-# INLINE tail #-}
tail :: Applicative m => StreamK m a -> m (Maybe (StreamK m a))
tail :: forall (m :: * -> *) a.
Applicative m =>
StreamK m a -> m (Maybe (StreamK m a))
tail =
    let stop :: m (Maybe a)
stop      = Maybe a -> m (Maybe a)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe a
forall a. Maybe a
Nothing
        single :: p -> f (Maybe (StreamK m a))
single p
_  = Maybe (StreamK m a) -> f (Maybe (StreamK m a))
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (StreamK m a) -> f (Maybe (StreamK m a)))
-> Maybe (StreamK m a) -> f (Maybe (StreamK m a))
forall a b. (a -> b) -> a -> b
$ StreamK m a -> Maybe (StreamK m a)
forall a. a -> Maybe a
Just StreamK m a
forall (m :: * -> *) a. StreamK m a
nil
        yieldk :: p -> a -> f (Maybe a)
yieldk p
_ a
r = Maybe a -> f (Maybe a)
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe a -> f (Maybe a)) -> Maybe a -> f (Maybe a)
forall a b. (a -> b) -> a -> b
$ a -> Maybe a
forall a. a -> Maybe a
Just a
r
    in State StreamK m a
-> (a -> StreamK m a -> m (Maybe (StreamK m a)))
-> (a -> m (Maybe (StreamK m a)))
-> m (Maybe (StreamK m a))
-> StreamK m a
-> m (Maybe (StreamK m a))
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState a -> StreamK m a -> m (Maybe (StreamK m a))
forall {f :: * -> *} {p} {a}.
Applicative f =>
p -> a -> f (Maybe a)
yieldk a -> m (Maybe (StreamK m a))
forall {f :: * -> *} {p} {m :: * -> *} {a}.
Applicative f =>
p -> f (Maybe (StreamK m a))
single m (Maybe (StreamK m a))
forall {a}. m (Maybe a)
stop

-- Note that this is not a StreamK -> StreamK because then we cannot handle the
-- empty stream case without making this a partial function.
--
-- XXX How do we implement unsnoc? Make StreamK a monad and return the
-- remaining stream as a result value in the monad?

-- | Extract all but the last element of the stream, if any. This will end up
-- evaluating the last element as well to find out that it is last.
--
-- /Pre-release/
{-# INLINE init #-}
init :: Applicative m => StreamK m a -> m (Maybe (StreamK m a))
init :: forall (m :: * -> *) a.
Applicative m =>
StreamK m a -> m (Maybe (StreamK m a))
init = StreamK m a -> m (Maybe (StreamK m a))
forall (m :: * -> *) a.
Applicative m =>
StreamK m a -> m (Maybe (StreamK m a))
go1
    where
    go1 :: StreamK f t -> f (Maybe (StreamK f t))
go1 StreamK f t
m1 = do
        (\case
            Maybe (t, StreamK f t)
Nothing -> Maybe (StreamK f t)
forall a. Maybe a
Nothing
            Just (t
h, StreamK f t
t) -> StreamK f t -> Maybe (StreamK f t)
forall a. a -> Maybe a
Just (StreamK f t -> Maybe (StreamK f t))
-> StreamK f t -> Maybe (StreamK f t)
forall a b. (a -> b) -> a -> b
$ t -> StreamK f t -> StreamK f t
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
go t
h StreamK f t
t) (Maybe (t, StreamK f t) -> Maybe (StreamK f t))
-> f (Maybe (t, StreamK f t)) -> f (Maybe (StreamK f t))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> StreamK f t -> f (Maybe (t, StreamK f t))
forall (m :: * -> *) a.
Applicative m =>
StreamK m a -> m (Maybe (a, StreamK m a))
uncons StreamK f t
m1
    go :: t -> StreamK m t -> StreamK m t
go t
p StreamK m t
m1 = (forall r.
 State StreamK m t
 -> (t -> StreamK m t -> m r) -> (t -> m r) -> m r -> m r)
-> StreamK m t
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m t
  -> (t -> StreamK m t -> m r) -> (t -> m r) -> m r -> m r)
 -> StreamK m t)
-> (forall r.
    State StreamK m t
    -> (t -> StreamK m t -> m r) -> (t -> m r) -> m r -> m r)
-> StreamK m t
forall a b. (a -> b) -> a -> b
$ \State StreamK m t
_ t -> StreamK m t -> m r
yld t -> m r
sng m r
stp ->
        let single :: p -> m r
single p
_ = t -> m r
sng t
p
            yieldk :: t -> StreamK m t -> m r
yieldk t
a StreamK m t
x = t -> StreamK m t -> m r
yld t
p (StreamK m t -> m r) -> StreamK m t -> m r
forall a b. (a -> b) -> a -> b
$ t -> StreamK m t -> StreamK m t
go t
a StreamK m t
x
         in State StreamK m t
-> (t -> StreamK m t -> m r)
-> (t -> m r)
-> m r
-> StreamK m t
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m t
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a. State t m a
defState t -> StreamK m t -> m r
yieldk t -> m r
forall {p}. p -> m r
single m r
stp StreamK m t
m1

------------------------------------------------------------------------------
-- Reordering
------------------------------------------------------------------------------

-- | Lazy left fold to a stream.
{-# INLINE foldlS #-}
foldlS ::
    (StreamK m b -> a -> StreamK m b) -> StreamK m b -> StreamK m a -> StreamK m b
foldlS :: forall (m :: * -> *) b a.
(StreamK m b -> a -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldlS StreamK m b -> a -> StreamK m b
step = StreamK m b -> StreamK m a -> StreamK m b
go
    where
    go :: StreamK m b -> StreamK m a -> StreamK m b
go StreamK m b
acc StreamK m a
rest = (forall r.
 State StreamK m b
 -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m b
  -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
 -> StreamK m b)
-> (forall r.
    State StreamK m b
    -> (b -> StreamK m b -> m r) -> (b -> m r) -> m r -> m r)
-> StreamK m b
forall a b. (a -> b) -> a -> b
$ \State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp ->
        let run :: StreamK m b -> m r
run StreamK m b
x = State StreamK m b
-> (b -> StreamK m b -> m r)
-> (b -> m r)
-> m r
-> StreamK m b
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream State StreamK m b
st b -> StreamK m b -> m r
yld b -> m r
sng m r
stp StreamK m b
x
            stop :: m r
stop = StreamK m b -> m r
run StreamK m b
acc
            single :: a -> m r
single a
a = StreamK m b -> m r
run (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> a -> StreamK m b
step StreamK m b
acc a
a
            yieldk :: a -> StreamK m a -> m r
yieldk a
a StreamK m a
r = StreamK m b -> m r
run (StreamK m b -> m r) -> StreamK m b -> m r
forall a b. (a -> b) -> a -> b
$ StreamK m b -> StreamK m a -> StreamK m b
go (StreamK m b -> a -> StreamK m b
step StreamK m b
acc a
a) StreamK m a
r
         in State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStream (State StreamK m b -> State StreamK m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a (n :: * -> *) b.
State t m a -> State t n b
adaptState State StreamK m b
st) a -> StreamK m a -> m r
yieldk a -> m r
single m r
stop StreamK m a
rest

{-# INLINE reverse #-}
reverse :: StreamK m a -> StreamK m a
reverse :: forall (m :: * -> *) a. StreamK m a -> StreamK m a
reverse = (StreamK m a -> a -> StreamK m a)
-> StreamK m a -> StreamK m a -> StreamK m a
forall (m :: * -> *) b a.
(StreamK m b -> a -> StreamK m b)
-> StreamK m b -> StreamK m a -> StreamK m b
foldlS ((a -> StreamK m a -> StreamK m a)
-> StreamK m a -> a -> StreamK m a
forall a b c. (a -> b -> c) -> b -> a -> c
flip a -> StreamK m a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a -> StreamK m a
cons) StreamK m a
forall (m :: * -> *) a. StreamK m a
nil

------------------------------------------------------------------------------
-- Running effects
------------------------------------------------------------------------------

-- | Run an action before evaluating the stream.
{-# INLINE before #-}
before :: Monad m => m b -> StreamK m a -> StreamK m a
before :: forall (m :: * -> *) b a.
Monad m =>
m b -> StreamK m a -> StreamK m a
before m b
action StreamK m a
stream =
    (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
        m b
action m b -> m r -> m r
forall a b. m a -> m b -> m b
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp StreamK m a
stream

-- XXX Rename to "impure" (opposite of pure) or "purely".
{-# INLINE concatEffect #-}
concatEffect :: Monad m => m (StreamK m a) -> StreamK m a
concatEffect :: forall (m :: * -> *) a. Monad m => m (StreamK m a) -> StreamK m a
concatEffect m (StreamK m a)
action =
    (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
        m (StreamK m a)
action m (StreamK m a) -> (StreamK m a -> m r) -> m r
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp

{-# INLINE concatMapEffect #-}
concatMapEffect :: Monad m => (b -> StreamK m a) -> m b -> StreamK m a
concatMapEffect :: forall (m :: * -> *) b a.
Monad m =>
(b -> StreamK m a) -> m b -> StreamK m a
concatMapEffect b -> StreamK m a
f m b
action =
    (forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall (m :: * -> *) a.
(forall r.
 State StreamK m a
 -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
mkStream ((forall r.
  State StreamK m a
  -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
 -> StreamK m a)
-> (forall r.
    State StreamK m a
    -> (a -> StreamK m a -> m r) -> (a -> m r) -> m r -> m r)
-> StreamK m a
forall a b. (a -> b) -> a -> b
$ \State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp ->
        m b
action m b -> (b -> m r) -> m r
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
forall (m :: * -> *) a r.
State StreamK m a
-> (a -> StreamK m a -> m r)
-> (a -> m r)
-> m r
-> StreamK m a
-> m r
foldStreamShared State StreamK m a
st a -> StreamK m a -> m r
yld a -> m r
sng m r
stp (StreamK m a -> m r) -> (b -> StreamK m a) -> b -> m r
forall b c a. (b -> c) -> (a -> b) -> a -> c
. b -> StreamK m a
f

------------------------------------------------------------------------------
-- Stream with a cross product style monad instance
------------------------------------------------------------------------------

-- XXX add Alternative, MonadPlus - should we use interleave as the Semigroup
-- append operation in FairNested?

-- | 'Nested' is a list-transformer monad, it serves the same purpose as the
-- @ListT@ type from the @list-t@ package. It is similar to the standard
-- Haskell lists' monad instance. 'Nested' monad behaves like nested @for@ loops
-- implementing a computation based on a cross product over multiple streams.
--
-- >>> mk = StreamK.Nested . StreamK.fromStream . Stream.fromList
-- >>> un = Stream.toList . StreamK.toStream . StreamK.unNested
--
-- == Looping
--
-- In the following code the variable @x@ assumes values of the elements of the
-- stream one at a time and runs the code that follows; using that value. It is
-- equivalent to a @for@ loop:
--
-- >>> :{
-- un $ do
--     x <- mk [1,2,3] -- for each element in the stream
--     return x
-- :}
-- [1,2,3]
--
-- == Nested Looping
--
-- Multiple streams can be nested like nested @for@ loops. The result is a
-- cross product of the streams.
--
-- >>> :{
-- un $ do
--     x <- mk [1,2,3] -- outer loop, for each element in the stream
--     y <- mk [4,5,6] -- inner loop, for each element in the stream
--     return (x, y)
-- :}
-- [(1,4),(1,5),(1,6),(2,4),(2,5),(2,6),(3,4),(3,5),(3,6)]
--
-- Note that an infinite stream in an inner loop will block the outer streams
-- from moving to the next iteration.
--
-- == How it works?
--
-- The bind operation of the monad is flipped 'concatMapWith' 'append'. The
-- concatMap operation maps the lines involving y as a function of x over the
-- stream [1,2,3]. The streams generated so are combined using the 'append'
-- operation. If we desugar the above monad code using bind explicitly, it
-- becomes clear how it works:
--
-- >>> import Streamly.Internal.Data.StreamK (Nested(..))
-- >>> (Nested m) >>= f = Nested $ StreamK.concatMapWith StreamK.append (unNested . f) m
-- >>> un (mk [1,2,3] >>= (\x -> (mk [4,5,6] >>= \y -> return (x,y))))
-- [(1,4),(1,5),(1,6),(2,4),(2,5),(2,6),(3,4),(3,5),(3,6)]
--
-- You can achieve the looping and nested looping by directly using concatMap
-- but the monad and the \"do notation\" gives you better ergonomics.
--
-- == Interleaving of loop iterations
--
-- If we look at the cross product of [1,2,3], [4,5,6], the streams being
-- combined using 'append' are the @for@ loop iterations as follows:
--
-- @
-- (1,4) (1,5) (1,6) -- first iteration of the outer loop
-- (2,4) (2,5) (2,6) -- second iteration of the outer loop
-- (3,4) (3,5) (3,6) -- third iteration of the outer loop
-- @
--
-- The result is equivalent to sequentially appending all the iterations of the
-- nested @for@ loop:
--
-- @
-- [(1,4),(1,5),(1,6),(2,4),(2,5),(2,6),(3,4),(3,5),(3,6)]
-- @
--
-- == Logic Programming
--
-- 'Nested' also serves the purpose of 'LogicT' type from the 'logict' package.
-- The @MonadLogic@ operations can be implemented using the available stream
-- operations. For example, 'uncons' is @msplit@, 'interleave' corresponds to
-- the @interleave@ operation of MonadLogic, 'fairConcatFor' is the
-- fair bind (@>>-@) operation. The 'FairNested' type provides a monad with fair
-- bind.
--
-- == Related Functionality
--
-- A custom type can be created using 'bfsConcatMap' as the monad bind
-- operation then the nested loops would get inverted - the innermost loop
-- becomes the outermost and vice versa.
--
-- See 'FairNested' if you want all the streams to get equal chance to execute
-- even if they are infinite.
newtype Nested m a = Nested {forall (m :: * -> *) a. Nested m a -> StreamK m a
unNested :: StreamK m a}
        deriving ((forall a b. (a -> b) -> Nested m a -> Nested m b)
-> (forall a b. a -> Nested m b -> Nested m a)
-> Functor (Nested m)
forall a b. a -> Nested m b -> Nested m a
forall a b. (a -> b) -> Nested m a -> Nested m b
forall (m :: * -> *) a b. Monad m => a -> Nested m b -> Nested m a
forall (m :: * -> *) a b.
Monad m =>
(a -> b) -> Nested m a -> Nested m b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall (m :: * -> *) a b.
Monad m =>
(a -> b) -> Nested m a -> Nested m b
fmap :: forall a b. (a -> b) -> Nested m a -> Nested m b
$c<$ :: forall (m :: * -> *) a b. Monad m => a -> Nested m b -> Nested m a
<$ :: forall a b. a -> Nested m b -> Nested m a
Functor, NonEmpty (Nested m a) -> Nested m a
Nested m a -> Nested m a -> Nested m a
(Nested m a -> Nested m a -> Nested m a)
-> (NonEmpty (Nested m a) -> Nested m a)
-> (forall b. Integral b => b -> Nested m a -> Nested m a)
-> Semigroup (Nested m a)
forall b. Integral b => b -> Nested m a -> Nested m a
forall a.
(a -> a -> a)
-> (NonEmpty a -> a)
-> (forall b. Integral b => b -> a -> a)
-> Semigroup a
forall (m :: * -> *) a. NonEmpty (Nested m a) -> Nested m a
forall (m :: * -> *) a. Nested m a -> Nested m a -> Nested m a
forall (m :: * -> *) a b.
Integral b =>
b -> Nested m a -> Nested m a
$c<> :: forall (m :: * -> *) a. Nested m a -> Nested m a -> Nested m a
<> :: Nested m a -> Nested m a -> Nested m a
$csconcat :: forall (m :: * -> *) a. NonEmpty (Nested m a) -> Nested m a
sconcat :: NonEmpty (Nested m a) -> Nested m a
$cstimes :: forall (m :: * -> *) a b.
Integral b =>
b -> Nested m a -> Nested m a
stimes :: forall b. Integral b => b -> Nested m a -> Nested m a
Semigroup, Semigroup (Nested m a)
Nested m a
Semigroup (Nested m a) =>
Nested m a
-> (Nested m a -> Nested m a -> Nested m a)
-> ([Nested m a] -> Nested m a)
-> Monoid (Nested m a)
[Nested m a] -> Nested m a
Nested m a -> Nested m a -> Nested m a
forall a.
Semigroup a =>
a -> (a -> a -> a) -> ([a] -> a) -> Monoid a
forall (m :: * -> *) a. Semigroup (Nested m a)
forall (m :: * -> *) a. Nested m a
forall (m :: * -> *) a. [Nested m a] -> Nested m a
forall (m :: * -> *) a. Nested m a -> Nested m a -> Nested m a
$cmempty :: forall (m :: * -> *) a. Nested m a
mempty :: Nested m a
$cmappend :: forall (m :: * -> *) a. Nested m a -> Nested m a -> Nested m a
mappend :: Nested m a -> Nested m a -> Nested m a
$cmconcat :: forall (m :: * -> *) a. [Nested m a] -> Nested m a
mconcat :: [Nested m a] -> Nested m a
Monoid, (forall m. Monoid m => Nested m m -> m)
-> (forall m a. Monoid m => (a -> m) -> Nested m a -> m)
-> (forall m a. Monoid m => (a -> m) -> Nested m a -> m)
-> (forall a b. (a -> b -> b) -> b -> Nested m a -> b)
-> (forall a b. (a -> b -> b) -> b -> Nested m a -> b)
-> (forall b a. (b -> a -> b) -> b -> Nested m a -> b)
-> (forall b a. (b -> a -> b) -> b -> Nested m a -> b)
-> (forall a. (a -> a -> a) -> Nested m a -> a)
-> (forall a. (a -> a -> a) -> Nested m a -> a)
-> (forall a. Nested m a -> [a])
-> (forall a. Nested m a -> Bool)
-> (forall a. Nested m a -> Int)
-> (forall a. Eq a => a -> Nested m a -> Bool)
-> (forall a. Ord a => Nested m a -> a)
-> (forall a. Ord a => Nested m a -> a)
-> (forall a. Num a => Nested m a -> a)
-> (forall a. Num a => Nested m a -> a)
-> Foldable (Nested m)
forall a. Eq a => a -> Nested m a -> Bool
forall a. Num a => Nested m a -> a
forall a. Ord a => Nested m a -> a
forall m. Monoid m => Nested m m -> m
forall a. Nested m a -> Bool
forall a. Nested m a -> Int
forall a. Nested m a -> [a]
forall a. (a -> a -> a) -> Nested m a -> a
forall m a. Monoid m => (a -> m) -> Nested m a -> m
forall b a. (b -> a -> b) -> b -> Nested m a -> b
forall a b. (a -> b -> b) -> b -> Nested m a -> b
forall (m :: * -> *) a.
(Foldable m, Monad m, Eq a) =>
a -> Nested m a -> Bool
forall (m :: * -> *) a.
(Foldable m, Monad m, Num a) =>
Nested m a -> a
forall (m :: * -> *) a.
(Foldable m, Monad m, Ord a) =>
Nested m a -> a
forall (m :: * -> *) m.
(Foldable m, Monad m, Monoid m) =>
Nested m m -> m
forall (m :: * -> *) a. (Foldable m, Monad m) => Nested m a -> Bool
forall (m :: * -> *) a. (Foldable m, Monad m) => Nested m a -> Int
forall (m :: * -> *) a. (Foldable m, Monad m) => Nested m a -> [a]
forall (m :: * -> *) a.
(Foldable m, Monad m) =>
(a -> a -> a) -> Nested m a -> a
forall (m :: * -> *) m a.
(Foldable m, Monad m, Monoid m) =>
(a -> m) -> Nested m a -> m
forall (m :: * -> *) b a.
(Foldable m, Monad m) =>
(b -> a -> b) -> b -> Nested m a -> b
forall (m :: * -> *) a b.
(Foldable m, Monad m) =>
(a -> b -> b) -> b -> Nested m a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall (m :: * -> *) m.
(Foldable m, Monad m, Monoid m) =>
Nested m m -> m
fold :: forall m. Monoid m => Nested m m -> m
$cfoldMap :: forall (m :: * -> *) m a.
(Foldable m, Monad m, Monoid m) =>
(a -> m) -> Nested m a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> Nested m a -> m
$cfoldMap' :: forall (m :: * -> *) m a.
(Foldable m, Monad m, Monoid m) =>
(a -> m) -> Nested m a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> Nested m a -> m
$cfoldr :: forall (m :: * -> *) a b.
(Foldable m, Monad m) =>
(a -> b -> b) -> b -> Nested m a -> b
foldr :: forall a b. (a -> b -> b) -> b -> Nested m a -> b
$cfoldr' :: forall (m :: * -> *) a b.
(Foldable m, Monad m) =>
(a -> b -> b) -> b -> Nested m a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> Nested m a -> b
$cfoldl :: forall (m :: * -> *) b a.
(Foldable m, Monad m) =>
(b -> a -> b) -> b -> Nested m a -> b
foldl :: forall b a. (b -> a -> b) -> b -> Nested m a -> b
$cfoldl' :: forall (m :: * -> *) b a.
(Foldable m, Monad m) =>
(b -> a -> b) -> b -> Nested m a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> Nested m a -> b
$cfoldr1 :: forall (m :: * -> *) a.
(Foldable m, Monad m) =>
(a -> a -> a) -> Nested m a -> a
foldr1 :: forall a. (a -> a -> a) -> Nested m a -> a
$cfoldl1 :: forall (m :: * -> *) a.
(Foldable m, Monad m) =>
(a -> a -> a) -> Nested m a -> a
foldl1 :: forall a. (a -> a -> a) -> Nested m a -> a
$ctoList :: forall (m :: * -> *) a. (Foldable m, Monad m) => Nested m a -> [a]
toList :: forall a. Nested m a -> [a]
$cnull :: forall (m :: * -> *) a. (Foldable m, Monad m) => Nested m a -> Bool
null :: forall a. Nested m a -> Bool
$clength :: forall (m :: * -> *) a. (Foldable m, Monad m) => Nested m a -> Int
length :: forall a. Nested m a -> Int
$celem :: forall (m :: * -> *) a.
(Foldable m, Monad m, Eq a) =>
a -> Nested m a -> Bool
elem :: forall a. Eq a => a -> Nested m a -> Bool
$cmaximum :: forall (m :: * -> *) a.
(Foldable m, Monad m, Ord a) =>
Nested m a -> a
maximum :: forall a. Ord a => Nested m a -> a
$cminimum :: forall (m :: * -> *) a.
(Foldable m, Monad m, Ord a) =>
Nested m a -> a
minimum :: forall a. Ord a => Nested m a -> a
$csum :: forall (m :: * -> *) a.
(Foldable m, Monad m, Num a) =>
Nested m a -> a
sum :: forall a. Num a => Nested m a -> a
$cproduct :: forall (m :: * -> *) a.
(Foldable m, Monad m, Num a) =>
Nested m a -> a
product :: forall a. Num a => Nested m a -> a
Foldable)

{-# DEPRECATED CrossStreamK "Use Nested instead." #-}
type CrossStreamK = Nested

{-# DEPRECATED mkCross "Use Nested instead." #-}
-- | Wrap the 'StreamK' type in a 'Nested' newtype to enable cross
-- product style applicative and monad instances.
--
-- This is a type level operation with no runtime overhead.
{-# INLINE mkCross #-}
mkCross :: StreamK m a -> Nested m a
mkCross :: forall (m :: * -> *) a. StreamK m a -> Nested m a
mkCross = StreamK m a -> Nested m a
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested

-- | Unwrap the 'StreamK' type from 'CrossStreamK' newtype.
--
-- This is a type level operation with no runtime overhead.
{-# INLINE unCross #-}
unCross :: CrossStreamK m a -> StreamK m a
unCross :: forall (m :: * -> *) a. Nested m a -> StreamK m a
unCross = Nested m a -> StreamK m a
forall (m :: * -> *) a. Nested m a -> StreamK m a
unNested

-- Pure (Identity monad) stream instances
deriving instance Traversable (Nested Identity)
deriving instance IsList (Nested Identity a)
deriving instance (a ~ Char) => IsString (Nested Identity a)
-- deriving instance Eq a => Eq (Nested Identity a)
-- deriving instance Ord a => Ord (Nested Identity a)

-- Do not use automatic derivation for this to show as "fromList" rather than
-- "fromList Identity".
instance Show a => Show (Nested Identity a) where
    {-# INLINE show #-}
    show :: Nested Identity a -> String
show (Nested StreamK Identity a
xs) = StreamK Identity a -> String
forall a. Show a => a -> String
show StreamK Identity a
xs

instance Read a => Read (Nested Identity a) where
    {-# INLINE readPrec #-}
    readPrec :: ReadPrec (Nested Identity a)
readPrec = (StreamK Identity a -> Nested Identity a)
-> ReadPrec (StreamK Identity a) -> ReadPrec (Nested Identity a)
forall a b. (a -> b) -> ReadPrec a -> ReadPrec b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap StreamK Identity a -> Nested Identity a
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested ReadPrec (StreamK Identity a)
forall a. Read a => ReadPrec a
readPrec

------------------------------------------------------------------------------
-- Applicative
------------------------------------------------------------------------------

-- Note: we need to define all the typeclass operations because we want to
-- INLINE them.
instance Monad m => Applicative (Nested m) where
    {-# INLINE pure #-}
    pure :: forall a. a -> Nested m a
pure a
x = StreamK m a -> Nested m a
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested (a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a
fromPure a
x)

    {-# INLINE (<*>) #-}
    (Nested StreamK m (a -> b)
s1) <*> :: forall a b. Nested m (a -> b) -> Nested m a -> Nested m b
<*> (Nested StreamK m a
s2) =
        StreamK m b -> Nested m b
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested (StreamK m (a -> b) -> StreamK m a -> StreamK m b
forall (m :: * -> *) a b.
StreamK m (a -> b) -> StreamK m a -> StreamK m b
crossApply StreamK m (a -> b)
s1 StreamK m a
s2)

    {-# INLINE liftA2 #-}
    liftA2 :: forall a b c.
(a -> b -> c) -> Nested m a -> Nested m b -> Nested m c
liftA2 a -> b -> c
f Nested m a
x = Nested m (b -> c) -> Nested m b -> Nested m c
forall a b. Nested m (a -> b) -> Nested m a -> Nested m b
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
(<*>) ((a -> b -> c) -> Nested m a -> Nested m (b -> c)
forall a b. (a -> b) -> Nested m a -> Nested m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap a -> b -> c
f Nested m a
x)

    {-# INLINE (*>) #-}
    (Nested StreamK m a
s1) *> :: forall a b. Nested m a -> Nested m b -> Nested m b
*> (Nested StreamK m b
s2) =
        StreamK m b -> Nested m b
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested (StreamK m a -> StreamK m b -> StreamK m b
forall (m :: * -> *) a b. StreamK m a -> StreamK m b -> StreamK m b
crossApplySnd StreamK m a
s1 StreamK m b
s2)

    {-# INLINE (<*) #-}
    (Nested StreamK m a
s1) <* :: forall a b. Nested m a -> Nested m b -> Nested m a
<* (Nested StreamK m b
s2) =
        StreamK m a -> Nested m a
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested (StreamK m a -> StreamK m b -> StreamK m a
forall (m :: * -> *) a b. StreamK m a -> StreamK m b -> StreamK m a
crossApplyFst StreamK m a
s1 StreamK m b
s2)

------------------------------------------------------------------------------
-- Monad
------------------------------------------------------------------------------

instance Monad m => Monad (Nested m) where
    return :: forall a. a -> Nested m a
return = a -> Nested m a
forall a. a -> Nested m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure

    -- Benchmarks better with CPS bind and pure:
    -- Prime sieve (25x)
    -- n binds, breakAfterSome, filterAllIn, state transformer (~2x)
    --
    {-# INLINE (>>=) #-}
    >>= :: forall a b. Nested m a -> (a -> Nested m b) -> Nested m b
(>>=) (Nested StreamK m a
m) a -> Nested m b
f =
        StreamK m b -> Nested m b
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested ((StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
forall (m :: * -> *) b a.
(StreamK m b -> StreamK m b -> StreamK m b)
-> StreamK m a -> (a -> StreamK m b) -> StreamK m b
bindWith StreamK m b -> StreamK m b -> StreamK m b
forall (m :: * -> *) a. StreamK m a -> StreamK m a -> StreamK m a
append StreamK m a
m (Nested m b -> StreamK m b
forall (m :: * -> *) a. Nested m a -> StreamK m a
unNested (Nested m b -> StreamK m b)
-> (a -> Nested m b) -> a -> StreamK m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> Nested m b
f))

    {-# INLINE (>>) #-}
    >> :: forall a b. Nested m a -> Nested m b -> Nested m b
(>>) = Nested m a -> Nested m b -> Nested m b
forall a b. Nested m a -> Nested m b -> Nested m b
forall (f :: * -> *) a b. Applicative f => f a -> f b -> f b
(*>)

------------------------------------------------------------------------------
-- Alternative and MonadPlus
------------------------------------------------------------------------------

instance (Monad m) => Fail.MonadFail (Nested m) where
  fail :: forall a. String -> Nested m a
fail String
_ = Nested m a -> Nested m a
forall a. a -> a
inline Nested m a
forall a. Monoid a => a
mempty

instance (Monad m, Functor m) => Alternative (Nested m) where
  empty :: forall a. Nested m a
empty = Nested m a -> Nested m a
forall a. a -> a
inline Nested m a
forall a. Monoid a => a
mempty
  <|> :: forall a. Nested m a -> Nested m a -> Nested m a
(<|>) = (Nested m a -> Nested m a -> Nested m a)
-> Nested m a -> Nested m a -> Nested m a
forall a. a -> a
inline Nested m a -> Nested m a -> Nested m a
forall a. Monoid a => a -> a -> a
mappend

instance (Monad m) => MonadPlus (Nested m) where
  mzero :: forall a. Nested m a
mzero = Nested m a -> Nested m a
forall a. a -> a
inline Nested m a
forall a. Monoid a => a
mempty
  mplus :: forall a. Nested m a -> Nested m a -> Nested m a
mplus = (Nested m a -> Nested m a -> Nested m a)
-> Nested m a -> Nested m a -> Nested m a
forall a. a -> a
inline Nested m a -> Nested m a -> Nested m a
forall a. Monoid a => a -> a -> a
mappend

------------------------------------------------------------------------------
-- Transformers
------------------------------------------------------------------------------

instance (MonadIO m) => MonadIO (Nested m) where
    liftIO :: forall a. IO a -> Nested m a
liftIO IO a
x = StreamK m a -> Nested m a
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested (m a -> StreamK m a
forall (m :: * -> *) a. Monad m => m a -> StreamK m a
fromEffect (m a -> StreamK m a) -> m a -> StreamK m a
forall a b. (a -> b) -> a -> b
$ IO a -> m a
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO IO a
x)

instance MonadTrans Nested where
    {-# INLINE lift #-}
    lift :: forall (m :: * -> *) a. Monad m => m a -> Nested m a
lift m a
x = StreamK m a -> Nested m a
forall (m :: * -> *) a. StreamK m a -> Nested m a
Nested (m a -> StreamK m a
forall (m :: * -> *) a. Monad m => m a -> StreamK m a
fromEffect m a
x)

instance (MonadThrow m) => MonadThrow (Nested m) where
    throwM :: forall e a. (HasCallStack, Exception e) => e -> Nested m a
throwM = m a -> Nested m a
forall (m :: * -> *) a. Monad m => m a -> Nested m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m a -> Nested m a) -> (e -> m a) -> e -> Nested m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. e -> m a
forall e a. (HasCallStack, Exception e) => e -> m a
forall (m :: * -> *) e a.
(MonadThrow m, HasCallStack, Exception e) =>
e -> m a
throwM

------------------------------------------------------------------------------
-- Stream with a fair cross product style monad instance
------------------------------------------------------------------------------

-- XXX We can fix the termination issues by adding a "skip" continuation in the
-- stream. Adding a "block" continuation can allow for blocking IO. Both of
-- these together will provide a co-operative scheduling. However, adding skip
-- will regress performance in heavy filtering cases. If that's important we
-- can create another type StreamK' for skip continuation. That type can use
-- conversion from Stream type for everything except append and concatMap.

-- | 'FairNested' is like the 'Nested' type but explores the depth and breadth of
-- the cross product grid equally, so that each of the stream being crossed is
-- consumed equally. It can be used to nest infinite streams without starving
-- one due to the other.
--
-- >>> mk = StreamK.FairNested . StreamK.fromStream . Stream.fromList
-- >>> un = Stream.toList . StreamK.toStream . StreamK.unFairNested
--
-- == Looping
--
-- A single stream case is equivalent to 'Nested', it is a simple @for@ loop
-- over the stream:
--
-- >>> :{
-- un $ do
--     x <- mk [1,2] -- for each element in the stream
--     return x
-- :}
-- [1,2]
--
-- == Fair Nested Looping
--
-- Multiple streams nest like @for@ loops. The result is a cross product of the
-- streams. However, the ordering of the results of the cross product is such
-- that each stream gets consumed equally. In other words, inner iterations of
-- a nested loop get the same priority as the outer iterations. Inner
-- iterations do not finish completely before the outer iterations start.
--
-- >>> :{
-- un $ do
--     x <- mk [1,2,3] -- outer, for each element in the stream
--     y <- mk [4,5,6] -- inner, for each element in the stream
--     -- Perform the following actions for each x, for each y
--     return (x, y)
-- :}
-- [(1,4),(1,5),(2,4),(1,6),(2,5),(3,4),(2,6),(3,5),(3,6)]
--
-- == Nesting Infinite Streams
--
-- Example with infinite streams. Print all pairs in the cross product with sum
-- less than a specified number.
--
-- >>> :{
-- Stream.toList
--  $ Stream.takeWhile (\(x,y) -> x + y < 6)
--  $ StreamK.toStream $ StreamK.unFairNested
--  $ do
--     x <- mk [1..] -- infinite stream
--     y <- mk [1..] -- infinite stream
--     return (x, y)
-- :}
-- [(1,1),(1,2),(2,1),(1,3),(2,2),(3,1),(1,4),(2,3),(3,2),(4,1)]
--
-- == How it works?
--
-- 'FairNested' uses 'fairConcatFor' as the monad bind operation.
-- If we look at the cross product of [1,2,3], [4,5,6], the streams being
-- combined using 'concatMapDigaonal' are the sequential loop iterations:
--
-- @
-- (1,4) (1,5) (1,6) -- first iteration of the outer loop
-- (2,4) (2,5) (2,6) -- second iteration of the outer loop
-- (3,4) (3,5) (3,6) -- third iteration of the outer loop
-- @
--
-- The result is a triangular or diagonal traversal of these iterations:
--
-- @
-- [(1,4),(1,5),(2,4),(1,6),(2,5),(3,4),(2,6),(3,5),(3,6)]
-- @
--
-- == Associativity Issues
--
-- WARNING! The FairNested monad breaks the associativity law intentionally for
-- usefulness, it is associative only up to permutation equivalence. In this
-- monad the association order of statements might make a difference to the
-- ordering of the results because of changing the way in which streams are
-- scheduled. The same issues arise when you use the 'interleave' operation
-- directly, association order matters - however, here it can be more subtle as
-- the programmer may not see it directly.
--
-- >>> un (mk [1,2] >>= (\x -> mk [x, x + 1] >>= (\y -> mk [y, y + 2])))
-- [1,3,2,2,4,4,3,5]
-- >>> un ((mk [1,2] >>= (\x -> mk [x, x + 1])) >>= (\y -> mk [y, y + 2]))
-- [1,3,2,4,2,4,3,5]
--
-- This type is designed to be used for use cases where ordering of results
-- does not matter, we want to explore different streams to find specific
-- results, but the order in which we find or present the results may not be
-- important. Re-association of statements in this monad may change how different
-- branches are scheduled, which may change the scheduling priority of some
-- streams over others, this may end up starving some branches - in the worst
-- case some branches may be fully starved by some infinite branches producing
-- nothing - resulting in a non-terminating program.
--
-- == Non-Termination Cases
--
-- If an infinite stream that does not produce a value at all is interleaved
-- with another stream then the entire computation gets stuck forever because
-- the interleave operation schedules the second stream only after the first
-- stream yields a value. This can lead to non-terminating programs, an example
-- is provided below.
--
-- >>> :{
-- toS = StreamK.toStream . StreamK.unFairNested
-- odds x = mk (if x then [1,3..] else [2,4..])
-- filterEven x = if even x then pure x else StreamK.FairNested StreamK.nil
-- :}
--
-- When writing code with do notation, keep in mind that when we bind a
-- variable to a monadic value, all the following code that depends on this
-- variable is associated together and connected to it via a monad bind.
-- Consider the following code:
--
-- >>> :{
-- evens = toS $ do
--     r <- mk [True,False]
--     -- The next two statements depending on the variable r are associated
--     -- together and bound to the previous line using a monad bind.
--     x <- odds r
--     filterEven x
-- :}
--
-- This code does not terminate because, when r is True, @odds@ and
-- @filterEven@ together constitute an infinite inner loop, coninuously working
-- but not yielding any value at all, this stream is interleaved with the outer
-- loop, therefore, the outer loop does not get a chance to move to the next
-- iteration.
--
-- But the following code works as expected:
--
-- >>> :{
-- evens = toS $ do
--     x <- mk [True,False] >>= odds
--     filterEven x
-- :}
--
-- >>> Stream.toList $ Stream.take 3 $ evens
-- [2,4,6]
--
-- This works because both the lists being interleaved continue to produce
-- values in the outer loop and the inner loop keeps filtering them.
--
-- Care should be taken how you write your program, keep in mind the scheduling
-- implications. To avoid such scheduling problems in the serial FairNested type
-- use the concurrent version i.e. FairParallel described in
-- 'Streamly.Data.Stream.MkType' module. Due to concurrent evaluation each
-- branch will make progress even if one is an infinite loop producing nothing.
--
-- == Related Operations
--
-- We can create a custom type with 'concatMapWith' 'interleave' as the monad
-- bind operation then the inner loop iterations get exponentially more
-- priority over the outer iterations of the nested loop. This is not fully
-- fair, it is biased - this is exactly how the logic-t and list-t
-- implementation of fair bind works.

newtype FairNested m a = FairNested {forall (m :: * -> *) a. FairNested m a -> StreamK m a
unFairNested :: StreamK m a}
        deriving ((forall a b. (a -> b) -> FairNested m a -> FairNested m b)
-> (forall a b. a -> FairNested m b -> FairNested m a)
-> Functor (FairNested m)
forall a b. a -> FairNested m b -> FairNested m a
forall a b. (a -> b) -> FairNested m a -> FairNested m b
forall (m :: * -> *) a b.
Monad m =>
a -> FairNested m b -> FairNested m a
forall (m :: * -> *) a b.
Monad m =>
(a -> b) -> FairNested m a -> FairNested m b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall (m :: * -> *) a b.
Monad m =>
(a -> b) -> FairNested m a -> FairNested m b
fmap :: forall a b. (a -> b) -> FairNested m a -> FairNested m b
$c<$ :: forall (m :: * -> *) a b.
Monad m =>
a -> FairNested m b -> FairNested m a
<$ :: forall a b. a -> FairNested m b -> FairNested m a
Functor, (forall m. Monoid m => FairNested m m -> m)
-> (forall m a. Monoid m => (a -> m) -> FairNested m a -> m)
-> (forall m a. Monoid m => (a -> m) -> FairNested m a -> m)
-> (forall a b. (a -> b -> b) -> b -> FairNested m a -> b)
-> (forall a b. (a -> b -> b) -> b -> FairNested m a -> b)
-> (forall b a. (b -> a -> b) -> b -> FairNested m a -> b)
-> (forall b a. (b -> a -> b) -> b -> FairNested m a -> b)
-> (forall a. (a -> a -> a) -> FairNested m a -> a)
-> (forall a. (a -> a -> a) -> FairNested m a -> a)
-> (forall a. FairNested m a -> [a])
-> (forall a. FairNested m a -> Bool)
-> (forall a. FairNested m a -> Int)
-> (forall a. Eq a => a -> FairNested m a -> Bool)
-> (forall a. Ord a => FairNested m a -> a)
-> (forall a. Ord a => FairNested m a -> a)
-> (forall a. Num a => FairNested m a -> a)
-> (forall a. Num a => FairNested m a -> a)
-> Foldable (FairNested m)
forall a. Eq a => a -> FairNested m a -> Bool
forall a. Num a => FairNested m a -> a
forall a. Ord a => FairNested m a -> a
forall m. Monoid m => FairNested m m -> m
forall a. FairNested m a -> Bool
forall a. FairNested m a -> Int
forall a. FairNested m a -> [a]
forall a. (a -> a -> a) -> FairNested m a -> a
forall m a. Monoid m => (a -> m) -> FairNested m a -> m
forall b a. (b -> a -> b) -> b -> FairNested m a -> b
forall a b. (a -> b -> b) -> b -> FairNested m a -> b
forall (m :: * -> *) a.
(Foldable m, Monad m, Eq a) =>
a -> FairNested m a -> Bool
forall (m :: * -> *) a.
(Foldable m, Monad m, Num a) =>
FairNested m a -> a
forall (m :: * -> *) a.
(Foldable m, Monad m, Ord a) =>
FairNested m a -> a
forall (m :: * -> *) m.
(Foldable m, Monad m, Monoid m) =>
FairNested m m -> m
forall (m :: * -> *) a.
(Foldable m, Monad m) =>
FairNested m a -> Bool
forall (m :: * -> *) a.
(Foldable m, Monad m) =>
FairNested m a -> Int
forall (m :: * -> *) a.
(Foldable m, Monad m) =>
FairNested m a -> [a]
forall (m :: * -> *) a.
(Foldable m, Monad m) =>
(a -> a -> a) -> FairNested m a -> a
forall (m :: * -> *) m a.
(Foldable m, Monad m, Monoid m) =>
(a -> m) -> FairNested m a -> m
forall (m :: * -> *) b a.
(Foldable m, Monad m) =>
(b -> a -> b) -> b -> FairNested m a -> b
forall (m :: * -> *) a b.
(Foldable m, Monad m) =>
(a -> b -> b) -> b -> FairNested m a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> Int)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall (m :: * -> *) m.
(Foldable m, Monad m, Monoid m) =>
FairNested m m -> m
fold :: forall m. Monoid m => FairNested m m -> m
$cfoldMap :: forall (m :: * -> *) m a.
(Foldable m, Monad m, Monoid m) =>
(a -> m) -> FairNested m a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> FairNested m a -> m
$cfoldMap' :: forall (m :: * -> *) m a.
(Foldable m, Monad m, Monoid m) =>
(a -> m) -> FairNested m a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> FairNested m a -> m
$cfoldr :: forall (m :: * -> *) a b.
(Foldable m, Monad m) =>
(a -> b -> b) -> b -> FairNested m a -> b
foldr :: forall a b. (a -> b -> b) -> b -> FairNested m a -> b
$cfoldr' :: forall (m :: * -> *) a b.
(Foldable m, Monad m) =>
(a -> b -> b) -> b -> FairNested m a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> FairNested m a -> b
$cfoldl :: forall (m :: * -> *) b a.
(Foldable m, Monad m) =>
(b -> a -> b) -> b -> FairNested m a -> b
foldl :: forall b a. (b -> a -> b) -> b -> FairNested m a -> b
$cfoldl' :: forall (m :: * -> *) b a.
(Foldable m, Monad m) =>
(b -> a -> b) -> b -> FairNested m a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> FairNested m a -> b
$cfoldr1 :: forall (m :: * -> *) a.
(Foldable m, Monad m) =>
(a -> a -> a) -> FairNested m a -> a
foldr1 :: forall a. (a -> a -> a) -> FairNested m a -> a
$cfoldl1 :: forall (m :: * -> *) a.
(Foldable m, Monad m) =>
(a -> a -> a) -> FairNested m a -> a
foldl1 :: forall a. (a -> a -> a) -> FairNested m a -> a
$ctoList :: forall (m :: * -> *) a.
(Foldable m, Monad m) =>
FairNested m a -> [a]
toList :: forall a. FairNested m a -> [a]
$cnull :: forall (m :: * -> *) a.
(Foldable m, Monad m) =>
FairNested m a -> Bool
null :: forall a. FairNested m a -> Bool
$clength :: forall (m :: * -> *) a.
(Foldable m, Monad m) =>
FairNested m a -> Int
length :: forall a. FairNested m a -> Int
$celem :: forall (m :: * -> *) a.
(Foldable m, Monad m, Eq a) =>
a -> FairNested m a -> Bool
elem :: forall a. Eq a => a -> FairNested m a -> Bool
$cmaximum :: forall (m :: * -> *) a.
(Foldable m, Monad m, Ord a) =>
FairNested m a -> a
maximum :: forall a. Ord a => FairNested m a -> a
$cminimum :: forall (m :: * -> *) a.
(Foldable m, Monad m, Ord a) =>
FairNested m a -> a
minimum :: forall a. Ord a => FairNested m a -> a
$csum :: forall (m :: * -> *) a.
(Foldable m, Monad m, Num a) =>
FairNested m a -> a
sum :: forall a. Num a => FairNested m a -> a
$cproduct :: forall (m :: * -> *) a.
(Foldable m, Monad m, Num a) =>
FairNested m a -> a
product :: forall a. Num a => FairNested m a -> a
Foldable)

-- Pure (Identity monad) stream instances
deriving instance Traversable (FairNested Identity)
deriving instance IsList (FairNested Identity a)
deriving instance (a ~ Char) => IsString (FairNested Identity a)
-- deriving instance Eq a => Eq (FairNested Identity a)
-- deriving instance Ord a => Ord (FairNested Identity a)

-- Do not use automatic derivation for this to show as "fromList" rather than
-- "fromList Identity".
instance Show a => Show (FairNested Identity a) where
    {-# INLINE show #-}
    show :: FairNested Identity a -> String
show (FairNested StreamK Identity a
xs) = StreamK Identity a -> String
forall a. Show a => a -> String
show StreamK Identity a
xs

instance Read a => Read (FairNested Identity a) where
    {-# INLINE readPrec #-}
    readPrec :: ReadPrec (FairNested Identity a)
readPrec = (StreamK Identity a -> FairNested Identity a)
-> ReadPrec (StreamK Identity a)
-> ReadPrec (FairNested Identity a)
forall a b. (a -> b) -> ReadPrec a -> ReadPrec b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap StreamK Identity a -> FairNested Identity a
forall (m :: * -> *) a. StreamK m a -> FairNested m a
FairNested ReadPrec (StreamK Identity a)
forall a. Read a => ReadPrec a
readPrec

------------------------------------------------------------------------------
-- Applicative
------------------------------------------------------------------------------

-- Note: we need to define all the typeclass operations because we want to
-- INLINE them.
instance Monad m => Applicative (FairNested m) where
    {-# INLINE pure #-}
    pure :: forall a. a -> FairNested m a
pure a
x = StreamK m a -> FairNested m a
forall (m :: * -> *) a. StreamK m a -> FairNested m a
FairNested (a -> StreamK m a
forall a (m :: * -> *). a -> StreamK m a
fromPure a
x)

    -- XXX implement more efficient version of these
    <*> :: forall a b.
FairNested m (a -> b) -> FairNested m a -> FairNested m b
(<*>) = FairNested m (a -> b) -> FairNested m a -> FairNested m b
forall (m :: * -> *) a b. Monad m => m (a -> b) -> m a -> m b
ap
    {-
    {-# INLINE (<*>) #-}
    (FairNested s1) <*> (FairNested s2) =
        FairNested (crossApply s1 s2)

    {-# INLINE liftA2 #-}
    liftA2 f x = (<*>) (fmap f x)

    {-# INLINE (*>) #-}
    (FairNested s1) *> (FairNested s2) =
        FairNested (crossApplySnd s1 s2)

    {-# INLINE (<*) #-}
    (FairNested s1) <* (FairNested s2) =
        FairNested (crossApplyFst s1 s2)
    -}

------------------------------------------------------------------------------
-- Monad
------------------------------------------------------------------------------

instance Monad m => Monad (FairNested m) where
    return :: forall a. a -> FairNested m a
return = a -> FairNested m a
forall a. a -> FairNested m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure

    {-# INLINE (>>=) #-}
    >>= :: forall a b.
FairNested m a -> (a -> FairNested m b) -> FairNested m b
(>>=) (FairNested StreamK m a
m) a -> FairNested m b
f = StreamK m b -> FairNested m b
forall (m :: * -> *) a. StreamK m a -> FairNested m a
FairNested ((a -> StreamK m b) -> StreamK m a -> StreamK m b
forall a (m :: * -> *) b.
(a -> StreamK m b) -> StreamK m a -> StreamK m b
fairConcatMap (FairNested m b -> StreamK m b
forall (m :: * -> *) a. FairNested m a -> StreamK m a
unFairNested (FairNested m b -> StreamK m b)
-> (a -> FairNested m b) -> a -> StreamK m b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> FairNested m b
f) StreamK m a
m)

    -- {-# INLINE (>>) #-}
    -- (>>) = (*>)

------------------------------------------------------------------------------
-- Transformers
------------------------------------------------------------------------------

instance (MonadIO m) => MonadIO (FairNested m) where
    liftIO :: forall a. IO a -> FairNested m a
liftIO IO a
x = StreamK m a -> FairNested m a
forall (m :: * -> *) a. StreamK m a -> FairNested m a
FairNested (m a -> StreamK m a
forall (m :: * -> *) a. Monad m => m a -> StreamK m a
fromEffect (m a -> StreamK m a) -> m a -> StreamK m a
forall a b. (a -> b) -> a -> b
$ IO a -> m a
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO IO a
x)

instance MonadTrans FairNested where
    {-# INLINE lift #-}
    lift :: forall (m :: * -> *) a. Monad m => m a -> FairNested m a
lift m a
x = StreamK m a -> FairNested m a
forall (m :: * -> *) a. StreamK m a -> FairNested m a
FairNested (m a -> StreamK m a
forall (m :: * -> *) a. Monad m => m a -> StreamK m a
fromEffect m a
x)

instance (MonadThrow m) => MonadThrow (FairNested m) where
    throwM :: forall e a. (HasCallStack, Exception e) => e -> FairNested m a
throwM = m a -> FairNested m a
forall (m :: * -> *) a. Monad m => m a -> FairNested m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (m a -> FairNested m a) -> (e -> m a) -> e -> FairNested m a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. e -> m a
forall e a. (HasCallStack, Exception e) => e -> m a
forall (m :: * -> *) e a.
(MonadThrow m, HasCallStack, Exception e) =>
e -> m a
throwM