├── .gitignore ├── .gitlab-ci.yml ├── CMakeLists.txt ├── COPYING ├── Dockerfile ├── README.md ├── cmake ├── Fetch3rdParty.cmake ├── FindEigen3.cmake ├── FindJeMalloc.cmake ├── FindTcmalloc.cmake ├── FindZeroMQ.cmake └── Testing.cmake.in ├── documentation ├── Docker.md ├── Embedding.md ├── Installation.md ├── Network.md ├── Operators.md ├── Partitioning.md ├── Python.md ├── QueryCompiler.md ├── Tables.md ├── Tutorial.md ├── Usage.md └── UseCases.md ├── src ├── cep │ ├── CEPEngine.hpp │ ├── EventBuffer.hpp │ ├── Instance.hpp │ ├── MatchProducer.hpp │ ├── Matcher.cpp │ ├── Matcher.hpp │ ├── NFAController.hpp │ ├── NFAStructure.hpp │ ├── StructurePool.hpp │ ├── dsl │ │ └── CEPState.hpp │ ├── edge │ │ ├── ForwardEdge.hpp │ │ ├── LoopEdge.hpp │ │ └── NFAEdge.hpp │ ├── engine │ │ ├── FirstMatchEngine.hpp │ │ └── NextMatchEngine.hpp │ ├── relatedvalue │ │ ├── RelatedStateValue.hpp │ │ ├── RelatedStateValueMax.hpp │ │ ├── RelatedStateValueMin.hpp │ │ ├── RelatedStateValuePrevious.hpp │ │ └── RelatedStateValueSum.hpp │ ├── state │ │ ├── FinalState.hpp │ │ ├── KleeneState.hpp │ │ ├── NFAState.hpp │ │ ├── NegationState.hpp │ │ ├── NormalState.hpp │ │ └── StartState.hpp │ └── util │ │ ├── GCStructures.cpp │ │ ├── GCStructures.hpp │ │ ├── Partition.hpp │ │ ├── ThreadPool.hpp │ │ ├── ValueIDMap.hpp │ │ └── ValueIDMultimap.hpp ├── core │ ├── ElementSerializable.hpp │ ├── PFabricTypes.hpp │ ├── Punctuation.cpp │ ├── Punctuation.hpp │ ├── StreamElementTraits.hpp │ ├── TimestampHelper.cpp │ ├── TimestampHelper.hpp │ ├── Tuple.hpp │ ├── TupleFactoryTraits.hpp │ ├── TuplePrinter.hpp │ ├── TuplePtrFactory.hpp │ ├── parser │ │ ├── AttributeParserBase.hpp │ │ ├── SelectAttributeParser.hpp │ │ ├── StringAttributeParser.hpp │ │ ├── StringRef.hpp │ │ ├── StringRefAttributeParser.hpp │ │ └── TupleParser.hpp │ └── serialize.hpp ├── demo │ ├── CMakeLists.txt │ └── RestDemo.cpp ├── dsl │ ├── Dataflow.cpp │ ├── Dataflow.hpp │ ├── PFabricContext.cpp │ ├── PFabricContext.hpp │ ├── Pipe.hpp │ ├── Topology.cpp │ ├── Topology.hpp │ └── TopologyException.hpp ├── libcpp │ ├── algorithms │ │ ├── materializeStringRefs.hpp │ │ └── splitStringRef.hpp │ ├── mpl │ │ ├── Forward.hpp │ │ ├── algorithms.hpp │ │ ├── algorithms │ │ │ ├── ConstexprFunction.hpp │ │ │ └── StaticForEach.hpp │ │ ├── relational.hpp │ │ ├── relational │ │ │ ├── CrossProduct.hpp │ │ │ ├── JoinTupleWithRelation.hpp │ │ │ ├── JoinTuples.hpp │ │ │ ├── MakeRelation.hpp │ │ │ ├── Relation.hpp │ │ │ └── Tuple.hpp │ │ ├── sequences.hpp │ │ └── sequences │ │ │ ├── Append.hpp │ │ │ ├── Appender.hpp │ │ │ ├── Flatten.hpp │ │ │ ├── GenerateIndexes.hpp │ │ │ ├── GetDuplicates.hpp │ │ │ ├── HasDuplicates.hpp │ │ │ ├── IndexOf.hpp │ │ │ ├── InsertAssertUnique.hpp │ │ │ ├── RemoveDuplicates.hpp │ │ │ ├── SequenceJoiner.hpp │ │ │ └── ToSequence.hpp │ ├── preprocessor │ │ ├── MacroEnd.hpp │ │ ├── MakeString.hpp │ │ └── MakeStrings.hpp │ ├── test_utilities │ │ ├── TestDataFixtureBase.hpp │ │ ├── TestFixtureException.hpp │ │ └── test_utilities_config.hpp.in │ ├── types │ │ ├── detail │ │ │ ├── Function.hpp │ │ │ ├── IntrusivePtr.hpp │ │ │ ├── SharedInstance.hpp │ │ │ ├── SharedPtr.hpp │ │ │ ├── SubstringRef.hpp │ │ │ ├── TupleType.hpp │ │ │ ├── UniqueInstance.hpp │ │ │ └── UniquePtr.hpp │ │ ├── traits │ │ │ ├── FunctionTraits.hpp │ │ │ ├── GetPointedElementType.hpp │ │ │ └── PointerTraits.hpp │ │ └── types.hpp │ └── utilities │ │ ├── BindVariadic.hpp │ │ ├── ConvertTimeDurationToDouble.hpp │ │ ├── DynamicPointerMove.hpp │ │ ├── EnvironmentVariable.hpp │ │ ├── GetTypeName.hpp │ │ ├── PrintCSV.hpp │ │ ├── Timer.hpp │ │ ├── TypePrinter.hpp │ │ ├── exceptions.hpp │ │ ├── exceptions │ │ ├── ErrorInfo.hpp │ │ ├── ExceptionBase.hpp │ │ └── NullPointerException.hpp │ │ └── utilities.hpp ├── matrix │ ├── BaseMatrix.hpp │ ├── DenseMatrix.hpp │ ├── Matrix.hpp │ ├── ReaderValue.hpp │ ├── SparseMatrix.hpp │ └── VectorParser.hpp ├── net │ ├── KafkaSource.cpp │ ├── KafkaSource.hpp │ ├── MQTTSource.cpp │ ├── MQTTSource.hpp │ ├── RabbitMQSource.cpp │ ├── RabbitMQSource.hpp │ ├── ZMQSocket.cpp │ └── ZMQSocket.hpp ├── pfabric.hpp ├── pfabric_config.h.in ├── pubsub │ ├── Flow.hpp │ ├── Sink.hpp │ ├── SinkHelperMacros.hpp │ ├── Source.hpp │ ├── SourceHelperMacros.hpp │ ├── channels │ │ ├── Channel.hpp │ │ ├── ChannelGroup.hpp │ │ ├── ChannelID.hpp │ │ ├── ChannelTraits.hpp │ │ ├── ConnectChannels.hpp │ │ ├── InputChannel.hpp │ │ ├── OutputChannel.hpp │ │ ├── PublisherTraits.hpp │ │ ├── SubscriberTraits.hpp │ │ ├── Subscription.hpp │ │ ├── SubscriptionBase.hpp │ │ ├── impl │ │ │ ├── ChannelConsumer.hpp │ │ │ ├── ChannelCreator.hpp │ │ │ ├── CreateChannelInstanceTypes.hpp │ │ │ ├── GenerateChannelConsumerInterface.hpp │ │ │ ├── GenerateInputChannelGroup.hpp │ │ │ └── GenerateOutputChannelGroup.hpp │ │ └── parameters │ │ │ ├── ChannelParameterTraits.hpp │ │ │ ├── ChannelParameters.hpp │ │ │ ├── InputChannelParameterTraits.hpp │ │ │ ├── InputChannelParameters.hpp │ │ │ ├── IsInputChannelParameter.hpp │ │ │ ├── IsOutputChannelParameter.hpp │ │ │ ├── OutputChannelParameterTraits.hpp │ │ │ ├── OutputChannelParameters.hpp │ │ │ ├── SelectInputChannelParameters.hpp │ │ │ └── SelectOutputChannelParameters.hpp │ ├── publish_subscribe.hpp │ └── signals │ │ ├── BoostSignal.hpp │ │ ├── BoostSlot.hpp │ │ ├── DefaultSlotFunction.hpp │ │ ├── DefaultSourceSignal.hpp │ │ ├── OneToManySignal.hpp │ │ ├── OneToOneSignal.hpp │ │ ├── SignalTraits.hpp │ │ ├── StdSlot.hpp │ │ └── SynchronizedSlot.hpp ├── python │ ├── PyAggregateState.cpp │ ├── PyTopology.cpp │ └── PyTopology.hpp ├── qcomp │ ├── CMakeLists.txt │ ├── CompilerMain.cpp │ ├── Plan.cpp │ ├── Plan.hpp │ ├── PlanCache.cpp │ ├── PlanCache.hpp │ ├── QueryCompileException.hpp │ ├── QueryCompiler.cpp │ ├── QueryCompiler.hpp │ ├── SQLParser.cpp │ ├── SQLParser.hpp │ ├── TopologyBuilder.hpp │ ├── TypeManager.cpp │ ├── TypeManager.hpp │ ├── UniqueNameGenerator.cpp │ └── UniqueNameGenerator.hpp ├── qop │ ├── AggregateFunctions.hpp │ ├── AggregateStateBase.hpp │ ├── Aggregation.hpp │ ├── Barrier.hpp │ ├── BaseOp.hpp │ ├── Batcher.hpp │ ├── BinaryTransform.hpp │ ├── ConsoleWriter.hpp │ ├── DataSink.hpp │ ├── DataSource.hpp │ ├── DefaultElementJoin.hpp │ ├── EagerElementJoin.hpp │ ├── ElementJoinTraits.hpp │ ├── FileWriter.hpp │ ├── FromMatrix.hpp │ ├── FromTable.hpp │ ├── FromTxTables.hpp │ ├── GroupedAggregation.hpp │ ├── JsonExtractor.hpp │ ├── Map.hpp │ ├── MatrixMerge.hpp │ ├── MatrixSlice.hpp │ ├── MemorySource.hpp │ ├── Merge.hpp │ ├── Notify.hpp │ ├── OperatorMacros.hpp │ ├── PartitionBy.hpp │ ├── Queue.hpp │ ├── RESTSource.cpp │ ├── RESTSource.hpp │ ├── SHJoin.hpp │ ├── ScaleJoin.hpp │ ├── SelectFromMVCCTable.hpp │ ├── SelectFromTable.hpp │ ├── SelectFromTxTable.hpp │ ├── SlidingWindow.hpp │ ├── StatefulMap.hpp │ ├── StreamGenerator.hpp │ ├── TextFileSource.cpp │ ├── TextFileSource.hpp │ ├── ToMVCCTable.hpp │ ├── ToMatrix.hpp │ ├── ToTable.hpp │ ├── ToTxTable.hpp │ ├── TriggerNotifier.cpp │ ├── TriggerNotifier.hpp │ ├── TumblingWindow.hpp │ ├── TupleDeserializer.hpp │ ├── TupleExtractor.hpp │ ├── Tuplifier.hpp │ ├── UnaryTransform.hpp │ ├── Where.hpp │ ├── Window.cpp │ ├── Window.hpp │ ├── ZMQSink.hpp │ ├── ZMQSource.cpp │ ├── ZMQSource.hpp │ └── aggr_functions │ │ ├── AggrAvg.hpp │ │ ├── AggrCount.hpp │ │ ├── AggrDCount.hpp │ │ ├── AggrGlobalMax.hpp │ │ ├── AggrGlobalMin.hpp │ │ ├── AggrIdentity.hpp │ │ ├── AggrLRecent.hpp │ │ ├── AggrMRecent.hpp │ │ ├── AggrMedian.hpp │ │ ├── AggrMinMax.hpp │ │ ├── AggrSum.hpp │ │ └── AggregateFunc.hpp ├── table │ ├── BDCCPTable.hpp │ ├── BOCCTable.hpp │ ├── BaseTable.hpp │ ├── CuckooTable.hpp │ ├── HashMapTable.hpp │ ├── LogBuffer.hpp │ ├── MVCCTable.hpp │ ├── PBPTreeTable.hpp │ ├── RDBTable.hpp │ ├── S2PLTable.hpp │ ├── StateContext.cpp │ ├── StateContext.hpp │ ├── Table.hpp │ ├── TableException.hpp │ ├── TableInfo.cpp │ ├── TableInfo.hpp │ └── TxTable.hpp └── usecases │ ├── CMakeLists.txt │ ├── DEBS2017 │ ├── CMakeLists.txt │ └── DEBS2017.cpp │ ├── FreqTrajectories │ ├── CMakeLists.txt │ ├── GeoUtils.cpp │ ├── Pattern.cpp │ ├── Pattern.hpp │ ├── PrefixSpan.cpp │ ├── PrefixSpan.hpp │ ├── QueryLoop.cpp │ ├── TrackerServer.cpp │ ├── TrajectoryDB.cpp │ ├── TrajectoryDB.hpp │ ├── WebServer.cpp │ ├── WebServer.hpp │ └── config.ini │ ├── LinearRoad │ ├── CMakeLists.txt │ ├── DataDriverLR.hpp │ ├── DataProvider │ │ └── CMakeLists.txt │ ├── FunctionsFromPaper.cpp │ └── LinRoad.cpp │ ├── MatrixProcessing │ ├── CMakeLists.txt │ ├── GraphTest.cpp │ ├── ImageProcessingTest.cpp │ ├── data │ │ ├── blur_image_test.in │ │ └── blur_image_test.res │ └── operations │ │ ├── Graph.hpp │ │ ├── GraphAlgorithms.hpp │ │ ├── ImageFilter.hpp │ │ └── filters │ │ ├── BaseImageFilter.hpp │ │ ├── GaussianFilter.hpp │ │ ├── ImageProcessingFunc.hpp │ │ ├── LaplacianFilter.hpp │ │ ├── MediaBlurFilter.hpp │ │ └── SmoothFilter.hpp │ ├── SECRET │ ├── CMakeLists.txt │ ├── README.md │ └── SECRET.cpp │ └── TxSupport │ ├── BOCCExample.cpp │ ├── CMakeLists.txt │ ├── Cuckoo.cpp │ ├── MVCCExample.cpp │ ├── README.md │ ├── S2PLExample.cpp │ ├── TxExample.hpp │ ├── TxProcessing.cpp │ ├── Workload.cpp │ ├── Workload.h │ ├── accounting.csv │ ├── accounting2.csv │ ├── common.h │ └── generateWorkloads.cpp └── test ├── AggregateFuncTest.cpp ├── AggregationTest.cpp ├── BDCCPTableTest.cpp ├── BarrierTest.cpp ├── CEPDSLTest.cpp ├── CMakeLists.txt ├── ChannelGroupTest.cpp ├── ContextTest.cpp ├── FlowTest.cpp ├── FromMatrixTest.cpp ├── FromTableTest.cpp ├── GroupedAggregationTest.cpp ├── HashMapTableTest.cpp ├── KafkaSourceTest.cpp ├── LinearRoadTest.cpp ├── MQTTSourceTest.cpp ├── MapTest.cpp ├── MatrixSliceTest.cpp ├── MatrixTest.cpp ├── MemorySourceTest.cpp ├── NotifyTest.cpp ├── PBPTreeTableTest.cpp ├── PartitionTest.cpp ├── PyTest.py ├── QueueTest.cpp ├── RDBTableTest.cpp ├── RESTSourceTest.cpp ├── RabbitMQSourceTest.cpp ├── RocksDBTest.cpp ├── SHJoinTest.cpp ├── SelectChannelParametersTest.cpp ├── SeqCEPTest.cpp ├── SignalTest.cpp ├── SinkTest.cpp ├── SourceTest.cpp ├── StreamElementTraitsTest.cpp ├── StreamGeneratorTest.cpp ├── StreamMockup.hpp ├── SynchronizedSlotTest.cpp ├── TestDataGenerator.hpp ├── TestMain.cpp ├── TextFileSourceTest.cpp ├── TimestampHelperTest.cpp ├── ToTableTest.cpp ├── TopologyAggregationTest.cpp ├── TopologyBenchmarks.cpp ├── TopologyGroupByTest.cpp ├── TopologyJoinTest.cpp ├── TopologyTest.cpp ├── TupleExtractorTest.cpp ├── TupleTest.cpp ├── TuplifierTest.cpp ├── WhereTest.cpp ├── WindowTest.cpp ├── WriterTest.cpp ├── ZMQPubSubTest.cpp ├── ZMQSourceTest.cpp └── test_data ├── cep_test.in ├── cep_test.res ├── tuplifier_test1.in ├── tuplifier_test1.res ├── tuplifier_test2.res └── vector_test.in /.gitignore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/.gitignore -------------------------------------------------------------------------------- /.gitlab-ci.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/.gitlab-ci.yml -------------------------------------------------------------------------------- /CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/CMakeLists.txt -------------------------------------------------------------------------------- /COPYING: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/COPYING -------------------------------------------------------------------------------- /Dockerfile: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/Dockerfile -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/README.md -------------------------------------------------------------------------------- /cmake/Fetch3rdParty.cmake: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/cmake/Fetch3rdParty.cmake -------------------------------------------------------------------------------- /cmake/FindEigen3.cmake: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/cmake/FindEigen3.cmake -------------------------------------------------------------------------------- /cmake/FindJeMalloc.cmake: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/cmake/FindJeMalloc.cmake -------------------------------------------------------------------------------- /cmake/FindTcmalloc.cmake: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/cmake/FindTcmalloc.cmake -------------------------------------------------------------------------------- /cmake/FindZeroMQ.cmake: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/cmake/FindZeroMQ.cmake -------------------------------------------------------------------------------- /cmake/Testing.cmake.in: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/cmake/Testing.cmake.in -------------------------------------------------------------------------------- /documentation/Docker.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Docker.md -------------------------------------------------------------------------------- /documentation/Embedding.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Embedding.md -------------------------------------------------------------------------------- /documentation/Installation.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Installation.md -------------------------------------------------------------------------------- /documentation/Network.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Network.md -------------------------------------------------------------------------------- /documentation/Operators.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Operators.md -------------------------------------------------------------------------------- /documentation/Partitioning.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Partitioning.md -------------------------------------------------------------------------------- /documentation/Python.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Python.md -------------------------------------------------------------------------------- /documentation/QueryCompiler.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/QueryCompiler.md -------------------------------------------------------------------------------- /documentation/Tables.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Tables.md -------------------------------------------------------------------------------- /documentation/Tutorial.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Tutorial.md -------------------------------------------------------------------------------- /documentation/Usage.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/Usage.md -------------------------------------------------------------------------------- /documentation/UseCases.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/documentation/UseCases.md -------------------------------------------------------------------------------- /src/cep/CEPEngine.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/CEPEngine.hpp -------------------------------------------------------------------------------- /src/cep/EventBuffer.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/EventBuffer.hpp -------------------------------------------------------------------------------- /src/cep/Instance.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/Instance.hpp -------------------------------------------------------------------------------- /src/cep/MatchProducer.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/MatchProducer.hpp -------------------------------------------------------------------------------- /src/cep/Matcher.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/Matcher.cpp -------------------------------------------------------------------------------- /src/cep/Matcher.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/Matcher.hpp -------------------------------------------------------------------------------- /src/cep/NFAController.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/NFAController.hpp -------------------------------------------------------------------------------- /src/cep/NFAStructure.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/NFAStructure.hpp -------------------------------------------------------------------------------- /src/cep/StructurePool.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/StructurePool.hpp -------------------------------------------------------------------------------- /src/cep/dsl/CEPState.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/dsl/CEPState.hpp -------------------------------------------------------------------------------- /src/cep/edge/ForwardEdge.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/edge/ForwardEdge.hpp -------------------------------------------------------------------------------- /src/cep/edge/LoopEdge.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/edge/LoopEdge.hpp -------------------------------------------------------------------------------- /src/cep/edge/NFAEdge.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/edge/NFAEdge.hpp -------------------------------------------------------------------------------- /src/cep/engine/FirstMatchEngine.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/engine/FirstMatchEngine.hpp -------------------------------------------------------------------------------- /src/cep/engine/NextMatchEngine.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/engine/NextMatchEngine.hpp -------------------------------------------------------------------------------- /src/cep/relatedvalue/RelatedStateValue.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/relatedvalue/RelatedStateValue.hpp -------------------------------------------------------------------------------- /src/cep/relatedvalue/RelatedStateValueMax.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/relatedvalue/RelatedStateValueMax.hpp -------------------------------------------------------------------------------- /src/cep/relatedvalue/RelatedStateValueMin.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/relatedvalue/RelatedStateValueMin.hpp -------------------------------------------------------------------------------- /src/cep/relatedvalue/RelatedStateValuePrevious.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/relatedvalue/RelatedStateValuePrevious.hpp -------------------------------------------------------------------------------- /src/cep/relatedvalue/RelatedStateValueSum.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/relatedvalue/RelatedStateValueSum.hpp -------------------------------------------------------------------------------- /src/cep/state/FinalState.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/state/FinalState.hpp -------------------------------------------------------------------------------- /src/cep/state/KleeneState.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/state/KleeneState.hpp -------------------------------------------------------------------------------- /src/cep/state/NFAState.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/state/NFAState.hpp -------------------------------------------------------------------------------- /src/cep/state/NegationState.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/state/NegationState.hpp -------------------------------------------------------------------------------- /src/cep/state/NormalState.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/state/NormalState.hpp -------------------------------------------------------------------------------- /src/cep/state/StartState.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/state/StartState.hpp -------------------------------------------------------------------------------- /src/cep/util/GCStructures.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/util/GCStructures.cpp -------------------------------------------------------------------------------- /src/cep/util/GCStructures.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/util/GCStructures.hpp -------------------------------------------------------------------------------- /src/cep/util/Partition.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/util/Partition.hpp -------------------------------------------------------------------------------- /src/cep/util/ThreadPool.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/util/ThreadPool.hpp -------------------------------------------------------------------------------- /src/cep/util/ValueIDMap.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/util/ValueIDMap.hpp -------------------------------------------------------------------------------- /src/cep/util/ValueIDMultimap.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/cep/util/ValueIDMultimap.hpp -------------------------------------------------------------------------------- /src/core/ElementSerializable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/ElementSerializable.hpp -------------------------------------------------------------------------------- /src/core/PFabricTypes.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/PFabricTypes.hpp -------------------------------------------------------------------------------- /src/core/Punctuation.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/Punctuation.cpp -------------------------------------------------------------------------------- /src/core/Punctuation.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/Punctuation.hpp -------------------------------------------------------------------------------- /src/core/StreamElementTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/StreamElementTraits.hpp -------------------------------------------------------------------------------- /src/core/TimestampHelper.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/TimestampHelper.cpp -------------------------------------------------------------------------------- /src/core/TimestampHelper.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/TimestampHelper.hpp -------------------------------------------------------------------------------- /src/core/Tuple.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/Tuple.hpp -------------------------------------------------------------------------------- /src/core/TupleFactoryTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/TupleFactoryTraits.hpp -------------------------------------------------------------------------------- /src/core/TuplePrinter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/TuplePrinter.hpp -------------------------------------------------------------------------------- /src/core/TuplePtrFactory.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/TuplePtrFactory.hpp -------------------------------------------------------------------------------- /src/core/parser/AttributeParserBase.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/parser/AttributeParserBase.hpp -------------------------------------------------------------------------------- /src/core/parser/SelectAttributeParser.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/parser/SelectAttributeParser.hpp -------------------------------------------------------------------------------- /src/core/parser/StringAttributeParser.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/parser/StringAttributeParser.hpp -------------------------------------------------------------------------------- /src/core/parser/StringRef.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/parser/StringRef.hpp -------------------------------------------------------------------------------- /src/core/parser/StringRefAttributeParser.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/parser/StringRefAttributeParser.hpp -------------------------------------------------------------------------------- /src/core/parser/TupleParser.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/parser/TupleParser.hpp -------------------------------------------------------------------------------- /src/core/serialize.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/core/serialize.hpp -------------------------------------------------------------------------------- /src/demo/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/demo/CMakeLists.txt -------------------------------------------------------------------------------- /src/demo/RestDemo.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/demo/RestDemo.cpp -------------------------------------------------------------------------------- /src/dsl/Dataflow.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/Dataflow.cpp -------------------------------------------------------------------------------- /src/dsl/Dataflow.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/Dataflow.hpp -------------------------------------------------------------------------------- /src/dsl/PFabricContext.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/PFabricContext.cpp -------------------------------------------------------------------------------- /src/dsl/PFabricContext.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/PFabricContext.hpp -------------------------------------------------------------------------------- /src/dsl/Pipe.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/Pipe.hpp -------------------------------------------------------------------------------- /src/dsl/Topology.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/Topology.cpp -------------------------------------------------------------------------------- /src/dsl/Topology.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/Topology.hpp -------------------------------------------------------------------------------- /src/dsl/TopologyException.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/dsl/TopologyException.hpp -------------------------------------------------------------------------------- /src/libcpp/algorithms/materializeStringRefs.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/algorithms/materializeStringRefs.hpp -------------------------------------------------------------------------------- /src/libcpp/algorithms/splitStringRef.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/algorithms/splitStringRef.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/Forward.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/Forward.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/algorithms.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/algorithms.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/algorithms/ConstexprFunction.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/algorithms/ConstexprFunction.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/algorithms/StaticForEach.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/algorithms/StaticForEach.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/relational.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/relational.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/relational/CrossProduct.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/relational/CrossProduct.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/relational/JoinTupleWithRelation.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/relational/JoinTupleWithRelation.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/relational/JoinTuples.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/relational/JoinTuples.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/relational/MakeRelation.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/relational/MakeRelation.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/relational/Relation.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/relational/Relation.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/relational/Tuple.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/relational/Tuple.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/Append.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/Append.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/Appender.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/Appender.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/Flatten.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/Flatten.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/GenerateIndexes.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/GenerateIndexes.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/GetDuplicates.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/GetDuplicates.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/HasDuplicates.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/HasDuplicates.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/IndexOf.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/IndexOf.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/InsertAssertUnique.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/InsertAssertUnique.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/RemoveDuplicates.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/RemoveDuplicates.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/SequenceJoiner.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/SequenceJoiner.hpp -------------------------------------------------------------------------------- /src/libcpp/mpl/sequences/ToSequence.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/mpl/sequences/ToSequence.hpp -------------------------------------------------------------------------------- /src/libcpp/preprocessor/MacroEnd.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/preprocessor/MacroEnd.hpp -------------------------------------------------------------------------------- /src/libcpp/preprocessor/MakeString.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/preprocessor/MakeString.hpp -------------------------------------------------------------------------------- /src/libcpp/preprocessor/MakeStrings.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/preprocessor/MakeStrings.hpp -------------------------------------------------------------------------------- /src/libcpp/test_utilities/TestDataFixtureBase.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/test_utilities/TestDataFixtureBase.hpp -------------------------------------------------------------------------------- /src/libcpp/test_utilities/TestFixtureException.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/test_utilities/TestFixtureException.hpp -------------------------------------------------------------------------------- /src/libcpp/test_utilities/test_utilities_config.hpp.in: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/test_utilities/test_utilities_config.hpp.in -------------------------------------------------------------------------------- /src/libcpp/types/detail/Function.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/Function.hpp -------------------------------------------------------------------------------- /src/libcpp/types/detail/IntrusivePtr.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/IntrusivePtr.hpp -------------------------------------------------------------------------------- /src/libcpp/types/detail/SharedInstance.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/SharedInstance.hpp -------------------------------------------------------------------------------- /src/libcpp/types/detail/SharedPtr.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/SharedPtr.hpp -------------------------------------------------------------------------------- /src/libcpp/types/detail/SubstringRef.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/SubstringRef.hpp -------------------------------------------------------------------------------- /src/libcpp/types/detail/TupleType.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/TupleType.hpp -------------------------------------------------------------------------------- /src/libcpp/types/detail/UniqueInstance.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/UniqueInstance.hpp -------------------------------------------------------------------------------- /src/libcpp/types/detail/UniquePtr.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/detail/UniquePtr.hpp -------------------------------------------------------------------------------- /src/libcpp/types/traits/FunctionTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/traits/FunctionTraits.hpp -------------------------------------------------------------------------------- /src/libcpp/types/traits/GetPointedElementType.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/traits/GetPointedElementType.hpp -------------------------------------------------------------------------------- /src/libcpp/types/traits/PointerTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/traits/PointerTraits.hpp -------------------------------------------------------------------------------- /src/libcpp/types/types.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/types/types.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/BindVariadic.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/BindVariadic.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/ConvertTimeDurationToDouble.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/ConvertTimeDurationToDouble.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/DynamicPointerMove.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/DynamicPointerMove.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/EnvironmentVariable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/EnvironmentVariable.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/GetTypeName.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/GetTypeName.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/PrintCSV.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/PrintCSV.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/Timer.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/Timer.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/TypePrinter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/TypePrinter.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/exceptions.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/exceptions.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/exceptions/ErrorInfo.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/exceptions/ErrorInfo.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/exceptions/ExceptionBase.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/exceptions/ExceptionBase.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/exceptions/NullPointerException.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/exceptions/NullPointerException.hpp -------------------------------------------------------------------------------- /src/libcpp/utilities/utilities.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/libcpp/utilities/utilities.hpp -------------------------------------------------------------------------------- /src/matrix/BaseMatrix.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/matrix/BaseMatrix.hpp -------------------------------------------------------------------------------- /src/matrix/DenseMatrix.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/matrix/DenseMatrix.hpp -------------------------------------------------------------------------------- /src/matrix/Matrix.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/matrix/Matrix.hpp -------------------------------------------------------------------------------- /src/matrix/ReaderValue.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/matrix/ReaderValue.hpp -------------------------------------------------------------------------------- /src/matrix/SparseMatrix.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/matrix/SparseMatrix.hpp -------------------------------------------------------------------------------- /src/matrix/VectorParser.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/matrix/VectorParser.hpp -------------------------------------------------------------------------------- /src/net/KafkaSource.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/KafkaSource.cpp -------------------------------------------------------------------------------- /src/net/KafkaSource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/KafkaSource.hpp -------------------------------------------------------------------------------- /src/net/MQTTSource.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/MQTTSource.cpp -------------------------------------------------------------------------------- /src/net/MQTTSource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/MQTTSource.hpp -------------------------------------------------------------------------------- /src/net/RabbitMQSource.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/RabbitMQSource.cpp -------------------------------------------------------------------------------- /src/net/RabbitMQSource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/RabbitMQSource.hpp -------------------------------------------------------------------------------- /src/net/ZMQSocket.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/ZMQSocket.cpp -------------------------------------------------------------------------------- /src/net/ZMQSocket.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/net/ZMQSocket.hpp -------------------------------------------------------------------------------- /src/pfabric.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pfabric.hpp -------------------------------------------------------------------------------- /src/pfabric_config.h.in: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pfabric_config.h.in -------------------------------------------------------------------------------- /src/pubsub/Flow.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/Flow.hpp -------------------------------------------------------------------------------- /src/pubsub/Sink.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/Sink.hpp -------------------------------------------------------------------------------- /src/pubsub/SinkHelperMacros.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/SinkHelperMacros.hpp -------------------------------------------------------------------------------- /src/pubsub/Source.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/Source.hpp -------------------------------------------------------------------------------- /src/pubsub/SourceHelperMacros.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/SourceHelperMacros.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/Channel.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/Channel.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/ChannelGroup.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/ChannelGroup.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/ChannelID.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/ChannelID.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/ChannelTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/ChannelTraits.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/ConnectChannels.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/ConnectChannels.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/InputChannel.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/InputChannel.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/OutputChannel.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/OutputChannel.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/PublisherTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/PublisherTraits.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/SubscriberTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/SubscriberTraits.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/Subscription.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/Subscription.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/SubscriptionBase.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/SubscriptionBase.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/impl/ChannelConsumer.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/impl/ChannelConsumer.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/impl/ChannelCreator.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/impl/ChannelCreator.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/impl/CreateChannelInstanceTypes.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/impl/CreateChannelInstanceTypes.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/impl/GenerateChannelConsumerInterface.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/impl/GenerateChannelConsumerInterface.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/impl/GenerateInputChannelGroup.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/impl/GenerateInputChannelGroup.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/impl/GenerateOutputChannelGroup.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/impl/GenerateOutputChannelGroup.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/ChannelParameterTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/ChannelParameterTraits.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/ChannelParameters.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/ChannelParameters.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/InputChannelParameterTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/InputChannelParameterTraits.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/InputChannelParameters.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/InputChannelParameters.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/IsInputChannelParameter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/IsInputChannelParameter.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/IsOutputChannelParameter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/IsOutputChannelParameter.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/OutputChannelParameterTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/OutputChannelParameterTraits.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/OutputChannelParameters.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/OutputChannelParameters.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/SelectInputChannelParameters.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/SelectInputChannelParameters.hpp -------------------------------------------------------------------------------- /src/pubsub/channels/parameters/SelectOutputChannelParameters.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/channels/parameters/SelectOutputChannelParameters.hpp -------------------------------------------------------------------------------- /src/pubsub/publish_subscribe.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/publish_subscribe.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/BoostSignal.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/BoostSignal.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/BoostSlot.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/BoostSlot.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/DefaultSlotFunction.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/DefaultSlotFunction.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/DefaultSourceSignal.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/DefaultSourceSignal.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/OneToManySignal.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/OneToManySignal.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/OneToOneSignal.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/OneToOneSignal.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/SignalTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/SignalTraits.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/StdSlot.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/StdSlot.hpp -------------------------------------------------------------------------------- /src/pubsub/signals/SynchronizedSlot.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/pubsub/signals/SynchronizedSlot.hpp -------------------------------------------------------------------------------- /src/python/PyAggregateState.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/python/PyAggregateState.cpp -------------------------------------------------------------------------------- /src/python/PyTopology.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/python/PyTopology.cpp -------------------------------------------------------------------------------- /src/python/PyTopology.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/python/PyTopology.hpp -------------------------------------------------------------------------------- /src/qcomp/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/CMakeLists.txt -------------------------------------------------------------------------------- /src/qcomp/CompilerMain.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/CompilerMain.cpp -------------------------------------------------------------------------------- /src/qcomp/Plan.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/Plan.cpp -------------------------------------------------------------------------------- /src/qcomp/Plan.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/Plan.hpp -------------------------------------------------------------------------------- /src/qcomp/PlanCache.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/PlanCache.cpp -------------------------------------------------------------------------------- /src/qcomp/PlanCache.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/PlanCache.hpp -------------------------------------------------------------------------------- /src/qcomp/QueryCompileException.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/QueryCompileException.hpp -------------------------------------------------------------------------------- /src/qcomp/QueryCompiler.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/QueryCompiler.cpp -------------------------------------------------------------------------------- /src/qcomp/QueryCompiler.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/QueryCompiler.hpp -------------------------------------------------------------------------------- /src/qcomp/SQLParser.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/SQLParser.cpp -------------------------------------------------------------------------------- /src/qcomp/SQLParser.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/SQLParser.hpp -------------------------------------------------------------------------------- /src/qcomp/TopologyBuilder.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/TopologyBuilder.hpp -------------------------------------------------------------------------------- /src/qcomp/TypeManager.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/TypeManager.cpp -------------------------------------------------------------------------------- /src/qcomp/TypeManager.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/TypeManager.hpp -------------------------------------------------------------------------------- /src/qcomp/UniqueNameGenerator.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/UniqueNameGenerator.cpp -------------------------------------------------------------------------------- /src/qcomp/UniqueNameGenerator.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qcomp/UniqueNameGenerator.hpp -------------------------------------------------------------------------------- /src/qop/AggregateFunctions.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/AggregateFunctions.hpp -------------------------------------------------------------------------------- /src/qop/AggregateStateBase.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/AggregateStateBase.hpp -------------------------------------------------------------------------------- /src/qop/Aggregation.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Aggregation.hpp -------------------------------------------------------------------------------- /src/qop/Barrier.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Barrier.hpp -------------------------------------------------------------------------------- /src/qop/BaseOp.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/BaseOp.hpp -------------------------------------------------------------------------------- /src/qop/Batcher.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Batcher.hpp -------------------------------------------------------------------------------- /src/qop/BinaryTransform.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/BinaryTransform.hpp -------------------------------------------------------------------------------- /src/qop/ConsoleWriter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ConsoleWriter.hpp -------------------------------------------------------------------------------- /src/qop/DataSink.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/DataSink.hpp -------------------------------------------------------------------------------- /src/qop/DataSource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/DataSource.hpp -------------------------------------------------------------------------------- /src/qop/DefaultElementJoin.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/DefaultElementJoin.hpp -------------------------------------------------------------------------------- /src/qop/EagerElementJoin.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/EagerElementJoin.hpp -------------------------------------------------------------------------------- /src/qop/ElementJoinTraits.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ElementJoinTraits.hpp -------------------------------------------------------------------------------- /src/qop/FileWriter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/FileWriter.hpp -------------------------------------------------------------------------------- /src/qop/FromMatrix.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/FromMatrix.hpp -------------------------------------------------------------------------------- /src/qop/FromTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/FromTable.hpp -------------------------------------------------------------------------------- /src/qop/FromTxTables.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/FromTxTables.hpp -------------------------------------------------------------------------------- /src/qop/GroupedAggregation.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/GroupedAggregation.hpp -------------------------------------------------------------------------------- /src/qop/JsonExtractor.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/JsonExtractor.hpp -------------------------------------------------------------------------------- /src/qop/Map.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Map.hpp -------------------------------------------------------------------------------- /src/qop/MatrixMerge.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/MatrixMerge.hpp -------------------------------------------------------------------------------- /src/qop/MatrixSlice.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/MatrixSlice.hpp -------------------------------------------------------------------------------- /src/qop/MemorySource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/MemorySource.hpp -------------------------------------------------------------------------------- /src/qop/Merge.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Merge.hpp -------------------------------------------------------------------------------- /src/qop/Notify.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Notify.hpp -------------------------------------------------------------------------------- /src/qop/OperatorMacros.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/OperatorMacros.hpp -------------------------------------------------------------------------------- /src/qop/PartitionBy.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/PartitionBy.hpp -------------------------------------------------------------------------------- /src/qop/Queue.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Queue.hpp -------------------------------------------------------------------------------- /src/qop/RESTSource.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/RESTSource.cpp -------------------------------------------------------------------------------- /src/qop/RESTSource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/RESTSource.hpp -------------------------------------------------------------------------------- /src/qop/SHJoin.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/SHJoin.hpp -------------------------------------------------------------------------------- /src/qop/ScaleJoin.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ScaleJoin.hpp -------------------------------------------------------------------------------- /src/qop/SelectFromMVCCTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/SelectFromMVCCTable.hpp -------------------------------------------------------------------------------- /src/qop/SelectFromTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/SelectFromTable.hpp -------------------------------------------------------------------------------- /src/qop/SelectFromTxTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/SelectFromTxTable.hpp -------------------------------------------------------------------------------- /src/qop/SlidingWindow.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/SlidingWindow.hpp -------------------------------------------------------------------------------- /src/qop/StatefulMap.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/StatefulMap.hpp -------------------------------------------------------------------------------- /src/qop/StreamGenerator.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/StreamGenerator.hpp -------------------------------------------------------------------------------- /src/qop/TextFileSource.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/TextFileSource.cpp -------------------------------------------------------------------------------- /src/qop/TextFileSource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/TextFileSource.hpp -------------------------------------------------------------------------------- /src/qop/ToMVCCTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ToMVCCTable.hpp -------------------------------------------------------------------------------- /src/qop/ToMatrix.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ToMatrix.hpp -------------------------------------------------------------------------------- /src/qop/ToTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ToTable.hpp -------------------------------------------------------------------------------- /src/qop/ToTxTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ToTxTable.hpp -------------------------------------------------------------------------------- /src/qop/TriggerNotifier.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/TriggerNotifier.cpp -------------------------------------------------------------------------------- /src/qop/TriggerNotifier.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/TriggerNotifier.hpp -------------------------------------------------------------------------------- /src/qop/TumblingWindow.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/TumblingWindow.hpp -------------------------------------------------------------------------------- /src/qop/TupleDeserializer.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/TupleDeserializer.hpp -------------------------------------------------------------------------------- /src/qop/TupleExtractor.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/TupleExtractor.hpp -------------------------------------------------------------------------------- /src/qop/Tuplifier.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Tuplifier.hpp -------------------------------------------------------------------------------- /src/qop/UnaryTransform.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/UnaryTransform.hpp -------------------------------------------------------------------------------- /src/qop/Where.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Where.hpp -------------------------------------------------------------------------------- /src/qop/Window.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Window.cpp -------------------------------------------------------------------------------- /src/qop/Window.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/Window.hpp -------------------------------------------------------------------------------- /src/qop/ZMQSink.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ZMQSink.hpp -------------------------------------------------------------------------------- /src/qop/ZMQSource.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ZMQSource.cpp -------------------------------------------------------------------------------- /src/qop/ZMQSource.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/ZMQSource.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrAvg.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrAvg.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrCount.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrCount.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrDCount.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrDCount.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrGlobalMax.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrGlobalMax.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrGlobalMin.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrGlobalMin.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrIdentity.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrIdentity.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrLRecent.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrLRecent.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrMRecent.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrMRecent.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrMedian.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrMedian.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrMinMax.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrMinMax.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggrSum.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggrSum.hpp -------------------------------------------------------------------------------- /src/qop/aggr_functions/AggregateFunc.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/qop/aggr_functions/AggregateFunc.hpp -------------------------------------------------------------------------------- /src/table/BDCCPTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/BDCCPTable.hpp -------------------------------------------------------------------------------- /src/table/BOCCTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/BOCCTable.hpp -------------------------------------------------------------------------------- /src/table/BaseTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/BaseTable.hpp -------------------------------------------------------------------------------- /src/table/CuckooTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/CuckooTable.hpp -------------------------------------------------------------------------------- /src/table/HashMapTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/HashMapTable.hpp -------------------------------------------------------------------------------- /src/table/LogBuffer.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/LogBuffer.hpp -------------------------------------------------------------------------------- /src/table/MVCCTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/MVCCTable.hpp -------------------------------------------------------------------------------- /src/table/PBPTreeTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/PBPTreeTable.hpp -------------------------------------------------------------------------------- /src/table/RDBTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/RDBTable.hpp -------------------------------------------------------------------------------- /src/table/S2PLTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/S2PLTable.hpp -------------------------------------------------------------------------------- /src/table/StateContext.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/StateContext.cpp -------------------------------------------------------------------------------- /src/table/StateContext.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/StateContext.hpp -------------------------------------------------------------------------------- /src/table/Table.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/Table.hpp -------------------------------------------------------------------------------- /src/table/TableException.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/TableException.hpp -------------------------------------------------------------------------------- /src/table/TableInfo.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/TableInfo.cpp -------------------------------------------------------------------------------- /src/table/TableInfo.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/TableInfo.hpp -------------------------------------------------------------------------------- /src/table/TxTable.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/table/TxTable.hpp -------------------------------------------------------------------------------- /src/usecases/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/DEBS2017/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/DEBS2017/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/DEBS2017/DEBS2017.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/DEBS2017/DEBS2017.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/GeoUtils.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/GeoUtils.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/Pattern.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/Pattern.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/Pattern.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/Pattern.hpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/PrefixSpan.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/PrefixSpan.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/PrefixSpan.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/PrefixSpan.hpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/QueryLoop.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/QueryLoop.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/TrackerServer.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/TrackerServer.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/TrajectoryDB.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/TrajectoryDB.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/TrajectoryDB.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/TrajectoryDB.hpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/WebServer.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/WebServer.cpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/WebServer.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/WebServer.hpp -------------------------------------------------------------------------------- /src/usecases/FreqTrajectories/config.ini: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/FreqTrajectories/config.ini -------------------------------------------------------------------------------- /src/usecases/LinearRoad/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/LinearRoad/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/LinearRoad/DataDriverLR.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/LinearRoad/DataDriverLR.hpp -------------------------------------------------------------------------------- /src/usecases/LinearRoad/DataProvider/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/LinearRoad/DataProvider/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/LinearRoad/FunctionsFromPaper.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/LinearRoad/FunctionsFromPaper.cpp -------------------------------------------------------------------------------- /src/usecases/LinearRoad/LinRoad.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/LinearRoad/LinRoad.cpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/GraphTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/GraphTest.cpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/ImageProcessingTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/ImageProcessingTest.cpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/data/blur_image_test.in: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/data/blur_image_test.in -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/data/blur_image_test.res: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/data/blur_image_test.res -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/Graph.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/Graph.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/GraphAlgorithms.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/GraphAlgorithms.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/ImageFilter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/ImageFilter.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/filters/BaseImageFilter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/filters/BaseImageFilter.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/filters/GaussianFilter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/filters/GaussianFilter.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/filters/ImageProcessingFunc.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/filters/ImageProcessingFunc.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/filters/LaplacianFilter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/filters/LaplacianFilter.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/filters/MediaBlurFilter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/filters/MediaBlurFilter.hpp -------------------------------------------------------------------------------- /src/usecases/MatrixProcessing/operations/filters/SmoothFilter.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/MatrixProcessing/operations/filters/SmoothFilter.hpp -------------------------------------------------------------------------------- /src/usecases/SECRET/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/SECRET/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/SECRET/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/SECRET/README.md -------------------------------------------------------------------------------- /src/usecases/SECRET/SECRET.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/SECRET/SECRET.cpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/BOCCExample.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/BOCCExample.cpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/CMakeLists.txt -------------------------------------------------------------------------------- /src/usecases/TxSupport/Cuckoo.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/Cuckoo.cpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/MVCCExample.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/MVCCExample.cpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/README.md -------------------------------------------------------------------------------- /src/usecases/TxSupport/S2PLExample.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/S2PLExample.cpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/TxExample.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/TxExample.hpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/TxProcessing.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/TxProcessing.cpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/Workload.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/Workload.cpp -------------------------------------------------------------------------------- /src/usecases/TxSupport/Workload.h: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/Workload.h -------------------------------------------------------------------------------- /src/usecases/TxSupport/accounting.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/accounting.csv -------------------------------------------------------------------------------- /src/usecases/TxSupport/accounting2.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/accounting2.csv -------------------------------------------------------------------------------- /src/usecases/TxSupport/common.h: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/common.h -------------------------------------------------------------------------------- /src/usecases/TxSupport/generateWorkloads.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/src/usecases/TxSupport/generateWorkloads.cpp -------------------------------------------------------------------------------- /test/AggregateFuncTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/AggregateFuncTest.cpp -------------------------------------------------------------------------------- /test/AggregationTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/AggregationTest.cpp -------------------------------------------------------------------------------- /test/BDCCPTableTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/BDCCPTableTest.cpp -------------------------------------------------------------------------------- /test/BarrierTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/BarrierTest.cpp -------------------------------------------------------------------------------- /test/CEPDSLTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/CEPDSLTest.cpp -------------------------------------------------------------------------------- /test/CMakeLists.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/CMakeLists.txt -------------------------------------------------------------------------------- /test/ChannelGroupTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/ChannelGroupTest.cpp -------------------------------------------------------------------------------- /test/ContextTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/ContextTest.cpp -------------------------------------------------------------------------------- /test/FlowTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/FlowTest.cpp -------------------------------------------------------------------------------- /test/FromMatrixTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/FromMatrixTest.cpp -------------------------------------------------------------------------------- /test/FromTableTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/FromTableTest.cpp -------------------------------------------------------------------------------- /test/GroupedAggregationTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/GroupedAggregationTest.cpp -------------------------------------------------------------------------------- /test/HashMapTableTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/HashMapTableTest.cpp -------------------------------------------------------------------------------- /test/KafkaSourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/KafkaSourceTest.cpp -------------------------------------------------------------------------------- /test/LinearRoadTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/LinearRoadTest.cpp -------------------------------------------------------------------------------- /test/MQTTSourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/MQTTSourceTest.cpp -------------------------------------------------------------------------------- /test/MapTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/MapTest.cpp -------------------------------------------------------------------------------- /test/MatrixSliceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/MatrixSliceTest.cpp -------------------------------------------------------------------------------- /test/MatrixTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/MatrixTest.cpp -------------------------------------------------------------------------------- /test/MemorySourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/MemorySourceTest.cpp -------------------------------------------------------------------------------- /test/NotifyTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/NotifyTest.cpp -------------------------------------------------------------------------------- /test/PBPTreeTableTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/PBPTreeTableTest.cpp -------------------------------------------------------------------------------- /test/PartitionTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/PartitionTest.cpp -------------------------------------------------------------------------------- /test/PyTest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/PyTest.py -------------------------------------------------------------------------------- /test/QueueTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/QueueTest.cpp -------------------------------------------------------------------------------- /test/RDBTableTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/RDBTableTest.cpp -------------------------------------------------------------------------------- /test/RESTSourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/RESTSourceTest.cpp -------------------------------------------------------------------------------- /test/RabbitMQSourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/RabbitMQSourceTest.cpp -------------------------------------------------------------------------------- /test/RocksDBTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/RocksDBTest.cpp -------------------------------------------------------------------------------- /test/SHJoinTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/SHJoinTest.cpp -------------------------------------------------------------------------------- /test/SelectChannelParametersTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/SelectChannelParametersTest.cpp -------------------------------------------------------------------------------- /test/SeqCEPTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/SeqCEPTest.cpp -------------------------------------------------------------------------------- /test/SignalTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/SignalTest.cpp -------------------------------------------------------------------------------- /test/SinkTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/SinkTest.cpp -------------------------------------------------------------------------------- /test/SourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/SourceTest.cpp -------------------------------------------------------------------------------- /test/StreamElementTraitsTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/StreamElementTraitsTest.cpp -------------------------------------------------------------------------------- /test/StreamGeneratorTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/StreamGeneratorTest.cpp -------------------------------------------------------------------------------- /test/StreamMockup.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/StreamMockup.hpp -------------------------------------------------------------------------------- /test/SynchronizedSlotTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/SynchronizedSlotTest.cpp -------------------------------------------------------------------------------- /test/TestDataGenerator.hpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TestDataGenerator.hpp -------------------------------------------------------------------------------- /test/TestMain.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TestMain.cpp -------------------------------------------------------------------------------- /test/TextFileSourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TextFileSourceTest.cpp -------------------------------------------------------------------------------- /test/TimestampHelperTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TimestampHelperTest.cpp -------------------------------------------------------------------------------- /test/ToTableTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/ToTableTest.cpp -------------------------------------------------------------------------------- /test/TopologyAggregationTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TopologyAggregationTest.cpp -------------------------------------------------------------------------------- /test/TopologyBenchmarks.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TopologyBenchmarks.cpp -------------------------------------------------------------------------------- /test/TopologyGroupByTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TopologyGroupByTest.cpp -------------------------------------------------------------------------------- /test/TopologyJoinTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TopologyJoinTest.cpp -------------------------------------------------------------------------------- /test/TopologyTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TopologyTest.cpp -------------------------------------------------------------------------------- /test/TupleExtractorTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TupleExtractorTest.cpp -------------------------------------------------------------------------------- /test/TupleTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TupleTest.cpp -------------------------------------------------------------------------------- /test/TuplifierTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/TuplifierTest.cpp -------------------------------------------------------------------------------- /test/WhereTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/WhereTest.cpp -------------------------------------------------------------------------------- /test/WindowTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/WindowTest.cpp -------------------------------------------------------------------------------- /test/WriterTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/WriterTest.cpp -------------------------------------------------------------------------------- /test/ZMQPubSubTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/ZMQPubSubTest.cpp -------------------------------------------------------------------------------- /test/ZMQSourceTest.cpp: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/ZMQSourceTest.cpp -------------------------------------------------------------------------------- /test/test_data/cep_test.in: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/test_data/cep_test.in -------------------------------------------------------------------------------- /test/test_data/cep_test.res: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/test_data/cep_test.res -------------------------------------------------------------------------------- /test/test_data/tuplifier_test1.in: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/test_data/tuplifier_test1.in -------------------------------------------------------------------------------- /test/test_data/tuplifier_test1.res: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/test_data/tuplifier_test1.res -------------------------------------------------------------------------------- /test/test_data/tuplifier_test2.res: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dbis-ilm/pipefabric/HEAD/test/test_data/tuplifier_test2.res -------------------------------------------------------------------------------- /test/test_data/vector_test.in: -------------------------------------------------------------------------------- 1 | 1 4 5 2 6 8 3 4 6 1 --------------------------------------------------------------------------------