├── .asf.yaml ├── .gitignore ├── .gitlab-ci.yml ├── .idea └── runConfigurations │ ├── all_pipeline_elements_flink.xml │ ├── all_pipeline_elements_jvm.xml │ ├── connect_adapters.xml │ ├── connect_adapters_iiot.xml │ ├── extensions_all_jvm.xml │ ├── processors_aggregation_flink.xml │ ├── processors_enricher_flink.xml │ ├── processors_enricher_jvm.xml │ ├── processors_filters_jvm.xml │ ├── processors_filters_siddhi.xml │ ├── processors_geo_flink.xml │ ├── processors_geo_jvm.xml │ ├── processors_imageprocessing_jvm.xml │ ├── processors_pattern_detection_flink.xml │ ├── processors_statistics_flink.xml │ ├── processors_textmining_flink.xml │ ├── processors_textmining_jvm.xml │ ├── processors_transformation_flink.xml │ ├── processors_transformation_jvm.xml │ ├── sinks_brokers_jvm.xml │ ├── sinks_databases_flink.xml │ ├── sinks_databases_jvm.xml │ ├── sinks_internal_jvm.xml │ ├── sinks_notifications_jvm.xml │ ├── sources_vehicle_simulator_jvm.xml │ └── sources_watertank_simulator_jvm.xml ├── .spignore ├── CONTRIBUTING.md ├── DISCLAIMER ├── LICENSE ├── NOTICE ├── README.md ├── RELEASE_NOTES.md ├── RELEASE_VALIDATION.md ├── docker-compose.yml ├── pom.xml ├── streampipes-connect-adapters-iiot ├── .idea │ └── workspace.xml ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── connect │ │ └── iiot │ │ ├── ConnectAdapterIiotInit.java │ │ ├── adapters │ │ ├── PullAdapter.java │ │ ├── PullRestAdapter.java │ │ ├── influxdb │ │ │ ├── InfluxDbClient.java │ │ │ ├── InfluxDbSetAdapter.java │ │ │ └── InfluxDbStreamAdapter.java │ │ ├── mysql │ │ │ ├── Column.java │ │ │ ├── MySqlAdapter.java │ │ │ ├── MySqlClient.java │ │ │ ├── MySqlSetAdapter.java │ │ │ └── MySqlStreamAdapter.java │ │ ├── netio │ │ │ ├── NetioMQTTAdapter.java │ │ │ ├── NetioRestAdapter.java │ │ │ ├── NetioUtils.java │ │ │ └── model │ │ │ │ ├── NetioAllPowerOutputs.java │ │ │ │ ├── NetioGlobalMeasure.java │ │ │ │ └── NetioPowerOutput.java │ │ ├── opcua │ │ │ ├── OpcNode.java │ │ │ ├── OpcUa.java │ │ │ ├── OpcUaAdapter.java │ │ │ └── utils │ │ │ │ ├── OpcUaNodeVariants.java │ │ │ │ ├── OpcUaTypes.java │ │ │ │ └── OpcUaUtil.java │ │ ├── plc4x │ │ │ ├── modbus │ │ │ │ ├── ModbusConfigFile.java │ │ │ │ └── Plc4xModbusAdapter.java │ │ │ ├── passive │ │ │ │ └── Plc4xPassiveAdapter.java │ │ │ └── s7 │ │ │ │ └── Plc4xS7Adapter.java │ │ ├── ros │ │ │ └── RosBridgeAdapter.java │ │ └── simulator │ │ │ └── machine │ │ │ ├── MachineDataSimulator.java │ │ │ ├── MachineDataSimulatorUtils.java │ │ │ └── MachineDataStreamAdapter.java │ │ ├── protocol │ │ ├── set │ │ │ ├── FileProtocol.java │ │ │ └── HttpProtocol.java │ │ └── stream │ │ │ ├── BrokerProtocol.java │ │ │ ├── FileStreamProtocol.java │ │ │ ├── HttpServerEventReceiver.java │ │ │ ├── HttpServerProtocol.java │ │ │ ├── HttpStreamProtocol.java │ │ │ ├── KafkaProtocol.java │ │ │ ├── MqttProtocol.java │ │ │ ├── PullProtocol.java │ │ │ └── pulsar │ │ │ ├── PulsarConsumer.java │ │ │ ├── PulsarProtocol.java │ │ │ └── PulsarUtils.java │ │ └── utils │ │ └── FileProtocolUtils.java │ └── resources │ ├── org.apache.streampipes.connect.iiot.adapters.influxdb.set │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.influxdb.stream │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.mysql.set │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.mysql.stream │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.netio.mqtt │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.netio.rest │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.opcua │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.plc4x.modbus │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.plc4x.s7 │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.ros │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.adapters.simulator.machine │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.protocol.set.file │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.protocol.set.http │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.protocol.stream.file │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.protocol.stream.http │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.protocol.stream.httpserver │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.protocol.stream.kafka │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.connect.iiot.protocol.stream.mqtt │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.connect.iiot.protocol.stream.pulsar │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-connect-adapters ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── streampipes │ │ │ └── connect │ │ │ ├── ConnectAdapterInit.java │ │ │ ├── adapters │ │ │ ├── PullAdapter.java │ │ │ ├── PullRestAdapter.java │ │ │ ├── coindesk │ │ │ │ ├── CoindeskBitcoinAdapter.java │ │ │ │ ├── Currency.java │ │ │ │ └── model │ │ │ │ │ ├── Bpi.java │ │ │ │ │ ├── CoindeskRawModel.java │ │ │ │ │ ├── EUR.java │ │ │ │ │ ├── GBP.java │ │ │ │ │ ├── Time.java │ │ │ │ │ └── USD.java │ │ │ ├── flic │ │ │ │ ├── FlicMQTTAdapter.java │ │ │ │ ├── FlicOutput.java │ │ │ │ └── FlicUtils.java │ │ │ ├── gdelt │ │ │ │ └── GdeltAdapter.java │ │ │ ├── iex │ │ │ │ ├── IexCloudAdapter.java │ │ │ │ ├── IexCloudNewsAdapter.java │ │ │ │ ├── IexCloudStockAdapter.java │ │ │ │ └── model │ │ │ │ │ ├── IexNewsData.java │ │ │ │ │ └── IexStockData.java │ │ │ ├── image │ │ │ │ ├── ImageZipAdapter.java │ │ │ │ ├── ImageZipUtils.java │ │ │ │ ├── ZipFileImageIterator.java │ │ │ │ ├── set │ │ │ │ │ └── ImageSetAdapter.java │ │ │ │ └── stream │ │ │ │ │ └── ImageStreamAdapter.java │ │ │ ├── iss │ │ │ │ ├── IssAdapter.java │ │ │ │ └── model │ │ │ │ │ ├── IssModel.java │ │ │ │ │ └── IssPosition.java │ │ │ ├── sensemap │ │ │ │ ├── OpenSenseMapAdapter.java │ │ │ │ ├── SensorNames.java │ │ │ │ └── model │ │ │ │ │ ├── CurrentLocation.java │ │ │ │ │ ├── Geometry.java │ │ │ │ │ ├── LastMeasurement.java │ │ │ │ │ ├── Loc.java │ │ │ │ │ ├── SenseBox.java │ │ │ │ │ └── Sensor.java │ │ │ ├── simulator │ │ │ │ └── random │ │ │ │ │ ├── RandomDataSetAdapter.java │ │ │ │ │ ├── RandomDataSimulator.java │ │ │ │ │ ├── RandomDataSimulatorUtils.java │ │ │ │ │ └── RandomDataStreamAdapter.java │ │ │ ├── slack │ │ │ │ ├── SlackAdapter.java │ │ │ │ └── SlackConsumer.java │ │ │ ├── ti │ │ │ │ └── TISensorTag.java │ │ │ └── wikipedia │ │ │ │ ├── WikipediaAdapter.java │ │ │ │ ├── WikipediaEditedArticlesAdapter.java │ │ │ │ ├── WikipediaModelConverter.java │ │ │ │ ├── WikipediaNewArticlesAdapter.java │ │ │ │ ├── WikipediaSseConsumer.java │ │ │ │ └── model │ │ │ │ ├── Length.java │ │ │ │ ├── Meta.java │ │ │ │ ├── Revision.java │ │ │ │ └── WikipediaModel.java │ │ │ ├── config │ │ │ ├── ConfigKeys.java │ │ │ └── ConnectWorkerConfig.java │ │ │ └── protocol │ │ │ └── stream │ │ │ └── HDFSProtocol.java │ └── resources │ │ ├── org.apache.streampipes.connect.adapters.coindesk │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.flic.mqtt │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.gdelt │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.iex.news │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.iex.stocks │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.image.set │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.image.stream │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.iss │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.simulator.randomdataset │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.simulator.randomdatastream │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.slack │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.ti │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.wikipedia.edit │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.connect.adapters.wikipedia.new │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ └── org.apache.streampipes.connect.protocol.stream.httpserver │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ └── test │ └── java │ └── org │ └── apache │ └── streampipes │ └── connect │ └── adapters │ ├── PullAdapterTest.java │ ├── generic │ ├── Mock.java │ ├── UtilTest.java │ ├── elements │ │ ├── AddTimestampTest.java │ │ └── DuplicateFilterTest.java │ └── protocol │ │ ├── set │ │ └── HttpProtocolTest.java │ │ └── stream │ │ └── HttpProtocolTest.java │ ├── sensemap │ ├── OpenSenseMapAdapterTest.java │ └── SensorNamesTest.java │ └── ti │ └── TISensorTagTest.java ├── streampipes-extensions-all-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── streampipes │ └── extensions │ └── all │ └── jvm │ └── AllExtensionsInit.java ├── streampipes-pipeline-elements-all-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── streampipes │ └── pe │ └── flink │ ├── AllFlinkPipelineElementsInit.java │ └── config │ ├── Config.java │ └── ConfigKeys.java ├── streampipes-pipeline-elements-all-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── streampipes │ └── pe │ └── jvm │ ├── AllPipelineElementsInit.java │ └── config │ ├── AllPipelineElementsConfig.java │ └── ConfigKeys.java ├── streampipes-pipeline-elements-data-simulator ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── streampipes │ └── pe │ └── simulator │ ├── StreamPipesSimulationRunner.java │ └── TopicAwareWorkflow.java ├── streampipes-pipeline-elements-shared ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── streampipes │ └── pe │ └── shared │ ├── PlaceholderExtractor.java │ └── config │ ├── kafka │ ├── KafkaConfig.java │ └── KafkaConnectUtils.java │ └── mqtt │ ├── MqttConfig.java │ ├── MqttConnectUtils.java │ └── MqttConsumer.java ├── streampipes-processors-aggregation-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── streampipes │ │ │ └── processors │ │ │ └── aggregation │ │ │ └── flink │ │ │ ├── AbstractAggregationProgram.java │ │ │ ├── AggregationFlinkInit.java │ │ │ ├── config │ │ │ ├── AggregationFlinkConfig.java │ │ │ └── ConfigKeys.java │ │ │ └── processor │ │ │ ├── aggregation │ │ │ ├── Aggregation.java │ │ │ ├── AggregationController.java │ │ │ ├── AggregationParameters.java │ │ │ ├── AggregationProgram.java │ │ │ ├── AggregationType.java │ │ │ ├── CountAggregation.java │ │ │ ├── TimeAggregation.java │ │ │ └── config │ │ │ │ └── aggregation.md │ │ │ ├── count │ │ │ ├── CountController.java │ │ │ ├── CountMapper.java │ │ │ ├── CountParameters.java │ │ │ ├── CountProgram.java │ │ │ ├── CountTrigger.java │ │ │ ├── TimeWindowConverter.java │ │ │ └── Tuple2MapMapper.java │ │ │ ├── eventcount │ │ │ ├── EventCountController.java │ │ │ ├── EventCountMapper.java │ │ │ ├── EventCountOutputMapper.java │ │ │ ├── EventCountParameters.java │ │ │ └── EventCountProgram.java │ │ │ └── rate │ │ │ ├── EventRate.java │ │ │ ├── EventRateController.java │ │ │ ├── EventRateParameter.java │ │ │ └── EventRateProgram.java │ └── resources │ │ ├── org.apache.streampipes.processors.aggregation.flink.aggregation │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.aggregation.flink.count │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.aggregation.flink.eventcount │ │ ├── documentation.md │ │ └── strings.en │ │ └── org.apache.streampipes.processors.aggregation.flink.rate │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ └── test │ └── java │ └── org │ └── apache │ └── streampipes │ └── processors │ └── aggregation │ └── flink │ └── processor │ ├── aggregation │ ├── AggregationTestData.java │ └── TestTimeAggregationProgram.java │ ├── count │ └── TestCountProgram.java │ └── rate │ └── TestRateProgram.java ├── streampipes-processors-change-detection-jvm ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── changedetection │ │ └── jvm │ │ ├── ChangeDetectionJvmInit.java │ │ ├── config │ │ ├── ChangeDetectionJvmConfig.java │ │ └── ConfigKeys.java │ │ └── cusum │ │ ├── Cusum.java │ │ ├── CusumController.java │ │ ├── CusumEventFields.java │ │ ├── CusumParameters.java │ │ └── WelfordAggregate.java │ └── resources │ └── org.apache.streampipes.processors.changedetection.jvm.cusum │ ├── documentation.md │ └── strings.en ├── streampipes-processors-enricher-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── enricher │ │ └── flink │ │ ├── AbstractEnricherProgram.java │ │ ├── EnricherFlinkInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── EnricherFlinkConfig.java │ │ └── processor │ │ ├── math │ │ ├── mathop │ │ │ ├── MathOp.java │ │ │ ├── MathOpController.java │ │ │ ├── MathOpParameters.java │ │ │ └── MathOpProgram.java │ │ ├── operation │ │ │ ├── Operation.java │ │ │ ├── OperationAddition.java │ │ │ ├── OperationDivide.java │ │ │ ├── OperationModulo.java │ │ │ ├── OperationMultiply.java │ │ │ └── OperationSubtracting.java │ │ └── staticmathop │ │ │ ├── StaticMathOp.java │ │ │ ├── StaticMathOpController.java │ │ │ ├── StaticMathOpParameters.java │ │ │ └── StaticMathOpProgram.java │ │ ├── timestamp │ │ ├── TimestampController.java │ │ ├── TimestampEnricher.java │ │ ├── TimestampParameters.java │ │ └── TimestampProgram.java │ │ ├── trigonometry │ │ ├── Operation.java │ │ ├── Trigonometry.java │ │ ├── TrigonometryController.java │ │ ├── TrigonometryParameters.java │ │ └── TrigonometryProgram.java │ │ └── urldereferencing │ │ ├── UrlDereferencing.java │ │ ├── UrlDereferencingController.java │ │ ├── UrlDereferencingParameter.java │ │ └── UrlDereferencingProgram.java │ └── resources │ ├── org.apache.streampipes.processors.enricher.flink.processor.math.mathop │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.enricher.flink.processor.math.staticmathop │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.enricher.flink.processor.trigonometry │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.enricher.flink.processor.urldereferencing │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processors.enricher.flink.timestamp │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-processors-enricher-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── .env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── enricher │ │ └── jvm │ │ ├── EnricherJvmInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── EnricherJvmConfig.java │ │ └── processor │ │ ├── jseval │ │ ├── JSEval.java │ │ ├── JSEvalController.java │ │ └── JSEvalParameters.java │ │ ├── sizemeasure │ │ ├── SizeMeasure.java │ │ ├── SizeMeasureController.java │ │ └── SizeMeasureParameters.java │ │ └── valueChange │ │ └── ValueChangeProcessor.java │ └── resources │ ├── org.apache.streampipes.processors.enricher.jvm.jseval │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.enricher.jvm.sizemeasure │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processors.enricher.jvm.valueChange │ ├── documentation.md │ └── strings.en ├── streampipes-processors-filters-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── filters │ │ └── jvm │ │ ├── FiltersJvmInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── FiltersJvmConfig.java │ │ └── processor │ │ ├── booleanfilter │ │ └── BooleanFilterProcessor.java │ │ ├── compose │ │ └── ComposeProcessor.java │ │ ├── enrich │ │ └── MergeByEnrichProcessor.java │ │ ├── limit │ │ ├── RateLimit.java │ │ ├── RateLimitController.java │ │ ├── RateLimitParameters.java │ │ ├── util │ │ │ ├── EventSelection.java │ │ │ ├── ProcessJob.java │ │ │ ├── SchedulerUtil.java │ │ │ ├── WindowFactory.java │ │ │ └── WindowType.java │ │ └── window │ │ │ ├── CronWindow.java │ │ │ ├── LengthWindow.java │ │ │ ├── ScheduleWindow.java │ │ │ ├── TimeWindow.java │ │ │ └── Window.java │ │ ├── merge │ │ ├── MergeByTimeProcessor.java │ │ └── StreamBuffer.java │ │ ├── numericalfilter │ │ ├── NumericalFilterProcessor.java │ │ └── NumericalOperator.java │ │ ├── numericaltextfilter │ │ ├── NumericalOperator.java │ │ ├── NumericalTextFilterProcessor.java │ │ └── StringOperator.java │ │ ├── projection │ │ └── ProjectionProcessor.java │ │ ├── schema │ │ └── MergeBySchemaProcessor.java │ │ ├── textfilter │ │ ├── StringOperator.java │ │ └── TextFilterProcessor.java │ │ └── threshold │ │ ├── ThresholdDetectionOperator.java │ │ └── ThresholdDetectionProcessor.java │ └── resources │ ├── org.apache.streampipes.processors.filters.jvm.compose │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.enrich │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.limit │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.merge │ ├── documentation.md │ ├── icon.png │ ├── merge_description.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.numericalfilter │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.numericaltextfilter │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.processor.booleanfilter │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.project │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.schema │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processors.filters.jvm.textfilter │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processors.filters.jvm.threshold │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-processors-filters-siddhi ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── streampipes │ │ │ └── processors │ │ │ └── siddhi │ │ │ ├── FiltersSiddhiInit.java │ │ │ ├── config │ │ │ ├── ConfigKeys.java │ │ │ └── FilterSiddhiConfig.java │ │ │ ├── count │ │ │ └── CountAggregation.java │ │ │ ├── filter │ │ │ ├── NumericalFilter.java │ │ │ ├── NumericalFilterController.java │ │ │ └── NumericalFilterParameters.java │ │ │ ├── frequency │ │ │ ├── Frequency.java │ │ │ ├── FrequencyController.java │ │ │ └── FrequencyParameters.java │ │ │ ├── frequencychange │ │ │ ├── FrequencyChange.java │ │ │ ├── FrequencyChangeController.java │ │ │ └── FrequencyChangeParameters.java │ │ │ ├── listcollector │ │ │ └── ListCollector.java │ │ │ ├── listfilter │ │ │ └── ListFilter.java │ │ │ ├── sequence │ │ │ ├── Sequence.java │ │ │ ├── SequenceController.java │ │ │ └── SequenceParameters.java │ │ │ ├── stop │ │ │ ├── StreamStop.java │ │ │ ├── StreamStopController.java │ │ │ └── StreamStopParameters.java │ │ │ ├── topk │ │ │ └── TopK.java │ │ │ └── trend │ │ │ ├── Trend.java │ │ │ ├── TrendController.java │ │ │ ├── TrendOperator.java │ │ │ └── TrendParameters.java │ └── resources │ │ ├── org.apache.streampipes.processors.siddhi.count │ │ ├── documentation.md │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.frequency │ │ ├── documentation.md │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.frequencychange │ │ ├── documentation.md │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.increase │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.listcollector │ │ ├── documentation.md │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.listfilter │ │ ├── documentation.md │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.numericalfilter │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.sequence │ │ ├── documentation.md │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.siddhi.stop │ │ ├── documentation.md │ │ └── strings.en │ │ └── org.apache.streampipes.processors.siddhi.topk │ │ ├── documentation.md │ │ └── strings.en │ └── test │ └── java │ └── org │ └── apache │ └── streampipes │ └── processors │ └── siddhi │ └── trend │ └── TestTrendProcessor.java ├── streampipes-processors-geo-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processor │ │ └── geo │ │ └── flink │ │ ├── AbstractGeoProgram.java │ │ ├── GeoFlinkInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── GeoFlinkConfig.java │ │ └── processor │ │ └── gridenricher │ │ ├── CellOption.java │ │ ├── EnrichmentSettings.java │ │ ├── SpatialGridCalculator.java │ │ ├── SpatialGridConstants.java │ │ ├── SpatialGridEnricher.java │ │ ├── SpatialGridEnrichmentController.java │ │ ├── SpatialGridEnrichmentParameters.java │ │ └── SpatialGridEnrichmentProgram.java │ └── resources │ └── org.apache.streampipes.processor.geo.flink │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-processors-geo-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── geo │ │ └── jvm │ │ ├── GeoJvmInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── GeoJvmConfig.java │ │ ├── jts │ │ ├── helper │ │ │ ├── SpGeometryBuilder.java │ │ │ └── SpTrajectoryBuilder.java │ │ └── processor │ │ │ ├── latLngToGeo │ │ │ ├── LatLngToGeo.java │ │ │ ├── LatLngToGeoController.java │ │ │ └── LatLngToGeoParameter.java │ │ │ ├── setEPSG │ │ │ ├── SetEPSG.java │ │ │ ├── SetEpsgController.java │ │ │ └── SetEpsgParameter.java │ │ │ └── trajectory │ │ │ ├── CreateTrajectoryFromPoints.java │ │ │ ├── CreateTrajectoryFromPointsController.java │ │ │ └── CreateTrajectoryFromPointsParameter.java │ │ └── processor │ │ ├── distancecalculator │ │ ├── DistanceCalculator.java │ │ ├── DistanceCalculatorController.java │ │ └── DistanceCalculatorParameters.java │ │ ├── geocoder │ │ ├── GoogleMapsGeocoding.java │ │ ├── GoogleMapsGeocodingController.java │ │ └── GoogleMapsGeocodingParameters.java │ │ ├── revgeocoder │ │ ├── ReverseGeocoding.java │ │ ├── ReverseGeocodingController.java │ │ ├── ReverseGeocodingParameters.java │ │ └── geocode │ │ │ ├── GeoName.java │ │ │ ├── ReverseGeoCode.java │ │ │ └── kdtree │ │ │ ├── KDNode.java │ │ │ ├── KDNodeComparator.java │ │ │ └── KDTree.java │ │ ├── speed │ │ ├── SpeedCalculator.java │ │ ├── SpeedCalculatorController.java │ │ └── SpeedCalculatorParameters.java │ │ ├── staticdistancecalculator │ │ ├── StaticDistanceCalculator.java │ │ ├── StaticDistanceCalculatorController.java │ │ └── StaticDistanceCalculatorParameters.java │ │ ├── staticgeocoder │ │ ├── StaticGoogleMapsGeocoder.java │ │ ├── StaticGoogleMapsGeocodingController.java │ │ └── StaticGoogleMapsGeocodingParameters.java │ │ └── util │ │ └── DistanceUtil.java │ └── resources │ ├── org.apache.streampipes.processor.geo.jvm.geocoding │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processor.geo.jvm.reversegeocoding │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processor.geo.jvm.staticgeocoding │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processors.geo.jvm.jts.processor.latLngToGeo │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.geo.jvm.jts.processor.setEPSG │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.geo.jvm.jts.processor.trajectory │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.geo.jvm.processor.distancecalculator │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processors.geo.jvm.processor.speed │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processors.geo.jvm.processor.staticdistancecalculator │ ├── documentation.md │ └── strings.en ├── streampipes-processors-image-processing-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── imageprocessing │ │ └── jvm │ │ ├── ImageProcessingJvmInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── ImageProcessingJvmConfig.java │ │ └── processor │ │ ├── commons │ │ ├── ImageTransformer.java │ │ ├── PlainImageTransformer.java │ │ └── RequiredBoxStream.java │ │ ├── genericclassification │ │ ├── GenericImageClassification.java │ │ ├── GenericImageClassificationController.java │ │ └── GenericImageClassificationParameters.java │ │ ├── imagecropper │ │ ├── ImageCropper.java │ │ ├── ImageCropperController.java │ │ └── ImageCropperParameters.java │ │ ├── imageenrichment │ │ ├── BoxCoordinates.java │ │ ├── ColorUtil.java │ │ ├── ImageEnricher.java │ │ ├── ImageEnrichmentController.java │ │ └── ImageEnrichmentParameters.java │ │ └── qrreader │ │ ├── QrCodeReader.java │ │ ├── QrCodeReaderController.java │ │ └── QrCodeReaderParameters.java │ └── resources │ ├── org.apache.streampipes.processor.imageclassification.jvm.generic-image-classification │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processor.imageclassification.jvm.image-cropper │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processor.imageclassification.jvm.image-enricher │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processor.imageclassification.qrcode │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-processors-pattern-detection-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── streampipes │ │ │ └── processors │ │ │ └── pattern │ │ │ └── detection │ │ │ └── flink │ │ │ ├── AbstractPatternDetectionProgram.java │ │ │ ├── PatternDetectionFlinkInit.java │ │ │ ├── config │ │ │ ├── ConfigKeys.java │ │ │ └── PatternDetectionFlinkConfig.java │ │ │ └── processor │ │ │ ├── absence │ │ │ ├── Absence.java │ │ │ ├── AbsenceController.java │ │ │ ├── AbsenceParameters.java │ │ │ └── AbsenceProgram.java │ │ │ ├── and │ │ │ ├── And.java │ │ │ ├── AndController.java │ │ │ ├── AndParameters.java │ │ │ ├── AndProgram.java │ │ │ ├── TimeUnit.java │ │ │ └── TimeUnitConverter.java │ │ │ ├── common │ │ │ └── TimestampExtractor.java │ │ │ ├── peak │ │ │ ├── PeakDetectionCalculator.java │ │ │ ├── PeakDetectionController.java │ │ │ ├── PeakDetectionParameters.java │ │ │ ├── PeakDetectionProgram.java │ │ │ └── utils │ │ │ │ └── SlidingBatchWindow.java │ │ │ └── sequence │ │ │ ├── EventStorage.java │ │ │ ├── Sequence.java │ │ │ ├── SequenceController.java │ │ │ ├── SequenceParameters.java │ │ │ └── SequenceProgram.java │ └── resources │ │ ├── org.apache.streampipes.processors.pattern-detection.flink.absence │ │ ├── documentation.md │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.pattern-detection.flink.and │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.pattern-detection.flink.peak-detection │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ └── org.apache.streampipes.processors.pattern-detection.flink.sequence │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ └── test │ └── java │ └── org │ └── apache │ └── streampipes │ └── processors │ └── pattern │ └── detection │ └── processor │ ├── absence │ └── TestAbsence.java │ └── and │ └── TestAnd.java ├── streampipes-processors-statistics-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── .env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── statistics │ │ └── flink │ │ ├── AbstractStatisticsProgram.java │ │ ├── StatisticsFlinkInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── StatisticsFlinkConfig.java │ │ ├── extensions │ │ ├── MapKeySelector.java │ │ ├── SlidingBatchWindow.java │ │ ├── SlidingEventTimeWindow.java │ │ └── TimestampMappingFunction.java │ │ └── processor │ │ └── stat │ │ ├── summary │ │ ├── StatisticsSummaryCalculator.java │ │ ├── StatisticsSummaryController.java │ │ ├── StatisticsSummaryParameters.java │ │ └── StatisticsSummaryProgram.java │ │ └── window │ │ ├── StatisticsSummaryCalculatorWindow.java │ │ ├── StatisticsSummaryControllerWindow.java │ │ ├── StatisticsSummaryParametersWindow.java │ │ ├── StatisticsSummaryParamsSerializable.java │ │ └── StatisticsSummaryProgramWindow.java │ └── resources │ ├── org.apache.streampipes.processors.statistics.flink.statistics-summary-window │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processors.statistics.flink.statistics-summary │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-processors-text-mining-flink ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── textmining │ │ └── flink │ │ ├── AbstractTextMiningProgram.java │ │ ├── TextMiningFlinkInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── TextMiningFlinkConfig.java │ │ └── processor │ │ ├── entity │ │ ├── EntityExtraction.java │ │ └── EntityExtractionModel.java │ │ ├── language │ │ ├── LanguageDetection.java │ │ ├── LanguageDetectionController.java │ │ ├── LanguageDetectionParameters.java │ │ └── LanguageDetectionProgram.java │ │ ├── sentiment │ │ ├── SentimentDetection.java │ │ ├── SentimentDetectionController.java │ │ ├── SentimentDetectionParameters.java │ │ └── SentimentDetectionProgram.java │ │ └── wordcount │ │ ├── Word.java │ │ ├── WordCountController.java │ │ ├── WordCountParameters.java │ │ ├── WordCountProgram.java │ │ ├── WordSplitter.java │ │ └── WordToEventConverter.java │ └── resources │ ├── en-ner-location.bin │ ├── en-ner-organization.bin │ ├── en-ner-person.bin │ ├── org.apache.streampipes.processors.textmining.flink.languagedetection │ ├── documentation.md │ └── strings.en │ └── org.apache.streampipes.processors.textmining.flink.wordcount │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-processors-text-mining-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── textmining │ │ └── jvm │ │ ├── TextMiningJvmInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── TextMiningJvmConfig.java │ │ └── processor │ │ ├── TextMiningUtil.java │ │ ├── chunker │ │ ├── Chunker.java │ │ ├── ChunkerController.java │ │ └── ChunkerParameters.java │ │ ├── language │ │ ├── LanguageDetection.java │ │ ├── LanguageDetectionController.java │ │ └── LanguageDetectionParameters.java │ │ ├── namefinder │ │ ├── NameFinder.java │ │ ├── NameFinderController.java │ │ └── NameFinderParameters.java │ │ ├── partofspeech │ │ ├── PartOfSpeech.java │ │ ├── PartOfSpeechController.java │ │ └── PartOfSpeechParameters.java │ │ ├── sentencedetection │ │ ├── SentenceDetection.java │ │ ├── SentenceDetectionController.java │ │ └── SentenceDetectionParameters.java │ │ └── tokenizer │ │ ├── Tokenizer.java │ │ ├── TokenizerController.java │ │ └── TokenizerParameters.java │ └── resources │ ├── org.apache.streampipes.processors.textmining.jvm.chunker │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.textmining.jvm.languagedetection │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.textmining.jvm.namefinder │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.textmining.jvm.partofspeech │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.textmining.jvm.sentencedetection │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processors.textmining.jvm.tokenizer │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-processors-transformation-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ ├── main │ ├── java │ │ ├── com │ │ │ └── kohlschutter │ │ │ │ └── boilerpipe │ │ │ │ ├── BoilerpipeDocumentSource.java │ │ │ │ ├── BoilerpipeExtractor.java │ │ │ │ ├── BoilerpipeFilter.java │ │ │ │ ├── BoilerpipeInput.java │ │ │ │ ├── BoilerpipeProcessingException.java │ │ │ │ ├── conditions │ │ │ │ └── TextBlockCondition.java │ │ │ │ ├── document │ │ │ │ ├── Image.java │ │ │ │ ├── TextBlock.java │ │ │ │ ├── TextDocument.java │ │ │ │ ├── TextDocumentStatistics.java │ │ │ │ └── package-info.java │ │ │ │ ├── estimators │ │ │ │ └── SimpleEstimator.java │ │ │ │ ├── extractors │ │ │ │ ├── ArticleExtractor.java │ │ │ │ ├── ArticleSentencesExtractor.java │ │ │ │ ├── CanolaExtractor.java │ │ │ │ ├── CommonExtractors.java │ │ │ │ ├── DefaultExtractor.java │ │ │ │ ├── ExtractorBase.java │ │ │ │ ├── KeepEverythingExtractor.java │ │ │ │ ├── KeepEverythingWithMinKWordsExtractor.java │ │ │ │ ├── LargestContentExtractor.java │ │ │ │ ├── NumWordsRulesExtractor.java │ │ │ │ └── package-info.java │ │ │ │ ├── filters │ │ │ │ ├── debug │ │ │ │ │ └── PrintDebugFilter.java │ │ │ │ ├── english │ │ │ │ │ ├── DensityRulesClassifier.java │ │ │ │ │ ├── HeuristicFilterBase.java │ │ │ │ │ ├── IgnoreBlocksAfterContentFilter.java │ │ │ │ │ ├── IgnoreBlocksAfterContentFromEndFilter.java │ │ │ │ │ ├── KeepLargestFulltextBlockFilter.java │ │ │ │ │ ├── MinFulltextWordsFilter.java │ │ │ │ │ ├── NumWordsRulesClassifier.java │ │ │ │ │ ├── TerminatingBlocksFinder.java │ │ │ │ │ └── package-info.java │ │ │ │ ├── heuristics │ │ │ │ │ ├── AddPrecedingLabelsFilter.java │ │ │ │ │ ├── ArticleMetadataFilter.java │ │ │ │ │ ├── BlockProximityFusion.java │ │ │ │ │ ├── ContentFusion.java │ │ │ │ │ ├── DocumentTitleMatchClassifier.java │ │ │ │ │ ├── ExpandTitleToContentFilter.java │ │ │ │ │ ├── KeepLargestBlockFilter.java │ │ │ │ │ ├── LabelFusion.java │ │ │ │ │ ├── LargeBlockSameTagLevelToContentFilter.java │ │ │ │ │ ├── ListAtEndFilter.java │ │ │ │ │ ├── SimpleBlockFusionProcessor.java │ │ │ │ │ ├── TrailingHeadlineToBoilerplateFilter.java │ │ │ │ │ └── package-info.java │ │ │ │ └── simple │ │ │ │ │ ├── BoilerplateBlockFilter.java │ │ │ │ │ ├── InvertedFilter.java │ │ │ │ │ ├── LabelToBoilerplateFilter.java │ │ │ │ │ ├── LabelToContentFilter.java │ │ │ │ │ ├── MarkEverythingBoilerplateFilter.java │ │ │ │ │ ├── MarkEverythingContentFilter.java │ │ │ │ │ ├── MinClauseWordsFilter.java │ │ │ │ │ ├── MinWordsFilter.java │ │ │ │ │ ├── SplitParagraphBlocksFilter.java │ │ │ │ │ ├── SurroundingToContentFilter.java │ │ │ │ │ └── package-info.java │ │ │ │ ├── labels │ │ │ │ ├── ConditionalLabelAction.java │ │ │ │ ├── DefaultLabels.java │ │ │ │ └── LabelAction.java │ │ │ │ ├── package-info.java │ │ │ │ ├── sax │ │ │ │ ├── BoilerpipeHTMLContentHandler.java │ │ │ │ ├── BoilerpipeHTMLParser.java │ │ │ │ ├── BoilerpipeSAXInput.java │ │ │ │ ├── CommonTagActions.java │ │ │ │ ├── DefaultTagActionMap.java │ │ │ │ ├── HTMLDocument.java │ │ │ │ ├── HTMLFetcher.java │ │ │ │ ├── HTMLHighlighter.java │ │ │ │ ├── ImageExtractor.java │ │ │ │ ├── InputSourceable.java │ │ │ │ ├── MarkupTagAction.java │ │ │ │ ├── TagAction.java │ │ │ │ ├── TagActionMap.java │ │ │ │ └── package-info.java │ │ │ │ └── util │ │ │ │ ├── UnicodeTokenizer.java │ │ │ │ └── package-info.java │ │ └── org │ │ │ └── apache │ │ │ └── streampipes │ │ │ └── processors │ │ │ └── transformation │ │ │ └── flink │ │ │ ├── AbstractFlinkTransformationProgram.java │ │ │ ├── TransformationFlinkInit.java │ │ │ ├── config │ │ │ ├── ConfigKeys.java │ │ │ └── TransformationFlinkConfig.java │ │ │ └── processor │ │ │ ├── boilerplate │ │ │ ├── BoilerplateController.java │ │ │ ├── BoilerplateParameters.java │ │ │ ├── BoilerplateProgram.java │ │ │ ├── BoilerplateRemover.java │ │ │ ├── ExtractorMode.java │ │ │ └── OutputMode.java │ │ │ ├── converter │ │ │ ├── FieldConverter.java │ │ │ ├── FieldConverterController.java │ │ │ ├── FieldConverterParameters.java │ │ │ └── FieldConverterProgram.java │ │ │ ├── hasher │ │ │ ├── FieldHasher.java │ │ │ ├── FieldHasherController.java │ │ │ ├── FieldHasherParameters.java │ │ │ ├── FieldHasherProgram.java │ │ │ └── algorithm │ │ │ │ ├── HashAlgorithm.java │ │ │ │ ├── HashAlgorithmType.java │ │ │ │ ├── Md5HashAlgorithm.java │ │ │ │ ├── Sha1HashAlgorithm.java │ │ │ │ └── Sha2HashAlgorithm.java │ │ │ ├── mapper │ │ │ ├── FieldMapper.java │ │ │ ├── FieldMapperController.java │ │ │ ├── FieldMapperParameters.java │ │ │ └── FieldMapperProgram.java │ │ │ ├── measurementUnitConverter │ │ │ ├── MeasurementUnitConverter.java │ │ │ ├── MeasurementUnitConverterController.java │ │ │ ├── MeasurementUnitConverterParameters.java │ │ │ └── MeasurementUnitConverterProgram.java │ │ │ └── rename │ │ │ ├── FieldRenamer.java │ │ │ ├── FieldRenamerController.java │ │ │ ├── FieldRenamerParameters.java │ │ │ └── FieldRenamerProgram.java │ └── resources │ │ ├── org.apache.streampipes.processors.transformation.flink.field-converter │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.transformation.flink.field-mapper │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.transformation.flink.field-renamer │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.transformation.flink.fieldhasher │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ ├── org.apache.streampipes.processors.transformation.flink.measurement-unit-converter │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ │ └── org.apache.streampipes.processors.transformation.flink.processor.boilerplate │ │ ├── documentation.md │ │ ├── icon.png │ │ └── strings.en │ └── test │ └── java │ └── org │ └── apache │ └── streampipes │ └── processors │ └── transformation │ └── flink │ ├── processor │ ├── converter │ │ └── TestConverterProgram.java │ ├── hasher │ │ ├── TestFieldHasher.java │ │ ├── TestFieldHasherController.java │ │ ├── TestFieldHasherProgram.java │ │ └── TestFieldHasherUtils.java │ └── rename │ │ └── TestRenameProgram.java │ └── utils │ └── DummyCollector.java ├── streampipes-processors-transformation-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── processors │ │ └── transformation │ │ └── jvm │ │ ├── TransformationJvmInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── TransformationJvmConfig.java │ │ └── processor │ │ ├── array │ │ ├── count │ │ │ ├── CountArray.java │ │ │ ├── CountArrayController.java │ │ │ └── CountArrayParameters.java │ │ └── split │ │ │ ├── SplitArray.java │ │ │ ├── SplitArrayController.java │ │ │ └── SplitArrayParameters.java │ │ ├── booloperator │ │ ├── counter │ │ │ ├── BooleanCounter.java │ │ │ ├── BooleanCounterController.java │ │ │ └── BooleanCounterParameters.java │ │ ├── edge │ │ │ ├── SignalEdgeFilter.java │ │ │ ├── SignalEdgeFilterController.java │ │ │ └── SignalEdgeFilterParameters.java │ │ ├── inverter │ │ │ ├── BooleanInverter.java │ │ │ ├── BooleanInverterController.java │ │ │ └── BooleanInverterParameters.java │ │ ├── logical │ │ │ ├── BooleanOperationInputConfigs.java │ │ │ ├── BooleanOperatorProcessor.java │ │ │ ├── enums │ │ │ │ └── BooleanOperatorType.java │ │ │ └── operations │ │ │ │ ├── ANDBoolOperation.java │ │ │ │ ├── IBoolOperation.java │ │ │ │ ├── NORBoolOperation.java │ │ │ │ ├── NOTBooleanOperation.java │ │ │ │ ├── ORBooleanOperation.java │ │ │ │ ├── XNORBoolOperation.java │ │ │ │ ├── XORBooleanOperation.java │ │ │ │ └── factory │ │ │ │ └── BoolOperationFactory.java │ │ ├── state │ │ │ ├── BooleanToState.java │ │ │ ├── BooleanToStateController.java │ │ │ └── BooleanToStateParameters.java │ │ ├── timekeeping │ │ │ ├── BooleanTimekeeping.java │ │ │ ├── BooleanTimekeepingController.java │ │ │ └── BooleanTimekeepingParameters.java │ │ └── timer │ │ │ ├── BooleanTimer.java │ │ │ ├── BooleanTimerController.java │ │ │ └── BooleanTimerParameters.java │ │ ├── csvmetadata │ │ ├── CsvMetadataEnrichment.java │ │ ├── CsvMetadataEnrichmentController.java │ │ ├── CsvMetadataEnrichmentParameters.java │ │ └── CsvMetadataEnrichmentUtils.java │ │ ├── fieldrename │ │ └── FiledRenameProcessor.java │ │ ├── state │ │ ├── StateUtils.java │ │ ├── buffer │ │ │ ├── StateBuffer.java │ │ │ ├── StateBufferController.java │ │ │ └── StateBufferParameters.java │ │ └── labeler │ │ │ ├── LabelerUtils.java │ │ │ ├── buffer │ │ │ ├── StateBufferLabeler.java │ │ │ ├── StateBufferLabelerController.java │ │ │ └── StateBufferLabelerParameters.java │ │ │ ├── model │ │ │ ├── Statement.java │ │ │ └── StatementUtils.java │ │ │ └── number │ │ │ ├── NumberLabeler.java │ │ │ ├── NumberLabelerController.java │ │ │ └── NumberLabelerParameters.java │ │ ├── stringoperator │ │ ├── counter │ │ │ ├── StringCounter.java │ │ │ ├── StringCounterController.java │ │ │ └── StringCounterParameters.java │ │ ├── state │ │ │ ├── StringToState.java │ │ │ ├── StringToStateController.java │ │ │ └── StringToStateParameters.java │ │ └── timer │ │ │ ├── StringTimer.java │ │ │ ├── StringTimerController.java │ │ │ └── StringTimerParameters.java │ │ ├── task │ │ ├── TaskDuration.java │ │ ├── TaskDurationController.java │ │ └── TaskDurationParameters.java │ │ ├── timestampextractor │ │ ├── OutputFields.java │ │ ├── TimestampExtractor.java │ │ ├── TimestampExtractorController.java │ │ └── TimestampExtractorParameters.java │ │ ├── transformtoboolean │ │ ├── TransformToBoolean.java │ │ ├── TransformToBooleanController.java │ │ └── TransformToBooleanParameters.java │ │ └── value │ │ ├── change │ │ ├── ChangedValueDetection.java │ │ ├── ChangedValueDetectionController.java │ │ └── ChangedValueDetectionParameters.java │ │ └── duration │ │ ├── CalculateDuration.java │ │ ├── CalculateDurationController.java │ │ └── CalculateDurationParameters.java │ └── resources │ ├── org.apache.streampipes.processors.transformation.jvm.booloperator.counter │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.booloperator.inverter │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.booloperator.logical │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping │ ├── documentation.md │ ├── icon.png │ ├── strings.en │ └── time_measure_example.png │ ├── org.apache.streampipes.processors.transformation.jvm.booloperator.timer │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.changed-value │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.count-array │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.csvmetadata │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.duration-value │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.fieldrename │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.processor.booloperator.edge │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.processor.state.buffer │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.buffer │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.processor.timestampextractor │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.split-array │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.stringoperator.counter │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.stringoperator.timer │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.processors.transformation.jvm.taskduration │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.processors.transformation.jvm.transform-to-boolean │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-sinks-brokers-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── sinks │ │ └── brokers │ │ └── jvm │ │ ├── BrokersJvmInit.java │ │ ├── bufferrest │ │ ├── BufferRest.java │ │ ├── BufferRestController.java │ │ ├── BufferRestParameters.java │ │ └── buffer │ │ │ ├── BufferListener.java │ │ │ └── MessageBuffer.java │ │ ├── config │ │ ├── BrokersJvmConfig.java │ │ └── ConfigKeys.java │ │ ├── jms │ │ ├── JmsController.java │ │ ├── JmsParameters.java │ │ └── JmsPublisher.java │ │ ├── kafka │ │ ├── KafkaController.java │ │ ├── KafkaParameters.java │ │ └── KafkaPublisher.java │ │ ├── mqtt │ │ ├── MqttClient.java │ │ ├── MqttPublisherSink.java │ │ └── common │ │ │ ├── MqttOptions.java │ │ │ └── MqttUtils.java │ │ ├── nats │ │ ├── NatsController.java │ │ ├── NatsParameters.java │ │ └── NatsPublisher.java │ │ ├── pulsar │ │ ├── Pulsar.java │ │ ├── PulsarController.java │ │ └── PulsarParameters.java │ │ ├── rabbitmq │ │ ├── RabbitMqConsumer.java │ │ ├── RabbitMqController.java │ │ ├── RabbitMqParameters.java │ │ └── RabbitMqPublisher.java │ │ ├── rest │ │ ├── RestController.java │ │ ├── RestParameters.java │ │ └── RestPublisher.java │ │ └── websocket │ │ ├── SocketServer.java │ │ └── WebsocketServerSink.java │ └── resources │ ├── org.apache.streampipes.sinks.brokers.jvm.bufferrest │ ├── documentation.md │ └── strings.en │ ├── org.apache.streampipes.sinks.brokers.jvm.jms │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.brokers.jvm.kafka │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.brokers.jvm.mqtt │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.brokers.jvm.nats │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.brokers.jvm.pulsar │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.brokers.jvm.rabbitmq │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.brokers.jvm.rest │ ├── documentation.md │ └── strings.en │ └── org.apache.streampipes.sinks.brokers.jvm.websocket │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-sinks-databases-flink ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── sinks │ │ └── databases │ │ └── flink │ │ ├── DatabasesFlinkInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── DatabasesFlinkConfig.java │ │ └── elasticsearch │ │ ├── ElasticSearchController.java │ │ ├── ElasticSearchParameters.java │ │ ├── ElasticSearchProgram.java │ │ ├── ElasticsearchIndexRequestBuilder.java │ │ └── elastic │ │ ├── ActionRequestFailureHandler.java │ │ ├── BulkProcessorIndexer.java │ │ ├── Elasticsearch6ApiCallBridge.java │ │ ├── ElasticsearchApiCallBridge.java │ │ ├── ElasticsearchSink.java │ │ ├── ElasticsearchSinkBase.java │ │ ├── ElasticsearchSinkFunction.java │ │ ├── NoOpFailureHandler.java │ │ └── RequestIndexer.java │ └── resources │ └── org.apache.streampipes.sinks.databases.flink.elasticsearch │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-sinks-databases-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── sinks │ │ └── databases │ │ └── jvm │ │ ├── DatabasesJvmInit.java │ │ ├── config │ │ ├── ConfigKeys.java │ │ └── DatabasesJvmConfig.java │ │ ├── couchdb │ │ ├── CouchDb.java │ │ ├── CouchDbController.java │ │ └── CouchDbParameters.java │ │ ├── ditto │ │ ├── Ditto.java │ │ ├── DittoController.java │ │ └── DittoParameters.java │ │ ├── influxdb │ │ ├── InfluxDb.java │ │ ├── InfluxDbClient.java │ │ ├── InfluxDbController.java │ │ └── InfluxDbParameters.java │ │ ├── iotdb │ │ ├── IotDb.java │ │ ├── IotDbController.java │ │ └── IotDbParameters.java │ │ ├── jdbcclient │ │ ├── JdbcClient.java │ │ ├── model │ │ │ ├── DbDataTypeFactory.java │ │ │ ├── DbDataTypes.java │ │ │ ├── DbDescription.java │ │ │ ├── JdbcConnectionParameters.java │ │ │ ├── ParameterInformation.java │ │ │ ├── StatementHandler.java │ │ │ ├── SupportedDbEngines.java │ │ │ └── TableDescription.java │ │ └── utils │ │ │ ├── SQLStatementUtils.java │ │ │ └── StatementUtils.java │ │ ├── mysql │ │ ├── Mysql.java │ │ ├── MysqlController.java │ │ └── MysqlParameters.java │ │ ├── opcua │ │ ├── OpcUa.java │ │ ├── OpcUaParameters.java │ │ └── UpcUaController.java │ │ ├── postgresql │ │ ├── PostgreSql.java │ │ ├── PostgreSqlController.java │ │ └── PostgreSqlParameters.java │ │ └── redis │ │ ├── Redis.java │ │ ├── RedisController.java │ │ └── RedisParameters.java │ └── resources │ ├── org.apache.streampipes.sinks.databases.ditto │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.databases.jvm.couchdb │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.databases.jvm.influxdb │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.databases.jvm.iotdb │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.databases.jvm.mysql │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.databases.jvm.opcua │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.databases.jvm.postgresql │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.sinks.databases.jvm.redis │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-sinks-internal-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── sinks │ │ └── internal │ │ └── jvm │ │ ├── SinksInternalJvmInit.java │ │ ├── config │ │ └── ConfigKeys.java │ │ ├── dashboard │ │ ├── Dashboard.java │ │ ├── DashboardController.java │ │ └── DashboardParameters.java │ │ ├── datalake │ │ ├── DataLake.java │ │ ├── DataLakeController.java │ │ ├── DataLakeInfluxDbClient.java │ │ └── DataLakeParameters.java │ │ └── notification │ │ ├── NotificationController.java │ │ ├── NotificationParameters.java │ │ └── NotificationProducer.java │ └── resources │ ├── org.apache.streampipes.sinks.internal.jvm.dashboard │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.internal.jvm.datalake │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.sinks.internal.jvm.notification │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-sinks-notifications-jvm ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── sinks │ │ └── notifications │ │ └── jvm │ │ ├── SinksNotificationsJvmInit.java │ │ ├── config │ │ └── ConfigKeys.java │ │ ├── email │ │ ├── EmailController.java │ │ ├── EmailParameters.java │ │ └── EmailPublisher.java │ │ ├── onesignal │ │ ├── OneSignalController.java │ │ ├── OneSignalParameters.java │ │ └── OneSignalProducer.java │ │ ├── slack │ │ ├── SlackNotification.java │ │ ├── SlackNotificationController.java │ │ └── SlackNotificationParameters.java │ │ └── telegram │ │ ├── TelegramController.java │ │ ├── TelegramParameters.java │ │ └── TelegramPublisher.java │ └── resources │ ├── org.apache.streampipes.sinks.notifications.jvm.email │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.notifications.jvm.onesignal │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sinks.notifications.jvm.slack │ ├── documentation.md │ ├── icon.png │ └── strings.en │ └── org.apache.streampipes.sinks.notifications.jvm.telegram │ ├── documentation.md │ ├── icon.png │ └── strings.en ├── streampipes-sources-vehicle-simulator ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── sources │ │ └── vehicle │ │ └── simulator │ │ ├── VehicleSimulatorInit.java │ │ ├── config │ │ └── ConfigKeys.java │ │ ├── simulator │ │ └── VehicleDataSimulator.java │ │ └── vehicle │ │ └── streams │ │ └── VehicleStream.java │ └── resources │ ├── streampipesDemoConfig.json │ └── streampipesVehicleWorkflow.json ├── streampipes-sources-watertank-simulator ├── Dockerfile ├── aarch64.Dockerfile ├── arm.Dockerfile ├── development │ └── env ├── pom.xml └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── streampipes │ │ └── sources │ │ └── watertank │ │ └── simulator │ │ ├── WatertankSimulatorInit.java │ │ ├── config │ │ └── ConfigKeys.java │ │ ├── utils │ │ └── WatertankDataSimulator.java │ │ ├── vocabulary │ │ └── WaterTankVocabulary.java │ │ └── watertank │ │ └── streams │ │ ├── FlowRate1Stream.java │ │ ├── FlowRate2Stream.java │ │ ├── PressureTankStream.java │ │ ├── WaterLevel1Stream.java │ │ └── WaterLevel2Stream.java │ └── resources │ ├── org.apache.streampipes.sources.simulator.flowrate1 │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sources.simulator.flowrate2 │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sources.simulator.pressure │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sources.simulator.waterlevel1 │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── org.apache.streampipes.sources.simulator.waterlevel2 │ ├── documentation.md │ ├── icon.png │ └── strings.en │ ├── streampipesDemoConfig.json │ ├── streampipesFestoFlowRateWorkflow.json │ ├── streampipesFestoLevel101Workflow.json │ ├── streampipesFestoLevel102Workflow.json │ ├── streampipesFestoPressureWorkflow.json │ └── streampipesSiemensFlowRateWorkflow.json └── tools └── maven └── assembly.xml /.idea/runConfigurations/all_pipeline_elements_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/all_pipeline_elements_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 20 | 21 | -------------------------------------------------------------------------------- /.idea/runConfigurations/connect_adapters.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/connect_adapters_iiot.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_aggregation_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_enricher_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_enricher_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_filters_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_filters_siddhi.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_geo_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_geo_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_imageprocessing_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_pattern_detection_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_statistics_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_textmining_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_textmining_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_transformation_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 14 | 15 | -------------------------------------------------------------------------------- /.idea/runConfigurations/processors_transformation_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/sinks_brokers_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/sinks_databases_flink.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 15 | 16 | -------------------------------------------------------------------------------- /.idea/runConfigurations/sinks_databases_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 13 | 14 | -------------------------------------------------------------------------------- /.idea/runConfigurations/sinks_internal_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 21 | 22 | -------------------------------------------------------------------------------- /.idea/runConfigurations/sinks_notifications_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 22 | 23 | -------------------------------------------------------------------------------- /.idea/runConfigurations/sources_vehicle_simulator_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 15 | 16 | -------------------------------------------------------------------------------- /.idea/runConfigurations/sources_watertank_simulator_jvm.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 15 | 16 | -------------------------------------------------------------------------------- /.spignore: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.jvm.jseval 2 | org.apache.streampipes.sinks.databases.jvm.redis -------------------------------------------------------------------------------- /DISCLAIMER: -------------------------------------------------------------------------------- 1 | Apache StreamPipes is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the 2 | Apache Incubator. Incubation is required of all newly accepted projects until a further review indicates that the 3 | infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful 4 | ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it 5 | does indicate that the project has yet to be fully endorsed by the ASF. -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.influxdb.set/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.influxdb.set/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.influxdb.stream/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.influxdb.stream/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.mysql.set/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.mysql.set/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.mysql.set/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.adapters.mysql.set.title=MySQL Set Adapter 2 | org.apache.streampipes.connect.iiot.adapters.mysql.set.description=Creates a data set from an SQL table 3 | 4 | mysqlHost.title=Hostname 5 | mysqlHost.description=Hostname of the MySql Server 6 | 7 | mysqlPort.title=Port 8 | mysqlPort.description=Port of the MySql Server. Default: 3306 9 | 10 | mysqlDatabase.title=Database 11 | mysqlDatabase.description=Database in which the table is located 12 | 13 | mysqlTable.title=Table 14 | mysqlTable.description=Table which should be watched 15 | 16 | mysqlUser.title=Username 17 | mysqlUser.description=Username of the user 18 | 19 | mysqlPassword.title=Password 20 | mysqlPassword.description=Password of the user 21 | 22 | replaceNullValues.title=Replace Null Values 23 | replaceNullValues.description=Should null values in the incoming data be replace by defaults? If not, these events are skipped -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.mysql.stream/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.mysql.stream/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.mysql.stream/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.adapters.mysql.stream.title=MySql Stream Adapter 2 | org.apache.streampipes.connect.iiot.adapters.mysql.stream.description=Creates a data stream for a SQL table 3 | 4 | 5 | mysqlHost.title=Hostname 6 | mysqlHost.description=Hostname of the MySql Server 7 | 8 | mysqlPort.title=Port 9 | mysqlPort.description=Port of the MySql Server. Default: 3306 10 | 11 | mysqlDatabase.title=Database 12 | mysqlDatabase.description=Database in which the table is located 13 | 14 | mysqlTable.title=Table 15 | mysqlTable.description=Table which should be watched 16 | 17 | mysqlUser.title=Username 18 | mysqlUser.description=Username of the user 19 | 20 | mysqlPassword.title=Password 21 | mysqlPassword.description=Password of the user 22 | 23 | replaceNullValues.title=Replace Null Values 24 | replaceNullValues.description=Should null values in the incoming data be replace by defaults? If not, these events are skipped -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.netio.mqtt/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.netio.mqtt/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.netio.mqtt/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.adapters.netio.mqtt.title=NETIO MQTT M2M 2 | org.apache.streampipes.connect.iiot.adapters.netio.mqtt.description=Connect a NETIO power plugs over MQTT 3 | 4 | access-mode.title=Access Mode 5 | access-mode.description= 6 | 7 | anonymous-alternative.title=Unauthenticated 8 | anonymous-alternative.description= 9 | 10 | username-alternative.title=Username/Password 11 | username-alternative.description= 12 | 13 | username-group.title=User Group 14 | username-group.description= 15 | 16 | username.title=Username 17 | username.description= 18 | 19 | password.title=Password 20 | password.description= 21 | 22 | broker_url.title=Broker URL 23 | broker_url.description=Example: tcp://test-server.com:1883 (Protocol required. Port required)" 24 | 25 | topic.title=Topic 26 | topic.description=Example: test/topic 27 | 28 | -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.netio.rest/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.netio.rest/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.netio.rest/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.adapters.netio.rest.title=NETIO http JSON 2 | org.apache.streampipes.connect.iiot.adapters.netio.rest.description=Connect a NETIO REST power plug 3 | 4 | NETIO_IP.title=Netio endpoint 5 | NETIO_IP.description=Example: 192.168.34.56 6 | 7 | NETIO_USERNAME.title=Username 8 | NETIO_USERNAME.description=Can be configured over the NETIO configurations 9 | 10 | NETIO_PASSWORD.title=Password 11 | NETIO_PASSWORD.description=Can be configured over the NETIO configurations 12 | 13 | NETIO_POLLING_INTERVAL.title=Polling Interval [s] 14 | NETIO_POLLING_INTERVAL.description=Polling interval in seconds 15 | -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.opcua/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.opcua/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.plc4x.modbus/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.plc4x.modbus/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.plc4x.s7/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.plc4x.s7/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.plc4x.s7/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.adapters.plc4x.s7.title=PLC4X S7 2 | org.apache.streampipes.connect.iiot.adapters.plc4x.s7.description=Connect directly to your PLC 3 | 4 | plc_ip.title=PLC Address 5 | plc_ip.description=Example: 192.168.34.56 6 | 7 | plc_polling_interval.title=Polling Interval [ms] 8 | plc_polling_interval.description=Polling Interval of adapter in milliseconds. Minimum value is 10. 9 | 10 | configure.title=Configuration 11 | configure.description= 12 | 13 | manually.title=Enter Manually 14 | manually.description=Enter the addresses to read the data from manually 15 | 16 | plc_nodes.title=Nodes 17 | plc_nodes.description=Define the nodes of the PLC 18 | 19 | plc_node_runtime_name.title=Runtime Name 20 | plc_node_runtime_name.description=example: temperature 21 | 22 | plc_node_name.title=Node Name 23 | plc_node_name.description=example: %Q0.4 24 | 25 | plc_node_type.title=Data Type 26 | plc_node_type.description=example: bool -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.ros/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.ros/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.ros/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.adapters.ros.title=ROS Bridge 2 | org.apache.streampipes.connect.iiot.adapters.ros.description=Connect Robots running on ROS. 3 | 4 | ROS_HOST_KEY.title=Ros Bridge 5 | ROS_HOST_KEY.description=Example: test-server.com (No protocol) 6 | 7 | ROS_PORT_KEY.title=Port 8 | ROS_PORT_KEY.description=Example: 9090 9 | 10 | TOPIC_KEY.title=Topic 11 | TOPIC_KEY.description=Example: /battery (Starts with /) -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.simulator.machine/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.simulator.machine/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.adapters.simulator.machine/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.adapters.simulator.machine.title=Machine Data Simulator 2 | org.apache.streampipes.connect.iiot.adapters.simulator.machine.description=Publishes various simulated machine sensor data 3 | 4 | wait-time-ms.title=Wait Time (MS) 5 | wait-time-ms.description=The time to wait between two events in milliseconds 6 | 7 | selected-simulator-option.title=Select sensor 8 | selected-simulator-option.description=Select simulated sensor data to be published -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.set.file/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.set.file/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.set.file/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.protocol.set.file.title=File Set 2 | org.apache.streampipes.connect.iiot.protocol.set.file.description=Reads the content from a local file. 3 | 4 | interval-key.title=Interval [ms] 5 | interval-key.description=Define waiting time between the files 6 | 7 | filePath.title=File 8 | filePath.description=File Path 9 | 10 | 11 | -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.set.http/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.set.http/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.set.http/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.protocol.set.http.title=HTTP Set 2 | org.apache.streampipes.connect.iiot.protocol.set.http.description=Regularly poll an HTTP endpoint 3 | 4 | url.title=URL 5 | url.description=Example: http(s)://test-server.com 6 | 7 | 8 | -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.file/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.file/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.file/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.protocol.stream.file.title=File Stream 2 | org.apache.streampipes.connect.iioot.protocol.stream.file=Continuously streams the content from a file. 3 | 4 | filePath.title=File 5 | filePath.description=File Path 6 | 7 | replaceTimestamp.title=Use current time 8 | replaceTimestamp.description=Replace Event Time with Current Timestamp 9 | 10 | speed.title=Replay Speed 11 | speed.description=original = 1; speedup 2x = 2; half speed = 0.5 12 | 13 | -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.http/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.http/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.http/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.protocol.stream.http.title=HTTP Stream 2 | org.apache.streampipes.connect.iiot.protocol.stream.http.description=Continuously fetched events from an HTTP REST endpoint. 3 | 4 | url.title=URL 5 | url.description=Example: http(s)://test-server.com 6 | 7 | interval.title=Interval [sec] 8 | interval.description=Example: 5 (Polling interval in seconds) 9 | 10 | 11 | -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.httpserver/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.httpserver/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.kafka/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.kafka/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.kafka/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.protocol.stream.kafka.title=Apache Kafka 2 | org.apache.streampipes.connect.iiot.protocol.stream.kafka.description=Consumes messages from an Apache Kafka broker 3 | 4 | host.title=Broker Hostname 5 | host.description=Example: test.server.com (No protocol) 6 | 7 | port.title=Broker Port 8 | port.description=9092 9 | 10 | topic.title=Topic 11 | topic.description=Example: test.topic 12 | 13 | access-mode.title=Access Mode 14 | access-mode.description= 15 | 16 | anonymous-alternative.title=Unauthenticated 17 | anonymous-alternative.description= 18 | 19 | username-alternative.title=Username/Password 20 | username-alternative.description= 21 | 22 | username-group.title=User Group 23 | username-group.description= 24 | 25 | username.title=Username 26 | username.description= 27 | 28 | password.title=Password 29 | password.description= -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.mqtt/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.mqtt/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.mqtt/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.protocol.stream.mqtt.title=MQTT 2 | org.apache.streampipes.connect.iiot.protocol.stream.mqtt.description=Consumes messages from a broker using the MQTT protocol 3 | 4 | access-mode.title=Access Mode 5 | access-mode.description=0 6 | 7 | anonymous-alternative.title=Unauthenticated 8 | anonymous-alternative.description= 9 | 10 | username-alternative.title=Username/Password 11 | username-alternative.description= 12 | 13 | username-group.title=User Group 14 | username-group.description= 15 | 16 | username.title=Username 17 | username.description= 18 | 19 | password.title=Password 20 | password.description= 21 | 22 | broker_url.title=Broker URL 23 | broker_url.description=Example: tcp://test-server.com:1883 (Protocol required. Port required)" 24 | 25 | topic.title=Topic 26 | topic.description=Example: test/topic 27 | 28 | -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.pulsar/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.pulsar/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters-iiot/src/main/resources/org.apache.streampipes.connect.iiot.protocol.stream.pulsar/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.iiot.protocol.stream.pulsar.title=Apache Pulsar 2 | org.apache.streampipes.connect.iiot.protocol.stream.pulsar.description=Consumes messages from an Apache Pulsar broker 3 | 4 | pulsar-broker-host.title=Broker Hostname 5 | pulsar-broker-host.description=Example: test.server.com (No protocol) 6 | 7 | pulsar-broker-port.title=Broker Port 8 | pulsar-broker-port.description=Example: 6650 9 | 10 | pulsar-topic.title=Topic 11 | pulsar-topic.description=Example: test.topic 12 | 13 | 14 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/java/org/apache/streampipes/connect/adapters/coindesk/Currency.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.streampipes.connect.adapters.coindesk; 20 | 21 | public enum Currency { 22 | EUR, 23 | GBP, 24 | USD; 25 | 26 | } 27 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.coindesk/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.coindesk/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.coindesk/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.coindesk.title=Coindesk Bitcoin Price 2 | org.apache.streampipes.connect.adapters.coindesk.description=The current bitcoin price from the Coindesk API. 3 | 4 | currency.title=Currency 5 | currency.description=The currency of the bitcoin rate 6 | 7 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.flic.mqtt/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.flic.mqtt/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.flic.mqtt/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.flic.mqtt.title= Flic MQTT 2 | org.apache.streampipes.connect.adapters.flic.mqtt.description= Connect a Flic Smart Button over MQTT 3 | 4 | access-mode.title=Access Mode 5 | access-mode.description= 6 | 7 | anonymous-alternative.title=Unauthenticated 8 | anonymous-alternative.description= 9 | 10 | username-alternative.title=Username/Password 11 | username-alternative.description= 12 | 13 | username-group.title=User Group 14 | username-group.description= 15 | 16 | username.title=Username 17 | username.description= 18 | 19 | password.title=Password 20 | password.description= 21 | 22 | broker_url.title=Broker URL 23 | broker_url.description=Example: tcp://test-server.com:1883 (Protocol required. Port required)" 24 | 25 | topic.title=Topic 26 | topic.description=Example: test/topic 27 | 28 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.gdelt/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.gdelt/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.gdelt/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.gdelt.title=GDELT 2 | org.apache.streampipes.connect.adapters.gdelt.description=Global Database of Society 3 | 4 | 5 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iex.news/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iex.news/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iex.news/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.iex.news.title=IEX Cloud News 2 | org.apache.streampipes.connect.adapters.iex.news.description=Fetches news for a given company (10 news / minutes maximum) 3 | 4 | token.title=API Token 5 | token.description=The IEXCloud API token 6 | 7 | stock-symbol.title=Stock Symbol 8 | stock-symbol.description=The stock symbol (e.g., AAPL) 9 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iex.stocks/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iex.stocks/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iex.stocks/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.iex.stocks.title=IEX Cloud Stock Quotes 2 | org.apache.streampipes.connect.adapters.iex.stocks.description=Live stock data provided by the IEXCloud API 3 | 4 | token.title=API Token 5 | token.description=The IEXCloud API token 6 | 7 | stock-symbol.title=Stock Symbol 8 | stock-symbol.description=The stock symbol (e.g., AAPL) 9 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.image.set/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.image.set/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.image.set/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.image.set.title=Image Upload (Set) 2 | org.apache.streampipes.connect.adapters.image.set.description= 3 | 4 | interval-key.title=Interval [ms] 5 | interval-key.description=Define waiting time between the images 6 | 7 | zip-file-key.title=Zipped Folder With Images 8 | zip-file-key.description=A zip file that contains the images to stream 9 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.image.stream/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.image.stream/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.image.stream/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.image.stream.title=Image Upload (Stream) 2 | org.apache.streampipes.connect.adapters.image.stream.description= 3 | 4 | interval-key.title=Interval [ms] 5 | interval-key.description=Define waiting time between the images 6 | 7 | zip-file-key.title=Zipped Folder With Images 8 | zip-file-key.description=A zip file that contains the images to stream 9 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iss/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iss/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.iss/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.iss.title=ISS Location 2 | org.apache.streampipes.connect.adapters.iss.description=Current Location of the International Space Station (ISS) 3 | 4 | polling-interval.title=Polling interval 5 | polling-interval.description=The update interval in seconds -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.simulator.randomdataset/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.simulator.randomdataset/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.simulator.randomdataset/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.simulator.randomdataset.title=Random Data Simulator (Set) 2 | org.apache.streampipes.connect.adapters.simulator.randomdataset.description=Publishes a bounded stream of random events 3 | 4 | 5 | wait-time-ms.title=Wait Time (MS) 6 | wait-time-ms.description=The time to wait between two events in milliseconds 7 | 8 | number-of-events.title=Number of Events 9 | number-of-events.description=The number of events to send -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.simulator.randomdatastream/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.simulator.randomdatastream/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.simulator.randomdatastream/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.simulator.randomdatastream.title=Random Data Simulator (Stream) 2 | org.apache.streampipes.connect.adapters.simulator.randomdatastream.description=Publishes a continuous stream of random events 3 | 4 | 5 | wait-time-ms.title=Wait Time (MS) 6 | wait-time-ms.description=The time to wait between two events in milliseconds" -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.slack/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.slack/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.slack/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.slack.title=Slack 2 | org.apache.streampipes.connect.adapters.slack.description=Subscribes to a Slack channel 3 | 4 | 5 | slack-token.title=Slack API Token 6 | slack-token.description=The API token of your Slack workspace -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.ti/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.ti/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.ti/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.ti.title=TI Sensor Tag 2 | org.apache.streampipes.connect.adapters.ti.mqttdescription= 3 | 4 | access-mode.title=Access Mode 5 | access-mode.description= 6 | 7 | anonymous-alternative.title=Unauthenticated 8 | anonymous-alternative.description= 9 | 10 | username-alternative.title=Username/Password 11 | username-alternative.description= 12 | 13 | username-group.title=User Group 14 | username-group.description= 15 | 16 | username.title=Username 17 | username.description= 18 | 19 | password.title=Password 20 | password.description= 21 | 22 | broker_url.title=Broker URL 23 | broker_url.description=Example: tcp://test-server.com:1883 (Protocol required. Port required)" 24 | 25 | topic.title=Topic 26 | topic.description=Example: test/topic 27 | 28 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.wikipedia.edit/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.wikipedia.edit/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.wikipedia.edit/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.wikipedia.edit.title=Wikipedia Edits 2 | org.apache.streampipes.connect.adapters.wikipedia.edit.description=Continuously publishes recent Wikipedia edits 3 | 4 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.wikipedia.new/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.wikipedia.new/icon.png -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.adapters.wikipedia.new/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.connect.adapters.wikipedia.new.title=Wikipedia New Articles 2 | org.apache.streampipes.connect.adapters.wikipedia.new.description=Continuously publishes articles created on Wikipedia 3 | 4 | -------------------------------------------------------------------------------- /streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.protocol.stream.httpserver/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-connect-adapters/src/main/resources/org.apache.streampipes.connect.protocol.stream.httpserver/icon.png -------------------------------------------------------------------------------- /streampipes-pipeline-elements-all-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=8005 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_FLINK_DEBUG=true -------------------------------------------------------------------------------- /streampipes-processors-aggregation-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6005 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_FLINK_DEBUG=true 21 | -------------------------------------------------------------------------------- /streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.aggregation/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.aggregation/icon.png -------------------------------------------------------------------------------- /streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.count/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.count/icon.png -------------------------------------------------------------------------------- /streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.count/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.aggregation.flink.count.title=Count Aggregation 2 | org.apache.streampipes.processors.aggregation.flink.count.description=Performs an aggregation based on a given event property and outputs the number of occurrences. 3 | 4 | count-mapping.title=Field to count 5 | count-mapping.description=The field that contains the values which should be counted 6 | 7 | time-window.title=Time Window Size 8 | time-window.description=Size of the time window 9 | 10 | scale.title=Time Window Scale 11 | scale.description= -------------------------------------------------------------------------------- /streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.eventcount/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.aggregation.flink.eventcount.title=Event Counter 2 | org.apache.streampipes.processors.aggregation.flink.eventcount.description=Counts the number of events within a time window. 3 | 4 | time-window.title=Time Window Size 5 | time-window.description=Size of the time window 6 | 7 | scale.title=Time Window Scale 8 | scale.description= -------------------------------------------------------------------------------- /streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.rate/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.rate/icon.png -------------------------------------------------------------------------------- /streampipes-processors-aggregation-flink/src/main/resources/org.apache.streampipes.processors.aggregation.flink.rate/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.aggregation.flink.rate.title=Event Rate 2 | org.apache.streampipes.processors.aggregation.flink.rate.description=Computes current event rate. Output is a number representing events per second. 3 | 4 | rate.title=Time Baseline 5 | rate.description=Time window size used for calculating the rate in seconds, also defines the output rate -------------------------------------------------------------------------------- /streampipes-processors-change-detection-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6018 18 | SP_HOST=localhost 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-processors-change-detection-jvm/src/main/resources/org.apache.streampipes.processors.changedetection.jvm.cusum/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.changedetection.jvm.cusum.title=Cusum 2 | org.apache.streampipes.processors.changedetection.jvm.cusum.description= 3 | 4 | number-mapping.title=Value to observe 5 | number-mapping.description=Specifies the monitored dimension. 6 | 7 | param-k.title=Parameter k 8 | param-k.description=The sensitivity parameter. High value indicates low sensitivity. Unit: Standard deviations 9 | 10 | param-h.title=Parameter h 11 | param-h.description=The threshold above which a change is detected. Unit: Standard deviations -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6010 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_FLINK_DEBUG=true -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.math.mathop/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.math.mathop/icon.png -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.math.mathop/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.flink.processor.math.mathop.title=Math 2 | org.apache.streampipes.processors.enricher.flink.processor.math.mathop.description=Performs calculations on event properties (+, -, *, /, %) 3 | 4 | leftOperand.title=Left operand 5 | leftOperand.description=Select left operand 6 | 7 | rightOperand.title=Right operand 8 | rightOperand.description=Select right operand 9 | 10 | operation.title=Select Operation 11 | operation.description= -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.math.staticmathop/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.math.staticmathop/icon.png -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.math.staticmathop/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.flink.processor.math.staticmathop.title=Static Math 2 | org.apache.streampipes.processors.enricher.flink.processor.math.staticmathop.description=Performs calculation on an event property with a static value (+, -, *, /, %) 3 | 4 | leftOperand.title=Left operand 5 | leftOperand.description=Select left operand 6 | 7 | rightOperandValue.title=Right operand value 8 | rightOperandValue.description=Specify the value of the right operand. 9 | 10 | operation.title=Select operation -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.trigonometry/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.trigonometry/icon.png -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.trigonometry/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.flink.processor.trigonometry.title=Trigonometry Functions 2 | org.apache.streampipes.processors.enricher.flink.processor.trigonometry.description=Performs Trigonometric functions on event properties 3 | 4 | operand.title=Alpha 5 | operand.description=Select the alpha parameter 6 | 7 | operation.title=Operation 8 | operation.description= 9 | 10 | -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.urldereferencing/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.urldereferencing/icon.png -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.processor.urldereferencing/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.flink.processor.urldereferencing.title=URL Dereferencing 2 | org.apache.streampipes.processors.enricher.flink.processor.urldereferencing.description=Append the html page as a string to event 3 | 4 | url.title=URL 5 | url.description=The server URL -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.timestamp/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.timestamp/icon.png -------------------------------------------------------------------------------- /streampipes-processors-enricher-flink/src/main/resources/org.apache.streampipes.processors.enricher.flink.timestamp/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.flink.timestamp.title=Timestamp Enricher 2 | org.apache.streampipes.processors.enricher.flink.timestamp.description=Appends the current time in ms to the event payload -------------------------------------------------------------------------------- /streampipes-processors-enricher-jvm/development/.env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6015 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-processors-enricher-jvm/src/main/resources/org.apache.streampipes.processors.enricher.jvm.jseval/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-enricher-jvm/src/main/resources/org.apache.streampipes.processors.enricher.jvm.jseval/icon.png -------------------------------------------------------------------------------- /streampipes-processors-enricher-jvm/src/main/resources/org.apache.streampipes.processors.enricher.jvm.jseval/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.jvm.jseval.title=JavaScript Eval 2 | org.apache.streampipes.processors.enricher.jvm.jseval.description=Pipeline element that allows writing user defined JavaScript function to enrich events. 3 | 4 | jsFunction.title=JavaScript Function 5 | jsFunction.description=Function to enrich event data. -------------------------------------------------------------------------------- /streampipes-processors-enricher-jvm/src/main/resources/org.apache.streampipes.processors.enricher.jvm.sizemeasure/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-enricher-jvm/src/main/resources/org.apache.streampipes.processors.enricher.jvm.sizemeasure/icon.png -------------------------------------------------------------------------------- /streampipes-processors-enricher-jvm/src/main/resources/org.apache.streampipes.processors.enricher.jvm.sizemeasure/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.jvm.sizemeasure.title=Size Measure 2 | org.apache.streampipes.processors.enricher.jvm.sizemeasure.description=Measures the size of an event in Bytes, KB or MB 3 | 4 | sizeUnit.title=Size unit 5 | sizeUnit.description=The unit in which the size of the event should be added -------------------------------------------------------------------------------- /streampipes-processors-enricher-jvm/src/main/resources/org.apache.streampipes.processors.enricher.jvm.valueChange/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.enricher.jvm.valueChange.title=Value Change 2 | org.apache.streampipes.processors.enricher.jvm.valueChange.description=The processing element should be able to detect when a numeric property change from one configured value to another 3 | 4 | change-value-mapping.title=Property to monitor 5 | change-value-mapping.description=The property where the values are monitored 6 | 7 | from-property-value.title=From Value 8 | from-property-value.description=Result is true, when the property value switches from this value to the To Value 9 | 10 | to-property-value.title=To Value 11 | to-property-value.description=Result is true, wWhen the property value switches from the from value to this 12 | 13 | is-changed.title=isChanged 14 | is-changed.description= -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6015 18 | #SP_HOST=localhost 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.compose/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.compose/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.compose/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.compose.title=Compose 2 | org.apache.streampipes.processors.filters.jvm.compose.description=Merges two event streams -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.enrich/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.enrich/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.enrich/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.enrich.title=Merge Two Streams 2 | org.apache.streampipes.processors.filters.jvm.enrich.description=One data stream is enriched by the values of the other 3 | 4 | select-stream.title=Select Output Frequency 5 | select-stream.description=The output frequency of the selected stream is maintained 6 | -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.limit/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.limit/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.merge/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.merge/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.merge/merge_description.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.merge/merge_description.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.merge/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.merge.title=Synchronize Two Streams 2 | org.apache.streampipes.processors.filters.jvm.merge.description=Event streams are merged when events occur at the same time 3 | 4 | timestamp_mapping_stream_1.title=Timestamp Field Stream 1 5 | timestamp_mapping_stream_1.description=The value which contains a timestamp 6 | 7 | timestamp_mapping_stream_2.title=Timestamp Field Stream 2 8 | timestamp_mapping_stream_2.description=The value which contains a timestamp 9 | 10 | time-interval.title=Time Interval [ms] 11 | time-interval.description=Time interval to match stream timestamps in milliseconds 12 | -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericalfilter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericalfilter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericalfilter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.numericalfilter.title=Numerical Filter 2 | org.apache.streampipes.processors.filters.jvm.numericalfilter.description=Numerical Filter Description 3 | 4 | number-mapping.title=Field 5 | number-mapping.description=Specifies the field name where the filter operation should be applied on. 6 | 7 | operation.title=Filter Operation 8 | operation.description=Specifies the filter operation that should be applied on the field 9 | 10 | value.title=Threshold value 11 | value.description=Specifies a threshold value -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.numericaltextfilter.title=Numerical Text Filter 2 | org.apache.streampipes.processors.filters.jvm.numericaltextfilter.description=Filters numerical and text values. 3 | 4 | number-mapping.title=Number Field 5 | number-mapping.description=Specifies the field name where the filter operation should be applied on. 6 | 7 | number-operation.title=Number Filter Operation 8 | number-operation.description=Specifies the filter operation that should be applied on the field 9 | 10 | number-value.title=Number Threshold 11 | number-value.description=Specifies a threshold value 12 | 13 | text-mapping.title=Text Field 14 | text.description= 15 | 16 | text-operation.title=Text Operation 17 | text-operation.description= 18 | 19 | text-keyword.title=Text keyword 20 | text-keyword.description= -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.processor.booleanfilter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.processor.booleanfilter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.processor.booleanfilter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.processor.booleanfilter.title=Boolean Filter 2 | org.apache.streampipes.processors.filters.jvm.processor.booleanfilter.description=Keeps events with the selected value 3 | 4 | boolean-mapping.title=Boolean Field 5 | boolean-mapping.description=Specifies the field name where the filter operation should be applied on. 6 | 7 | value.title=Value 8 | value.description=Events with the selected value are forwarded (E.g. when false, all events with value true are filtered out) 9 | 10 | -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.project/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.project/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.project/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.project.title=Projection 2 | org.apache.streampipes.processors.filters.jvm.project.description=Outputs a selectable subset of an input event type -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.schema/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.schema.title=Merge stream with same schema 2 | org.apache.streampipes.processors.filters.jvm.schema.description=Event streams are merged when their schemas are same 3 | 4 | -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.textfilter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.textfilter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.textfilter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.textfilter.title=Text Filter 2 | org.apache.streampipes.processors.filters.jvm.textfilter.description=Text Filter Description 3 | 4 | text.title=Select Text Field 5 | text.description= 6 | 7 | operation.title=Select Operation 8 | operation.description= 9 | 10 | keyword.title=Select Keyword 11 | keyword.description= -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.threshold/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.threshold/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.threshold/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.filters.jvm.threshold.title=Threshold Detector 2 | org.apache.streampipes.processors.filters.jvm.threshold.description=When threshold rule is fulfilled true is appended 3 | 4 | number-mapping.title=Field 5 | number-mapping.description=Specifies the field name where the filter operation should be applied on. 6 | 7 | operation.title=Filter Operation 8 | operation.description=Specifies the filter operation that should be applied on the field 9 | 10 | value.title=Threshold value 11 | value.description=Specifies a threshold value -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6020 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/java/org/apache/streampipes/processors/siddhi/trend/TrendOperator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.streampipes.processors.siddhi.trend; 20 | 21 | public enum TrendOperator { 22 | INCREASE, DECREASE 23 | } 24 | -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.count/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.count.title=Count Aggregation 2 | org.apache.streampipes.processors.siddhi.count.description=Performs an aggregation based on a given event property and outputs the number of occurrences. 3 | 4 | count-mapping.title=Field to count 5 | count-mapping.description=The field that contains the values which should be counted 6 | 7 | time-window.title=Time Window Size 8 | time-window.description=Size of the time window 9 | 10 | scale.title=Time Window Scale 11 | scale.description= 12 | -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.frequency/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.frequency.title=Frequency Calculation 2 | org.apache.streampipes.processors.siddhi.frequency.description=Calculates the frequency of the event stream 3 | 4 | duration.title=Time Window Length (Seconds) 5 | duration.description=Specifies the size of the time window in seconds. 6 | 7 | timeUnit.title=Time Unit 8 | timeUnit.description=Specifies a unit for the time window. -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.frequencychange/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.frequencychange.title=Frequency Change 2 | org.apache.streampipes.processors.siddhi.frequencychange.description=Detects when the frequency of the event stream changes 3 | 4 | duration.title=Time Window Length (Seconds) 5 | duration.description=Specifies the size of the time window in seconds. 6 | 7 | timeUnit.title=Time Unit 8 | timeUnit.description=Specifies a unit for the time window of the sequence. 9 | 10 | increase.title=Percentage of Increase/Decrease 11 | increase.description=Specifies the increase in percent (e.g., 100 indicates an increase by 100 percent within the specified time window. -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.increase/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.increase/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.increase/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.increase.title=Trend 2 | org.apache.streampipes.processors.siddhi.increase.description=Detects the increase of a numerical field over a customizable time window. Example: A temperature value increases by 10 percent within 5 minutes. 3 | 4 | mapping.title=Value to Observe 5 | mapping.description=Specifies the value that should be monitored. 6 | 7 | increase.title=Percentage of Increase/Decrease 8 | increase.description=Specifies the increase in percent (e.g., 100 indicates an increase by 100 percent within the specified time window. 9 | 10 | duration.title=Time Window Length (Seconds) 11 | duration.description=Specifies the size of the time window in seconds. 12 | 13 | operation.title=Increase/Decrease 14 | operation.description=Specifies the type of operation the processor should perform. -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.listcollector/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.listcollector.title=List Collector 2 | org.apache.streampipes.processors.siddhi.listcollector.description=Collects all values from a field within a specified batch window into a list. 3 | 4 | list-key.title=Field 5 | list-key.description=The field where values should be collected into a list. 6 | 7 | window-size.title=Batch Window Size 8 | window-size.description=The batch window size. 9 | -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.listfilter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.listfilter.title=List Filter 2 | org.apache.streampipes.processors.siddhi.listfilter.description=Filters a field from an input event that represents a list for the presence of a given value. 3 | 4 | list-key.title=Value to Observe 5 | list-key.description=Specifies the value that should be monitored. 6 | 7 | required-value.title=Required value 8 | required-value.description=The value (a primitive data type such as a number or text) that should be part of the list. 9 | -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.numericalfilter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.numericalfilter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.numericalfilter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.numericalfilter.title=Numerical Filter (Siddhi) 2 | org.apache.streampipes.processors.siddhi.numericalfilter.description=Numerical Filter based on CEP engine Siddhi 3 | 4 | number-mapping.title=Field to Filter 5 | number-mapping.description=Specifies the field name where the filter operation should be applied on. 6 | 7 | value.title=Threshold value 8 | value.description=Specifies a threshold value. 9 | 10 | operation.title=Filter Operation 11 | operation.description=Specifies the filter operation that should be applied on the field -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.sequence/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.sequence.title=Sequence Detection 2 | org.apache.streampipes.processors.siddhi.sequence.description=Merges events from two event streams, when the top event arrives first and then the bottom event 3 | 4 | duration.title=Time Window Length (Seconds) 5 | duration.description=Specifies the size of the time window in seconds. -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.stop/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.stop.title=Stream Stop Detection 2 | org.apache.streampipes.processors.siddhi.stop.description=Triggers an event when the input data stream stops sending events 3 | 4 | message.title=Message 5 | message.description=Message that stream stopped 6 | 7 | duration.title=Time Window Length (Seconds) 8 | duration.description=Specifies the size of the time window in seconds. -------------------------------------------------------------------------------- /streampipes-processors-filters-siddhi/src/main/resources/org.apache.streampipes.processors.siddhi.topk/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.siddhi.topk.title=Top k 2 | org.apache.streampipes.processors.siddhi.topk.description=Collects incoming events in a batch window of configurable size and outputs the top-k events of a given count value. 3 | 4 | list-key.title=Field 5 | list-key.description=The field where values should be collected into a list. 6 | 7 | window-size.title=Batch Window Size 8 | window-size.description=The batch window size. 9 | 10 | scale.title=Time Window Scale 11 | scale.description= 12 | 13 | limit.title=Limit 14 | limit.description=The maximum size of the list items that are forwarded 15 | 16 | order.title=Order 17 | order.description=The order direction (top-k or bottom-k) 18 | -------------------------------------------------------------------------------- /streampipes-processors-geo-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6025 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_FLINK_DEBUG=true 21 | -------------------------------------------------------------------------------- /streampipes-processors-geo-flink/src/main/resources/org.apache.streampipes.processor.geo.flink/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-geo-flink/src/main/resources/org.apache.streampipes.processor.geo.flink/icon.png -------------------------------------------------------------------------------- /streampipes-processors-geo-flink/src/main/resources/org.apache.streampipes.processor.geo.flink/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.geo.flink.title=Spatial Grid Enrichment 2 | org.apache.streampipes.processor.geo.flink.description=Groups spatial events into cells of a given size 3 | 4 | mapping-latitude.title=Latitude Field 5 | mapping-latitude.description= 6 | 7 | mapping-longitude.title=Longitude Field 8 | mapping-longitude.description= 9 | 10 | cellSize.title=Cell Size 11 | cellSize.description=The size of a cell in meters 12 | 13 | startingCell.title=Starting Location 14 | startingCell.description=The upper-left corner of the starting cell -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=8005 18 | SP_HOST=172.17.0.1 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processor.geo.jvm.geocoding/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.geo.jvm.geocoding.title=Google Maps Geocoder 2 | org.apache.streampipes.processor.geo.jvm.geocoding.description=Geocodes a placename to a latitude/longitude combination 3 | 4 | place-mapping.title=Place 5 | place-mapping.description=The field of the input event that should be used to compute the lat/lng values. -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processor.geo.jvm.reversegeocoding/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.geo.jvm.reversegeocoding.title=Reverse Geocoder 2 | org.apache.streampipes.processor.geo.jvm.reversegeocoding.description=Finds the nearest place based on latitude and longitude 3 | values. 4 | 5 | latitude-mapping.title=Latitude 6 | latitude-mapping.description=The field containing the latitude value. 7 | 8 | longitude-mapping.title=Longitude 9 | longitude-mapping.description=The field containing the longitude value. -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processor.geo.jvm.staticgeocoding/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.geo.jvm.staticgeocoding.title=Static Google Maps Geocoder 2 | org.apache.streampipes.processor.geo.jvm.staticgeocoding.description=Geocodes a fixed placename to lat/lng coordinates and 3 | appends these coordinates to every input event. 4 | 5 | place.title=Place 6 | place.description=The place name that should be converted to a lat/lng combination -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.latLngToGeo/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.latLngToGeo/icon.png -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.latLngToGeo/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.geo.jvm.jts.processor.latLngToGeo.title=Creates JTS Point 2 | org.apache.streampipes.processors.geo.jvm.jts.processor.latLngToGeo.description=Creates a JTS Point from Latitude and Longitude values 3 | 4 | latitude-key=Latitude value 5 | longitude-key=Longitude value 6 | epsg-key= EPSG Code 7 | -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.setEPSG/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.setEPSG/icon.png -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.setEPSG/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.geo.jvm.jts.processor.setEPSG.title=EPSG Code 2 | org.apache.streampipes.processors.geo.jvm.jts.processor.setEPSG.description=Adds a corresponding EPSG Code to the Geo-Event 3 | 4 | epsg-key.title=EPSG Code field 5 | epsg-key.description=EPSG Code 6 | -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.trajectory/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.trajectory/icon.png -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.jts.processor.trajectory/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.geo.jvm.jts.processor.trajectory.title=Single Trajectory Creator 2 | org.apache.streampipes.processors.geo.jvm.jts.processor.trajectory.description=Creates a trajectory from JTS point events 3 | 4 | point-key.title=JTS Point Event 5 | point-key.description=Single Point Event which will be added to the trajectory 6 | 7 | point-key.title=JTS Point Event 8 | point-key.description=Single Point Event which will be added to the trajectory 9 | 10 | epsg-key.title=CRS of Input Point 11 | epsg-key.description=EPSG-Code of input point 12 | 13 | m-key.title=measurement value 14 | m-key.description=Measurement value which will be stored with each point event 15 | 16 | description-key.title=description text of trajectory 17 | description-key.description=A description text for the trajectory 18 | 19 | subpoints-key.title=number of allowed sub-points 20 | subpoints-key.description=amount of allowed sub-points, creating the trajectory 21 | -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.processor.distancecalculator/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.geo.jvm.processor.distancecalculator.title=Distance Calculator 2 | org.apache.streampipes.processors.geo.jvm.processor.distancecalculator.description=Calculates the distance between two geo points 3 | 4 | lat1.title=First Latitude 5 | lat1.description= 6 | 7 | long1.title=First Longitude 8 | long1.description= 9 | 10 | lat2.title=Second Latitude 11 | lat2.description= 12 | 13 | long2.title=Second Longitude 14 | long2.description= 15 | 16 | calculatedDistance.title=Distance Field 17 | startingCell.description= -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.processor.speed/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.processor.speed/icon.png -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.processor.speed/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.geo.jvm.processor.speed.title=Speed Calculator 2 | org.apache.streampipes.processors.geo.jvm.processor.speed.description=Calculates the speed of two latlng pairs in kilometers per hour 3 | 4 | timestamp-key.title=Timestamp field 5 | timestamp-key.description=Used to calculate the time between events 6 | 7 | latitude-key.title=Latitude field 8 | latitude-key.description=Used to calculate the distance between events 9 | 10 | longitude-key.title=Longitude field 11 | longitude-key.description=Used to calculate the distance between events 12 | 13 | count-window-key.title=Count window 14 | count-window-key.description=Describes the number of stored events, used for the calculation -------------------------------------------------------------------------------- /streampipes-processors-geo-jvm/src/main/resources/org.apache.streampipes.processors.geo.jvm.processor.staticdistancecalculator/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.geo.jvm.processor.staticdistancecalculator.title=Static Distance Calculator 2 | org.apache.streampipes.processors.geo.jvm.processor.staticdistancecalculator.description=Calculates the distance between a fixed location and a moving location 3 | 4 | latitude-key.title=Latitude field 5 | latitude-key.description= 6 | 7 | longitude-key.title=Longitude field 8 | longitude-key.description= 9 | 10 | selected-latitude-key.title=Latitude 11 | selected-latitude-key.description=The latitude value of the fixed location 12 | 13 | selected-longitude-key.title=Longitude 14 | selected-longitude-key.description=The longitude value of the fixed location -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6035 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.generic-image-classification/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.generic-image-classification/icon.png -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.generic-image-classification/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.imageclassification.jvm.generic-image-classification.title=Generic Image Classification 2 | org.apache.streampipes.processor.imageclassification.jvm.generic-image-classification.description=Classification Description (Generic Model) 3 | 4 | image-mapping.title=Image field -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.image-cropper/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.image-cropper/icon.png -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.image-cropper/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.imageclassification.jvm.image-cropper.title=Image Cropper 2 | org.apache.streampipes.processor.imageclassification.jvm.image-cropper.description=Crops an image based on given bounding box coordinates 3 | 4 | image-property.title=Image Field 5 | image-property.description=The field containing the image. 6 | 7 | box-array-property.title=Bounding box field 8 | box-array-property.description=The field containing a list of bounding boxes. -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.image-enricher/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.image-enricher/icon.png -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.jvm.image-enricher/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.imageclassification.jvm.image-enricher.title=Image Enricher 2 | org.apache.streampipes.processor.imageclassification.jvm.image-enricher.description=Enriches an image with a bounding box of given coordinates 3 | 4 | image-property.title=Image Field 5 | image-property.description=The field containing the image. 6 | 7 | box-array-property.title=Bounding box field 8 | box-array-property.description=The field containing a list of bounding boxes. -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.qrcode/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.qrcode/icon.png -------------------------------------------------------------------------------- /streampipes-processors-image-processing-jvm/src/main/resources/org.apache.streampipes.processor.imageclassification.qrcode/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processor.imageclassification.qrcode.title=QR Code Reader 2 | org.apache.streampipes.processor.imageclassification.qrcode.description=Detects a QR Code in an image 3 | 4 | send-if-no-result.title=Send placeholder value if no qr code is detected 5 | send-if-no-result.description= 6 | 7 | placeholder-value.title=Placeholder value 8 | placeholder-value.description= 9 | 10 | qr-value.title=QR code value 11 | qr-value.description= 12 | 13 | image-property.title=Image 14 | image-property.description= -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6040 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_FLINK_DEBUG=true 21 | -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/java/org/apache/streampipes/processors/pattern/detection/flink/processor/absence/Absence.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | package org.apache.streampipes.processors.pattern.detection.flink.processor.absence; 19 | 20 | public class Absence { 21 | } 22 | -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/java/org/apache/streampipes/processors/pattern/detection/flink/processor/and/And.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | package org.apache.streampipes.processors.pattern.detection.flink.processor.and; 19 | 20 | public class And { 21 | } 22 | -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.absence/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.pattern-detection.flink.absence.title=Absence 2 | org.apache.streampipes.processors.pattern-detection.flink.absence.description=Detects whether an event does not arrive within a specified time after the occurrence of another event. 3 | 4 | time-window.title=Time Window 5 | time-window.description=Time window size (seconds) 6 | 7 | time-unit.title=Time Unit 8 | time-unit.description=The time unit used for detecting the co-occurrence. -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.and/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.and/icon.png -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.and/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.pattern-detection.flink.and.title=And 2 | org.apache.streampipes.processors.pattern-detection.flink.and.description=Detects whether an event co-occurs with another event within a given time. 3 | 4 | time-window.title=Time Window 5 | time-window.description=Time window size (seconds) 6 | 7 | time-unit.title=Time Unit 8 | time-unit.description=The time unit used for detecting the co-occurrence. 9 | 10 | left-mapping.title=Left Mapping 11 | left-mapping.description= 12 | 13 | right-mapping.title=Right Mapping 14 | right-mapping.description= -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.peak-detection/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.peak-detection/icon.png -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.peak-detection/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.pattern-detection.flink.peak-detection.title=Peak Detection 2 | org.apache.streampipes.processors.pattern-detection.flink.peak-detection.description=Detect peaks in time series data 3 | 4 | value-to-observe.title=Value to Observe 5 | value-to-observe.description=Provide a value where statistics are calculated upon 6 | 7 | partition-by.title=Group by 8 | partition-by.description=Partition the stream by a given id 9 | 10 | timestamp-mapping.title=Time 11 | timestam-mapping.description=Provide a time parameter 12 | 13 | sp-lag.title=Lag 14 | sp-lag.description=Defines the lag of the smoothing function 15 | 16 | sp-threshold.title=Threshold 17 | sp-threshold.description=Defines the standard deviation threshold 18 | 19 | sp-count-window.title=Count Window Size 20 | sp-count-window.description=Defines the size of the count window 21 | 22 | sp-influence.title=Influence 23 | sp-influence.description=Defines the influence -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.sequence/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.sequence/icon.png -------------------------------------------------------------------------------- /streampipes-processors-pattern-detection-flink/src/main/resources/org.apache.streampipes.processors.pattern-detection.flink.sequence/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.pattern-detection.flink.sequence.title=Sequence 2 | org.apache.streampipes.processors.pattern-detection.flink.sequence.description=Detects a sequence of events. 3 | 4 | timeWindow.title=Time Window Size 5 | timeWindow.description=Size of the time window 6 | 7 | timeUnit.title=Time Unit 8 | timeUnit.description=Specifies a unit for the time window of the sequence. -------------------------------------------------------------------------------- /streampipes-processors-statistics-flink/development/.env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6045 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_FLINK_DEBUG=true 21 | -------------------------------------------------------------------------------- /streampipes-processors-statistics-flink/src/main/resources/org.apache.streampipes.processors.statistics.flink.statistics-summary-window/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-statistics-flink/src/main/resources/org.apache.streampipes.processors.statistics.flink.statistics-summary-window/icon.png -------------------------------------------------------------------------------- /streampipes-processors-statistics-flink/src/main/resources/org.apache.streampipes.processors.statistics.flink.statistics-summary-window/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.statistics.flink.statistics-summary-window.title=Sliding Descriptive Statistics 2 | org.apache.streampipes.processors.statistics.flink.statistics-summary-window.description=Calculate simple descriptive summary statistics based on a configurable time window 3 | 4 | value-to-observe.title=Value to Observe 5 | value-to-observe.description=Provide a value where statistics are calculated upon 6 | 7 | partition-by.title=Group by 8 | partition-by.description=Partition the stream by a given id 9 | 10 | timestamp-mapping.title=Time Field 11 | timestamp-mapping.description=Provide a time parameter 12 | 13 | time-window.title=Time Window Size 14 | time-window.description=Size of the time window 15 | 16 | time-scale.title=Time Window Scale 17 | time-scale.description= 18 | 19 | -------------------------------------------------------------------------------- /streampipes-processors-statistics-flink/src/main/resources/org.apache.streampipes.processors.statistics.flink.statistics-summary/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-statistics-flink/src/main/resources/org.apache.streampipes.processors.statistics.flink.statistics-summary/icon.png -------------------------------------------------------------------------------- /streampipes-processors-statistics-flink/src/main/resources/org.apache.streampipes.processors.statistics.flink.statistics-summary/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.statistics.flink.statistics-summary.title=Statistics Summary 2 | org.apache.streampipes.processors.statistics.flink.statistics-summary.description=Calculate simple descriptive summary statistics 3 | 4 | list-property.title=Field Name 5 | list-property.description=Select a list property -------------------------------------------------------------------------------- /streampipes-processors-text-mining-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6050 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_FLINK_DEBUG=true 21 | -------------------------------------------------------------------------------- /streampipes-processors-text-mining-flink/src/main/resources/en-ner-location.bin: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-flink/src/main/resources/en-ner-location.bin -------------------------------------------------------------------------------- /streampipes-processors-text-mining-flink/src/main/resources/en-ner-organization.bin: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-flink/src/main/resources/en-ner-organization.bin -------------------------------------------------------------------------------- /streampipes-processors-text-mining-flink/src/main/resources/en-ner-person.bin: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-flink/src/main/resources/en-ner-person.bin -------------------------------------------------------------------------------- /streampipes-processors-text-mining-flink/src/main/resources/org.apache.streampipes.processors.textmining.flink.languagedetection/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.flink.languagedetection.title=Language Detection 2 | org.apache.streampipes.processors.textmining.flink.languagedetection.description=Detects the language of a written text. 3 | 4 | detectionField.title=Field Selection 5 | detectionField.description=Specifies the field where the language detection should be applied upon. 6 | 7 | 8 | -------------------------------------------------------------------------------- /streampipes-processors-text-mining-flink/src/main/resources/org.apache.streampipes.processors.textmining.flink.wordcount/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-flink/src/main/resources/org.apache.streampipes.processors.textmining.flink.wordcount/icon.png -------------------------------------------------------------------------------- /streampipes-processors-text-mining-flink/src/main/resources/org.apache.streampipes.processors.textmining.flink.wordcount/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.flink.wordcount.title=Word Count 2 | org.apache.streampipes.processors.textmining.flink.wordcount.description=Counts words on continuous text-based streams 3 | 4 | wordcountField.title=Property Selection 5 | wordcountField.description=Specifies the event property from your stream that should be counted. 6 | 7 | timeWindow.title=Time Window Size 8 | timeWindow.description=Size of the time window in seconds 9 | 10 | 11 | -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6065 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.chunker/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.chunker/icon.png -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.chunker/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.jvm.chunker.title=Chunker (English) 2 | org.apache.streampipes.processors.textmining.jvm.chunker.description=Segments given tokens of a text into different chunks 3 | 4 | tagsField.title=Part of Speech tags 5 | tagsField.description=The tags which mark every token with a part of speech tag 6 | 7 | tokensField.title=Tokens 8 | tokensField.description=The tokens of the text 9 | 10 | confidence.title=Confidence Chunker 11 | confidence.description=The confidence with which each Chunk is tagged 12 | 13 | chunk.title=Chunk 14 | chunk.description=The chunk each token belongs to 15 | 16 | binary-file.title=Model File 17 | binary-file.description=Provide a model from the OpenNLP project. You can find the link in the documentation of this processor. -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.languagedetection/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.languagedetection/icon.png -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.languagedetection/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.jvm.languagedetection.title=Language Detection 2 | org.apache.streampipes.processors.textmining.jvm.languagedetection.description=Detects the language of a given text 3 | 4 | detectionField.title=Text detection 5 | detectionField.description=Specifies the event property from your stream that contains the text in an unknown language. 6 | 7 | language.title=Language 8 | language.description=The acronym of the detected language 9 | 10 | confidence.title=Confidence 11 | confidence.description=The probability that the detected language is correct. Between 0 (no confidence) and 1 (highly confident). 12 | 13 | binary-file.title=Model File 14 | binary-file.description=Provide a model from the OpenNLP project. You can find the link in the documentation of this processor. -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.namefinder/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.namefinder/icon.png -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.namefinder/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.jvm.namefinder.title=Name Finder 2 | org.apache.streampipes.processors.textmining.jvm.namefinder.description=Finds a name in given Tokens of one document 3 | 4 | model.title=Model 5 | model.description=Select the model which should be used 6 | 7 | tokensField.title=List of Tokens 8 | tokensField.description=List of the tokens of the document 9 | 10 | foundNames.title=Found Names 11 | foundNames.description=The found names in the tokens 12 | 13 | -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.partofspeech/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.partofspeech/icon.png -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.partofspeech/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.jvm.partofspeech.title=Part of Speech (English) 2 | org.apache.streampipes.processors.textmining.jvm.partofspeech.description=Takes in a stream of tokens and appends a tag at the end of the word corresponding to the part of speech 3 | 4 | detectionField.title=Tokens 5 | detectionField.description=Specifies the event property from your stream that contains the tokens 6 | 7 | tagPos.title=Part of Speech Tag 8 | tagPos.description=The part of speech tag which was detected 9 | 10 | confidencePos.title=Confidence 11 | confidencePos.description=The probability that the detected part of speech is correct. Between 0 (no confidence) and 1 (highly confident). 12 | 13 | binary-file.title=Model File 14 | binary-file.description=Provide a model from the OpenNLP project. You can find the link in the documentation of this processor. 15 | -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.sentencedetection/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.sentencedetection/icon.png -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.sentencedetection/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.jvm.sentencedetection.title=Sentence Detection (English) 2 | org.apache.streampipes.processors.textmining.jvm.sentencedetection.description=Splits a text up in sentences 3 | 4 | detectionField.title=Sentence detection 5 | detectionField.description=Specifies the event property from your stream that contains the text that should be split up 6 | 7 | binary-file.title=Model File 8 | binary-file.description=Provide a model from the OpenNLP project. You can find the link in the documentation of this processor. 9 | -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.tokenizer/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.tokenizer/icon.png -------------------------------------------------------------------------------- /streampipes-processors-text-mining-jvm/src/main/resources/org.apache.streampipes.processors.textmining.jvm.tokenizer/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.textmining.jvm.tokenizer.title=Tokenizer (English) 2 | org.apache.streampipes.processors.textmining.jvm.tokenizer.description=Segments a text into different tokens 3 | 4 | detectionField.title=Text 5 | detectionField.description=Specifies the event property of your stream that contains the text to tokenize 6 | 7 | tokenList.title=List of Tokens 8 | tokenList.description=List of all tokens in the text 9 | 10 | binary-file.title=Model File 11 | binary-file.description=Provide a model from the OpenNLP project. You can find the link in the documentation of this processor. 12 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=6055 18 | SP_HOST=localhost 19 | SP_FLINK_DEBUG=true 20 | SP_DEBUG=true 21 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/document/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * The Boilerpipe document model. 3 | */ 4 | package com.kohlschutter.boilerpipe.document; 5 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/extractors/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Some standard extractors (i.e., completely piped BoilerpipeFilters) 3 | */ 4 | package com.kohlschutter.boilerpipe.extractors; 5 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/filters/english/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * These BoilerpipeFilters have only been tested on English text. 3 | * 4 | * That is, they will probably work with other Western languages, but maybe need some parameter tuning to perform well. 5 | */ 6 | package com.kohlschutter.boilerpipe.filters.english; 7 | 8 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/filters/heuristics/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * These BoilerpipeFilters are pure heuristics. 3 | */ 4 | package com.kohlschutter.boilerpipe.filters.heuristics; 5 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/filters/simple/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * These BoilerpipeFilters are straight-forward and probably not really specific to English. 3 | */ 4 | package com.kohlschutter.boilerpipe.filters.simple; 5 | 6 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * The Boilerpipe top-level package. 3 | */ 4 | package com.kohlschutter.boilerpipe; 5 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/sax/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Classes related to parsing and producing HTML from/to Boilerpipe TextDocuments. 3 | */ 4 | package com.kohlschutter.boilerpipe.sax; 5 | 6 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/java/com/kohlschutter/boilerpipe/util/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Some helper classes. 3 | */ 4 | package com.kohlschutter.boilerpipe.util; 5 | 6 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-converter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-converter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-converter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.flink.field-converter.title=Field Converter 2 | org.apache.streampipes.processors.transformation.flink.field-converter.description=Converts a string value to a number data type 3 | 4 | convert-property.title=Field 5 | convert-property.description=The field to convert 6 | 7 | target-type.title=Datatype 8 | target-type.description=The target datatype -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-mapper/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-mapper/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-mapper/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.flink.field-mapper.title=Field Mapper 2 | org.apache.streampipes.processors.transformation.flink.field-mapper.description=Replaces one or more field with a new field and computes a hash value of these fields 3 | 4 | replaceProperties.title=Fields 5 | replaceProperties.description=The fields to replace 6 | 7 | fieldName.title=New Field Name 8 | fieldName.description=The name of the new field -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-renamer/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-renamer/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.field-renamer/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.flink.field-renamer.title=Field Renamer 2 | org.apache.streampipes.processors.transformation.flink.field-renamer.description=Replaces the runtime name of an event property with a custom defined name. 3 | 4 | convert-property.title=Field 5 | convert-property.description=The field to rename 6 | 7 | field-name.title=The new field name 8 | field-name.description= -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.fieldhasher/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.fieldhasher/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.fieldhasher/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.flink.fieldhasher.title=Field Hasher 2 | org.apache.streampipes.processors.transformation.flink.fieldhasher.description=Hashes the value of a field using various hash functions. 3 | 4 | property-mapping.title=Field 5 | property-mapping.description=The field the hash function should be applied on 6 | 7 | hash-algorithm.title=Hash Algorithm 8 | hash-algorithm.description=The hash algorithm that should be used. -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.measurement-unit-converter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.measurement-unit-converter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.measurement-unit-converter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.flink.measurement-unit-converter.title=Measurement Unit Converter 2 | org.apache.streampipes.processors.transformation.flink.measurement-unit-converter.description=Converts a unit of measurement to another one 3 | 4 | convert-property.title=Field 5 | convert-property.description=The field to convert 6 | 7 | output-unit.title=Output Unit 8 | output-unit.description=The output type unit of measurement -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.processor.boilerplate/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.processor.boilerplate/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-flink/src/main/resources/org.apache.streampipes.processors.transformation.flink.processor.boilerplate/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.flink.processor.boilerplate.title=Boilerplate Removal 2 | org.apache.streampipes.processors.transformation.flink.processor.boilerplate.description=Removes boilerplate and extract fulltext from HTML 3 | 4 | stringProperty.title=HTML Field 5 | stringProperty.description=The field containing the HTML 6 | 7 | extractor.title=Extractor 8 | extractor.description=Common use: Article Extractor 9 | 10 | outputMode.title=Output Mode 11 | outputMode.description= -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | SP_PORT=6060 17 | SP_HOST=localhost 18 | SP_DEBUG=true 19 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.counter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.counter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.counter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.booloperator.counter.title=Boolean Counter 2 | org.apache.streampipes.processors.transformation.jvm.booloperator.counter.description=Increases a counter on each change of a boolean value 3 | 4 | field.title=Boolean Field 5 | field.description=The field of the boolean to monitor 6 | 7 | flank.title=Flank Parameter 8 | flank.description=The change in a sequence which triggers counting 9 | 10 | countField.title=Counter 11 | countField.description=Amount of changes of boolean value 12 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.inverter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.inverter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.inverter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.booloperator.inverter.title=Boolean Inverter 2 | org.apache.streampipes.processors.transformation.jvm.booloperator.inverter.description=Inverts the boolean value of the selected field 3 | 4 | invert-field.title=Invert field 5 | invert-field.description=The field which should be inverted -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.logical/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.booloperator.logical.title=Boolean Logical Operator 2 | org.apache.streampipes.processors.transformation.jvm.booloperator.logical.description=Performs a logical boolean operation b/w selected fields 3 | 4 | operator-field.title=Boolean Operator 5 | operator-field.description=The boolean operation that needs to be performed 6 | 7 | properties-field.title=Properties 8 | properties-field.description=List of properties(operands) on which operation needs to be performed 9 | 10 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping.title=Measure Time Between Two Sensors 2 | org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping.description=Requires two booleans and measures the time between two signals 3 | 4 | left-field.title=Left Field 5 | left-field.description=The boolean that starts the timer 6 | 7 | right-field.title=Right Field 8 | right-field.description=The boolean that triggers the event with the result of the timer 9 | 10 | time-field.title=Timer Field 11 | time-field.description=Time between the signal of the left and the right field 12 | 13 | count-field.title=Counter 14 | count-field.description=Amount of signals that where first on the left and then on the right sensor 15 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping/time_measure_example.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timekeeping/time_measure_example.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timer/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timer/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.booloperator.timer/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.booloperator.timer.title=Boolean Timer 2 | org.apache.streampipes.processors.transformation.jvm.booloperator.timer.description=Measures how long a boolean measure does not change 3 | 4 | field.title=Boolean Field 5 | field.description=The boolean field that is monitored 6 | 7 | timerField.title=Value to observe 8 | timerField.description=Define whether the time for true or false should be measured 9 | 10 | 11 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.changed-value/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.changed-value.title=Value Changed 2 | org.apache.streampipes.processors.transformation.jvm.changed-value.description=Outpus an event every time a specific field changes. 3 | 4 | compare.title=Keep Fields 5 | compare.description=The field which might change over time -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.count-array/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.count-array/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.count-array/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.count-array.title=Count Array 2 | org.apache.streampipes.processors.transformation.jvm.count-array.description=Counts the size of list items. 3 | 4 | array-field.title=List Field 5 | array-field.description=Contains a list of fields. -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.csvmetadata/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.csvmetadata/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.csvmetadata/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.csvmetadata.title=CSV Metadata Enricher 2 | org.apache.streampipes.processors.transformation.jvm.csvmetadata.description=Enriches an event with additional fields from a 3 | CSV file based on a common ID. 4 | 5 | mapping-field.title=ID Field 6 | mapping-field.description=The id field used to lookup the metadata 7 | 8 | csv-file.title=CSV File 9 | csv-file.description=The CSV file to upload 10 | 11 | fields-to-append.title=Fields to append 12 | fields-to-append.description=The fields from the CSV file that should be appended to the event. 13 | 14 | field-to-match.title=Field to match 15 | field-to-match.description=The field to match with the value of the selected ID field, used to lookup metadata 16 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.duration-value/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.duration-value/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.duration-value/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.duration-value.title=Calculate Duration 2 | org.apache.streampipes.processors.transformation.jvm.duration-value.description=Calculates the time between two timestamps. 3 | 4 | start-ts.title=Start Timestamp 5 | start-ts.description=The timestamp of the start event 6 | 7 | end-ts.title=End Timestamp 8 | end-ts.description=The timestamp of the end event 9 | 10 | unit-field.title=Time Unit 11 | unit-field.description= 12 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldrename/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.fieldrename.title=Field Renamer 2 | org.apache.streampipes.processors.transformation.jvm.fieldrename.description=Replaces the runtime name of an event property with a custom defined name. 3 | 4 | 5 | convert-property.title=Field 6 | convert-property.description=The field to rename 7 | 8 | field-name.title=The new field name 9 | field-name.description=The new field name 10 | 11 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.edge/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.edge/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.edge/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.processor.booloperator.edge.title=Signal Edge Filter 2 | org.apache.streampipes.processors.transformation.jvm.processor.booloperator.edge.description=Forwards the event when a signal edge is detected 3 | 4 | boolean_state_field.title=Boolean Signal 5 | boolean_state_field.description=The signal to observe and detect the signal edge 6 | 7 | flank.title=Signal Edge 8 | flank.description=What kind of signal edge should be detected 9 | 10 | delay.title=Delay 11 | delay.description=How many events should be waited to forward the result? 12 | 13 | event-selection-id.title=Output Event Selection 14 | event-selection-id.description=This specifies the event(s) that are selected to be emitted. 15 | 16 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state.title=Boolean To State 2 | org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state.description=Transforms boolean properties into a string state property 3 | 4 | state-field.title=State field 5 | state-field.description=Test 6 | 7 | collection-id.title=Collection Field 8 | collection-id.description=Test 9 | 10 | default-state-id.title=Default State 11 | default-state-id.description=Define a default state, when all values are false 12 | 13 | boolean_state_field.title=Current State 14 | boolean_state_field.description=When the boolean value is true the runtime name is used for the final state value 15 | 16 | json-configuration.title=Mapping Configuration 17 | json-configuration.description=Define mappings to replace the runtime names with own Strings 18 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.buffer/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.buffer/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.buffer/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.processor.state.buffer.title=State Buffer 2 | org.apache.streampipes.processors.transformation.jvm.processor.state.buffer.description=Buffers a sensor values during a state 3 | 4 | timestampId.title=Timestamp 5 | timestampId.description=Field with the timestamp of event 6 | 7 | stateId.title=State 8 | stateId.description=Field containing the state 9 | 10 | sensorValueFieldId.title=Sensor Value to Cache 11 | sensorValueFieldId.description=Select the sensor value that should be cached, while the state is active 12 | 13 | values.title=values 14 | values.description= 15 | 16 | state.title=state 17 | state.description= 18 | 19 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.buffer/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.buffer/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state.title=String To State 2 | org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state.description=Transforms string properties into a string state property 3 | 4 | state-field.title=State field 5 | state-field.description=Test 6 | 7 | collection-id.title=Collection Field 8 | collection-id.description=Test 9 | 10 | string_state.title=Current State 11 | string_state_field.description= 12 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.timestampextractor/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.processor.timestampextractor.title=Timestamp Extractor 2 | org.apache.streampipes.processors.transformation.jvm.processor.timestampextractor.description=Extracts a timestamp into its individual time fields. 3 | 4 | timestampField.title=Timestamp Field 5 | timestampField.description= 6 | 7 | selectedOutputFields.title=Extract Fields 8 | selectedOutputFields.description= -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.split-array/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.split-array/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.split-array/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.split-array.title=Split Array 2 | org.apache.streampipes.processors.transformation.jvm.split-array.description=Outputs an event for each entry of a list. 3 | 4 | keep.title=Keep fields 5 | keep.description=The fields that should be added to the events of array 6 | 7 | array-field.title=List Field 8 | array-field.description=Contains an array with events -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.stringoperator.counter/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.stringoperator.counter/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.stringoperator.counter/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.stringoperator.counter.title=String Counter 2 | org.apache.streampipes.processors.transformation.jvm.stringoperator.counter.description= Increases a counter on each change of a string value. 3 | 4 | field.title=String Field 5 | field.description=The field of the string to monitor 6 | 7 | countField.title=Counter 8 | countField.description= Number of changes of the given pair 9 | 10 | changeFromField.title = Change From Value 11 | changeFromField.description= Field value before the change 12 | 13 | changeFromField.title = Change To Value 14 | changeFromField.description= Field value after the change 15 | 16 | 17 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.stringoperator.timer/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.stringoperator.timer/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.stringoperator.timer/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.stringoperator.timer.title=String Timer 2 | org.apache.streampipes.processors.transformation.jvm.stringoperator.timer.description=Measures how long a string value does not change 3 | 4 | field.title=String Field 5 | field.description=The string field that is monitored 6 | 7 | measuredTime.title = Measured Time 8 | measuredTime.description = The time for the va 9 | 10 | fieldValue.title = Field Value 11 | fieldValue.description = The corresponding field value 12 | 13 | outputUnit.title= Output Unit 14 | outputUnit.description = The output unit of the measured time 15 | 16 | outputFrequency.title = Output Frequency 17 | outputFrequency.description = Define when the processor should emmit an output event. 18 | 19 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.taskduration/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.taskduration/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.taskduration/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.taskduration.title=Task Duration 2 | org.apache.streampipes.processors.transformation.jvm.taskduration.description=Computes the time difference between two events 3 | that contain fields indicating a current process step, i.e., id computes the time required for a state change. 4 | 5 | task-field.title=Task Field 6 | task-field.description=The field containing a description of the current process step. 7 | 8 | timestamp-field.title=Timestamp Field 9 | timestamp-field.description=The field containing a timestamp of the event. 10 | 11 | process-id.title=Process Id 12 | process-id.description=The ID of the state change detected. 13 | 14 | duration-id.title=Duration 15 | duration-id.description=The duration of the state change. 16 | 17 | 18 | -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.transform-to-boolean/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.transform-to-boolean/icon.png -------------------------------------------------------------------------------- /streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.transform-to-boolean/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.processors.transformation.jvm.transform-to-boolean.title=Transform to boolean 2 | org.apache.streampipes.processors.transformation.jvm.transform-to-boolean.description=Transforms a string or number to a boolean 3 | 4 | transform-fields.title=Transform fields 5 | transform-fields.description=The fields which should be transformed -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=7005 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.bufferrest/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.brokers.jvm.bufferrest.title=Buffered REST Publisher 2 | org.apache.streampipes.sinks.brokers.jvm.bufferrest.description=Once a given amount of events is reached, events are posted to the given REST interface. 3 | 4 | bufferrest.uri.title=REST URL 5 | bufferrest.uri.description=URL of the REST endpoint 6 | 7 | bufferrest.count.title=Buffer Size 8 | bufferrest.count=The amount of events to buffer before sending them on 9 | -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.jms/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.jms/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.jms/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.brokers.jvm.jms.title=JMS Publisher 2 | org.apache.streampipes.sinks.brokers.jvm.jms.description=Publishes events to a JMS topic 3 | 4 | topic.title=JMS Topic 5 | topic.description=Select a JMS topic 6 | 7 | broker-settings.title=JMS broker settings (use prefix tcp://) 8 | broker-settings.description=Provide settings of the JMS broker to connect with. 9 | 10 | host.title=JMS Endpoint 11 | host.description=IP address or hostname of JMS endpoint. (use prefix tcp://) 12 | 13 | port.title=Port 14 | port.description=Port of the JMS endpoint 15 | 16 | -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.kafka/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.kafka/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.mqtt/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.mqtt/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.nats/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.nats/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.pulsar/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.pulsar/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.pulsar/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.brokers.jvm.pulsar.title=Pulsar Publisher 2 | org.apache.streampipes.sinks.brokers.jvm.pulsar.description=Publish events to Apache Pulsar 3 | 4 | topic.title=Topic 5 | topic.description=Select a Pulsar topic 6 | 7 | pulsar-host.title=Pulsar Hostname 8 | pulsar-host.description=The hostname of the Pulsar broker 9 | 10 | pulsar-port.title=Pulsar Port 11 | pulsar-port.description=The port of the Pulsar broker (e.g., 6650) -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.rabbitmq/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.rabbitmq/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.rabbitmq/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.brokers.jvm.rabbitmq.title=RabbitMQ Publisher 2 | org.apache.streampipes.sinks.brokers.jvm.rabbitmq.description=Forwards events to a RabbitMQ broker 3 | 4 | topic.title=RabbitMQ Topic 5 | topic.description=Select a RabbitMQ topic 6 | 7 | broker-settings.title=RabbitMQ broker settings 8 | broker-settings.description=Provide settings of the RabbitMQ broker to connect with. 9 | 10 | host.title=Host 11 | host.description=Host or IP of the broker 12 | 13 | port.title=Port 14 | port.description=Port of the broker (e.g. 5672) 15 | 16 | user.title=User 17 | user.description=User to log in to the broker 18 | 19 | password.title=Password 20 | password.description=Password of the user 21 | 22 | exchange-name.title=Exchange Name 23 | exchange-name.description=Leave empty for default exchange 24 | 25 | -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.rest/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.brokers.jvm.rest.title=REST Publisher 2 | org.apache.streampipes.sinks.brokers.jvm.rest.description=Posts events to a REST interface 3 | 4 | url-key.title=REST URL 5 | url-key.description=URL of the REST endoint -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.websocket/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.websocket/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-brokers-jvm/src/main/resources/org.apache.streampipes.sinks.brokers.jvm.websocket/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.brokers.jvm.websocket.title=Websocket Server 2 | org.apache.streampipes.sinks.brokers.jvm.websocket.description=Sends events to a connected client 3 | 4 | port.title=Port 5 | port.description=Port of the websocket server. -------------------------------------------------------------------------------- /streampipes-sinks-databases-flink/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | # Those parameters are used by IntelliJ to set the default consul parameters for development 16 | SP_PORT=7010 17 | SP_HOST=host.docker.internal 18 | SP_DEBUG=true 19 | SP_FLINK_DEBUG=true 20 | SP_ELASTICSEARCH_HOST=localhost 21 | -------------------------------------------------------------------------------- /streampipes-sinks-databases-flink/src/main/resources/org.apache.streampipes.sinks.databases.flink.elasticsearch/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-flink/src/main/resources/org.apache.streampipes.sinks.databases.flink.elasticsearch/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-flink/src/main/resources/org.apache.streampipes.sinks.databases.flink.elasticsearch/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.databases.flink.elasticsearch.title=Elasticsearch 2 | org.apache.streampipes.sinks.databases.flink.elasticsearch.description=Stores data in an elasticsearch cluster 3 | 4 | timestamp-mapping.title=Timestamp Field 5 | timestamp-mapping.description=The field containing the timestamp for an Elasticsearch index 6 | 7 | index-name.title=Index Name 8 | index-name.description=Elasticsearch index name -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=7015 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.ditto/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.ditto/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.ditto/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.databases.ditto.title=Eclipse Ditto 2 | org.apache.streampipes.sinks.databases.ditto.description=Publishes events to Eclipse Ditto. 3 | 4 | dittoApiEndpointKey.title=Ditto API endpoint 5 | dittoApiEndpointKey.description=The endpoint URL of the Ditto instance 6 | 7 | dittoUserKey.title=Username 8 | dittoUserKey.description=The Ditto username 9 | 10 | dittoPasswordKey.title=Password 11 | dittoPasswordKey.description=The Ditto password 12 | 13 | dittoThingIdKey.title=Thing ID 14 | dittoThingIdKey.description=The Ditto thing ID 15 | 16 | dittoFeatureIdKey.title=Feature ID 17 | dittoFeatureIdKey.description=The Ditto feature ID 18 | 19 | selectedFieldsKey.title=Fields to send 20 | selectedFieldsKey.description=The fields that should be stored as a property 21 | -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.couchdb/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.couchdb/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.couchdb/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.databases.jvm.couchdb.title=CouchDB 2 | org.apache.streampipes.sinks.databases.jvm.couchdb.description=Stores events in an Apache CouchDB database. 3 | 4 | db_host.title=Hostname 5 | db_host.description=The hostname of the CouchDB instance 6 | 7 | db_port.title=Port 8 | db_port.description=The port of the CouchDB instance 9 | 10 | db_name.title=Database Name 11 | db_name.description=The name of the database where events will be stored -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.influxdb/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.influxdb/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.iotdb/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.iotdb/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.iotdb/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.databases.jvm.iotdb.title=IoTDB 2 | org.apache.streampipes.sinks.databases.jvm.iotdb.description=Stores events in a IoTDB database. 3 | 4 | db_host.title=Hostname 5 | db_host.description=The hostname of the IoTDB instance 6 | 7 | db_port.title=Port 8 | db_port.description=The port of the IoTDB instance (default 6667) 9 | 10 | db_storage_group.title=Storage Group Name 11 | db_storage_group.description=The name of the storage group where events will be stored 12 | 13 | db_user.title=Username 14 | db_user.description=The username for the IoTDB Server 15 | 16 | db_password.title=Password 17 | db_password.description=The password for the IoTDB Server 18 | 19 | timestamp_mapping.title=Timestamp 20 | timestamp_mapping.description=The timestamp of the event 21 | -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.mysql/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.mysql/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.mysql/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.databases.jvm.mysql.title = MySQL Database 2 | org.apache.streampipes.sinks.databases.jvm.mysql.description = Stores events in a MySQL database. 3 | 4 | host.title= Hostname 5 | host.description= Hostname of the MySQL server 6 | 7 | user.title = Username 8 | user.description = Username of the user 9 | 10 | password.title = Password 11 | password.description = Password of the user 12 | 13 | db.title = Database 14 | db.description = Database where the table is located 15 | 16 | table.title = Table 17 | table.description = Table that the data is written to 18 | 19 | port.title= Port 20 | port.description= Port of the MySQL server. Default port: 3306 21 | 22 | -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.opcua/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.opcua/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.opcua/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.databases.jvm.opcua.title=OPC-UA 2 | org.apache.streampipes.sinks.databases.jvm.opcua.description=Writes values in an OPC-UA server 3 | 4 | opc_host.title=Host Address 5 | opc_host.description=The host address of the OPC-UA server (Example: opc-ua-server.com, opc.tcp://opc-ua-server.com) 6 | 7 | opc_port.title=Port 8 | opc_port.description=The port of the OPC-UA server (Example: 4840) 9 | 10 | opc_namespace_index.title=Namespace Index 11 | opc_namespace_index.description=The number of the namespace 12 | 13 | opc_node_id_index.title=Node Id 14 | opc_node_id_index.description=Id of the node the result should be written to 15 | 16 | mapping_property_key.title=Event Property 17 | mapping_property_key.description=The event property that should be written to the OPC-UA server 18 | -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.postgresql/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.postgresql/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.postgresql/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.databases.jvm.postgresql.title=PostgreSQL 2 | org.apache.streampipes.sinks.databases.jvm.postgresql.description=Stores events in a Postgres database. 3 | 4 | db_host.title=Hostname 5 | db_host.description=The hostname of the PostgreSQL instance 6 | 7 | db_port.title=Port 8 | db_port.description=The port of the PostgreSQL instance (default 5432) 9 | 10 | db_name.title=Database Name 11 | db_name.description=The name of the database where events will be stored 12 | 13 | db_table.title=Table Name 14 | db_table.description=The name of the table where events will be stored (will be created if it does not exist) 15 | 16 | db_user.title=Username 17 | db_user.description=The username for the PostgreSQL Server 18 | 19 | db_password.title=Password 20 | db_password.description=The password for the PostgreSQL Server 21 | 22 | ssl_mode.title=SSL Mode 23 | ssl_mode.description=Should the connection be secured via SSL? 24 | -------------------------------------------------------------------------------- /streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.redis/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-databases-jvm/src/main/resources/org.apache.streampipes.sinks.databases.jvm.redis/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.dashboard/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.dashboard/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.dashboard/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.internal.jvm.dashboard.title=Dashboard Sink 2 | org.apache.streampipes.sinks.internal.jvm.dashboard.description=Visualizes data streams in the StreamPipes dashboard 3 | 4 | visualization-name.title=Visualization Name 5 | visualization-name.description=A name that identifies your visualization. -------------------------------------------------------------------------------- /streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.datalake/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.datalake/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.datalake/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.internal.jvm.datalake.title=Data Lake 2 | org.apache.streampipes.sinks.internal.jvm.datalake.description=Stores events in the StreamPipes data lake. 3 | 4 | db_measurement.title=Identifier 5 | db_measurement.description=The name of the identifier under which the data is to be stored. 6 | 7 | timestamp_mapping.title=Timestamp Field 8 | timestamp_mapping.description=The value which contains a timestamp 9 | -------------------------------------------------------------------------------- /streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.notification/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.notification/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-internal-jvm/src/main/resources/org.apache.streampipes.sinks.internal.jvm.notification/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.internal.jvm.notification.title=Notification 2 | org.apache.streampipes.sinks.internal.jvm.notification.description=Publish a notification to the StreamPipes UI. 3 | 4 | title.title=Notification Title 5 | title.description= 6 | 7 | content.title=Content 8 | content.description=Enter the notification text. You can use placeholders like #fieldName# to add the value of a stream variable. -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.email/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.email/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.email/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.notifications.jvm.email.title=Email Notification 2 | org.apache.streampipes.sinks.notifications.jvm.email.description=Email service to send notifications emails 3 | 4 | to_email.title=To 5 | to_email.description=Receiver E-mail address 6 | 7 | email_subject.title=Subject 8 | email_subject.description=The subject of the email 9 | 10 | email_content.title=Content 11 | email_content.description=Enter the email text. You can use place holders like #fieldName# to add the value of a field. 12 | 13 | silent-period.title=Silent Period [min] 14 | silent-period.description=The minimum number of minutes between two consecutive mails that are sent. 15 | -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.onesignal/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.onesignal/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.onesignal/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.notifications.jvm.onesignal.title=OneSignal 2 | org.apache.streampipes.sinks.notifications.jvm.onesignal.description=Send Push Message to OneSignal-Application 3 | 4 | content.title=Content 5 | content.description=Push Message 6 | 7 | app_id.title=App-ID 8 | app_id.description=OneSignal App ID 9 | 10 | api_key.title=API-Key 11 | api_key.description=REST API Key -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.slack/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.slack/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.slack/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.notifications.jvm.slack.title=Slack Notification 2 | org.apache.streampipes.sinks.notifications.jvm.slack.description=Slack bot to send notifications to Slack 3 | 4 | receiver.title=Send to 5 | receiver.description=Enter the username or channel you want to notify 6 | 7 | content.title=Message 8 | content.description=The message that should be sent 9 | 10 | channel-type.title=User or Channel 11 | channel-type.description=Decide wether you want to sent a notification to a user or to a channel 12 | 13 | auth-token.title=Auth Token 14 | auth-token.description=The token to authenticate at Slack -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.telegram/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.telegram/icon.png -------------------------------------------------------------------------------- /streampipes-sinks-notifications-jvm/src/main/resources/org.apache.streampipes.sinks.notifications.jvm.telegram/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sinks.notifications.jvm.telegram.title=Telegram Publisher 2 | org.apache.streampipes.sinks.notifications.jvm.telegram.description=Publisher to send notifications to a Telegram channel. 3 | 4 | api-key.title=Bot API Key 5 | api-key.description=The API Key generated by @BotFather when you created your bot. 6 | 7 | channel-chat-name.title=Channel Name or Chat Id 8 | channel-chat-name.description=The handle name of your public channel (e.g. @channel_name). For private channels or groups use chat id. 9 | 10 | message-text.title=Message 11 | message-text.description=The message to be sent. 12 | -------------------------------------------------------------------------------- /streampipes-sources-vehicle-simulator/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=5010 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_KAFKA_HOST=localhost 21 | SP_ZOOKEEPER_HOST=localhost 22 | -------------------------------------------------------------------------------- /streampipes-sources-vehicle-simulator/src/main/resources/streampipesDemoConfig.json: -------------------------------------------------------------------------------- 1 | { 2 | "workflows": [ 3 | { 4 | "workflowName": "vehicle", 5 | "workflowFilename": "streampipesVehicleWorkflow.json" 6 | } 7 | ], 8 | "producers": [ 9 | { 10 | "type": "logger" 11 | }, 12 | { 13 | "type": "kafka", 14 | "broker.server": "kafka", 15 | "broker.port": 9092, 16 | "topic": "xyz.use.step.specific.topic", 17 | "flatten": true, 18 | "sync": false 19 | } 20 | ] 21 | } 22 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/development/env: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # Those parameters are used by IntelliJ to set the default consul parameters for development 17 | SP_PORT=5015 18 | SP_HOST=host.docker.internal 19 | SP_DEBUG=true 20 | SP_KAFKA_HOST=localhost 21 | SP_ZOOKEEPER_HOST=localhost 22 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.flowrate1/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.flowrate1/icon.png -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.flowrate1/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sources.simulator.flowrate1.title=Flow Rate 1 2 | org.apache.streampipes.sources.simulator.flowrate1.description=Simulates a water flow rate sensor 3 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.flowrate2/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.flowrate2/icon.png -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.flowrate2/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sources.simulator.flowrate2.title=Flow Rate 2 2 | org.apache.streampipes.sources.simulator.flowrate2.description=Simulates a water flow rate sensor 3 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.pressure/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.pressure/icon.png -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.pressure/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sources.simulator.pressure.title=Pressure Sensor 2 | org.apache.streampipes.sources.simulator.pressure.description=Simulates a water tank pressure sensor 3 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.waterlevel1/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.waterlevel1/icon.png -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.waterlevel1/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sources.simulator.waterlevel1.title=Water Level 1 2 | org.apache.streampipes.sources.simulator.waterlevel1.description=Simulates a water level sensor 3 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.waterlevel2/icon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/streampipes-extensions/03b07cb064116d982310d7d529022e51dade256f/streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.waterlevel2/icon.png -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/org.apache.streampipes.sources.simulator.waterlevel2/strings.en: -------------------------------------------------------------------------------- 1 | org.apache.streampipes.sources.simulator.waterlevel2.title=Water Level 2 2 | org.apache.streampipes.sources.simulator.waterlevel2.description=Simulates a water level sensor 3 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/streampipesDemoConfig.json: -------------------------------------------------------------------------------- 1 | { 2 | "workflows": [ 3 | { 4 | "workflowName": "festoFlowRate", 5 | "workflowFilename": "streampipesFestoFlowRateWorkflow.json" 6 | }, 7 | { 8 | "workflowName": "siemensFlowRate", 9 | "workflowFilename": "streampipesSiemensFlowRateWorkflow.json" 10 | }, 11 | { 12 | "workflowName": "festoLevel101", 13 | "workflowFilename": "streampipesFestoLevel101Workflow.json" 14 | }, 15 | { 16 | "workflowName": "festoLevel102", 17 | "workflowFilename": "streampipesFestoLevel102Workflow.json" 18 | }, 19 | { 20 | "workflowName": "festoPressure", 21 | "workflowFilename": "streampipesFestoPressureWorkflow.json" 22 | } 23 | ], 24 | "producers": [ 25 | { 26 | "type": "logger" 27 | }, 28 | { 29 | "type": "kafka", 30 | "broker.server": "kafka", 31 | "broker.port": 9092, 32 | "topic": "xyz.use.step.specific.topic", 33 | "flatten": true, 34 | "sync": false 35 | } 36 | ] 37 | } 38 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/streampipesFestoFlowRateWorkflow.json: -------------------------------------------------------------------------------- 1 | { 2 | "eventFrequency": 1000, 3 | "varyEventFrequency": true, 4 | "repeatWorkflow": true, 5 | "timeBetweenRepeat": 1000, 6 | "varyRepeatFrequency": true, 7 | "targetTopic" : "org.apache.streampipes.examples.flowrate-1", 8 | "steps": [{ 9 | "config": [{ 10 | "timestamp": "nowTimestamp()", 11 | "sensorId" : "flowrate01", 12 | "mass_flow": "double(0,10)", 13 | "temperature": "double(40,50)" 14 | }], 15 | "duration": 20000 16 | },{ 17 | "config": [{ 18 | "timestamp": "nowTimestamp()", 19 | "sensorId" : "flowrate01", 20 | "mass_flow": "double(0,10)", 21 | "temperature": "double(50,60)" 22 | }], 23 | "duration": 10000 24 | }, { 25 | "config": [{ 26 | "timestamp": "nowTimestamp()", 27 | "sensorId" : "flowrate01", 28 | "mass_flow": "double(0,10)", 29 | "temperature": "double(60,70)" 30 | }], 31 | "duration": 10000 32 | }] 33 | } 34 | 35 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/streampipesFestoLevel101Workflow.json: -------------------------------------------------------------------------------- 1 | { 2 | "eventFrequency": 1000, 3 | "varyEventFrequency": true, 4 | "repeatWorkflow": true, 5 | "timeBetweenRepeat": 1000, 6 | "varyRepeatFrequency": true, 7 | "targetTopic" : "org.apache.streampipes.examples.waterlevel", 8 | "steps": [{ 9 | "config": [{ 10 | "timestamp": "nowTimestamp()", 11 | "sensorId" : "level01", 12 | "level": "double(70,75)", 13 | "underflow": false, 14 | "overflow" : false 15 | }], 16 | "duration": 20000 17 | },{ 18 | "config": [{ 19 | "timestamp": "nowTimestamp()", 20 | "sensorId" : "level01", 21 | "level": "double(60,70)", 22 | "underflow": false, 23 | "overflow" : false 24 | }], 25 | "duration": 10000 26 | }, { 27 | "config": [{ 28 | "timestamp": "nowTimestamp()", 29 | "sensorId" : "level01", 30 | "level": "double(50,60)", 31 | "underflow": true, 32 | "overflow" : false 33 | }], 34 | "duration": 10000 35 | }] 36 | } 37 | -------------------------------------------------------------------------------- /streampipes-sources-watertank-simulator/src/main/resources/streampipesFestoLevel102Workflow.json: -------------------------------------------------------------------------------- 1 | { 2 | "eventFrequency": 1000, 3 | "varyEventFrequency": true, 4 | "repeatWorkflow": true, 5 | "timeBetweenRepeat": 1000, 6 | "varyRepeatFrequency": true, 7 | "targetTopic" : "org.apache.streampipes.examples.waterlevel2", 8 | "steps": [{ 9 | "config": [{ 10 | "timestamp": "nowTimestamp()", 11 | "sensorId" : "level02", 12 | "level": "double(70,75)", 13 | "underflow": false 14 | }], 15 | "duration": 20000 16 | },{ 17 | "config": [{ 18 | "timestamp": "nowTimestamp()", 19 | "sensorId" : "level02", 20 | "level": "double(60,70)", 21 | "underflow": false 22 | }], 23 | "duration": 10000 24 | }, { 25 | "config": [{ 26 | "timestamp": "nowTimestamp()", 27 | "sensorId" : "level02", 28 | "level": "double(50,60)", 29 | "underflow": true 30 | }], 31 | "duration": 10000 32 | }] 33 | } 34 | --------------------------------------------------------------------------------