├── .config └── nextest.toml ├── .dockerignore ├── .github └── workflows │ ├── binaries.yml │ ├── ci.yml │ └── docker.yaml ├── .gitignore ├── CONTRIBUTING.md ├── Cargo.lock ├── Cargo.toml ├── LICENSE-APACHE ├── LICENSE-MIT ├── README.md ├── crates ├── arroyo-api │ ├── Cargo.toml │ ├── build.rs │ ├── migrations │ │ ├── V10__merge_definitions.sql │ │ ├── V11__delete_cascade.sql │ │ ├── V12__add_committing.sql │ │ ├── V13__backfill_pub_id.sql │ │ ├── V14__unify_job_ids.sql │ │ ├── V15__rename_connections.sql │ │ ├── V16__restart_failed.sql │ │ ├── V17__udfs_table.sql │ │ ├── V18__udfs_drop_language.sql │ │ ├── V19__case_insensitive_connection_names.sql │ │ ├── V1__initial.sql │ │ ├── V20__pipeline_v2.sql │ │ ├── V21__add_udf_path.sql │ │ ├── V22__remove_cluster_info_pub_id.sql │ │ ├── V23__drop_schemas_table.sql │ │ ├── V24__add_udf_language.sql │ │ ├── V2__add_raw_json.sql │ │ ├── V3__add_udfs.sql │ │ ├── V4__add_sources.sql │ │ ├── V5__add_http_connection.sql │ │ ├── V6__add_cluster_table.sql │ │ ├── V7__add_job_log_messages.sql │ │ ├── V8__connector_redesign.sql │ │ └── V9__add_pub_ids.sql │ ├── queries │ │ └── api_queries.sql │ ├── sqlite_migrations │ │ ├── V1__initial.sql │ │ └── V2__add_udf_language.sql │ └── src │ │ ├── cloud.rs │ │ ├── connection_profiles.rs │ │ ├── connection_tables.rs │ │ ├── connectors.rs │ │ ├── jobs.rs │ │ ├── lib.rs │ │ ├── metrics.rs │ │ ├── pipelines.rs │ │ ├── rest.rs │ │ ├── rest_utils.rs │ │ ├── sql.rs │ │ └── udfs.rs ├── arroyo-compiler-service │ ├── Cargo.toml │ └── src │ │ └── lib.rs ├── arroyo-connectors │ ├── Cargo.toml │ ├── build.rs │ └── src │ │ ├── blackhole │ │ ├── blackhole.svg │ │ ├── mod.rs │ │ └── operator.rs │ │ ├── confluent │ │ ├── confluent.svg │ │ ├── mod.rs │ │ └── profile.json │ │ ├── filesystem │ │ ├── delta.rs │ │ ├── filesystem.svg │ │ ├── mod.rs │ │ ├── sink │ │ │ ├── arrow.rs │ │ │ ├── delta.rs │ │ │ ├── json.rs │ │ │ ├── local.rs │ │ │ ├── mod.rs │ │ │ ├── parquet.rs │ │ │ └── two_phase_committer.rs │ │ ├── source.rs │ │ └── table.json │ │ ├── fluvio │ │ ├── fluvio.svg │ │ ├── mod.rs │ │ ├── sink.rs │ │ ├── source.rs │ │ └── table.json │ │ ├── impulse │ │ ├── impulse.svg │ │ ├── mod.rs │ │ ├── operator.rs │ │ └── table.json │ │ ├── kafka │ │ ├── kafka.svg │ │ ├── mod.rs │ │ ├── profile.json │ │ ├── sink │ │ │ ├── mod.rs │ │ │ └── test.rs │ │ ├── source │ │ │ ├── mod.rs │ │ │ └── test.rs │ │ └── table.json │ │ ├── kinesis │ │ ├── kinesis.svg │ │ ├── mod.rs │ │ ├── sink.rs │ │ ├── source.rs │ │ └── table.json │ │ ├── lib.rs │ │ ├── mqtt │ │ ├── mod.rs │ │ ├── mqtt.svg │ │ ├── profile.json │ │ ├── sink │ │ │ ├── mod.rs │ │ │ └── test.rs │ │ ├── source │ │ │ ├── mod.rs │ │ │ └── test.rs │ │ └── table.json │ │ ├── nats │ │ ├── mod.rs │ │ ├── nats.svg │ │ ├── profile.json │ │ ├── sink │ │ │ └── mod.rs │ │ ├── source │ │ │ └── mod.rs │ │ └── table.json │ │ ├── nexmark │ │ ├── mod.rs │ │ ├── nexmark.svg │ │ ├── operator.rs │ │ ├── table.json │ │ └── test.rs │ │ ├── polling_http │ │ ├── http.svg │ │ ├── mod.rs │ │ ├── operator.rs │ │ └── table.json │ │ ├── preview │ │ ├── mod.rs │ │ └── operator.rs │ │ ├── rabbitmq │ │ ├── mod.rs │ │ ├── profile.json │ │ ├── rabbitmq.svg │ │ ├── source.rs │ │ └── table.json │ │ ├── redis │ │ ├── lookup.rs │ │ ├── mod.rs │ │ ├── profile.json │ │ ├── redis.svg │ │ ├── sink.rs │ │ └── table.json │ │ ├── single_file │ │ ├── mod.rs │ │ ├── sink.rs │ │ ├── source.rs │ │ └── table.json │ │ ├── sse │ │ ├── mod.rs │ │ ├── operator.rs │ │ ├── sse.svg │ │ └── table.json │ │ ├── stdout │ │ ├── mod.rs │ │ ├── operator.rs │ │ └── stdout.svg │ │ ├── webhook │ │ ├── mod.rs │ │ ├── operator.rs │ │ ├── table.json │ │ └── webhook.svg │ │ └── websocket │ │ ├── mod.rs │ │ ├── operator.rs │ │ ├── table.json │ │ └── websocket.svg ├── arroyo-controller │ ├── Cargo.toml │ ├── build.rs │ ├── queries │ │ └── controller_queries.sql │ └── src │ │ ├── job_controller │ │ ├── job_metrics.rs │ │ └── mod.rs │ │ ├── lib.rs │ │ ├── schedulers │ │ ├── embedded.rs │ │ ├── kubernetes │ │ │ ├── mod.rs │ │ │ └── quantities.rs │ │ └── mod.rs │ │ └── states │ │ ├── checkpoint_stopping.rs │ │ ├── compiling.rs │ │ ├── finishing.rs │ │ ├── mod.rs │ │ ├── recovering.rs │ │ ├── rescaling.rs │ │ ├── restarting.rs │ │ ├── running.rs │ │ ├── scheduling.rs │ │ └── stopping.rs ├── arroyo-datastream │ ├── Cargo.toml │ └── src │ │ ├── lib.rs │ │ ├── logical.rs │ │ └── optimizers.rs ├── arroyo-formats │ ├── Cargo.toml │ └── src │ │ ├── avro │ │ ├── de.rs │ │ ├── mod.rs │ │ ├── schema.rs │ │ └── ser.rs │ │ ├── de.rs │ │ ├── json │ │ ├── mod.rs │ │ └── schema.rs │ │ ├── lib.rs │ │ ├── proto │ │ ├── de.rs │ │ ├── mod.rs │ │ ├── schema.rs │ │ └── test │ │ │ ├── mod.rs │ │ │ ├── my-policy.json │ │ │ └── protos │ │ │ ├── basic_types.proto │ │ │ ├── enum_fields.proto │ │ │ ├── map_fields.proto │ │ │ ├── nested_message.proto │ │ │ ├── orders.proto │ │ │ ├── repeated_fields.proto │ │ │ └── string_and_bytes.proto │ │ └── ser.rs ├── arroyo-metrics │ ├── Cargo.toml │ └── src │ │ └── lib.rs ├── arroyo-node │ ├── Cargo.toml │ └── src │ │ └── lib.rs ├── arroyo-openapi │ ├── Cargo.toml │ ├── build.rs │ └── src │ │ └── lib.rs ├── arroyo-operator │ ├── Cargo.toml │ └── src │ │ ├── connector.rs │ │ ├── context.rs │ │ ├── inq_reader.rs │ │ ├── lib.rs │ │ ├── operator.rs │ │ └── udfs.rs ├── arroyo-planner │ ├── Cargo.toml │ ├── build.rs │ └── src │ │ ├── builder.rs │ │ ├── extension │ │ ├── aggregate.rs │ │ ├── debezium.rs │ │ ├── join.rs │ │ ├── key_calculation.rs │ │ ├── lookup.rs │ │ ├── mod.rs │ │ ├── remote_table.rs │ │ ├── sink.rs │ │ ├── table_source.rs │ │ ├── updating_aggregate.rs │ │ ├── watermark_node.rs │ │ └── window_fn.rs │ │ ├── external.rs │ │ ├── functions.rs │ │ ├── lib.rs │ │ ├── logical.rs │ │ ├── physical.rs │ │ ├── plan │ │ ├── aggregate.rs │ │ ├── join.rs │ │ ├── mod.rs │ │ └── window_fn.rs │ │ ├── rewriters.rs │ │ ├── schemas.rs │ │ ├── tables.rs │ │ ├── test │ │ ├── mod.rs │ │ ├── plan_tests.rs │ │ ├── queries │ │ │ ├── analytics_ingest.sql │ │ │ ├── analytics_tutorial.sql │ │ │ ├── async_udf_subquery.sql │ │ │ ├── basic_tumble_aggregate.sql │ │ │ ├── bitcoin_exchange_rate.sql │ │ │ ├── bitcoin_stats.sql │ │ │ ├── bluesky_trends.sql │ │ │ ├── create_table_updating.sql │ │ │ ├── crypto_function.sql │ │ │ ├── custom_ttls.sql │ │ │ ├── error_lookup_join_non_primary_key.sql │ │ │ ├── error_memory_table_double_write.sql │ │ │ ├── error_mismatched_nested_windows.sql │ │ │ ├── error_missing_redis_key.sql │ │ │ ├── error_missing_window_basic_tumble.sql │ │ │ ├── error_no_aggregate_over_debezium.sql │ │ │ ├── error_no_nested_updating_aggregates.sql │ │ │ ├── error_offset_sliding_window.sql │ │ │ ├── error_read_from_unwritten_memory_table.sql │ │ │ ├── error_second_aggregate_missing_window.sql │ │ │ ├── filesystem_invalid_partition.sql │ │ │ ├── filesystem_partition.sql │ │ │ ├── first_pipeline.sql │ │ │ ├── hop_to_tumble.sql │ │ │ ├── infer_debezium_sink.sql │ │ │ ├── kafka_metadata_udf.sql │ │ │ ├── lookup_join.sql │ │ │ ├── mastodon_trends.sql │ │ │ ├── merge_updates.sql │ │ │ ├── metadata_error.sql │ │ │ ├── metadata_fields.sql │ │ │ ├── metadata_raw_string.sql │ │ │ ├── mismatched_sink_schema.sql │ │ │ ├── misplaced_hop.sql │ │ │ ├── no_inserting_updates_into_non_updating.sql │ │ │ ├── no_updating_joins.sql │ │ │ ├── no_updating_window_functions.sql │ │ │ ├── no_virtual_fields_updating.sql │ │ │ ├── parse.sql │ │ │ ├── parse_log.sql │ │ │ ├── program_compilation.sql │ │ │ ├── prometheus.sql │ │ │ ├── release_post_json.sql │ │ │ ├── schema_inference.sql │ │ │ ├── second_aggregate_recall_window.sql │ │ │ ├── source_rewriter_join.sql │ │ │ ├── struct_ddl.sql │ │ │ ├── subscript_in_virtual.sql │ │ │ ├── test_merge_sink.sql │ │ │ ├── test_no_aggregates_in_window.sql │ │ │ ├── updating_filter_join.sql │ │ │ ├── virtual_bad_schema.sql │ │ │ ├── watermarks.sql │ │ │ ├── window_function.sql │ │ │ ├── window_partition_by_column.sql │ │ │ ├── window_with_expression.sql │ │ │ └── windowed_inner_join.sql │ │ └── udfs │ │ │ ├── cbor_to_json.rs │ │ │ ├── get_city.rs │ │ │ ├── irate.rs │ │ │ ├── parse_log.rs │ │ │ └── parse_prom.rs │ │ ├── types.rs │ │ ├── udafs.rs │ │ └── utils.rs ├── arroyo-rpc │ ├── Cargo.toml │ ├── build.rs │ ├── default.toml │ ├── proto │ │ ├── api.proto │ │ └── rpc.proto │ └── src │ │ ├── api_types │ │ ├── checkpoints.rs │ │ ├── connections.rs │ │ ├── metrics.rs │ │ ├── mod.rs │ │ ├── pipelines.rs │ │ └── udfs.rs │ │ ├── config.rs │ │ ├── df.rs │ │ ├── formats.rs │ │ ├── lib.rs │ │ ├── public_ids.rs │ │ ├── schema_resolver.rs │ │ └── var_str.rs ├── arroyo-server-common │ ├── Cargo.toml │ ├── build.rs │ └── src │ │ ├── lib.rs │ │ ├── profile.rs │ │ └── shutdown.rs ├── arroyo-sql-testing │ ├── Cargo.toml │ ├── golden_outputs │ │ ├── active_drivers.json │ │ ├── aggregates.json │ │ ├── async_udf.json │ │ ├── cast_to_sink_type.json │ │ ├── debezium_agg.json │ │ ├── debezium_coercion.json │ │ ├── debezium_pass_through.json │ │ ├── double_negative_udf.json │ │ ├── every_aggregate.json │ │ ├── filter_updating_aggregates.json │ │ ├── global_session_window.json │ │ ├── grouped_aggregates.json │ │ ├── hourly_by_event_type.json │ │ ├── json_operators.json │ │ ├── memory_table.json │ │ ├── month_loose_watermark.json │ │ ├── most_active_driver_last_hour.json │ │ ├── most_active_driver_last_hour_unaligned.json │ │ ├── nexmark_q5.json │ │ ├── offset_impulse_join.json │ │ ├── outer_join.json │ │ ├── reinvoke_window_function.json │ │ ├── select_star.json │ │ ├── session_window.json │ │ ├── sliding_window_end.json │ │ ├── test_merge_sink.json │ │ ├── test_unnest_in_view.json │ │ ├── tight_watermark.json │ │ ├── udaf.json │ │ ├── union.json │ │ ├── unnest_in_view.json │ │ ├── updating_full_join.json │ │ ├── updating_inner_join.json │ │ ├── updating_left_join.json │ │ ├── updating_right_join.json │ │ ├── windowed_inner_join.json │ │ └── windowed_outer_join.json │ ├── inputs │ │ ├── aggregate_updates.json │ │ ├── cars.json │ │ ├── impulse.json │ │ ├── nexmark_bids.json │ │ ├── session_window.json │ │ └── sorted_cars.json │ ├── outputs │ │ └── .gitignore │ └── src │ │ ├── lib.rs │ │ ├── smoke_tests.rs │ │ ├── test │ │ └── queries │ │ │ ├── active_drivers.sql │ │ │ ├── aggregates.sql │ │ │ ├── async_udf.sql │ │ │ ├── cast_to_sink_type.sql │ │ │ ├── debezium_agg.sql │ │ │ ├── debezium_coercion.sql │ │ │ ├── debezium_pass_through.sql │ │ │ ├── double_negative_udf.sql │ │ │ ├── every_aggregate.sql │ │ │ ├── filter_updating_aggregates.sql │ │ │ ├── global_session_window.sql │ │ │ ├── grouped_aggregates.sql │ │ │ ├── hourly_by_event_type.sql │ │ │ ├── json_operators.sql │ │ │ ├── memory_table.sql │ │ │ ├── month_loose_watermark.sql │ │ │ ├── most_active_driver_last_hour.sql │ │ │ ├── most_active_driver_last_hour_unaligned.sql │ │ │ ├── nexmark_q5.sql │ │ │ ├── offset_impulse_join.sql │ │ │ ├── reinvoke_window_function.sql │ │ │ ├── select_star.sql │ │ │ ├── session_window.sql │ │ │ ├── sliding_window_end.sql │ │ │ ├── test_merge_sink.sql │ │ │ ├── tight_watermark.sql │ │ │ ├── udaf.sql │ │ │ ├── union.sql │ │ │ ├── unnest_in_view.sql │ │ │ ├── updating_full_join.sql │ │ │ ├── updating_inner_join.sql │ │ │ ├── updating_inner_join_with_updating.sql │ │ │ ├── updating_left_join.sql │ │ │ ├── updating_right_join.sql │ │ │ ├── windowed_inner_join.sql │ │ │ └── windowed_outer_join.sql │ │ └── udfs.rs ├── arroyo-state │ ├── Cargo.toml │ └── src │ │ ├── checkpoint_state.rs │ │ ├── committing_state.rs │ │ ├── lib.rs │ │ ├── metrics.rs │ │ ├── parquet.rs │ │ ├── schemas │ │ └── mod.rs │ │ └── tables │ │ ├── expiring_time_key_map.rs │ │ ├── global_keyed_map.rs │ │ ├── mod.rs │ │ └── table_manager.rs ├── arroyo-storage │ ├── Cargo.toml │ └── src │ │ ├── aws.rs │ │ └── lib.rs ├── arroyo-types │ ├── Cargo.toml │ └── src │ │ └── lib.rs ├── arroyo-udf │ ├── arroyo-udf-common │ │ ├── Cargo.toml │ │ └── src │ │ │ ├── async_udf.rs │ │ │ ├── lib.rs │ │ │ └── parse.rs │ ├── arroyo-udf-host │ │ ├── Cargo.toml │ │ └── src │ │ │ ├── lib.rs │ │ │ └── test.rs │ ├── arroyo-udf-macros │ │ ├── Cargo.toml │ │ └── src │ │ │ └── lib.rs │ ├── arroyo-udf-plugin │ │ ├── Cargo.toml │ │ └── src │ │ │ ├── async_udf.rs │ │ │ └── lib.rs │ └── arroyo-udf-python │ │ ├── Cargo.toml │ │ ├── python │ │ └── arroyo_udf.py │ │ └── src │ │ ├── interpreter.rs │ │ ├── lib.rs │ │ ├── pyarrow.rs │ │ ├── threaded.rs │ │ └── types.rs ├── arroyo-worker │ ├── Cargo.toml │ └── src │ │ ├── arrow │ │ ├── async_udf.rs │ │ ├── incremental_aggregator.rs │ │ ├── instant_join.rs │ │ ├── join_with_expiration.rs │ │ ├── lookup_join.rs │ │ ├── mod.rs │ │ ├── session_aggregating_window.rs │ │ ├── sliding_aggregating_window.rs │ │ ├── sync │ │ │ ├── mod.rs │ │ │ └── streams.rs │ │ ├── tumbling_aggregating_window.rs │ │ ├── updating_cache.rs │ │ ├── watermark_generator.rs │ │ └── window_fn.rs │ │ ├── engine.rs │ │ ├── lib.rs │ │ ├── network_manager.rs │ │ └── utils.rs ├── arroyo │ ├── Cargo.toml │ └── src │ │ ├── main.rs │ │ └── run.rs └── integ │ ├── Cargo.toml │ ├── src │ └── lib.rs │ └── tests │ └── api_tests.rs ├── docker ├── Dockerfile ├── config.toml ├── install_deps.sh └── refinery.toml ├── images ├── arroyo_logo.png ├── header_image.png └── running_job.png ├── k8s └── arroyo │ ├── .helmignore │ ├── Chart.lock │ ├── Chart.yaml │ ├── README.md │ ├── charts │ └── postgresql-12.6.8.tgz │ ├── ci │ └── ci-test-values.yaml │ ├── ct.yaml │ ├── templates │ ├── NOTES.txt │ ├── _helpers.tpl │ ├── configmap.yaml │ ├── controller.yaml │ ├── role.yaml │ ├── serviceaccount.yaml │ └── tests │ │ ├── test-console-connection.yaml │ │ └── test-grpc-connection.yaml │ └── values.yaml └── webui ├── .eslintignore ├── .eslintrc.yml ├── .gitignore ├── .prettierignore ├── .prettierrc ├── index.html ├── package.json ├── pnpm-lock.yaml ├── public ├── apple-touch-icon.png ├── icon.png ├── icon.svg └── logo.svg ├── src ├── App.css ├── App.tsx ├── components │ ├── CheckpointDetails.tsx │ ├── Checkpoints.tsx │ ├── CreatePipelineTourModal.tsx │ ├── ExampleQueries.tsx │ ├── ExampleQueryCard.tsx │ ├── Loading.tsx │ ├── OperatorDetail.tsx │ ├── OperatorErrors.tsx │ ├── PaginatedContent.tsx │ ├── PipelineNotFound.tsx │ ├── PipelineRow.tsx │ ├── PipelinesTable.tsx │ ├── StartPipelineModal.tsx │ ├── TimeSeriesGraph.tsx │ ├── TourCompleteModal.tsx │ ├── UdfNodeViewer.tsx │ └── WelcomeModal.tsx ├── gen │ └── api-types.ts ├── index.css ├── lib │ ├── CloudComponents.tsx │ ├── RadioGroup.tsx │ ├── data_fetching.ts │ ├── example_queries.ts │ ├── types.ts │ └── util.ts ├── main.tsx ├── router.tsx ├── routes │ ├── connections │ │ ├── ChooseConnector.tsx │ │ ├── ConfigureConnection.tsx │ │ ├── ConfigureProfile.tsx │ │ ├── ConfluentSchemaEditor.tsx │ │ ├── ConnectionTester.tsx │ │ ├── Connections.tsx │ │ ├── CreateConnection.tsx │ │ ├── CreateProfile.tsx │ │ ├── DefineSchema.tsx │ │ ├── JsonForm.tsx │ │ └── SchemaEditor.tsx │ ├── home │ │ └── Home.tsx │ ├── not_found │ │ ├── ApiUnavailable.tsx │ │ └── PageNotFound.tsx │ ├── pipelines │ │ ├── Catalog.tsx │ │ ├── CatalogTab.tsx │ │ ├── CodeEditor.tsx │ │ ├── CreatePipeline.tsx │ │ ├── PipelineConfigModal.tsx │ │ ├── PipelineDetails.tsx │ │ ├── PipelineEditorTabs.tsx │ │ ├── PipelineGraph.tsx │ │ ├── PipelineOutputs.tsx │ │ ├── PipelinesIndex.tsx │ │ ├── ResourcePanel.tsx │ │ └── pipelines.css │ └── udfs │ │ ├── DeleteUdfModal.tsx │ │ ├── GlobalizeModal.tsx │ │ ├── UdfEditTab.tsx │ │ ├── UdfEditor.tsx │ │ ├── UdfLabel.tsx │ │ ├── UdfNodePopover.tsx │ │ └── UdfsResourceTab.tsx ├── styles │ └── data-grid-style.css ├── theming.ts ├── tour.ts ├── udf_state.ts └── vite-env.d.ts ├── tsconfig.json ├── tsconfig.node.json └── vite.config.ts /.config/nextest.toml: -------------------------------------------------------------------------------- 1 | [profile.default] 2 | slow-timeout = { period = "60s", terminate-after = 2 } 3 | -------------------------------------------------------------------------------- /.dockerignore: -------------------------------------------------------------------------------- 1 | target 2 | docker/Dockerfile 3 | webui/node_modules 4 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | target/ 2 | .idea 3 | .env 4 | .vscode/ 5 | 6 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributing to Arroyo 2 | 3 | We welcome contributions from the community! 4 | 5 | Please refer to the [dev setup](https://doc.arroyo.dev/developing/dev-setup) guide for how to get started. You can 6 | find help on our [discord](https://doc.arroyo.dev/developing/dev-setup) or via email at 7 | [support@arroyo.systems](mailto:support@arroyo.systems). 8 | -------------------------------------------------------------------------------- /LICENSE-MIT: -------------------------------------------------------------------------------- 1 | Copyright (c) 2014 The Rust Project Developers 2 | 3 | Permission is hereby granted, free of charge, to any 4 | person obtaining a copy of this software and associated 5 | documentation files (the "Software"), to deal in the 6 | Software without restriction, including without 7 | limitation the rights to use, copy, modify, merge, 8 | publish, distribute, sublicense, and/or sell copies of 9 | the Software, and to permit persons to whom the Software 10 | is furnished to do so, subject to the following 11 | conditions: 12 | 13 | The above copyright notice and this permission notice 14 | shall be included in all copies or substantial portions 15 | of the Software. 16 | 17 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF 18 | ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED 19 | TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A 20 | PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT 21 | SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY 22 | CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION 23 | OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR 24 | IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER 25 | DEALINGS IN THE SOFTWARE. 26 | -------------------------------------------------------------------------------- /crates/arroyo-api/build.rs: -------------------------------------------------------------------------------- 1 | use cornucopia::{CodegenSettings, Error}; 2 | use postgres::{Client, NoTls}; 3 | 4 | fn main() -> Result<(), Error> { 5 | let queries_path = "queries"; 6 | let destination = format!("{}/api-sql.rs", std::env::var("OUT_DIR").unwrap()); 7 | let settings = CodegenSettings { 8 | gen_async: true, 9 | derive_ser: true, 10 | gen_sync: false, 11 | gen_sqlite: true, 12 | }; 13 | 14 | println!("cargo:rerun-if-changed={queries_path}"); 15 | println!("cargo:rerun-if-changed=migrations"); 16 | println!("cargo:rerun-if-changed=sqlite_migrations"); 17 | 18 | let mut client = match std::env::var("DATABASE_URL") { 19 | Ok(database_url) => Client::connect(&database_url, NoTls) 20 | .unwrap_or_else(|e| panic!("Failed to connect to database: {}", e)), 21 | Err(_) => Client::configure() 22 | .dbname("arroyo") 23 | .host("localhost") 24 | .port(5432) 25 | .user("arroyo") 26 | .password("arroyo") 27 | .connect(NoTls) 28 | .unwrap_or_else(|e| panic!("Failed to connect to default database: {}", e)), 29 | }; 30 | 31 | let mut sqlite = 32 | rusqlite::Connection::open_in_memory().expect("Couldn't open sqlite memory connection"); 33 | let migrations = refinery::load_sql_migrations("sqlite_migrations").unwrap(); 34 | refinery::Runner::new(&migrations) 35 | .run(&mut sqlite) 36 | .expect("Failed to run migrations"); 37 | 38 | cornucopia::generate_live_with_sqlite( 39 | &mut client, 40 | queries_path, 41 | Some(&destination), 42 | &sqlite, 43 | settings, 44 | )?; 45 | 46 | Ok(()) 47 | } 48 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V11__delete_cascade.sql: -------------------------------------------------------------------------------- 1 | -- add missing ON DELETE CASCADE to connection_table_pipelines 2 | ALTER TABLE connection_table_pipelines RENAME COLUMN pipeline_id to pipeline_id_old; 3 | ALTER TABLE connection_table_pipelines 4 | ADD COLUMN pipeline_id BIGINT REFERENCES pipelines(id) ON DELETE CASCADE; 5 | UPDATE connection_table_pipelines SET pipeline_id = pipeline_id_old; 6 | ALTER TABLE connection_table_pipelines DROP COLUMN pipeline_id_old; 7 | 8 | -- add missing ON DELETE CASCADE to job_configs 9 | ALTER TABLE job_configs RENAME COLUMN pipeline_id to pipeline_id_old; 10 | ALTER TABLE job_configs 11 | ADD COLUMN pipeline_id BIGINT REFERENCES pipelines(id) ON DELETE CASCADE; 12 | UPDATE job_configs SET pipeline_id = pipeline_id_old; 13 | ALTER TABLE job_configs DROP COLUMN pipeline_id_old; 14 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V12__add_committing.sql: -------------------------------------------------------------------------------- 1 | ALTER TYPE checkpoint_state ADD VALUE 'committing'; 2 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V13__backfill_pub_id.sql: -------------------------------------------------------------------------------- 1 | UPDATE api_keys 2 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 3 | 4 | ALTER TABLE api_keys 5 | ALTER COLUMN pub_id SET NOT NULL; 6 | 7 | UPDATE connections 8 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 9 | 10 | ALTER TABLE connections 11 | ALTER COLUMN pub_id SET NOT NULL; 12 | 13 | UPDATE schemas 14 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 15 | 16 | ALTER TABLE schemas 17 | ALTER COLUMN pub_id SET NOT NULL; 18 | 19 | UPDATE pipelines 20 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 21 | 22 | ALTER TABLE pipelines 23 | ALTER COLUMN pub_id SET NOT NULL; 24 | 25 | UPDATE job_configs 26 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 27 | 28 | ALTER TABLE job_configs 29 | ALTER COLUMN pub_id SET NOT NULL; 30 | 31 | UPDATE checkpoints 32 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 33 | 34 | ALTER TABLE checkpoints 35 | ALTER COLUMN pub_id SET NOT NULL; 36 | 37 | UPDATE job_statuses 38 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 39 | 40 | ALTER TABLE job_statuses 41 | ALTER COLUMN pub_id SET NOT NULL; 42 | 43 | UPDATE cluster_info 44 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 45 | 46 | ALTER TABLE cluster_info 47 | ALTER COLUMN pub_id SET NOT NULL; 48 | 49 | UPDATE job_log_messages 50 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 51 | 52 | ALTER TABLE job_log_messages 53 | ALTER COLUMN pub_id SET NOT NULL; 54 | 55 | UPDATE connection_tables 56 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 57 | 58 | ALTER TABLE connection_tables 59 | ALTER COLUMN pub_id SET NOT NULL; 60 | 61 | UPDATE connection_table_pipelines 62 | SET pub_id = COALESCE(pub_id, CAST(id as VARCHAR)); 63 | 64 | ALTER TABLE connection_table_pipelines 65 | ALTER COLUMN pub_id SET NOT NULL; 66 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V14__unify_job_ids.sql: -------------------------------------------------------------------------------- 1 | -- going forward the 'id' column will be used to store the public id 2 | 3 | ALTER TABLE job_configs 4 | DROP COLUMN pub_id; 5 | 6 | ALTER TABLE job_configs 7 | ALTER COLUMN id TYPE VARCHAR; 8 | 9 | ALTER TABLE job_configs 10 | ADD CONSTRAINT job_configs_unique_id UNIQUE (id); 11 | 12 | ALTER TABLE checkpoints 13 | ALTER COLUMN job_id TYPE VARCHAR; 14 | 15 | ALTER TABLE job_statuses 16 | ALTER COLUMN id TYPE VARCHAR; 17 | 18 | ALTER TABLE job_log_messages 19 | ALTER COLUMN job_id TYPE VARCHAR; 20 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V15__rename_connections.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE connections RENAME TO connection_profiles; 2 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V16__restart_failed.sql: -------------------------------------------------------------------------------- 1 | CREATE TYPE restart_mode as ENUM ( 2 | 'safe', 'force'); 3 | 4 | ALTER TABLE job_configs 5 | ADD COLUMN restart_nonce int not null default 0, 6 | ADD COLUMN restart_mode restart_mode not null default 'safe'; 7 | 8 | ALTER TABLE job_statuses 9 | ADD COLUMN restart_nonce int not null default 0; -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V17__udfs_table.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE udfs ( 2 | pub_id VARCHAR PRIMARY KEY, 3 | organization_id VARCHAR NOT NULL, 4 | created_by VARCHAR NOT NULL, 5 | created_at TIMESTAMPTZ DEFAULT CURRENT_TIMESTAMP NOT NULL, 6 | updated_at TIMESTAMPTZ DEFAULT CURRENT_TIMESTAMP NOT NULL, 7 | prefix TEXT, 8 | name TEXT NOT NULL, 9 | definition TEXT NOT NULL, 10 | language TEXT NOT NULL, 11 | description TEXT, 12 | 13 | UNIQUE(organization_id, name) 14 | ); 15 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V18__udfs_drop_language.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE udfs DROP COLUMN language; 2 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V19__case_insensitive_connection_names.sql: -------------------------------------------------------------------------------- 1 | CREATE UNIQUE INDEX case_insensitive_connection_unique ON connection_tables (organization_id, LOWER(name)); -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V20__pipeline_v2.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE pipelines 2 | ADD COLUMN proto_version INT NOT NULL DEFAULT 1; -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V21__add_udf_path.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE udfs 2 | ADD COLUMN dylib_url TEXT NOT NULL DEFAULT ''; -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V22__remove_cluster_info_pub_id.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE cluster_info 2 | DROP COLUMN pub_id; -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V23__drop_schemas_table.sql: -------------------------------------------------------------------------------- 1 | DROP table schemas; -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V24__add_udf_language.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE udfs ADD COLUMN language VARCHAR(15) NOT NULL DEFAULT 'rust'; 2 | ALTER TABLE udfs ALTER COLUMN dylib_url DROP NOT NULL; 3 | ALTER TABLE udfs ALTER COLUMN dylib_url DROP DEFAULT; 4 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V2__add_raw_json.sql: -------------------------------------------------------------------------------- 1 | ALTER TYPE schema_type ADD VALUE 'raw_json'; 2 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V3__add_udfs.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE pipeline_definitions 2 | ADD COLUMN udfs JSONB NOT NULL DEFAULT '[]'; -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V4__add_sources.sql: -------------------------------------------------------------------------------- 1 | ALTER TYPE source_type 2 | ADD VALUE 'event_source'; 3 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V5__add_http_connection.sql: -------------------------------------------------------------------------------- 1 | ALTER TYPE connection_type 2 | ADD VALUE 'http'; 3 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V6__add_cluster_table.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE cluster_info ( 2 | id UUID PRIMARY KEY, 3 | name VARCHAR(255) NOT NULL, 4 | created_at TIMESTAMP WITH TIME ZONE NOT NULL DEFAULT NOW(), 5 | singleton INT NOT NULL DEFAULT 0 UNIQUE CHECK (singleton = 0) 6 | ); 7 | 8 | INSERT INTO cluster_info (id, name) VALUES (gen_random_uuid(), 'default'); 9 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V7__add_job_log_messages.sql: -------------------------------------------------------------------------------- 1 | create type log_level as ENUM ('info', 'warn', 'error'); 2 | 3 | CREATE TABLE job_log_messages ( 4 | id BIGSERIAL PRIMARY KEY, 5 | created_at TIMESTAMPTZ DEFAULT CURRENT_TIMESTAMP NOT NULL, 6 | job_id VARCHAR(8) REFERENCES job_configs(id) ON DELETE CASCADE, 7 | operator_id TEXT, 8 | task_index BIGINT, 9 | log_level log_level DEFAULT 'info', 10 | message TEXT NOT NULL, 11 | details TEXT 12 | ); 13 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V8__connector_redesign.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE connections 2 | ALTER COLUMN type TYPE TEXT using type::text; 3 | 4 | CREATE TABLE connection_tables ( 5 | id BIGSERIAL PRIMARY KEY, 6 | organization_id VARCHAR NOT NULL, 7 | created_by VARCHAR NOT NULL, 8 | created_at TIMESTAMPTZ DEFAULT CURRENT_TIMESTAMP NOT NULL, 9 | updated_by VARCHAR, 10 | updated_at TIMESTAMPTZ, 11 | deleted_at TIMESTAMPTZ, 12 | 13 | name TEXT NOT NULL, 14 | table_type TEXT NOT NULL, -- currently one of source or sink 15 | connector TEXT NOT NULL, 16 | connection_id BIGINT REFERENCES connections(id), 17 | config JSONB, 18 | -- json-serialized ConnectionSchema 19 | schema JSONB, 20 | 21 | UNIQUE (organization_id, name) 22 | ); 23 | 24 | CREATE TABLE connection_table_pipelines ( 25 | id BIGSERIAL PRIMARY KEY, 26 | pipeline_id BIGSERIAL REFERENCES pipeline_definitions(id) ON DELETE CASCADE, 27 | connection_table_id BIGSERIAL REFERENCES connection_tables(id) 28 | ); 29 | 30 | DROP TABLE pipeline_sinks; 31 | DROP TABLE pipeline_sources; 32 | DROP TABLE sinks; 33 | DROP TABLE sources; 34 | -------------------------------------------------------------------------------- /crates/arroyo-api/migrations/V9__add_pub_ids.sql: -------------------------------------------------------------------------------- 1 | ALTER TABLE api_keys 2 | ADD COLUMN pub_id VARCHAR UNIQUE; 3 | 4 | ALTER TABLE connections 5 | ADD COLUMN pub_id VARCHAR UNIQUE; 6 | 7 | ALTER TABLE schemas 8 | ADD COLUMN pub_id VARCHAR UNIQUE; 9 | 10 | ALTER TABLE pipelines 11 | ADD COLUMN pub_id VARCHAR UNIQUE; 12 | 13 | ALTER TABLE pipeline_definitions 14 | ADD COLUMN pub_id VARCHAR UNIQUE; 15 | 16 | ALTER TABLE job_configs 17 | ADD COLUMN pub_id VARCHAR UNIQUE; 18 | 19 | ALTER TABLE checkpoints 20 | ADD COLUMN pub_id VARCHAR UNIQUE; 21 | 22 | ALTER TABLE job_statuses 23 | ADD COLUMN pub_id VARCHAR UNIQUE; 24 | 25 | ALTER TABLE cluster_info 26 | ADD COLUMN pub_id VARCHAR UNIQUE; 27 | 28 | ALTER TABLE job_log_messages 29 | ADD COLUMN pub_id VARCHAR UNIQUE; 30 | 31 | ALTER TABLE connection_tables 32 | ADD COLUMN pub_id VARCHAR UNIQUE; 33 | 34 | ALTER TABLE connection_table_pipelines 35 | ADD COLUMN pub_id VARCHAR UNIQUE; 36 | -------------------------------------------------------------------------------- /crates/arroyo-api/sqlite_migrations/V2__add_udf_language.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE udfs_new ( 2 | pub_id TEXT PRIMARY KEY, 3 | organization_id TEXT NOT NULL, 4 | created_by TEXT NOT NULL, 5 | created_at TIMESTAMP DEFAULT CURRENT_TIMESTAMP NOT NULL, 6 | updated_at TIMESTAMP DEFAULT CURRENT_TIMESTAMP NOT NULL, 7 | prefix TEXT, 8 | name TEXT NOT NULL, 9 | definition TEXT NOT NULL, 10 | description TEXT, 11 | dylib_url TEXT, 12 | language VARCHAR(15) NOT NULL DEFAULT 'rust', 13 | UNIQUE (organization_id, name) 14 | ); 15 | 16 | -- Copy data from the old table to the nfew table 17 | INSERT INTO udfs_new (pub_id, organization_id, created_by, created_at, updated_at, 18 | prefix, name, definition, description, dylib_url, language) 19 | SELECT pub_id, organization_id, created_by, created_at, updated_at, 20 | prefix, name, definition, description, dylib_url, 'rust' 21 | from udfs; 22 | 23 | -- Drop the old table 24 | DROP TABLE udfs; 25 | 26 | -- Rename the new table to the old table name 27 | ALTER TABLE udfs_new RENAME TO udfs; 28 | -------------------------------------------------------------------------------- /crates/arroyo-api/src/cloud.rs: -------------------------------------------------------------------------------- 1 | use crate::{rest_utils::ErrorResp, AuthData, OrgMetadata, DEFAULT_ORG}; 2 | use axum_extra::headers::authorization::Bearer; 3 | use axum_extra::headers::Authorization; 4 | use axum_extra::TypedHeader; 5 | use cornucopia_async::Database; 6 | 7 | pub(crate) async fn authenticate( 8 | _client: &Database<'_>, 9 | _bearer_auth: Option>>, 10 | ) -> Result { 11 | Ok(AuthData { 12 | user_id: "user".to_string(), 13 | organization_id: DEFAULT_ORG.to_string(), 14 | role: "admin".to_string(), 15 | org_metadata: OrgMetadata { 16 | can_create_programs: true, 17 | max_nexmark_qps: f64::MAX, 18 | max_impulse_qps: f64::MAX, 19 | max_parallelism: u32::MAX, 20 | max_operators: u32::MAX, 21 | max_running_jobs: u32::MAX, 22 | kafka_qps: u32::MAX, 23 | }, 24 | }) 25 | } 26 | -------------------------------------------------------------------------------- /crates/arroyo-api/src/connectors.rs: -------------------------------------------------------------------------------- 1 | use crate::rest_utils::ErrorResp; 2 | use arroyo_connectors::connectors; 3 | use arroyo_rpc::api_types::ConnectorCollection; 4 | use axum::Json; 5 | 6 | /// List all connectors 7 | #[utoipa::path( 8 | get, 9 | path = "/v1/connectors", 10 | tag = "connectors", 11 | responses( 12 | (status = 200, description = "Got connectors collection", body = ConnectorCollection), 13 | ), 14 | )] 15 | pub async fn get_connectors() -> Result, ErrorResp> { 16 | let mut connectors: Vec<_> = connectors() 17 | .values() 18 | .map(|c| c.metadata()) 19 | .filter(|metadata| !metadata.hidden) 20 | .collect(); 21 | 22 | connectors.sort_by_cached_key(|c| c.name.clone()); 23 | Ok(Json(ConnectorCollection { data: connectors })) 24 | } 25 | -------------------------------------------------------------------------------- /crates/arroyo-api/src/sql.rs: -------------------------------------------------------------------------------- 1 | 2 | -------------------------------------------------------------------------------- /crates/arroyo-compiler-service/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-compiler-service" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | [dependencies] 7 | arroyo-rpc = { path = "../arroyo-rpc" } 8 | arroyo-server-common = { path = "../arroyo-server-common" } 9 | arroyo-storage = { path = "../arroyo-storage" } 10 | 11 | tonic = {workspace = true} 12 | tokio = { version = "1", features = ["full"] } 13 | tracing = "0.1" 14 | anyhow = "1.0.75" 15 | serde_json = "1.0.106" 16 | base64 = "0.22" 17 | dlopen2 = "0.7" 18 | toml = "0.8.12" 19 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/build.rs: -------------------------------------------------------------------------------- 1 | fn main() -> Result<(), String> { 2 | // recursively find all json files in the src directory 3 | glob::glob("src/**/*.json") 4 | .unwrap() 5 | .filter_map(Result::ok) 6 | .for_each(|path| { 7 | println!("cargo:rerun-if-changed={}", path.display()); 8 | }); 9 | 10 | Ok(()) 11 | } 12 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/blackhole/operator.rs: -------------------------------------------------------------------------------- 1 | use arrow::array::RecordBatch; 2 | use arroyo_operator::context::{Collector, OperatorContext}; 3 | use arroyo_operator::operator::ArrowOperator; 4 | use async_trait::async_trait; 5 | 6 | #[derive(Debug)] 7 | pub struct BlackholeSinkFunc {} 8 | 9 | impl BlackholeSinkFunc { 10 | pub fn new() -> BlackholeSinkFunc { 11 | BlackholeSinkFunc {} 12 | } 13 | } 14 | 15 | #[async_trait] 16 | impl ArrowOperator for BlackholeSinkFunc { 17 | fn name(&self) -> String { 18 | "BlackholeSink".to_string() 19 | } 20 | 21 | async fn process_batch( 22 | &mut self, 23 | _: RecordBatch, 24 | _: &mut OperatorContext, 25 | _: &mut dyn Collector, 26 | ) { 27 | // no-op 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/confluent/confluent.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/filesystem/filesystem.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/filesystem/sink/arrow.rs: -------------------------------------------------------------------------------- 1 | 2 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/fluvio/fluvio.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/fluvio/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "FluvioTable", 4 | "properties": { 5 | "endpoint": { 6 | "title": "Endpoint", 7 | "type": "string", 8 | "description": "The Fluvio endpoint to connect to; leave blank to use default" 9 | }, 10 | "topic": { 11 | "title": "Topic", 12 | "type": "string", 13 | "description": "The Fluvio topic to use for this table" 14 | }, 15 | "type": { 16 | "type": "object", 17 | "title": "Table Type", 18 | "oneOf": [ 19 | { 20 | "type": "object", 21 | "title": "Source", 22 | "properties": { 23 | "offset": { 24 | "type": "string", 25 | "description": "The offset to start reading from", 26 | "enum": [ 27 | "earliest", 28 | "latest" 29 | ] 30 | } 31 | }, 32 | "required": [ 33 | "offset" 34 | ], 35 | "additionalProperties": false 36 | }, 37 | { 38 | "type": "object", 39 | "title": "Sink", 40 | "properties": {}, 41 | "additionalProperties": false 42 | } 43 | ] 44 | } 45 | }, 46 | "required": [ 47 | "topic", 48 | "type" 49 | ] 50 | } 51 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/impulse/impulse.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/impulse/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "ImpulseTable", 4 | "properties": { 5 | "event_rate": { 6 | "title": "Event rate (messages / sec)", 7 | "type": "number", 8 | "description": "The number of messages the impulse source will emit per second", 9 | "examples": [ 10 | "100" 11 | ], 12 | "minimum": 0 13 | }, 14 | "event_time_interval": { 15 | "title": "Event time interval (µs)", 16 | "type": "integer", 17 | "description": "The number of microseconds in between the event times of subsequent events emmitted by the source; if not set wall-clock time is used" 18 | }, 19 | "message_count": { 20 | "title": "Message count", 21 | "type": "integer", 22 | "description": "The number of messages the impulse source will emit before stopping; if not set the source will run forever" 23 | } 24 | }, 25 | "required": [ 26 | "event_rate" 27 | ] 28 | } 29 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/kinesis/kinesis.svg: -------------------------------------------------------------------------------- 1 | 2 | 3 | Icon-Architecture/48/Arch_Amazon-Kinesis_48 4 | 5 | 6 | 7 | 8 | 9 | 10 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/mqtt/mqtt.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/mqtt/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "MqttTable", 4 | "properties": { 5 | "topic": { 6 | "title": "Topic", 7 | "type": "string", 8 | "description": "The MQTT topic to use for this table" 9 | }, 10 | "qos": { 11 | "type": "string", 12 | "title": "Quality of Service", 13 | "description": "The Quality of Service to use for this topic", 14 | "enum": ["AtMostOnce", "AtLeastOnce", "ExactlyOnce"] 15 | }, 16 | "type": { 17 | "type": "object", 18 | "title": "Table Type", 19 | "oneOf": [ 20 | { 21 | "type": "object", 22 | "title": "Source", 23 | "additionalProperties": false, 24 | "properties": {} 25 | }, 26 | { 27 | "type": "object", 28 | "title": "Sink", 29 | "properties": { 30 | "retain": { 31 | "type": "boolean", 32 | "title": "Retain", 33 | "description": "Whether to retain messages published to this topic" 34 | } 35 | }, 36 | "required": ["retain"], 37 | "additionalProperties": false 38 | } 39 | ] 40 | } 41 | }, 42 | "required": ["topic", "type"] 43 | } 44 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/nats/nats.svg: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/nexmark/nexmark.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/nexmark/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "NexmarkTable", 4 | "properties": { 5 | "event_rate": { 6 | "title": "Event rate (messages / sec)", 7 | "type": "number", 8 | "description": "The number of messages the Nexmark source will emit per second", 9 | "examples": ["100"], 10 | "minimum": 0 11 | }, 12 | "runtime": { 13 | "title": "Runtime (seconds)", 14 | "type": "number", 15 | "description": "If set, the source will finish after running for this many seconds" 16 | } 17 | }, 18 | "required": [ 19 | "event_rate" 20 | ] 21 | } 22 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/nexmark/test.rs: -------------------------------------------------------------------------------- 1 | use crate::nexmark::operator::{GeneratorConfig, NexmarkConfig, NexmarkGenerator}; 2 | use rand::{rngs::SmallRng, SeedableRng}; 3 | use std::{borrow::BorrowMut, time::SystemTime}; 4 | 5 | #[tokio::test] 6 | async fn test_nexmark() { 7 | let mut generator = NexmarkGenerator::new(10000.0, None); 8 | let mut random = SmallRng::seed_from_u64(1); 9 | for _ in 0..10000 { 10 | generator.next_event(random.borrow_mut()); 11 | } 12 | } 13 | 14 | #[test] 15 | fn test_auction_generation() { 16 | let mut random = SmallRng::seed_from_u64(1); 17 | let generator = NexmarkGenerator::new(10000.0, None); 18 | let _result = generator 19 | .generator_config 20 | .next_base0_auction_id(7, &mut random); 21 | } 22 | 23 | #[test] 24 | fn test_exit() { 25 | let mut random = SmallRng::seed_from_u64(1); 26 | let config = GeneratorConfig::new( 27 | NexmarkConfig::new(10000.0, Some(100), 1), 28 | SystemTime::now(), 29 | 1, 30 | Some(100), 31 | 1, 32 | ); 33 | let subconfig = config.split(8)[4].clone(); 34 | let mut generator = NexmarkGenerator::from_config(&subconfig, 0); 35 | while generator.has_next() { 36 | generator.next_event(&mut random); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/polling_http/http.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/polling_http/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "PollingHTTPTable", 4 | "properties": { 5 | "endpoint": { 6 | "title": "Endpoint", 7 | "type": "string", 8 | "description": "The endpoint to connect to", 9 | "examples": ["https://example.com:8080/sse"], 10 | "format": "uri" 11 | }, 12 | "headers": { 13 | "title": "Headers", 14 | "type": "string", 15 | "description": "Comma separated list of headers to send with the request", 16 | "examples": ["Authentication: digest 1234,Content-Type: application/json"], 17 | "format": "var-str" 18 | }, 19 | "method": { 20 | "title": "Method", 21 | "type": "string", 22 | "description": "HTTP method to use for the request", 23 | "enum": [ 24 | "GET", 25 | "POST", 26 | "PUT", 27 | "PATCH" 28 | ], 29 | "examples": [ 30 | "GET" 31 | ] 32 | }, 33 | "body": { 34 | "title": "Body", 35 | "type": "string", 36 | "description": "An optional body to send along with the request" 37 | }, 38 | "poll_interval_ms": { 39 | "title": "Polling Interval (ms)", 40 | "type": "integer", 41 | "description": "Number of milliseconds to wait between successful polls of the HTTP endpoint", 42 | "examples": [ 43 | "1000" 44 | ] 45 | }, 46 | "emit_behavior": { 47 | "title": "Emit Behavior", 48 | "type": "string", 49 | "description": "Controls whether all polls produce events or only those that differ from the previous poll", 50 | "enum": [ 51 | "all", 52 | "changed" 53 | ] 54 | } 55 | }, 56 | "required": [ 57 | "endpoint" 58 | ] 59 | } 60 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/rabbitmq/rabbitmq.svg: -------------------------------------------------------------------------------- 1 | 2 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/rabbitmq/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "RabbitmqStreamTable", 4 | "properties": { 5 | "stream": { 6 | "title": "Stream", 7 | "type": "string", 8 | "description": "The RabbitMQ stream to use for this table" 9 | }, 10 | "type": { 11 | "type": "object", 12 | "title": "Table Type", 13 | "oneOf": [ 14 | { 15 | "type": "object", 16 | "title": "Source", 17 | "properties": { 18 | "offset": { 19 | "type": "string", 20 | "description": "The offset to start reading from", 21 | "enum": ["first", "last", "next"] 22 | } 23 | }, 24 | "required": ["offset"], 25 | "additionalProperties": false 26 | }, 27 | { 28 | "type": "object", 29 | "title": "Sink", 30 | "properties": {}, 31 | "additionalProperties": false 32 | } 33 | ] 34 | } 35 | }, 36 | "required": ["stream", "type"] 37 | } 38 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/single_file/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "SingleFileTable", 4 | "properties": { 5 | "path": { 6 | "type": "string", 7 | "title": "Path" 8 | }, 9 | "table_type": { 10 | "type": "string", 11 | "title": "Table Type", 12 | "description": "Whether it is a source or a sink", 13 | "enum": [ 14 | "source", 15 | "sink" 16 | ] 17 | }, 18 | "wait_for_control": { 19 | "type": "boolean", 20 | "title": "Wait for control message", 21 | "description": "Whether to wait for a control message after each line" 22 | } 23 | }, 24 | "required": ["path", "table_type"], 25 | "additionalProperties": false 26 | } 27 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/sse/sse.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/sse/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "SSETable", 4 | "properties": { 5 | "endpoint": { 6 | "title": "Endpoint", 7 | "type": "string", 8 | "description": "The endpoint to connect to", 9 | "examples": ["https://example.com:8080/sse"], 10 | "format": "uri" 11 | }, 12 | "headers": { 13 | "title": "Headers", 14 | "type": "string", 15 | "description": "Comma separated list of headers to send with the request", 16 | "examples": ["Authentication: digest 1234,Content-Type: application/json"], 17 | "format": "var-str" 18 | }, 19 | "events": { 20 | "title": "Events", 21 | "type": "string", 22 | "description": "Comma separated list of events to listen for", 23 | "examples": ["event1,event2,event3"] 24 | } 25 | }, 26 | "required": [ 27 | "endpoint" 28 | ] 29 | } 30 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/stdout/operator.rs: -------------------------------------------------------------------------------- 1 | use arrow::array::RecordBatch; 2 | use arroyo_formats::ser::ArrowSerializer; 3 | use tokio::io::{AsyncWriteExt, BufWriter, Stdout}; 4 | 5 | use arroyo_operator::context::{Collector, OperatorContext}; 6 | use arroyo_operator::operator::ArrowOperator; 7 | use arroyo_types::SignalMessage; 8 | 9 | pub struct StdoutSink { 10 | pub stdout: BufWriter, 11 | pub serializer: ArrowSerializer, 12 | } 13 | 14 | #[async_trait::async_trait] 15 | impl ArrowOperator for StdoutSink { 16 | fn name(&self) -> String { 17 | "Stdout".to_string() 18 | } 19 | 20 | async fn process_batch( 21 | &mut self, 22 | batch: RecordBatch, 23 | _: &mut OperatorContext, 24 | _: &mut dyn Collector, 25 | ) { 26 | for value in self.serializer.serialize(&batch) { 27 | self.stdout.write_all(&value).await.unwrap(); 28 | self.stdout.write_u8(b'\n').await.unwrap(); 29 | } 30 | self.stdout.flush().await.unwrap(); 31 | } 32 | 33 | async fn on_close( 34 | &mut self, 35 | _: &Option, 36 | _: &mut OperatorContext, 37 | _: &mut dyn Collector, 38 | ) { 39 | self.stdout.flush().await.unwrap(); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/stdout/stdout.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/webhook/table.json: -------------------------------------------------------------------------------- 1 | { 2 | "type": "object", 3 | "title": "WebhookTable", 4 | "properties": { 5 | "endpoint": { 6 | "title": "Endpoint", 7 | "type": "string", 8 | "description": "The endpoint that should receive the webhook", 9 | "examples": [ 10 | "https://yourdomain.com/api/v1/webhooks" 11 | ], 12 | "format": "var-str" 13 | }, 14 | "headers": { 15 | "title": "Headers", 16 | "type": "string", 17 | "description": "Optional, comma separated list of headers to send with the webhook", 18 | "examples": [ 19 | "Authentication: Basic my-auth-secret,Content-Type: application/json" 20 | ], 21 | "format": "var-str" 22 | } 23 | }, 24 | "required": [ 25 | "endpoint" 26 | ] 27 | } 28 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/webhook/webhook.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-connectors/src/websocket/websocket.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /crates/arroyo-controller/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-controller" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | [features] 7 | default = [] 8 | 9 | [dependencies] 10 | arroyo-types = { path = "../arroyo-types" } 11 | arroyo-datastream = { path = "../arroyo-datastream" } 12 | arroyo-rpc = { path = "../arroyo-rpc" } 13 | arroyo-state = { path = "../arroyo-state" } 14 | arroyo-server-common = { path = "../arroyo-server-common" } 15 | arroyo-worker = { path = "../arroyo-worker" } 16 | 17 | tonic = {workspace = true} 18 | 19 | prost = {workspace = true} 20 | tokio = { version = "1", features = ["full"] } 21 | tokio-stream = "0.1.12" 22 | rand = { workspace = true } 23 | petgraph = {workspace = true, features = ["serde-1"]} 24 | prometheus = {workspace = true} 25 | async-trait = "0.1" 26 | lazy_static = "1.4.0" 27 | 28 | serde = "1" 29 | 30 | anyhow = "1.0.70" 31 | 32 | # Kubernetes 33 | kube = { version = "0.99", features = ["runtime", "derive"] } 34 | k8s-openapi = { workspace = true, features = ["v1_30"] } 35 | shlex = "1.3" 36 | 37 | # json-schema support 38 | serde_json = "1.0" 39 | 40 | # logging 41 | tracing = "0.1" 42 | 43 | # SQL 44 | postgres-types = { version = "*", features = ["derive"] } 45 | tokio-postgres = { version = "*", features = ["with-serde_json-1", "with-time-0_3", "with-uuid-1"] } 46 | futures = "0.3" 47 | time = "0.3" 48 | cornucopia_async = { workspace = true, features = ["with-serde_json-1"] } 49 | thiserror = "2" 50 | regex = "1.7.3" 51 | rusqlite = { workspace = true, features = ["serde_json", "time"] } 52 | 53 | [build-dependencies] 54 | cornucopia = { workspace = true } 55 | postgres = "0.19.5" 56 | arroyo-types = { path = "../arroyo-types" } 57 | rusqlite = { workspace = true } 58 | refinery = { version = "0.8.14", features = ["rusqlite"] } 59 | -------------------------------------------------------------------------------- /crates/arroyo-controller/build.rs: -------------------------------------------------------------------------------- 1 | use cornucopia::{CodegenSettings, Error}; 2 | use postgres::{Client, NoTls}; 3 | 4 | fn main() -> Result<(), Error> { 5 | let queries_path = "queries"; 6 | let destination = format!("{}/controller-sql.rs", std::env::var("OUT_DIR").unwrap()); 7 | let settings = CodegenSettings { 8 | gen_async: true, 9 | derive_ser: false, 10 | gen_sync: false, 11 | gen_sqlite: true, 12 | }; 13 | 14 | println!("cargo:rerun-if-changed={queries_path}"); 15 | println!("cargo:rerun-if-changed=../arroyo-api/migrations"); 16 | println!("cargo:rerun-if-changed=../arroyo-api/sqlite_migrations"); 17 | 18 | let mut client = match std::env::var("DATABASE_URL") { 19 | Ok(database_url) => Client::connect(&database_url, NoTls) 20 | .unwrap_or_else(|e| panic!("Failed to connect to database: {}", e)), 21 | Err(_) => Client::configure() 22 | .dbname("arroyo") 23 | .host("localhost") 24 | .port(5432) 25 | .user("arroyo") 26 | .password("arroyo") 27 | .connect(NoTls) 28 | .unwrap_or_else(|e| panic!("Failed to connect to default database: {}", e)), 29 | }; 30 | 31 | let mut sqlite = 32 | rusqlite::Connection::open_in_memory().expect("Couldn't open sqlite memory connection"); 33 | let migrations = refinery::load_sql_migrations("../arroyo-api/sqlite_migrations").unwrap(); 34 | refinery::Runner::new(&migrations) 35 | .run(&mut sqlite) 36 | .expect("Failed to run migrations"); 37 | 38 | cornucopia::generate_live_with_sqlite( 39 | &mut client, 40 | queries_path, 41 | Some(&destination), 42 | &sqlite, 43 | settings, 44 | )?; 45 | 46 | Ok(()) 47 | } 48 | -------------------------------------------------------------------------------- /crates/arroyo-controller/src/states/compiling.rs: -------------------------------------------------------------------------------- 1 | use crate::states::StateError; 2 | 3 | use super::{scheduling::Scheduling, JobContext, State, Transition}; 4 | 5 | #[derive(Debug)] 6 | pub struct Compiling; 7 | 8 | #[async_trait::async_trait] 9 | impl State for Compiling { 10 | fn name(&self) -> &'static str { 11 | "Compiling" 12 | } 13 | 14 | async fn next(self: Box, _ctx: &mut JobContext) -> Result { 15 | return Ok(Transition::next(*self, Scheduling {})); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /crates/arroyo-controller/src/states/finishing.rs: -------------------------------------------------------------------------------- 1 | use crate::states::StateError; 2 | 3 | use super::{Finished, JobContext, State, Transition}; 4 | 5 | #[derive(Debug)] 6 | pub struct Finishing {} 7 | 8 | #[async_trait::async_trait] 9 | impl State for Finishing { 10 | fn name(&self) -> &'static str { 11 | "Finishing" 12 | } 13 | 14 | async fn next(mut self: Box, ctx: &mut JobContext) -> Result { 15 | if let Err(e) = ctx 16 | .job_controller 17 | .as_mut() 18 | .unwrap() 19 | .wait_for_finish(ctx.rx) 20 | .await 21 | { 22 | return Err(ctx.retryable(self, "failed while waiting for job to finish", e, 10)); 23 | } 24 | 25 | Ok(Transition::next(*self, Finished {})) 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /crates/arroyo-datastream/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-datastream" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | 7 | [dependencies] 8 | arroyo-rpc = { path = "../arroyo-rpc" } 9 | arrow-schema = { workspace = true, features = ["serde"] } 10 | 11 | petgraph = {workspace = true, features = ["serde-1"]} 12 | serde = {version = "1", features = ["derive"]} 13 | syn = {version = "2", features = ["full"]} 14 | bincode = { version = "2.0.0-rc.3", features = ["serde"]} 15 | rand = { workspace = true } 16 | anyhow = "1.0.70" 17 | prost = {workspace = true} 18 | serde_json = "1.0.108" 19 | strum = { version = "0.27", features = ["derive"] } 20 | datafusion-proto = { workspace = true } 21 | itertools = "0.14" 22 | -------------------------------------------------------------------------------- /crates/arroyo-formats/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-formats" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | [dependencies] 7 | arroyo-types = { path = "../arroyo-types" } 8 | arroyo-rpc = { path = "../arroyo-rpc" } 9 | 10 | apache-avro = {workspace = true} 11 | serde = {version = "1.0", features = ["derive"]} 12 | serde_json = "1.0" 13 | arrow = { workspace = true } 14 | arrow-schema = { workspace = true } 15 | arrow-array = { workspace = true} 16 | arrow-json = { workspace = true } 17 | tokio = { version = "1", features = ["full"] } 18 | tracing = "0.1" 19 | anyhow = "1" 20 | memchr = "2" 21 | typify = "0.0.13" 22 | schemars = "0.8" 23 | prost = { workspace = true} 24 | prost-reflect = { workspace = true} 25 | prost-build = { workspace = true } 26 | base64 = "0.22.1" 27 | uuid = { version = "1.10.0", features = ["v4"] } 28 | regex = "1.10.6" 29 | integer-encoding = "4.0.2" 30 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/avro/mod.rs: -------------------------------------------------------------------------------- 1 | pub mod de; 2 | pub mod schema; 3 | pub mod ser; 4 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/lib.rs: -------------------------------------------------------------------------------- 1 | use arroyo_rpc::config::config; 2 | use serde_json::{json, Value}; 3 | use std::time::Instant; 4 | 5 | pub mod avro; 6 | pub mod json; 7 | 8 | pub mod de; 9 | pub mod proto; 10 | pub mod ser; 11 | 12 | pub fn should_flush(size: usize, time: Instant) -> bool { 13 | size > 0 14 | && (size >= config().pipeline.source_batch_size 15 | || time.elapsed() >= *config().pipeline.source_batch_linger) 16 | } 17 | 18 | pub(crate) fn float_to_json(f: f64) -> Value { 19 | match serde_json::Number::from_f64(f) { 20 | Some(n) => Value::Number(n), 21 | None => Value::String( 22 | (if f.is_infinite() && f.is_sign_positive() { 23 | "+Inf" 24 | } else if f.is_infinite() { 25 | "-Inf" 26 | } else { 27 | "NaN" 28 | }) 29 | .to_string(), 30 | ), 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/mod.rs: -------------------------------------------------------------------------------- 1 | pub mod de; 2 | pub mod schema; 3 | #[cfg(test)] 4 | mod test; 5 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/my-policy.json: -------------------------------------------------------------------------------- 1 | { 2 | "Version": "2012-10-17", 3 | "Statement": [ 4 | { 5 | "Effect": "Allow", 6 | "Action": "s3:*", 7 | "Resource": [ 8 | "arn:aws:s3:::arroyo-testing", 9 | "arn:aws:s3:::arroyo-testing/*" 10 | ] 11 | } 12 | ] 13 | } 14 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/protos/basic_types.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | message TestBasicTypes { 4 | bool bool_field = 1; 5 | int32 int32_field = 2; 6 | int64 int64_field = 3; 7 | uint32 uint32_field = 4; 8 | uint64 uint64_field = 5; 9 | float float_field = 6; 10 | double double_field = 7; 11 | } 12 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/protos/enum_fields.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | message TestEnumFields { 4 | enum TestEnum { 5 | UNKNOWN = 0; 6 | VALUE1 = 1; 7 | VALUE2 = 2; 8 | } 9 | 10 | TestEnum enum_field = 1; 11 | } -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/protos/map_fields.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | message TestMapFields { 4 | message NestedMessage { 5 | int32 inner_field = 1; 6 | } 7 | 8 | map int_to_string_map = 1; 9 | map string_to_message_map = 2; 10 | } 11 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/protos/nested_message.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | message TestNestedMessage { 4 | message NestedMessage { 5 | int32 inner_field = 1; 6 | } 7 | 8 | message DoubleNestedMessage { 9 | NestedMessage inner_nested = 1; 10 | } 11 | 12 | NestedMessage nested_field = 1; 13 | DoubleNestedMessage double_nested_field = 2; 14 | } 15 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/protos/orders.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package online_store; 4 | 5 | // Represents an order in the online store. 6 | message Order { 7 | string order_id = 1; 8 | string customer_id = 2; 9 | repeated OrderItem items = 3; 10 | string order_status = 4; 11 | double total_amount = 5; 12 | string currency = 6; 13 | PaymentDetails payment = 7; 14 | proto_common.Address shipping_address = 8; 15 | proto_common.Address billing_address = 9; 16 | } 17 | 18 | // Represents an item in the order. 19 | message OrderItem { 20 | string product_id = 1; 21 | string product_name = 2; 22 | int32 quantity = 3; 23 | double price_per_unit = 4; 24 | } 25 | 26 | // Represents payment details. 27 | message PaymentDetails { 28 | string payment_method = 1; 29 | string transaction_id = 2; 30 | } 31 | 32 | import "address.proto"; 33 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/protos/repeated_fields.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | message TestRepeatedFields { 4 | repeated int32 repeated_int = 1; 5 | repeated string repeated_string = 2; 6 | } 7 | -------------------------------------------------------------------------------- /crates/arroyo-formats/src/proto/test/protos/string_and_bytes.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | message TestStringAndBytes { 4 | string string_field = 1; 5 | bytes bytes_field = 2; 6 | } 7 | -------------------------------------------------------------------------------- /crates/arroyo-metrics/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-metrics" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | [dependencies] 7 | arroyo-types = { path = "../arroyo-types" } 8 | prometheus = { workspace = true, features = ["process"] } 9 | lazy_static = "1.4.0" 10 | -------------------------------------------------------------------------------- /crates/arroyo-node/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-node" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | [dependencies] 7 | arroyo-types = { path = "../arroyo-types" } 8 | arroyo-rpc = { path = "../arroyo-rpc" } 9 | arroyo-server-common = { path = "../arroyo-server-common" } 10 | 11 | tonic = { workspace = true } 12 | tokio = { version = "1", features = ["full"] } 13 | tracing = "0.1" 14 | rand = { workspace = true } 15 | local-ip-address = "0.6" 16 | lazy_static = "1.4.0" 17 | prometheus = { workspace = true } 18 | anyhow = "1.0.72" -------------------------------------------------------------------------------- /crates/arroyo-openapi/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-openapi" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | [dependencies] 7 | progenitor-client = { git = "https://github.com/oxidecomputer/progenitor" } 8 | reqwest = { version = "0.12", features = ["json", "stream"] } 9 | serde = { version = "1.0", features = ["derive"] } 10 | serde_json = "1.0" 11 | 12 | [build-dependencies] 13 | prettyplease = "0.2" 14 | progenitor = { version = '0.8' } 15 | serde_json = "1.0" 16 | syn = "2" 17 | 18 | utoipa = {workspace = true} 19 | arroyo-api = { path = "../arroyo-api" } -------------------------------------------------------------------------------- /crates/arroyo-openapi/build.rs: -------------------------------------------------------------------------------- 1 | use arroyo_api::ApiDoc; 2 | use progenitor::{GenerationSettings, InterfaceStyle}; 3 | use std::path::PathBuf; 4 | use std::{env, fs}; 5 | use utoipa::OpenApi; 6 | 7 | fn main() { 8 | // Generate the OpenAPI spec 9 | let api_spec = "../../target/api-spec.json"; 10 | 11 | let doc = ApiDoc::openapi().to_pretty_json().unwrap(); 12 | fs::write(api_spec, &doc).unwrap(); 13 | 14 | let spec = serde_json::from_str(&doc).unwrap(); 15 | let mut settings = GenerationSettings::new(); 16 | settings.with_interface(InterfaceStyle::Builder); 17 | let mut generator = progenitor::Generator::new(&settings); 18 | 19 | let tokens = generator.generate_tokens(&spec).unwrap(); 20 | let ast = syn::parse2(tokens).unwrap(); 21 | let content = prettyplease::unparse(&ast); 22 | 23 | let generated = PathBuf::from(env::var("OUT_DIR").unwrap()).join("generated"); 24 | fs::create_dir_all(&generated).unwrap(); 25 | fs::write(generated.join("api-client.rs"), content).unwrap(); 26 | } 27 | -------------------------------------------------------------------------------- /crates/arroyo-openapi/src/lib.rs: -------------------------------------------------------------------------------- 1 | // build.rs generates the OpenAPI spec 2 | #![allow(non_camel_case_types)] 3 | 4 | include!(concat!(env!("OUT_DIR"), "/generated/api-client.rs")); 5 | -------------------------------------------------------------------------------- /crates/arroyo-operator/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-operator" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html 7 | 8 | [dependencies] 9 | arroyo-formats = { path = "../arroyo-formats" } 10 | arroyo-metrics = { path = "../arroyo-metrics" } 11 | arroyo-rpc = { path = "../arroyo-rpc" } 12 | arroyo-state = { path = "../arroyo-state" } 13 | arroyo-types = { path = "../arroyo-types" } 14 | arroyo-datastream = { path = "../arroyo-datastream" } 15 | arroyo-storage = { path = "../arroyo-storage" } 16 | arroyo-udf-host = { path = "../arroyo-udf/arroyo-udf-host" } 17 | arroyo-udf-python = { path = "../arroyo-udf/arroyo-udf-python" } 18 | 19 | anyhow = "1.0.71" 20 | arrow = { workspace = true, features = ["ffi"] } 21 | async-trait = "0.1.68" 22 | bincode = "2.0.0-rc.3" 23 | datafusion = { workspace = true } 24 | futures = "0.3" 25 | prost = {workspace = true} 26 | rand = { workspace = true } 27 | tokio = { version = "1", features = ["full", "tracing"] } 28 | tokio-stream = { version = "0.1", features = ["full"] } 29 | tracing = "0.1" 30 | async-stream = "0.3.5" 31 | serde_json = "1.0.111" 32 | serde = "1.0.195" 33 | dlopen2 = "0.7.0" 34 | -------------------------------------------------------------------------------- /crates/arroyo-planner/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "arroyo-planner" 3 | version = "0.15.0-dev" 4 | edition = "2021" 5 | 6 | 7 | [dependencies] 8 | arroyo-types = { path = "../arroyo-types" } 9 | arroyo-rpc = {path = "../arroyo-rpc"} 10 | arroyo-datastream = { path = "../arroyo-datastream" } 11 | arroyo-connectors = { path = "../arroyo-connectors" } 12 | arroyo-operator = { path = "../arroyo-operator" } 13 | arroyo-udf-host = { path = "../arroyo-udf/arroyo-udf-host" } 14 | arroyo-udf-python = { path = "../arroyo-udf/arroyo-udf-python" } 15 | 16 | datafusion = { workspace = true } 17 | datafusion-proto = { workspace = true } 18 | datafusion-functions = { workspace = true } 19 | datafusion-functions-json = { workspace = true } 20 | 21 | prost = {workspace = true} 22 | arrow-schema = {workspace = true, features = ["serde"] } 23 | serde = {version = "1", features = ["derive"]} 24 | serde_json = "1" 25 | 26 | petgraph = { workspace = true } 27 | tokio = "1.27" 28 | tokio-stream = { version = "0.1", features = ["full"] } 29 | futures = "0.3" 30 | arrow = { workspace = true, features = ["ffi"] } 31 | arrow-array = { workspace = true} 32 | anyhow = {version = "1.0.70", features = ["backtrace"]} 33 | async-trait = "0.1" 34 | 35 | syn = {version = "2", features = ["full", "parsing", "extra-traits"]} 36 | tracing = "0.1.37" 37 | 38 | serde_json_path = "0.7" 39 | unicase = "2.7.0" 40 | xxhash-rust = { version = "0.8.12", features = ["xxh3", "std"] } 41 | itertools = "0.14.0" 42 | 43 | sqlparser = { workspace = true} 44 | 45 | [dev-dependencies] 46 | test-log = {version = "0.2.15", default-features = false, features = ["trace"]} 47 | rstest = { version = "0.25" } 48 | 49 | [build-dependencies] 50 | glob = "0.3.1" 51 | -------------------------------------------------------------------------------- /crates/arroyo-planner/build.rs: -------------------------------------------------------------------------------- 1 | fn main() -> Result<(), String> { 2 | // recursively find all json files in the src directory 3 | glob::glob("src/test/{queries,udf}/*") 4 | .unwrap() 5 | .filter_map(Result::ok) 6 | .for_each(|path| { 7 | println!("cargo:rerun-if-changed={}", path.display()); 8 | }); 9 | 10 | Ok(()) 11 | } 12 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/external.rs: -------------------------------------------------------------------------------- 1 | use arrow_schema::FieldRef; 2 | use arroyo_rpc::grpc::api::ConnectorOp; 3 | use std::time::Duration; 4 | 5 | #[derive(Clone, Debug, Eq, PartialEq)] 6 | pub enum ProcessingMode { 7 | Append, 8 | Update, 9 | } 10 | 11 | #[derive(Clone, Debug)] 12 | pub struct SqlSource { 13 | pub id: Option, 14 | pub struct_def: Vec, 15 | pub config: ConnectorOp, 16 | pub processing_mode: ProcessingMode, 17 | pub idle_time: Option, 18 | } 19 | 20 | #[derive(Clone, Debug, PartialEq, Eq)] 21 | pub enum SinkUpdateType { 22 | Allow, 23 | Disallow, 24 | Force, 25 | } 26 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/analytics_ingest.sql: -------------------------------------------------------------------------------- 1 | -- from https://www.arroyo.dev/blog/building-a-real-time-data-lake 2 | create table events ( 3 | id TEXT PRIMARY KEY, 4 | timestamp TIMESTAMP NOT NULL, 5 | ip TEXT, 6 | user_id TEXT, 7 | platform TEXT, 8 | app_version TEXT, 9 | type TEXT NOT NULL, 10 | properties JSON 11 | ) with ( 12 | connector = 'kafka', 13 | bootstrap_servers = 'localhost:9092', 14 | topic = 'analytics', 15 | format = 'json', 16 | type = 'source' 17 | ); 18 | 19 | create table account_created_sink with ( 20 | connector = 'delta', 21 | path = 's3://my-s3-bucket/data/account_created', 22 | format = 'parquet', 23 | 'filename.strategy' = 'uuid', 24 | 'parquet.compression' = 'zstd', 25 | time_partition_pattern = '%Y/%m/%d/%H', 26 | rollover_seconds = 6000 27 | ); 28 | 29 | INSERT INTO account_created_sink 30 | SELECT 31 | id, 32 | timestamp, 33 | ip, 34 | user_id, 35 | platform, 36 | app_version, 37 | properties->>'signup_method' as signup_method, 38 | properties->>'campaign_id' as campaign_id, 39 | properties->>'product' as product 40 | FROM events 41 | WHERE type = 'account_created'; 42 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/async_udf_subquery.sql: -------------------------------------------------------------------------------- 1 | create table logs ( 2 | ip TEXT 3 | ) with ( 4 | connector = 'kafka', 5 | bootstrap_servers = 'localhost:9092', 6 | format = 'json', 7 | type = 'source', 8 | topic ='logs', 9 | 'source.offset' = 'latest' 10 | ); 11 | 12 | SELECT count(*) as count, city, hop(interval '5 seconds', interval '15 minutes') as window 13 | FROM ( 14 | select get_city(logs.ip) as city 15 | FROM logs 16 | ) 17 | where city is not null 18 | group by city, window; 19 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/basic_tumble_aggregate.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE nexmark WITH ( 2 | connector = 'nexmark', 3 | event_rate = 10 4 | ); 5 | 6 | SELECT 7 | bid.auction as auction, 8 | tumble(INTERVAL '1' second) as window, 9 | count(*) as count 10 | FROM 11 | nexmark 12 | where 13 | bid is not null 14 | GROUP BY 15 | 1, 16 | 2 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/bitcoin_exchange_rate.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE coinbase ( 2 | type TEXT, 3 | price TEXT 4 | ) WITH ( 5 | connector = 'websocket', 6 | endpoint = 'wss://ws-feed.exchange.coinbase.com', 7 | subscription_message = '{ 8 | "type": "subscribe", 9 | "product_ids": [ 10 | "BTC-USD" 11 | ], 12 | "channels": ["ticker"] 13 | }', 14 | format = 'json' 15 | ); 16 | 17 | SELECT avg(CAST(price as FLOAT)) from coinbase 18 | WHERE type = 'ticker' 19 | GROUP BY hop(interval '5' second, interval '1 minute'); -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/bitcoin_stats.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE coinbase ( 2 | type TEXT, 3 | product_id TEXT, 4 | price FLOAT 5 | ) WITH ( 6 | connector = 'websocket', 7 | endpoint = 'wss://ws-feed.exchange.coinbase.com', 8 | subscription_message = '{ 9 | "type": "subscribe", 10 | "product_ids": [ 11 | "BTC-USD" 12 | ], 13 | "channels": ["ticker"] 14 | }', 15 | format = 'json' 16 | ); 17 | 18 | create view stats as ( 19 | SELECT 20 | tumble('5 seconds') as window, 21 | product_id, 22 | first_value(price) AS open, 23 | max(price) AS high, 24 | min(price) AS low, 25 | last_value(price) AS close 26 | FROM 27 | coinbase 28 | GROUP BY 29 | 1, 2); 30 | 31 | select window.start, product_id, open, high, low, close 32 | from stats 33 | where product_id is not null; 34 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/bluesky_trends.sql: -------------------------------------------------------------------------------- 1 | create table firesky ( 2 | value BYTEA 3 | ) with ( 4 | connector = 'websocket', 5 | endpoint = 'wss://firesky.tv/ws/app', 6 | format = 'raw_bytes' 7 | ); 8 | 9 | create view tags as ( 10 | select unnest(extract_json(cbor_to_json(value), 11 | '$.info.post.facets[*].features[*].tag')) as tag 12 | from firesky); 13 | 14 | 15 | SELECT * FROM ( 16 | SELECT *, ROW_NUMBER() OVER ( 17 | PARTITION BY window 18 | ORDER BY count DESC) as row_num 19 | FROM (SELECT count(*) as count, 20 | tag, 21 | hop(interval '5 seconds', interval '15 minutes') as window 22 | FROM tags 23 | group by tag, window)) WHERE row_num <= 5; 24 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/create_table_updating.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE nexmark ( 2 | auction bigint, 3 | bidder bigint, 4 | price bigint, 5 | channel text, 6 | url text, 7 | datetime timestamp, 8 | extra text 9 | ) WITH ( 10 | connector = 'filesystem', 11 | format = 'parquet', 12 | type = 'source', 13 | path = '/home/data', 14 | 'source.regex-pattern' = '00001-000.parquet', 15 | event_time_field = datetime 16 | ); 17 | 18 | CREATE TABLE counts as (SELECT count(*) FROM nexmark); 19 | 20 | SELECT * FROM counts -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/crypto_function.sql: -------------------------------------------------------------------------------- 1 | create table impulse with ( 2 | connector = 'impulse', 3 | event_rate = '10' 4 | ); 5 | 6 | select sha256(cast(counter as TEXT)) 7 | from impulse; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/custom_ttls.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE mastodon ( 2 | value TEXT 3 | ) WITH ( 4 | connector = 'sse', 5 | format = 'raw_string', 6 | endpoint = 'http://mastodon.arroyo.dev/api/v1/streaming/public', 7 | events = 'update' 8 | ); 9 | 10 | CREATE VIEW tags AS ( 11 | SELECT tag FROM ( 12 | SELECT extract_json_string(value, '$.tags[*].name') AS tag 13 | FROM mastodon) 14 | WHERE tag is not null 15 | ); 16 | 17 | set updating_ttl = interval '5' second; 18 | 19 | select count(distinct tag) from tags; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_lookup_join_non_primary_key.sql: -------------------------------------------------------------------------------- 1 | --fail=the right-side of a look-up join condition must be a PRIMARY KEY column, but 'value' is not 2 | create table impulse with ( 3 | connector = 'impulse', 4 | event_rate = '2' 5 | ); 6 | 7 | create temporary table lookup ( 8 | key TEXT METADATA FROM 'key' PRIMARY KEY, 9 | value TEXT, 10 | len INT 11 | ) with ( 12 | connector = 'redis', 13 | format = 'raw_string', 14 | address = 'redis://localhost:6379', 15 | format = 'json', 16 | 'lookup.cache.max_bytes' = 100000 17 | ); 18 | 19 | select A.counter, B.key, B.value, len 20 | from impulse A inner join lookup B 21 | on cast((A.counter % 10) as TEXT) = B.value; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_memory_table_double_write.sql: -------------------------------------------------------------------------------- 1 | --fail=Can only insert into a memory table once 2 | CREATE TABLE cars ( 3 | timestamp TIMESTAMP, 4 | driver_id BIGINT, 5 | event_type TEXT, 6 | location TEXT 7 | ) WITH ( 8 | connector = 'single_file', 9 | path = '$input_dir/cars.json', 10 | format = 'json', 11 | type = 'source' 12 | ); 13 | 14 | CREATE TABLE memory ( 15 | event_type TEXT, 16 | location TEXT, 17 | driver_id BIGINT 18 | ); 19 | 20 | CREATE TABLE cars_output ( 21 | driver_id BIGINT, 22 | event_type TEXT 23 | ) WITH ( 24 | connector = 'single_file', 25 | path = '$output_path', 26 | format = 'json', 27 | type = 'sink' 28 | ); 29 | INSERT INTO memory SELECT event_type, location, driver_id as other_driver_id FROM cars; 30 | INSERT INTO memory SELECT location, event_type, driver_id FROM cars; 31 | INSERT INTO cars_output SELECT driver_id as other_driver_id, event_type FROM memory; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_mismatched_nested_windows.sql: -------------------------------------------------------------------------------- 1 | --fail=This feature is not implemented: window in group by does not match input window 2 | CREATE TABLE nexmark WITH ( 3 | connector = 'nexmark', 4 | event_rate = '10' 5 | ); 6 | 7 | SELECT 8 | count(*) as auctions, 9 | tumble(interval '2 minute') as second_window 10 | FROM 11 | ( 12 | SELECT 13 | bid.auction as auction, 14 | tumble(interval '1 minute') as window, 15 | count(*) as count 16 | FROM 17 | nexmark 18 | where 19 | bid is not null 20 | GROUP BY 21 | 1, 22 | 2 23 | ) 24 | GROUP BY 25 | 2 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_missing_redis_key.sql: -------------------------------------------------------------------------------- 1 | --fail=Redis lookup tables must have a PRIMARY KEY field defined as `field_name TEXT METADATA FROM 'key'` 2 | create table impulse with ( 3 | connector = 'impulse', 4 | event_rate = '2' 5 | ); 6 | 7 | create table lookup ( 8 | key TEXT PRIMARY KEY, 9 | value TEXT 10 | ) with ( 11 | connector = 'redis', 12 | format = 'json', 13 | address = 'redis://localhost:6379', 14 | type = 'lookup' 15 | ); 16 | 17 | select A.counter, B.key, B.value 18 | from impulse A left join lookup B 19 | on cast((A.counter % 10) as TEXT) = B.key; 20 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_missing_window_basic_tumble.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE Nexmark WITH ( 2 | connector = 'nexmark', 3 | event_rate = '10' 4 | ); 5 | 6 | SELECT 7 | bid.auction as auction, 8 | count(*) as count 9 | FROM 10 | nexmark 11 | where 12 | bid is not null 13 | GROUP BY 14 | 1 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_no_aggregate_over_debezium.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE debezium_input ( 2 | id int PRIMARY KEY, 3 | count int 4 | ) WITH ( 5 | connector = 'kafka', 6 | bootstrap_servers = 'localhost:9092', 7 | type = 'source', 8 | topic = 'updating', 9 | format = 'debezium_json' 10 | ); 11 | 12 | SELECT count(*) FROM debezium_input -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_no_nested_updating_aggregates.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE impulse with ( 2 | connector = 'impulse', 3 | event_rate = '10' 4 | ); 5 | SELECT sum(count) , max(counter) FROM( 6 | SELECT count(*) as count, subtask_index, max(counter) as counter FROM impulse group by 2); -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_offset_sliding_window.sql: -------------------------------------------------------------------------------- 1 | --fail=Error during planning: hop() width 600s currently must be a multiple of slide 180s 2 | CREATE TABLE Nexmark WITH ( 3 | connector = 'nexmark', 4 | event_rate = '10' 5 | ); 6 | 7 | SELECT 8 | bid.auction as auction, 9 | hop(interval '3 minute', interval '10 minute') as window, 10 | count(*) as count 11 | FROM 12 | nexmark 13 | where 14 | bid is not null 15 | GROUP BY 16 | 1,2 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_read_from_unwritten_memory_table.sql: -------------------------------------------------------------------------------- 1 | --fail=Error during planning: Can't query from memory table memory without first inserting into it. 2 | CREATE TABLE memory ( 3 | event_type TEXT, 4 | location TEXT, 5 | driver_id BIGINT 6 | ); 7 | 8 | SELECT * FROM memory; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/error_second_aggregate_missing_window.sql: -------------------------------------------------------------------------------- 1 | --fail=must have window in aggregate. Make sure you are calling one of the windowing functions (hop, tumble, session) or using the window field of the input 2 | CREATE TABLE Nexmark WITH ( 3 | connector = 'nexmark', 4 | event_rate = '10' 5 | ); 6 | 7 | 8 | SELECT count(*) as auctions FROM ( 9 | SELECT 10 | bid.auction as auction, 11 | tumble(interval '1 minute') as window, 12 | count(*) as count 13 | FROM 14 | nexmark 15 | where 16 | bid is not null 17 | GROUP BY 18 | 1,2) -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/filesystem_invalid_partition.sql: -------------------------------------------------------------------------------- 1 | --fail=Failed to create table events_sink caused by Error during planning: partition field 'not_a_real_field' does not exist in the schema 2 | create table events ( 3 | id TEXT PRIMARY KEY, 4 | timestamp TIMESTAMP NOT NULL, 5 | ip TEXT, 6 | user_id TEXT, 7 | platform TEXT, 8 | app_version TEXT, 9 | type TEXT NOT NULL, 10 | properties JSON 11 | ) with ( 12 | connector = 'kafka', 13 | bootstrap_servers = 'localhost:9092', 14 | topic = 'analytics', 15 | format = 'json', 16 | type = 'source' 17 | ); 18 | 19 | create table events_sink ( 20 | id TEXT, 21 | timestamp TIMESTAMP NOT NULL, 22 | ip TEXT, 23 | user_id TEXT, 24 | platform TEXT, 25 | app_version TEXT, 26 | type TEXT NOT NULL, 27 | properties JSON 28 | ) with ( 29 | connector = 'delta', 30 | path = 's3://my-s3-bucket/data/events', 31 | format = 'parquet', 32 | 'filename.strategy' = 'uuid', 33 | 'parquet.compression' = 'zstd', 34 | time_partition_pattern = '%Y/%m/%d/%H', 35 | partition_fields = [type, not_a_real_field], 36 | rollover_seconds = 6000 37 | ); 38 | 39 | INSERT INTO events 40 | SELECT * from events; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/filesystem_partition.sql: -------------------------------------------------------------------------------- 1 | create table events ( 2 | id TEXT PRIMARY KEY, 3 | timestamp TIMESTAMP NOT NULL, 4 | ip TEXT, 5 | user_id TEXT, 6 | platform TEXT, 7 | app_version TEXT, 8 | type TEXT NOT NULL, 9 | properties JSON 10 | ) with ( 11 | connector = 'kafka', 12 | bootstrap_servers = 'localhost:9092', 13 | topic = 'analytics', 14 | format = 'json', 15 | type = 'source' 16 | ); 17 | 18 | create table events_sink ( 19 | id TEXT, 20 | timestamp TIMESTAMP NOT NULL, 21 | ip TEXT, 22 | user_id TEXT, 23 | platform TEXT, 24 | app_version TEXT, 25 | type TEXT NOT NULL, 26 | properties JSON 27 | ) with ( 28 | connector = 'delta', 29 | path = 's3://my-s3-bucket/data/events', 30 | format = 'parquet', 31 | 'filename.strategy' = 'uuid', 32 | 'parquet.compression' = 'zstd', 33 | time_partition_pattern = '%Y/%m/%d/%H', 34 | partition_fields = [type, app_version], 35 | rollover_seconds = 6000 36 | ); 37 | 38 | INSERT INTO events 39 | SELECT * from events; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/first_pipeline.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE nexmark with ( 2 | connector = 'nexmark', 3 | event_rate = '10' 4 | ); 5 | 6 | SELECT * FROM ( 7 | SELECT *, ROW_NUMBER() OVER ( 8 | PARTITION BY window 9 | ORDER BY count DESC) AS row_num 10 | FROM (SELECT count(*) AS count, bid.auction AS auction, 11 | hop(interval '2 seconds', interval '60 seconds') AS window 12 | FROM nexmark WHERE bid is not null 13 | GROUP BY 2, window)) WHERE row_num <= 5 14 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/hop_to_tumble.sql: -------------------------------------------------------------------------------- 1 | create table impulse with ( 2 | connector = 'impulse', 3 | event_rate = '10' 4 | ); 5 | 6 | select count(*) from impulse 7 | group by hop(interval '10 seconds', interval '10 seconds'); -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/infer_debezium_sink.sql: -------------------------------------------------------------------------------- 1 | create table impulse with ( 2 | connector = 'impulse', 3 | event_rate = '10' 4 | ); 5 | 6 | create table sink with ( 7 | connector = 'kafka', 8 | bootstrap_servers = 'localhost:9092', 9 | format = 'debezium_json', 10 | type = 'sink', 11 | topic = 'outputs' 12 | ); 13 | 14 | insert into sink 15 | select count(*) 16 | from impulse 17 | group by hop(interval '5 seconds', interval '30 seconds'); -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/kafka_metadata_udf.sql: -------------------------------------------------------------------------------- 1 | create table users ( 2 | id TEXT, 3 | name TEXT, 4 | offset BIGINT METADATA FROM 'offset_id', 5 | topic TEXT METADATA FROM 'topic', 6 | partition INT METADATA FROM 'partition' 7 | ) with ( 8 | connector = 'kafka', 9 | topic = 'order_topic', 10 | format='json', 11 | bootstrap_servers = '0.0.0.0:9092', 12 | type='source' 13 | ); 14 | 15 | SELECT * FROM users; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/lookup_join.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE events ( 2 | event_id TEXT, 3 | timestamp TIMESTAMP, 4 | customer_id TEXT, 5 | event_type TEXT 6 | ) WITH ( 7 | connector = 'kafka', 8 | topic = 'events', 9 | type = 'source', 10 | format = 'json', 11 | bootstrap_servers = 'broker:9092' 12 | ); 13 | 14 | create temporary table customers ( 15 | customer_id TEXT METADATA FROM 'key' PRIMARY KEY, 16 | customer_name TEXT, 17 | plan TEXT 18 | ) with ( 19 | connector = 'redis', 20 | format = 'raw_string', 21 | address = 'redis://localhost:6379', 22 | format = 'json', 23 | 'lookup.cache.max_bytes' = 1000000, 24 | 'lookup.cache.ttl' = interval '5' second 25 | ); 26 | 27 | SELECT e.event_id, e.timestamp, e.customer_id, e.event_type, c.customer_name, c.plan 28 | FROM events e 29 | LEFT JOIN customers c 30 | ON e.customer_id = c.customer_id 31 | WHERE c.plan = 'Premium'; 32 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/mastodon_trends.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE mastodon ( 2 | value TEXT 3 | ) WITH ( 4 | connector = 'sse', 5 | format = 'raw_string', 6 | endpoint = 'http://mastodon.arroyo.dev/api/v1/streaming/public', 7 | events = 'update' 8 | ); 9 | 10 | CREATE VIEW tags AS ( 11 | SELECT tag FROM ( 12 | SELECT extract_json_string(value, '$.tags[*].name') AS tag 13 | FROM mastodon) 14 | WHERE tag is not null 15 | ); 16 | 17 | SELECT * FROM ( 18 | SELECT *, ROW_NUMBER() OVER ( 19 | PARTITION BY window 20 | ORDER BY count DESC) as row_num 21 | FROM (SELECT count(*) as count, 22 | tag, 23 | hop(interval '5 seconds', interval '15 minutes') as window 24 | FROM tags 25 | group by tag, window)) WHERE row_num <= 5; 26 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/merge_updates.sql: -------------------------------------------------------------------------------- 1 | create table impulse with ( 2 | connector = 'impulse', 3 | event_rate = '10' 4 | ); 5 | 6 | select counter % 10, count(*) from impulse 7 | group by counter % 10; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/metadata_error.sql: -------------------------------------------------------------------------------- 1 | --fail=Failed to create table mqtt caused by Error during planning: incorrect data type for metadata field 'topic'; expected TEXT, but found INT 2 | create table mqtt ( 3 | name TEXT, 4 | value INT, 5 | topic INT METADATA FROM 'topic' 6 | ) with ( 7 | connector = 'mqtt', 8 | url = 'tcp://localhost:1883', 9 | topic = 'plant/#', 10 | type = 'source', 11 | format = 'json' 12 | ); 13 | 14 | select topic from mqtt; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/metadata_fields.sql: -------------------------------------------------------------------------------- 1 | create table mqtt ( 2 | name TEXT, 3 | value INT, 4 | my_topic TEXT METADATA FROM 'topic' 5 | ) with ( 6 | connector = 'mqtt', 7 | url = 'tcp://localhost:1883', 8 | topic = 'plant/#', 9 | type = 'source', 10 | format = 'json' 11 | ); 12 | 13 | select my_topic from mqtt; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/metadata_raw_string.sql: -------------------------------------------------------------------------------- 1 | create table mqtt ( 2 | value TEXT, 3 | my_topic TEXT METADATA FROM 'topic' 4 | ) with ( 5 | connector = 'mqtt', 6 | url = 'tcp://localhost:1883', 7 | topic = 'plant/#', 8 | type = 'source', 9 | format = 'raw_string' 10 | ); 11 | 12 | select my_topic from mqtt; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/mismatched_sink_schema.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE source (a int, b int) WITH ( 2 | connector = 'kafka', 3 | bootstrap_servers = 'localhost:9092', 4 | topic = 'source', 5 | format = 'json', 6 | type = 'source' 7 | ); 8 | 9 | CREATE TABLE sink (a text, b text) WITH ( 10 | connector = 'kafka', 11 | bootstrap_servers = 'localhost:9092', 12 | topic = 'sink', 13 | format = 'json', 14 | type = 'sink' 15 | ); 16 | 17 | 18 | INSERT INTO sink SELECT a, b FROM source; 19 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/misplaced_hop.sql: -------------------------------------------------------------------------------- 1 | --fail=Error during planning: time window function hop is not allowed in this context. Are you missing a GROUP BY clause? 2 | CREATE TABLE impulse WITH ( 3 | connector = 'impulse', 4 | event_rate = '10', 5 | message_count = '20' 6 | ); 7 | 8 | 9 | SELECT 10 | hop(interval '1 second', interval '10 second' ) as window, 11 | count(*) as count, 12 | min(counter) as min, 13 | max(counter) as max 14 | from impulse -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/no_inserting_updates_into_non_updating.sql: -------------------------------------------------------------------------------- 1 | CREATE table debezium_source ( 2 | id INT PRIMARY KEY, 3 | count int 4 | ) WITH ( 5 | connector = 'kafka', 6 | bootstrap_servers = 'localhost:9092', 7 | type = 'source', 8 | topic = 'updating', 9 | format = 'debezium_json' 10 | ); 11 | 12 | CREATE table sink ( 13 | count int 14 | ) WITH ( 15 | connector = 'kafka', 16 | bootstrap_servers = 'localhost:9092', 17 | type = 'sink', 18 | topic = 'sink', 19 | format = 'debezium_json' 20 | ); 21 | 22 | INSERT into sink 23 | SELECT count FROM debezium_source -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/no_updating_joins.sql: -------------------------------------------------------------------------------- 1 | --fail=Error during planning: can't handle updating left side of join 2 | CREATE TABLE nexmark ( 3 | auction bigint, 4 | bidder bigint, 5 | price bigint, 6 | channel text, 7 | url text, 8 | datetime timestamp, 9 | extra text 10 | ) WITH ( 11 | connector = 'filesystem', 12 | format = 'parquet', 13 | type = 'source', 14 | path = '/home/data', 15 | 'source.regex-pattern' = '00001-000.parquet', 16 | event_time_field = datetime 17 | ); 18 | 19 | CREATE TABLE counts as (SELECT count(*) as counts, bidder FROM nexmark GROUP BY 2); 20 | 21 | SELECT a.counts, b.counts FROM counts A join counts B on A.bidder = b.bidder -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/no_updating_window_functions.sql: -------------------------------------------------------------------------------- 1 | --fail=Window functions require already windowed input 2 | SELECT *, row_number() OVER (partition by bid.auction order by bid.datetime desc) as row_num 3 | FROM nexmark where bid is not null -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/no_virtual_fields_updating.sql: -------------------------------------------------------------------------------- 1 | --fail=Failed to create table debezium_source caused by Error during planning: can't use virtual fields with debezium format 2 | CREATE table debezium_source ( 3 | bids_auction int, 4 | price int, 5 | auctions_id int, 6 | initial_bid int, 7 | date_string text, 8 | datetime datetime GENERATED ALWAYS AS (CAST(date_string as timestamp)) STORED, 9 | "watermark" datetime GENERATED ALWAYS AS (CAST(date_string as timestamp) - interval '1 second') STORED 10 | ) WITH ( 11 | connector = 'kafka', 12 | bootstrap_servers = 'localhost:9092', 13 | type = 'source', 14 | topic = 'updating', 15 | format = 'debezium_json' 16 | ); 17 | SELECT * FROM debezium_source -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/parse.sql: -------------------------------------------------------------------------------- 1 | WITH bids as (SELECT bid.auction as auction, bid.datetime as datetime 2 | FROM (select bid from nexmark) where bid is not null) 3 | SELECT AuctionBids.auction as auction, AuctionBids.num as count 4 | FROM ( 5 | SELECT 6 | B1.auction, 7 | HOP(INTERVAL '2' SECOND, INTERVAL '10' SECOND) as window, 8 | count(*) AS num 9 | 10 | FROM bids B1 11 | GROUP BY 12 | 1,2 13 | ) AS AuctionBids 14 | JOIN ( 15 | SELECT 16 | max(num) AS maxn, 17 | window 18 | FROM ( 19 | SELECT 20 | count(*) AS num, 21 | HOP(INTERVAL '2' SECOND, INTERVAL '10' SECOND) AS window 22 | FROM bids B2 23 | GROUP BY 24 | B2.auction,2 25 | ) AS CountBids 26 | GROUP BY 2 27 | ) AS MaxBids 28 | ON 29 | AuctionBids.num = MaxBids.maxn 30 | and AuctionBids.window = MaxBids.window; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/parse_log.sql: -------------------------------------------------------------------------------- 1 | create table logs ( 2 | value TEXT NOT NULL, 3 | parsed TEXT GENERATED ALWAYS AS (parse_log(value)), 4 | event_time TIMESTAMP GENERATED ALWAYS AS 5 | (CAST(extract_json_string(parse_log(value), '$.timestamp') as TIMESTAMP)) 6 | ) with ( 7 | connector = 'kafka', 8 | type = 'source', 9 | format = 'raw_string', 10 | bootstrap_servers = 'localhost:9092', 11 | topic = 'apache_logs', 12 | 'source.offset' = 'earliest' 13 | ); 14 | 15 | SELECT count(*) 16 | FROM logs 17 | WHERE extract_json(parsed, '$.status_code')[1] = '500' 18 | GROUP BY hop(interval '5 seconds', interval '5 minutes'); 19 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/program_compilation.sql: -------------------------------------------------------------------------------- 1 | SELECT * FROM ( 2 | SELECT ROW_NUMBER() OVER ( 3 | PARTITION BY window 4 | ORDER BY count DESC) as row_number, auction FROM ( 5 | SELECT bid.auction as auction, 6 | hop(INTERVAL '10' minute, INTERVAL '20' minute ) as window, 7 | count(*) as count 8 | FROM (SELECT bid from nexmark where bid is not null) 9 | GROUP BY 1, 2)) where row_number = 1 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/release_post_json.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE events ( 2 | value JSON 3 | ) WITH ( 4 | connector = 'kafka', 5 | bootstrap_servers = 'kafka:9092', 6 | topic = 'events', 7 | format = 'json', 8 | type = 'source', 9 | 'json.unstructured' = 'true' 10 | ); 11 | 12 | SELECT 13 | -- using the json_get function 14 | json_get('user', 'name')::TEXT as name, 15 | -- or using the -> operator 16 | value->'user'->'email' as email, 17 | -- field presence check can be done with the ? operator 18 | value ? 'id' as has_id 19 | FROM events; 20 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/schema_inference.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE nexmark with ( 2 | connector = 'nexmark', 3 | event_rate = '100' 4 | ); 5 | 6 | CREATE TABLE output WITH ( 7 | connector = 'kafka', 8 | type = 'sink', 9 | bootstrap_servers = 'localhost:9092', 10 | format = 'json', 11 | 'topic' = 'outputs' 12 | ); 13 | 14 | INSERT INTO output 15 | SELECT count(*), auction.id from nexmark 16 | GROUP BY auction.id, hop(interval '5 seconds', interval '1 hour'); 17 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/second_aggregate_recall_window.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE Nexmark WITH ( 2 | connector = 'nexmark', 3 | event_rate = '10' 4 | ); 5 | 6 | SELECT 7 | count(*) as auction, 8 | tumble(interval '1 minute') as second_window 9 | FROM 10 | ( 11 | SELECT 12 | bid.auction as auction, 13 | tumble(interval '1 minute') as window, 14 | count(*) as count 15 | FROM 16 | nexmark 17 | where 18 | bid is not null 19 | GROUP BY 20 | 1, 21 | 2 22 | ) 23 | GROUP BY 24 | 2 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/source_rewriter_join.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE impulse WITH( 2 | connector = 'impulse', 3 | event_rate = '10000' 4 | ); 5 | 6 | 7 | SELECT evens.even_counter FROM 8 | (SELECT counter as even_counter FROM impulse where counter % 2 = 0) evens 9 | JOIN impulse on evens.even_counter = impulse.counter; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/struct_ddl.sql: -------------------------------------------------------------------------------- 1 | create table users ( 2 | id TEXT, 3 | t struct> 4 | ) with ( 5 | connector = 'kafka', 6 | format = 'json', 7 | bootstrap_servers = 'localhost:9092', 8 | type = 'source', 9 | topic = 'structs' 10 | ); 11 | 12 | select id, t.a, t.x.b from users; 13 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/subscript_in_virtual.sql: -------------------------------------------------------------------------------- 1 | create table input ( 2 | length JSON, 3 | diff INT GENERATED ALWAYS AS (extract_json(length, '$.old')[1]) STORED 4 | ) with ( 5 | connector = 'sse', 6 | endpoint = 'https://localhost:9091', 7 | format = 'json' 8 | ); 9 | 10 | select * from input; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/test_merge_sink.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE cars ( 2 | timestamp TIMESTAMP, 3 | driver_id BIGINT, 4 | event_type TEXT, 5 | location TEXT 6 | ) WITH ( 7 | connector = 'single_file', 8 | path = 'cars.json', 9 | format = 'json', 10 | type = 'source' 11 | ); 12 | 13 | CREATE TABLE cars_output ( 14 | timestamp TIMESTAMP, 15 | driver_id BIGINT, 16 | event_type TEXT, 17 | location TEXT 18 | ) WITH ( 19 | connector = 'single_file', 20 | path = 'cars_output.json', 21 | format = 'json', 22 | type = 'sink' 23 | ); 24 | INSERT INTO cars_output SELECT * FROM cars WHERE driver_id = 100 AND event_type = 'pickup'; 25 | INSERT INTO cars_output SELECT * FROM cars WHERE driver_id = 101 AND event_type = 'dropoff'; 26 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/test_no_aggregates_in_window.sql: -------------------------------------------------------------------------------- 1 | WITH bids as ( 2 | SELECT bid.auction as auction, bid.price as price, bid.bidder as bidder, bid.extra as extra, bid.datetime as datetime 3 | FROM nexmark where bid is not null) 4 | 5 | SELECT * FROM ( 6 | SELECT bidder, COUNT( distinct auction) as distinct_auctions 7 | FROM bids B1 8 | GROUP BY bidder, HOP(INTERVAL '3 second', INTERVAL '10' minute)) WHERE distinct_auctions > 2 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/updating_filter_join.sql: -------------------------------------------------------------------------------- 1 | --fail=Updating joins must include an equijoin condition 2 | 3 | CREATE TABLE cars ( 4 | timestamp TIMESTAMP, 5 | car_id TEXT, 6 | driver_id BIGINT, 7 | event_type TEXT, 8 | location TEXT 9 | ) WITH ( 10 | connector = 'single_file', 11 | path = '$input_dir/cars.json', 12 | format = 'json', 13 | type = 'source', 14 | event_time_field = 'timestamp' 15 | ); 16 | 17 | CREATE TABLE passengers ( 18 | timestamp TIMESTAMP, 19 | passenger_id BIGINT 20 | ) WITH ( 21 | connector = 'single_file', 22 | path = '$input_dir/cars.json', 23 | format = 'json', 24 | type = 'source', 25 | event_time_field = 'timestamp' 26 | ); 27 | 28 | select passenger_id, car_id 29 | from passengers 30 | join cars ON passenger_id < car_id; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/virtual_bad_schema.sql: -------------------------------------------------------------------------------- 1 | --fail=Schema error: No field named notfield. Valid fields are input.length. 2 | create table input ( 3 | length JSON, 4 | diff INT GENERATED ALWAYS AS (notfield) STORED 5 | ) with ( 6 | connector = 'sse', 7 | endpoint = 'https://localhost:9091', 8 | format = 'json' 9 | ); 10 | 11 | select * from input; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/watermarks.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE orders ( 2 | customer_id INT, 3 | order_id INT, 4 | date_string TEXT NOT NULL, 5 | timestamp TIMESTAMP GENERATED ALWAYS AS (CAST(date_string as TIMESTAMP)), 6 | watermark FOR timestamp as CAST(date_string as TIMESTAMP) - INTERVAL '5 seconds' 7 | ) WITH ( 8 | connector = 'kafka', 9 | format = 'json', 10 | type = 'source', 11 | bootstrap_servers = 'localhost:9092', 12 | topic = 'order_topic' 13 | ); 14 | 15 | CREATE TABLE users ( 16 | customer_id INT, 17 | timestamp TIMESTAMP, 18 | watermark FOR timestamp 19 | ) WITH ( 20 | connector = 'kafka', 21 | format = 'json', 22 | type = 'source', 23 | bootstrap_servers = 'localhost:9092', 24 | topic = 'order_topic' 25 | ); 26 | 27 | 28 | select * from orders; -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/window_function.sql: -------------------------------------------------------------------------------- 1 | SELECT * FROM ( 2 | SELECT *, ROW_NUMBER() OVER ( 3 | PARTITION BY window 4 | ORDER BY count DESC) as row_num 5 | FROM (SELECT count(*) as count, 6 | hop(interval '2 seconds', interval '10 seconds') as window 7 | FROM nexmark 8 | group by window)) WHERE row_num <= 5 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/window_partition_by_column.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE nexmark with ( 2 | connector = 'nexmark', 3 | event_rate = '500000', 4 | runtime = '20' 5 | ); 6 | 7 | 8 | SELECT * FROM ( 9 | SELECT *, row_number() OVER ( 10 | PARTITION BY window, auction 11 | ORDER BY count DESC) AS row_num 12 | FROM (SELECT count(*) AS count, bid.auction AS auction, 13 | hop(interval '2 seconds', interval '60 seconds') AS window 14 | FROM nexmark WHERE bid is not null 15 | GROUP BY 2, window)) WHERE row_num <= 5 -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/window_with_expression.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE impulse with ( 2 | connector = 'impulse', 3 | event_rate = '500000' 4 | ); 5 | 6 | SELECT 7 | *, 8 | row_number() OVER (PARTITION BY 2 * counter, window) as row_number 9 | FROM 10 | ( 11 | SELECT 12 | count(*) AS count, 13 | counter, 14 | hop(interval '2 seconds', interval '60 seconds') AS window 15 | FROM 16 | impulse 17 | GROUP BY 18 | 2, 19 | window 20 | ) -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/queries/windowed_inner_join.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE cars ( 2 | timestamp TIMESTAMP, 3 | driver_id BIGINT, 4 | event_type TEXT, 5 | location TEXT 6 | ) WITH ( 7 | connector = 'single_file', 8 | path = '$input_dir/cars.json', 9 | format = 'json', 10 | type = 'source', 11 | event_time_field = 'timestamp' 12 | ); 13 | CREATE TABLE hourly_aggregates ( 14 | hour TIMESTAMP, 15 | drivers BIGINT, 16 | pickups BIGINT 17 | ) WITH ( 18 | connector = 'single_file', 19 | path = '$output_path', 20 | format = 'json', 21 | type = 'sink' 22 | ); 23 | INSERT INTO hourly_aggregates 24 | SELECT window.start as hour, dropoff_drivers, pickup_drivers FROM ( 25 | SELECT dropoffs.window as window, dropoff_drivers, pickup_drivers 26 | FROM ( 27 | SELECT TUMBLE(INTERVAL '1' hour) as window, 28 | COUNT(distinct driver_id) as dropoff_drivers FROM cars where event_type = 'dropoff' 29 | GROUP BY 1 30 | ) dropoffs 31 | INNER JOIN ( 32 | SELECT TUMBLE(INTERVAL '1' hour) as window, 33 | COUNT(distinct driver_id) as pickup_drivers FROM cars where event_type = 'pickup' 34 | GROUP BY 1 35 | ) pickups 36 | ON dropoffs.window = pickups.window) -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/udfs/cbor_to_json.rs: -------------------------------------------------------------------------------- 1 | /* 2 | [dependencies] 3 | serde_cbor = "0.11" 4 | serde_json = "1" 5 | serde = {version = "1", features = ["derive"]} 6 | serde-transcode = "1" 7 | */ 8 | 9 | use arroyo_udf_plugin::udf; 10 | 11 | 12 | #[udf] 13 | fn cbor_to_json(data: &[u8]) -> Option { 14 | let mut deserializer = serde_cbor::Deserializer::from_slice(&data[..]); 15 | let mut buf = std::io::BufWriter::new(Vec::new()); 16 | let mut serializer = serde_json::Serializer::new(&mut buf); 17 | serde_transcode::transcode(&mut deserializer, &mut serializer).ok()?; 18 | let bytes = buf.into_inner().unwrap(); 19 | 20 | Some(String::from_utf8(bytes).ok()?) 21 | } 22 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/udfs/get_city.rs: -------------------------------------------------------------------------------- 1 | /* 2 | [dependencies] 3 | reqwest = { version = "0.11.23", features = ["json"] } 4 | serde_json = "1" 5 | */ 6 | 7 | use arroyo_udf_plugin::udf; 8 | use reqwest::Client; 9 | 10 | #[udf(allowed_in_flight=100000, timeout="180s")] 11 | pub async fn get_city(ip: &str) -> Option { 12 | use std::sync::OnceLock; 13 | static CLIENT: OnceLock = OnceLock::new(); 14 | let client = CLIENT.get_or_init(|| { 15 | Client::new() 16 | }); 17 | 18 | 19 | let body: serde_json::Value = 20 | client.get( 21 | format!("http://localhost:6006/{ip}")) 22 | .send() 23 | .await 24 | .ok()? 25 | .json() 26 | .await 27 | .ok()?; 28 | 29 | body.pointer("/names/en").and_then(|t| 30 | t.as_str() 31 | ).map(|t| t.to_string()) 32 | } -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/udfs/irate.rs: -------------------------------------------------------------------------------- 1 | #[udf] 2 | fn irate(values: Vec) -> Option { 3 | let start = values.first()?; 4 | let end = values.last()?; 5 | 6 | let rate = (end - start) / 60.0; 7 | if rate >= 0.0 { 8 | Some(rate) 9 | } else { 10 | // this implies there was a counter reset during this window; 11 | // just drop this sample 12 | None 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/udfs/parse_log.rs: -------------------------------------------------------------------------------- 1 | /* 2 | [dependencies] 3 | access_log_parser = "0.8" 4 | serde_json = "1" 5 | */ 6 | 7 | use access_log_parser::{parse, AccessLogError, LogType, LogEntry}; 8 | use serde_json::json; 9 | use std::time::{UNIX_EPOCH, Duration}; 10 | 11 | #[udf] 12 | pub fn parse_log(input: &str) -> Option { 13 | let LogEntry::CommonLog(entry) = 14 | parse(LogType::CommonLog, &input).ok()? else { 15 | return None 16 | }; 17 | 18 | Some(json!({ 19 | "ip": entry.ip, 20 | "user": entry.user, 21 | "timestamp": entry.timestamp.to_rfc3339(), 22 | "request": format!("{:?}", entry.request), 23 | "status_code": entry.status_code.as_u16() 24 | }).to_string()) 25 | } 26 | -------------------------------------------------------------------------------- /crates/arroyo-planner/src/test/udfs/parse_prom.rs: -------------------------------------------------------------------------------- 1 | /* 2 | [dependencies] 3 | regex = "1.10.2" 4 | serde_json = "1.0" 5 | */ 6 | 7 | #[udf] 8 | fn parse_prom(s: &str) -> Option { 9 | use regex::Regex; 10 | use std::collections::HashMap; 11 | 12 | // use OnceLock to prevent re-compilation of the regexes on every request 13 | use std::sync::OnceLock; 14 | static METRIC_REGEX: OnceLock = OnceLock::new(); 15 | let metric_regex = METRIC_REGEX.get_or_init(|| { 16 | Regex::new(r"(?P\w+)\{(?P[^}]+)\}\s+(?P[\d.]+)").unwrap() 17 | }); 18 | 19 | static LABEL_REGEX: OnceLock = OnceLock::new(); 20 | let label_regex = LABEL_REGEX.get_or_init(|| { 21 | regex::Regex::new(r##"(?P