├── .dockerignore ├── .github └── workflows │ ├── codecov.yml │ └── tests.yml ├── .gitignore ├── LICENSE ├── Makefile ├── README.md ├── code ├── .gitignore ├── Cargo.lock ├── Cargo.toml ├── build.rs ├── examples │ ├── delta-tbl-overwrite │ │ ├── _delta_log │ │ │ ├── 00000000000000000000.json │ │ │ └── 00000000000000000001.json │ │ ├── part-00000-74c070ed-19a6-4ab4-83b6-7a0f0d66f70d-c000.snappy.parquet │ │ └── part-00000-f4a247c9-a3bb-4b1e-adc7-7269808b8d73-c000.snappy.parquet │ ├── delta-tbl-partition │ │ ├── _delta_log │ │ │ └── 00000000000000000000.json │ │ ├── year=2020 │ │ │ └── part-00000-755eb925-59ad-4167-b46b-db694e7f3b2c.c000.snappy.parquet │ │ └── year=2021 │ │ │ └── part-00000-c22c8742-53c8-4ffa-b9c3-ba7705218ca2.c000.snappy.parquet │ ├── direct_s3.rs │ ├── query-delta-taxi.json │ ├── query-static-sample.json │ ├── query.json │ └── steps.json ├── proto │ └── buzz.proto ├── rustfmt.toml └── src │ ├── bin │ ├── main_fuse_lambda.rs │ ├── main_fuse_local.rs │ ├── main_hbee_lambda.rs │ ├── main_hbee_local.rs │ ├── main_hbee_tests.rs │ ├── main_hcomb.rs │ └── main_integ.rs │ ├── clients │ ├── cached_file.rs │ ├── fargate.rs │ ├── flight_client.rs │ ├── lambda.rs │ ├── mod.rs │ ├── range_cache.rs │ └── s3.rs │ ├── datasource │ ├── catalog │ │ ├── delta_catalog.rs │ │ ├── mod.rs │ │ ├── static_catalog.rs │ │ └── test_catalog.rs │ ├── hbee │ │ ├── mod.rs │ │ └── s3_parquet.rs │ ├── hcomb.rs │ └── mod.rs │ ├── error.rs │ ├── example_catalog.rs │ ├── execution_plan │ ├── mod.rs │ ├── parquet.rs │ └── stream.rs │ ├── flight_utils.rs │ ├── lib.rs │ ├── models │ ├── actions.rs │ ├── env.rs │ ├── hbee_event.rs │ ├── mod.rs │ └── query.rs │ ├── plan_utils.rs │ ├── serde │ ├── from_proto.rs │ ├── mod.rs │ └── to_proto.rs │ └── services │ ├── fuse │ ├── fuse_service.rs │ ├── hbee_scheduler.rs │ ├── hcomb_manager.rs │ ├── hcomb_scheduler.rs │ ├── mod.rs │ └── query_planner.rs │ ├── hbee │ ├── collector.rs │ ├── hbee_service.rs │ └── mod.rs │ ├── hcomb │ ├── flight_service.rs │ ├── hcomb_service.rs │ ├── mod.rs │ └── results_service.rs │ ├── mod.rs │ └── utils.rs ├── docker ├── Dockerfile └── docker-compose.yml └── infra ├── .terraform.lock.hcl ├── common.tf ├── env └── conf.tf ├── fargate ├── ecs.tf ├── iam.tf ├── inputs.tf ├── locals.tf └── outputs.tf ├── lambda ├── iam.tf ├── inputs.tf ├── lambda.tf └── outputs.tf ├── main.tf ├── modules.tf ├── output.tf ├── test.tf ├── versioning.tf └── versions.tf /.dockerignore: -------------------------------------------------------------------------------- 1 | docker/ 2 | infra/ 3 | code/target/ 4 | README.md 5 | Makefile -------------------------------------------------------------------------------- /.github/workflows/codecov.yml: -------------------------------------------------------------------------------- 1 | name: Codecov 2 | 3 | on: 4 | push: 5 | branches: [ master ] 6 | 7 | env: 8 | CARGO_TERM_COLOR: always 9 | 10 | jobs: 11 | build: 12 | 13 | runs-on: ubuntu-latest 14 | 15 | steps: 16 | - uses: actions/checkout@v2 17 | - name: Install Tarpaulin 18 | working-directory: ./code 19 | run: cargo install cargo-tarpaulin --version 0.18.0 20 | - name: Run Tarpaulin 21 | working-directory: ./code 22 | run: cargo tarpaulin --out Xml 23 | - name: Upload to codecov.io 24 | uses: codecov/codecov-action@v2 25 | with: 26 | working-directory: ./code 27 | 28 | -------------------------------------------------------------------------------- /.github/workflows/tests.yml: -------------------------------------------------------------------------------- 1 | name: Tests 2 | 3 | on: 4 | push: 5 | branches: [ master ] 6 | pull_request: 7 | branches: [ master ] 8 | 9 | env: 10 | CARGO_TERM_COLOR: always 11 | 12 | jobs: 13 | build: 14 | 15 | runs-on: ubuntu-latest 16 | 17 | steps: 18 | - uses: actions/checkout@v2 19 | - name: Build 20 | working-directory: ./code 21 | run: cargo build --verbose 22 | - name: Run tests 23 | working-directory: ./code 24 | run: cargo test --verbose 25 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | /infra/.terraform 2 | .vscode 3 | default.env -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2020 cloudfuse 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | SHELL := /bin/bash # Use bash syntax 2 | GIT_REVISION = `git rev-parse --short HEAD``git diff --quiet HEAD -- || echo "-dirty"` 3 | 4 | include default.env 5 | DEPLOY_PROFILE ?= $(eval DEPLOY_PROFILE := $(shell bash -c 'read -p "Deploy Profile: " input; echo $$input'))$(DEPLOY_PROFILE) 6 | BACKEND_PROFILE ?= $(eval BACKEND_PROFILE := $(shell bash -c 'read -p "Backend Profile: " input; echo $$input'))$(BACKEND_PROFILE) 7 | STAGE ?= $(eval STAGE := $(shell bash -c 'read -p "Stage: " input; echo $$input'))$(STAGE) 8 | REGION ?= $(eval REGION := $(shell bash -c 'read -p "Region: " input; echo $$input'))$(REGION) 9 | terraform = AWS_PROFILE=${BACKEND_PROFILE} terraform 10 | 11 | ## global commands 12 | 13 | check-dirty: 14 | @git diff --quiet HEAD -- || { echo "ERROR: commit first, or use 'make force-deploy' to deploy dirty"; exit 1; } 15 | 16 | ask-run-target: 17 | @echo "Running with profile ${DEPLOY_PROFILE}..." 18 | 19 | ask-deploy-target: 20 | @echo "Deploying ${GIT_REVISION} in ${STAGE} with profile ${DEPLOY_PROFILE}, backend profile ${BACKEND_PROFILE}..." 21 | 22 | # required with AWS CLI v2 23 | docker-login: 24 | aws ecr get-login-password --region "${REGION}" --profile=${DEPLOY_PROFILE} | \ 25 | docker login --username AWS --password-stdin \ 26 | "$(shell aws sts get-caller-identity --profile=${DEPLOY_PROFILE} --query 'Account' --output text).dkr.ecr.${REGION}.amazonaws.com" 27 | 28 | test: 29 | cd code; RUST_BACKTRACE=1 cargo test 30 | 31 | code/target/docker/%.zip: $(shell find code/src -type f) code/Cargo.toml docker/Dockerfile 32 | mkdir -p ./code/target/docker 33 | DOCKER_BUILDKIT=1 docker build \ 34 | -f docker/Dockerfile \ 35 | --build-arg BIN_NAME=$* \ 36 | --target export-stage \ 37 | --output ./code/target/docker \ 38 | . 39 | 40 | package-lambdas: code/target/docker/hbee_lambda.zip code/target/docker/fuse_lambda.zip code/target/docker/hbee_tests.zip 41 | 42 | package-hcomb: 43 | DOCKER_BUILDKIT=1 docker build \ 44 | -t cloudfuse/buzz-rust-hcomb:${GIT_REVISION} \ 45 | -f docker/Dockerfile \ 46 | --build-arg BIN_NAME=hcomb \ 47 | --build-arg PORT=3333 \ 48 | --target runtime-stage \ 49 | . 50 | 51 | run-integ-local: ask-run-target 52 | cd code; AWS_PROFILE=${DEPLOY_PROFILE} cargo run --bin integ 53 | 54 | run-integ-docker: ask-run-target 55 | COMPOSE_DOCKER_CLI_BUILD=1 DOCKER_BUILDKIT=1 docker-compose -f docker/docker-compose.yml build 56 | COMPOSE_DOCKER_CLI_BUILD=1 DOCKER_BUILDKIT=1 AWS_PROFILE=${DEPLOY_PROFILE} docker-compose -f docker/docker-compose.yml up --abort-on-container-exit 57 | 58 | example-direct-s3: ask-run-target 59 | cd code; RUST_BACKTRACE=1 AWS_PROFILE=${DEPLOY_PROFILE} cargo run --example direct_s3 60 | 61 | init: 62 | @cd infra; ${terraform} init 63 | @cd infra; ${terraform} workspace new ${STAGE} &>/dev/null || echo "${STAGE} already exists" 64 | 65 | destroy: ask-deploy-target 66 | cd infra; ${terraform} destroy \ 67 | --var profile=${DEPLOY_PROFILE} \ 68 | --var region_name=${REGION} 69 | 70 | deploy-all: ask-deploy-target package-hcomb package-lambdas 71 | @echo "DEPLOYING ${GIT_REVISION} on ${STAGE}..." 72 | @cd infra; ${terraform} workspace select ${STAGE} 73 | @cd infra; ${terraform} apply \ 74 | --var profile=${DEPLOY_PROFILE} \ 75 | --var region_name=${REGION} \ 76 | --var git_revision=${GIT_REVISION} 77 | @echo "${GIT_REVISION} DEPLOYED !!!" 78 | 79 | run-integ-aws: ask-run-target 80 | aws lambda invoke \ 81 | --function-name $(shell bash -c 'cd infra; ${terraform} output fuse_lambda_name') \ 82 | --log-type Tail \ 83 | --region ${REGION} \ 84 | --profile ${DEPLOY_PROFILE} \ 85 | --query 'LogResult' \ 86 | --output text \ 87 | --payload fileb://code/examples/query-delta-taxi.json \ 88 | /dev/null | base64 -d 89 | 90 | 91 | ## hbee-tests is a specific deployment of the hbee to run isolated tests 92 | 93 | # you can call this only if deploy-all was allready succesfully executed 94 | deploy-hbee-tests: ask-deploy-target code/target/docker/hbee_tests.zip 95 | @cd infra; ${terraform} workspace select dev 96 | cd infra; ${terraform} apply \ 97 | -auto-approve \ 98 | --var profile=${DEPLOY_PROFILE} \ 99 | --var region_name=${REGION} \ 100 | --var git_revision=${GIT_REVISION} \ 101 | --var push_hcomb=false 102 | 103 | run-hbee-tests: ask-run-target 104 | aws lambda invoke \ 105 | --function-name $(shell bash -c 'cd infra; ${terraform} output hbee_tests_lambda_name') \ 106 | --log-type Tail \ 107 | --region ${REGION} \ 108 | --profile ${DEPLOY_PROFILE} \ 109 | --query 'LogResult' \ 110 | --output text \ 111 | --payload fileb://code/examples/steps.json \ 112 | /dev/null | base64 -d -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # :honeybee: Buzz Rust :honeybee: 2 | 3 | [![License: MIT](https://img.shields.io/badge/License-MIT-green.svg)](LICENSE) 4 | ![Master](https://github.com/cloudfuse-io/buzz-rust/actions/workflows/tests.yml/badge.svg) 5 | [![codecov](https://codecov.io/gh/cloudfuse-io/buzz-rust/branch/master/graph/badge.svg?token=I5IDAW6VS6)](https://codecov.io/gh/cloudfuse-io/buzz-rust) 6 | 7 | > **Warning** 8 | > This project is a POC and is not actively maintained. It's dependencies are outdated, which might introduce security vulnerabilities. 9 | 10 | Buzz is best defined by the following key concepts: 11 | - **Interactive analytics query engine**, it quickly computes statistics or performs searches on huge amounts of data. 12 | - **Cloud serverless**, it does not use any resource when idle, but can scale instantly (and massively) to respond to incoming requests. 13 | 14 | Functionally, it can be compared to the managed analytics query services offered by all major cloud providers, but differs in that it is open source and uses lower-level components such as cloud functions and cloud storage. 15 | 16 | The current implementation is in Rust and is based on Apache Arrow with the DataFusion engine. 17 | 18 | ## Architecture 19 | 20 | Buzz is composed of three systems: 21 | - :honeybee: the HBees: cloud function workers that fetch data from the cloud storage and pre-aggregate it. They perform perform a highly scalable _map_ operation on the data. 22 | - :honey_pot: the HCombs: container based reducers that collect the intermediate state from the hbees and finialize the aggregation. They perform a low latency _reduce_ operation on the data. 23 | - :sparkler: the Fuse: a cloud function entrypoint that acts as scheduler and resource manager for a given query. 24 | 25 | ![Design overview](https://raw.githubusercontent.com/wiki/cloudfuse-io/buzz-rust/resources/design-principle-cropped.png) 26 | 27 | Buzz analytics queries are defined with SQL. The query is composed of different statements for the different stages (see example [here](code/examples/query-delta-taxi.json)). This is different from most distributed engines that have a scheduler that takes care of splitting a unique SQL query into multiple stages to be executed on different executors. The reason is that in Buzz, our executors (HBees and HCombs) have very different behaviors and capabilities. This is unusual and designing a query planner that understands this is not obvious. We prefer leaving it to our dear users, who are notoriously known to be smart, to decide what part of the query should be executed where. Further down the road, we might come up with a scheduler that is able to figure this out automatically. 28 | 29 | Note: the _h_ in hbee and hcomb stands for honey, of course ! :smiley: 30 | 31 | ## Tooling 32 | 33 | The Makefile contains all the usefull commands to build, deploy and test-run Buzz, so you will likely need `make` to be installed. 34 | 35 | Because Buzz is a cloud native query engine, many commands require an AWS account to be run. The Makefile uses the credentials stored in your `~/.aws/credentials` file (cf. [doc](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-files.html)) to access your cloud accounts on your behalf. 36 | 37 | When running commands from the Makefile, you will be prompted to specify the profiles (from the AWS creds file), region and stage you whish to use. If you don't want to specify these each time you run a make command, you can specify defaults by creating a file named `default.env` at the root of the project with the following content: 38 | ``` 39 | REGION:=eu-west-1|us-east-1|... 40 | DEPLOY_PROFILE:=profile-where-buzz-will-be-deployed 41 | BACKEND_PROFILE:=profile-that-has-access-to-the-s3-terraform-backend 42 | STAGE:=dev|prod|... 43 | ``` 44 | 45 | ### Build 46 | 47 | Build commands can be found in the Makefile. 48 | 49 | The AWS Lambda runtime runs a custom version of linux. To keep ourselves out of trouble we use *musl* instead of libc to make a static build. For reproducibility reasons, this build is done through docker. 50 | 51 | **Note:** docker builds require BuildKit (included in docker 18.09+) and the Docker Engine API 1.40+ with experimental features activated. 52 | 53 | ### Deploy 54 | 55 | The code can be deployed to AWS through terraform: 56 | - you need to configure your `~/.aws/credentials` 57 | - terraform is configured to use the S3 backend, so your backend AWS profile (`BACKEND_PROFILE`) should have access to a bucket where you will store your state. This bucket needs to be setup manually 58 | - you can customize properties (like networking or tags) relative to each stage in the `infra/env/conf.tf` files. Default settings should be sufficient in most cases. 59 | - you first need to init your terraform workspace. You will be prompted for: 60 | - the `STAGE` (dev/prod/...) so the associated terraform workspace can be created and the associated configs loaded. 61 | - the `BACKEND_PROFILE` 62 | - you also need to login docker to your ECR repository with `make docker-login`. For this to work, you need the AWS cli v2+ to be installed. 63 | - you can then deploy the Buzz resources. You will be prompted for 64 | - the `STAGE` 65 | - the `DEPLOY_PROFILE` corresponding to the creds of the account where you want to deploy 66 | - the `BACKEND_PROFILE` 67 | - if you want to cleanup you AWS account, you can run the destroy command. Buzz is a serverless query engine, so it should not consume many paying resources when it is not used. But there might be some residual costs (that we aim to reduce to 0) as long as the stack is kept in place. 68 | 69 | **Notes:** 70 | - remember that you can specify the `default.env` file to avoid being prompted at each command 71 | - *backend* and *deploy* profiles can be the same. In this case both the terraform backend and the buzz stack will reside in the same AWS account 72 | - Some resources might not be completely initialized after the terraform script completes. It generates errors such as `(AccessDeniedException) when calling the Invoke operation`. In that case wait a few minutes and try again. 73 | 74 | ## Running queries 75 | 76 | To run queries using the Makefile, you need the AWS cli v2+ to be installed. 77 | 78 | Example query (cf. [`code/examples/query-delta-taxi.json`](code/examples/query-delta-taxi.json)): 79 | ``` 80 | { 81 | "steps": [ 82 | { 83 | "sql": "SELECT payment_type, COUNT(payment_type) as payment_type_count FROM nyc_taxi GROUP BY payment_type", 84 | "name": "nyc_taxi_map", 85 | "step_type": "HBee", 86 | "partition_filter": "pickup_date<='2009-01-05'" 87 | }, 88 | { 89 | "sql": "SELECT payment_type, SUM(payment_type_count) FROM nyc_taxi_map GROUP BY payment_type", 90 | "name": "nyc_taxi_reduce", 91 | "step_type": "HComb" 92 | } 93 | ], 94 | "capacity": { 95 | "zones": 1 96 | }, 97 | "catalogs": [ 98 | { 99 | "name": "nyc_taxi", 100 | "type": "DeltaLake", 101 | "uri": "s3://cloudfuse-taxi-data/delta-tables/nyc-taxi-daily" 102 | } 103 | ] 104 | } 105 | ``` 106 | 107 | A query is a succession of steps. The `HBee` step type means that this part of the query runs in cloud functions (e.g AWS Lambda). The `HComb` step type means the associated query part runs on the container reducers (e.g AWS Fargate). The output of one step should be used as input (`FROM` statement) of the next step by refering to it by the step's name. 108 | 109 | A query takes the target list of file from a catalog. Supported catalogs are Static (file list compiled into the binary) and [DeltaLake](https://github.com/delta-io/delta). 110 | 111 | The `capacity.zone` field indicates the number of availability zones (and thus containers) used for `HComb` steps. This can be used to improve reducing capability and minimize cross-AZ data exchanges (both slower and more expensive). 112 | 113 | In the `HBee` step, you can specify a `partition_filter` field with an SQL filtering expression on partitioning dimensions. Currently partition values can only be strings. 114 | 115 | Current limitations: 116 | - only SQL supported by [DataFusion](https://github.com/apache/arrow-datafusion) is supported by Buzz 117 | - only single zone capacity is supported 118 | - only two-step queries are supported (`HBee` then `HComb`) 119 | - only single datasource queries can be run (no join) 120 | - a Buzz stack can only read S3 in its own region (because of S3 Gateway Endpoint) 121 | 122 | Note that the first query is slow (and might even timeout!) because it first needs to start a container for the HComb, which typically takes 15-25s on Fargate. Subsequent queries are much faster because they reuse the HComb container. The HComb is stopped after a configurable duration of inactivity (typically 5 minutes). 123 | -------------------------------------------------------------------------------- /code/.gitignore: -------------------------------------------------------------------------------- 1 | /target -------------------------------------------------------------------------------- /code/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "buzz-rust" 3 | version = "0.2.0" 4 | authors = ["rdettai "] 5 | edition = "2018" 6 | build = "build.rs" 7 | 8 | [lib] 9 | name = "buzz" 10 | 11 | [[bin]] 12 | name = "integ" 13 | path = "src/bin/main_integ.rs" 14 | 15 | [[bin]] 16 | name = "hcomb" 17 | path = "src/bin/main_hcomb.rs" 18 | 19 | [[bin]] 20 | name = "hbee_local" 21 | path = "src/bin/main_hbee_local.rs" 22 | 23 | [[bin]] 24 | name = "hbee_lambda" 25 | path = "src/bin/main_hbee_lambda.rs" 26 | 27 | [[bin]] 28 | name = "hbee_tests" 29 | path = "src/bin/main_hbee_tests.rs" 30 | 31 | [[bin]] 32 | name = "fuse_local" 33 | path = "src/bin/main_fuse_local.rs" 34 | 35 | [[bin]] 36 | name = "fuse_lambda" 37 | path = "src/bin/main_fuse_lambda.rs" 38 | 39 | [dependencies] 40 | arrow = { version = "4.3", features = ["prettyprint"] } 41 | arrow-parquet = { package = "parquet", version = "4.3", features = ["arrow"] } 42 | datafusion = { git = "https://github.com/apache/arrow-datafusion", rev = "c82c29c926e73b3c3b9c5351bcd7b01e4d0aa6a8" } 43 | arrow-flight = { version = "4.0" } 44 | deltalake = { git = "https://github.com/delta-io/delta-rs/", rev = "d56d10fac10088440e14da4abb9eeb2cf5e504a1", features = ["s3-rustls"] } 45 | 46 | chrono = { version = "0.4" } 47 | tokio = { version = "1.0", features = ["fs", "macros", "io-std"] } 48 | tokio-stream = { version = "0.1.2" } 49 | rusoto_core = { version = "0.46.0", default_features = false, features=["rustls"] } 50 | rusoto_s3 = { version = "0.46.0", default_features = false, features=["rustls"] } 51 | rusoto_lambda = { version = "0.46.0", default_features = false, features=["rustls"] } 52 | rusoto_ecs = { version = "0.46.0", default_features = false, features=["rustls"] } 53 | async-trait = "0.1" 54 | futures = "0.3" 55 | serde = { version = "1.0", features = ["derive"] } 56 | serde_json = "1.0" 57 | lambda_runtime = "0.2.1" 58 | tonic = "0.4" 59 | pin-project = "1.0" 60 | prost = "0.7" 61 | prost-types = "0.7" 62 | hyper = { version = "0.14.2", features = ["stream"] } 63 | base64 = { version = "0.13.0" } 64 | envy = {version = "0.4" } 65 | 66 | [build-dependencies] 67 | prost-build = { version = "0.7" } -------------------------------------------------------------------------------- /code/build.rs: -------------------------------------------------------------------------------- 1 | fn main() { 2 | prost_build::compile_protos(&["proto/buzz.proto"], &["proto"]) 3 | .unwrap_or_else(|e| panic!("protobuf compilation failed: {}", e)); 4 | } 5 | -------------------------------------------------------------------------------- /code/examples/delta-tbl-overwrite/_delta_log/00000000000000000000.json: -------------------------------------------------------------------------------- 1 | {"commitInfo":{"timestamp":1625060850086,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"isBlindAppend":false,"operationMetrics":{"numFiles":"1","numOutputBytes":"977","numOutputRows":"4"}}} 2 | {"protocol":{"minReaderVersion":1,"minWriterVersion":2}} 3 | {"metaData":{"id":"7cc4a548-effe-43f4-bba2-522ac7287a1b","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"year\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"dept_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"dept_staff\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1625060845706}} 4 | {"add":{"path":"part-00000-74c070ed-19a6-4ab4-83b6-7a0f0d66f70d-c000.snappy.parquet","partitionValues":{},"size":977,"modificationTime":1625060850000,"dataChange":true}} 5 | -------------------------------------------------------------------------------- /code/examples/delta-tbl-overwrite/_delta_log/00000000000000000001.json: -------------------------------------------------------------------------------- 1 | {"commitInfo":{"timestamp":1625061048043,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[]"},"readVersion":0,"isBlindAppend":false,"operationMetrics":{"numFiles":"1","numOutputBytes":"1006","numOutputRows":"8"}}} 2 | {"add":{"path":"part-00000-f4a247c9-a3bb-4b1e-adc7-7269808b8d73-c000.snappy.parquet","partitionValues":{},"size":1006,"modificationTime":1625061046000,"dataChange":true}} 3 | {"remove":{"path":"part-00000-74c070ed-19a6-4ab4-83b6-7a0f0d66f70d-c000.snappy.parquet","deletionTimestamp":1625061048041,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{},"size":977}} 4 | -------------------------------------------------------------------------------- /code/examples/delta-tbl-overwrite/part-00000-74c070ed-19a6-4ab4-83b6-7a0f0d66f70d-c000.snappy.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cloudfuse-io/buzz-rust/4c6b3972c26e81c49edb01d049e885e9a44cd8ec/code/examples/delta-tbl-overwrite/part-00000-74c070ed-19a6-4ab4-83b6-7a0f0d66f70d-c000.snappy.parquet -------------------------------------------------------------------------------- /code/examples/delta-tbl-overwrite/part-00000-f4a247c9-a3bb-4b1e-adc7-7269808b8d73-c000.snappy.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cloudfuse-io/buzz-rust/4c6b3972c26e81c49edb01d049e885e9a44cd8ec/code/examples/delta-tbl-overwrite/part-00000-f4a247c9-a3bb-4b1e-adc7-7269808b8d73-c000.snappy.parquet -------------------------------------------------------------------------------- /code/examples/delta-tbl-partition/_delta_log/00000000000000000000.json: -------------------------------------------------------------------------------- 1 | {"commitInfo":{"timestamp":1625061121471,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"year\"]"},"isBlindAppend":false,"operationMetrics":{"numFiles":"2","numOutputBytes":"1488","numOutputRows":"8"}}} 2 | {"protocol":{"minReaderVersion":1,"minWriterVersion":2}} 3 | {"metaData":{"id":"27eb2f99-ef7a-4f7a-b969-15f874716409","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"year\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"dept_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"dept_staff\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["year"],"configuration":{},"createdTime":1625061120333}} 4 | {"add":{"path":"year=2020/part-00000-755eb925-59ad-4167-b46b-db694e7f3b2c.c000.snappy.parquet","partitionValues":{"year":"2020"},"size":744,"modificationTime":1625061122000,"dataChange":true}} 5 | {"add":{"path":"year=2021/part-00000-c22c8742-53c8-4ffa-b9c3-ba7705218ca2.c000.snappy.parquet","partitionValues":{"year":"2021"},"size":744,"modificationTime":1625061122000,"dataChange":true}} 6 | -------------------------------------------------------------------------------- /code/examples/delta-tbl-partition/year=2020/part-00000-755eb925-59ad-4167-b46b-db694e7f3b2c.c000.snappy.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cloudfuse-io/buzz-rust/4c6b3972c26e81c49edb01d049e885e9a44cd8ec/code/examples/delta-tbl-partition/year=2020/part-00000-755eb925-59ad-4167-b46b-db694e7f3b2c.c000.snappy.parquet -------------------------------------------------------------------------------- /code/examples/delta-tbl-partition/year=2021/part-00000-c22c8742-53c8-4ffa-b9c3-ba7705218ca2.c000.snappy.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cloudfuse-io/buzz-rust/4c6b3972c26e81c49edb01d049e885e9a44cd8ec/code/examples/delta-tbl-partition/year=2021/part-00000-c22c8742-53c8-4ffa-b9c3-ba7705218ca2.c000.snappy.parquet -------------------------------------------------------------------------------- /code/examples/direct_s3.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use arrow_parquet::file::reader::{FileReader, Length, SerializedFileReader}; 4 | use buzz::clients::{s3, CachedFile, RangeCache}; 5 | 6 | async fn async_main() { 7 | let cache = RangeCache::new().await; 8 | // let file = CachedFile::new( 9 | // "us-east-2", 10 | // "cloudfuse-taxi-data", 11 | // // "raw_small/2009/01/data.parquet", 12 | // // 27301328, 13 | // "raw_5M/2009/01/data.parquet", 14 | // 388070114, 15 | // Arc::new(cache), 16 | // ); 17 | 18 | let (dler_id, dler_creator) = s3::downloader_creator("us-east-2"); 19 | let file_id = s3::file_id("ursa-labs-taxi-data", "2009/01/data.parquet"); 20 | 21 | let file = 22 | CachedFile::new(file_id, 461966527, Arc::new(cache), dler_id, dler_creator); 23 | 24 | // download footer 25 | let prefetch_size = 1024 * 1024; 26 | file.prefetch(file.len() - prefetch_size, prefetch_size as usize); 27 | 28 | let reader = 29 | SerializedFileReader::new(file).expect("Failed to create serialized reader"); 30 | println!("num_row_groups: {:?}", reader.metadata().num_row_groups()) 31 | } 32 | 33 | fn main() { 34 | tokio::runtime::Runtime::new() 35 | .unwrap() 36 | .block_on(async_main()); 37 | } 38 | -------------------------------------------------------------------------------- /code/examples/query-delta-taxi.json: -------------------------------------------------------------------------------- 1 | { 2 | "steps": [ 3 | { 4 | "sql": "SELECT payment_type, COUNT(payment_type) as payment_type_count FROM nyc_taxi GROUP BY payment_type", 5 | "name": "nyc_taxi_map", 6 | "step_type": "HBee", 7 | "partition_filter": "pickup_date<='2009-01-05'" 8 | }, 9 | { 10 | "sql": "SELECT payment_type, SUM(payment_type_count) FROM nyc_taxi_map GROUP BY payment_type", 11 | "name": "nyc_taxi_reduce", 12 | "step_type": "HComb" 13 | } 14 | ], 15 | "capacity": { 16 | "zones": 1 17 | }, 18 | "catalogs": [ 19 | { 20 | "name": "nyc_taxi", 21 | "type": "DeltaLake", 22 | "uri": "s3://cloudfuse-taxi-data/delta-tables/nyc-taxi-daily" 23 | } 24 | ] 25 | } -------------------------------------------------------------------------------- /code/examples/query-static-sample.json: -------------------------------------------------------------------------------- 1 | { 2 | "steps": [ 3 | { 4 | "sql": "SELECT payment_type, COUNT(payment_type) as payment_type_count FROM nyc_taxi GROUP BY payment_type", 5 | "name": "nyc_taxi_map", 6 | "step_type": "HBee", 7 | "partition_filter": "month<='2009/06'" 8 | }, 9 | { 10 | "sql": "SELECT payment_type, SUM(payment_type_count) FROM nyc_taxi_map GROUP BY payment_type", 11 | "name": "nyc_taxi_reduce", 12 | "step_type": "HComb" 13 | } 14 | ], 15 | "capacity": { 16 | "zones": 1 17 | }, 18 | "catalogs": [ 19 | { 20 | "name": "nyc_taxi", 21 | "type": "Static", 22 | "uri": "nyc_taxi_cloudfuse_sample" 23 | } 24 | ] 25 | } -------------------------------------------------------------------------------- /code/examples/query.json: -------------------------------------------------------------------------------- 1 | { 2 | "steps": [ 3 | { 4 | "sql": "SELECT payment_type, COUNT(payment_type) as payment_type_count FROM nyc_taxi GROUP BY payment_type", 5 | "name": "nyc_taxi_map", 6 | "step_type": "HBee", 7 | "partition_filter": "nyc_taxi.pickup_date<='2009/06'" 8 | }, 9 | { 10 | "sql": "SELECT payment_type, SUM(payment_type_count) FROM nyc_taxi_map GROUP BY payment_type", 11 | "name": "nyc_taxi_reduce", 12 | "step_type": "HComb" 13 | } 14 | ], 15 | "capacity": { 16 | "zones": 1 17 | }, 18 | "catalogs": [ 19 | { 20 | "name": "nyc_taxi", 21 | "type": "Static", 22 | "uri": "nyc_taxi_cloudfuse" 23 | } 24 | ] 25 | } -------------------------------------------------------------------------------- /code/examples/steps.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "sql": "SELECT payment_type, COUNT(payment_type) as payment_type_count, SUM(fare_amount) as fare_amount_sum FROM nyc_taxi_cloudfuse GROUP BY payment_type", 4 | "name": "nyc_taxi_map", 5 | "step_type": "HBee", 6 | "partition_filter": "month='2009/02'" 7 | }, 8 | { 9 | "sql": "SELECT payment_type, SUM(payment_type_count), SUM(fare_amount_sum) FROM nyc_taxi_map GROUP BY payment_type", 10 | "name": "nyc_taxi_reduce", 11 | "step_type": "HComb" 12 | } 13 | ] -------------------------------------------------------------------------------- /code/proto/buzz.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package buzz.protobuf; 4 | 5 | /////////////////////////////////////////////////////////////////////////////////////////////////// 6 | // Buzz Logical Plan 7 | /////////////////////////////////////////////////////////////////////////////////////////////////// 8 | 9 | message HBeeScanNode { 10 | string sql = 1; 11 | string source = 2; 12 | bytes schema = 3; 13 | 14 | oneof scan { 15 | S3ParquetScanNode s3_parquet = 10; 16 | } 17 | } 18 | 19 | message SizedFile { 20 | string key = 1; 21 | uint64 length = 2; 22 | } 23 | 24 | message S3ParquetScanNode { 25 | string region = 1; 26 | string bucket = 2; 27 | repeated SizedFile files = 3; 28 | } 29 | 30 | message HCombScanNode { 31 | string sql = 1; 32 | string source = 2; 33 | string query_id = 3; 34 | uint32 nb_hbee = 4; 35 | bytes schema = 5; 36 | } 37 | -------------------------------------------------------------------------------- /code/rustfmt.toml: -------------------------------------------------------------------------------- 1 | max_width = 90 -------------------------------------------------------------------------------- /code/src/bin/main_fuse_lambda.rs: -------------------------------------------------------------------------------- 1 | use std::error::Error; 2 | 3 | use buzz::error::{BuzzError, Result as BuzzResult}; 4 | use buzz::services::fuse::{ 5 | FargateHCombManager, FuseService, HttpHCombScheduler, LambdaHBeeScheduler, 6 | QueryPlanner, 7 | }; 8 | use lambda_runtime::{error::HandlerError, lambda, Context}; 9 | use serde_json::Value; 10 | 11 | pub async fn start_fuse(event: Value) -> BuzzResult<()> { 12 | let hbee_scheduler = LambdaHBeeScheduler::try_new()?; 13 | let hcomb_manager = FargateHCombManager::try_new()?; 14 | let hcomb_scheduler = HttpHCombScheduler {}; 15 | let query_planner = QueryPlanner::new(); 16 | let mut service = FuseService::new( 17 | Box::new(hbee_scheduler), 18 | Box::new(hcomb_manager), 19 | Box::new(hcomb_scheduler), 20 | query_planner, 21 | ); 22 | 23 | println!("[fuse] initialized, starting query..."); 24 | 25 | let query = serde_json::from_value(event) 26 | .map_err(|e| BuzzError::BadRequest(format!("{}", e)))?; 27 | 28 | service.run(query).await?; 29 | Ok(()) 30 | } 31 | 32 | fn main() -> Result<(), Box> { 33 | lambda!(my_handler); 34 | Ok(()) 35 | } 36 | 37 | fn my_handler(event: Value, _: Context) -> Result { 38 | println!("Input Event: {:?}", event); 39 | tokio::runtime::Runtime::new() 40 | .unwrap() 41 | .block_on(start_fuse(event)) 42 | .unwrap(); 43 | Ok(Value::String("Ok!".to_owned())) 44 | } 45 | -------------------------------------------------------------------------------- /code/src/bin/main_fuse_local.rs: -------------------------------------------------------------------------------- 1 | use buzz::services::fuse::{ 2 | FuseService, HttpHCombScheduler, QueryPlanner, TestHBeeScheduler, TestHCombManager, 3 | }; 4 | use std::fs::read_to_string; 5 | 6 | pub async fn start_fuse( 7 | hbee_addr: &str, 8 | hcomb_addr: &str, 9 | ) -> Result<(), Box> { 10 | // a fuse is all about the right delay, otherwise everything explodes to your face :) 11 | tokio::time::sleep(std::time::Duration::new(1, 0)).await; 12 | 13 | let query_string = read_to_string("./examples/query-static-sample.json")?; 14 | 15 | let hbee_scheduler = TestHBeeScheduler { 16 | domain: hbee_addr.to_owned(), 17 | }; 18 | let hcomb_manager = TestHCombManager { 19 | domain: hcomb_addr.to_owned(), 20 | }; 21 | let hcomb_scheduler = HttpHCombScheduler {}; 22 | let query_planner = QueryPlanner::new(); 23 | let mut service = FuseService::new( 24 | Box::new(hbee_scheduler), 25 | Box::new(hcomb_manager), 26 | Box::new(hcomb_scheduler), 27 | query_planner, 28 | ); 29 | 30 | let query = serde_json::from_str(&query_string)?; 31 | 32 | service.run(query).await?; 33 | Ok(()) 34 | } 35 | 36 | #[tokio::main] 37 | #[allow(dead_code)] 38 | async fn main() -> Result<(), Box> { 39 | start_fuse("hbee", "hcomb").await 40 | } 41 | -------------------------------------------------------------------------------- /code/src/bin/main_hbee_lambda.rs: -------------------------------------------------------------------------------- 1 | use std::error::Error; 2 | 3 | use buzz::models::HBeeEvent; 4 | use buzz::services::hbee::{HBeeService, HttpCollector}; 5 | use lambda_runtime::{error::HandlerError, lambda, Context}; 6 | use serde_json::Value; 7 | 8 | async fn exec(event: Value) -> Result<(), Box> { 9 | let hbee_event: HBeeEvent = serde_json::from_value(event)?; 10 | let (hbee_table_desc, sql, source) = hbee_event.plan.parse()?; 11 | let collector = Box::new(HttpCollector {}); 12 | let hbee_service = HBeeService::new(collector).await; 13 | hbee_service 14 | .execute_query( 15 | hbee_event.query_id, 16 | hbee_table_desc, 17 | sql, 18 | source, 19 | hbee_event.hcomb_address, 20 | ) 21 | .await 22 | .map_err(|e| Box::new(e).into()) 23 | } 24 | 25 | fn main() -> Result<(), Box> { 26 | lambda!(my_handler); 27 | Ok(()) 28 | } 29 | 30 | fn my_handler(event: Value, _: Context) -> Result { 31 | println!("Input Event: {:?}", event); 32 | tokio::runtime::Runtime::new() 33 | .unwrap() 34 | .block_on(exec(event)) 35 | .unwrap(); 36 | Ok(Value::String("Ok!".to_owned())) 37 | } 38 | -------------------------------------------------------------------------------- /code/src/bin/main_hbee_local.rs: -------------------------------------------------------------------------------- 1 | use std::error::Error; 2 | 3 | use buzz::models::HBeeEvent; 4 | use buzz::services::hbee::{HBeeService, HttpCollector}; 5 | use futures::StreamExt; 6 | use hyper::service::{make_service_fn, service_fn}; 7 | use hyper::{Body, Request, Response, Server}; 8 | 9 | type DynError = Box; 10 | 11 | async fn exec(mut req: Request) -> Result, DynError> { 12 | println!("[hbee] hbee_server.serve()"); 13 | let mut body = Vec::new(); 14 | while let Some(chunk) = req.body_mut().next().await { 15 | body.extend_from_slice(&chunk?); 16 | } 17 | let hbee_event: HBeeEvent = serde_json::from_slice(&body)?; 18 | let (hbee_table_desc, sql, source) = hbee_event.plan.parse()?; 19 | tokio::spawn(async { 20 | let collector = Box::new(HttpCollector {}); 21 | let hbee_service = HBeeService::new(collector).await; 22 | let res = hbee_service 23 | .execute_query( 24 | hbee_event.query_id, 25 | hbee_table_desc, 26 | sql, 27 | source, 28 | hbee_event.hcomb_address, 29 | ) 30 | .await; 31 | match res { 32 | Ok(_) => println!("[hbee] success"), 33 | Err(e) => println!("[hbee] exec error: {}", e), 34 | }; 35 | }); 36 | Ok(Response::new(Body::from("Ok!"))) 37 | } 38 | 39 | // this endpoint helps simulating lambda locally 40 | pub async fn start_hbee_server() -> Result<(), DynError> { 41 | let make_svc = make_service_fn(|_conn| async { Ok::<_, DynError>(service_fn(exec)) }); 42 | let addr = ([0, 0, 0, 0], 3000).into(); 43 | let server = Server::bind(&addr).serve(make_svc); 44 | println!("[hbee] Listening on http://{}", addr); 45 | server.await?; 46 | Ok(()) 47 | } 48 | 49 | #[tokio::main] 50 | #[allow(dead_code)] 51 | async fn main() -> Result<(), DynError> { 52 | start_hbee_server().await 53 | } 54 | -------------------------------------------------------------------------------- /code/src/bin/main_hbee_tests.rs: -------------------------------------------------------------------------------- 1 | use std::error::Error; 2 | 3 | use buzz::example_catalog; 4 | use buzz::services::fuse::{HBeePlan, QueryPlanner}; 5 | use buzz::services::hbee::{HBeeService, NoopCollector}; 6 | use lambda_runtime::{error::HandlerError, lambda, Context}; 7 | use serde_json::Value; 8 | 9 | const QUERY_ID: &str = "test_query"; 10 | 11 | async fn new_plan(event: Value) -> Result> { 12 | let mut qp = QueryPlanner::new(); 13 | qp.add_catalog("nyc_taxi_ursa", example_catalog::nyc_taxi_ursa())?; 14 | qp.add_catalog("nyc_taxi_cloudfuse", example_catalog::nyc_taxi_cloudfuse())?; 15 | qp.add_catalog( 16 | "nyc_taxi_cloudfuse_sample", 17 | example_catalog::nyc_taxi_cloudfuse_sample(), 18 | )?; 19 | let steps = serde_json::from_value(event)?; 20 | qp.plan(QUERY_ID.to_owned(), steps, 1) 21 | .await 22 | .map(|dist_plan| { 23 | dist_plan 24 | .zones 25 | .into_iter() 26 | .next() 27 | .unwrap() 28 | .hbee 29 | .into_iter() 30 | .next() 31 | .unwrap() 32 | }) 33 | .map_err(|e| Box::new(e).into()) 34 | } 35 | 36 | async fn exec(event: Value) -> Result<(), Box> { 37 | let plan = new_plan(event).await?; 38 | let collector = Box::new(NoopCollector {}); 39 | let hbee_service = HBeeService::new(collector).await; 40 | hbee_service 41 | .execute_query( 42 | QUERY_ID.to_owned(), 43 | plan.table, 44 | plan.sql, 45 | plan.source, 46 | "http://mock_endpoint".to_owned(), 47 | ) 48 | .await 49 | .map_err(|e| Box::new(e).into()) 50 | } 51 | 52 | fn main() -> Result<(), Box> { 53 | lambda!(my_handler); 54 | Ok(()) 55 | } 56 | 57 | fn my_handler(event: Value, _: Context) -> Result { 58 | println!("Input Event: {:?}", event); 59 | tokio::runtime::Runtime::new() 60 | .unwrap() 61 | .block_on(exec(event)) 62 | .unwrap(); 63 | Ok(Value::String("Ok!".to_owned())) 64 | } 65 | -------------------------------------------------------------------------------- /code/src/bin/main_hcomb.rs: -------------------------------------------------------------------------------- 1 | use buzz::services::hcomb::{FlightServiceImpl, HCombService}; 2 | 3 | pub async fn start_hcomb_server() -> Result<(), Box> { 4 | let hcomb_service = HCombService::new(); 5 | let flight_service = FlightServiceImpl::new(hcomb_service); 6 | let server_handle = flight_service.start().await; 7 | server_handle.await.unwrap(); 8 | 9 | Ok(()) 10 | } 11 | 12 | #[tokio::main] 13 | #[allow(dead_code)] 14 | async fn main() -> Result<(), Box> { 15 | start_hcomb_server().await 16 | } 17 | -------------------------------------------------------------------------------- /code/src/bin/main_integ.rs: -------------------------------------------------------------------------------- 1 | mod main_fuse_local; 2 | mod main_hbee_local; 3 | mod main_hcomb; 4 | 5 | #[tokio::main] 6 | async fn main() -> Result<(), Box> { 7 | tokio::select! { 8 | res = main_fuse_local::start_fuse("localhost", "localhost") => { 9 | println!("[integ] fuse result: {:?}", res); 10 | } 11 | res = main_hbee_local::start_hbee_server() => { 12 | println!("[integ] hbee server failed: {:?}", res); 13 | } 14 | res = main_hcomb::start_hcomb_server() => { 15 | println!("[integ] hcomb server failed: {:?}", res); 16 | } 17 | } 18 | Ok(()) 19 | } 20 | -------------------------------------------------------------------------------- /code/src/clients/cached_file.rs: -------------------------------------------------------------------------------- 1 | use std::fmt; 2 | use std::sync::Arc; 3 | 4 | use super::range_cache::{CachedRead, Downloader, RangeCache}; 5 | use crate::error::BuzzError; 6 | use arrow_parquet::errors::{ParquetError, Result as ParquetResult}; 7 | use arrow_parquet::file::reader::{ChunkReader, Length}; 8 | 9 | #[derive(Clone)] 10 | pub struct CachedFile { 11 | dler_id: String, 12 | file_id: String, 13 | length: u64, 14 | cache: Arc, 15 | } 16 | 17 | impl fmt::Debug for CachedFile { 18 | fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { 19 | f.debug_struct("CachedFile") 20 | .field("downloader_id", &self.dler_id) 21 | .field("file_id", &self.file_id) 22 | .field("length", &self.length) 23 | .finish() 24 | } 25 | } 26 | 27 | impl CachedFile { 28 | pub fn new( 29 | file_id: String, 30 | length: u64, 31 | cache: Arc, 32 | dler_id: String, 33 | dler_creator: F, 34 | ) -> Self 35 | where 36 | F: Fn() -> Arc, 37 | { 38 | cache.register_downloader(&dler_id, dler_creator); 39 | CachedFile { 40 | dler_id, 41 | file_id, 42 | length, 43 | cache, 44 | } 45 | } 46 | 47 | pub fn prefetch(&self, start: u64, length: usize) { 48 | self.cache 49 | .schedule(self.dler_id.clone(), self.file_id.clone(), start, length); 50 | } 51 | } 52 | 53 | impl Length for CachedFile { 54 | fn len(&self) -> u64 { 55 | self.length 56 | } 57 | } 58 | 59 | impl ChunkReader for CachedFile { 60 | type T = CachedRead; 61 | 62 | fn get_read(&self, start: u64, length: usize) -> ParquetResult { 63 | self.cache 64 | .get(self.dler_id.clone(), self.file_id.clone(), start, length) 65 | .map_err(|e| match e { 66 | BuzzError::ParquetError(err) => err, 67 | err => ParquetError::General(format!("{}", err)), 68 | }) 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /code/src/clients/fargate.rs: -------------------------------------------------------------------------------- 1 | use std::iter::IntoIterator; 2 | use std::str::FromStr; 3 | use std::sync::Arc; 4 | use std::time::{Duration, Instant}; 5 | 6 | use crate::error::{BuzzError, Result}; 7 | use crate::models::env::{self, FargateConfig}; 8 | use rusoto_core::Region; 9 | use rusoto_ecs::{ 10 | AwsVpcConfiguration, DescribeTasksRequest, Ecs, EcsClient, ListTasksRequest, 11 | ListTasksResponse, NetworkConfiguration, RunTaskRequest, 12 | }; 13 | use tokio::time::timeout; 14 | 15 | pub struct FargateCreationClient { 16 | client: Arc, 17 | config: Arc, 18 | } 19 | 20 | impl FargateCreationClient { 21 | pub fn try_new() -> Result { 22 | let config = Arc::new(env::get_fargate_config()?); 23 | Ok(Self { 24 | client: new_client(&config.aws_region), 25 | config, 26 | }) 27 | } 28 | } 29 | 30 | impl FargateCreationClient { 31 | /// Create a new fargate task and returns private IP. 32 | /// The task might not be ready to receive requests yet. 33 | pub async fn create_new(&self) -> Result { 34 | let start = Instant::now(); 35 | let config = Arc::clone(&self.config); 36 | 37 | let mut task_arn = self 38 | .get_existing_task(config.hcomb_cluster_name.clone()) 39 | .await; 40 | 41 | if task_arn.is_none() { 42 | task_arn = Some( 43 | self.start_task( 44 | config.hcomb_task_def_arn.clone(), 45 | config.hcomb_cluster_name.clone(), 46 | config.public_subnets.clone(), 47 | config.hcomb_task_sg_id.clone(), 48 | ) 49 | .await?, 50 | ); 51 | println!("[fuse] task started"); 52 | tokio::time::sleep(Duration::from_secs(1)).await; 53 | } 54 | 55 | let result = self 56 | .wait_for_provisioning(task_arn.unwrap(), config.hcomb_cluster_name.clone()) 57 | .await?; 58 | 59 | println!( 60 | "[fuse] took {}ms to create/find task", 61 | start.elapsed().as_millis() 62 | ); 63 | 64 | Ok(result) 65 | } 66 | 67 | /// Get existing task ARN if their is one 68 | /// /// TODO better error management 69 | async fn get_existing_task(&self, cluster_name: String) -> Option { 70 | let request = ListTasksRequest { 71 | cluster: Some(cluster_name), 72 | container_instance: None, 73 | desired_status: Some("RUNNING".to_owned()), 74 | family: None, 75 | launch_type: None, 76 | max_results: Some(1), 77 | next_token: None, 78 | service_name: None, 79 | started_by: None, 80 | }; 81 | 82 | let result = 83 | timeout(Duration::from_secs(2), self.client.list_tasks(request)).await; 84 | 85 | // if request for existing task failed for any reason, return None 86 | match result { 87 | Ok(Ok(ListTasksResponse { 88 | task_arns: Some(arns), 89 | .. 90 | })) if arns.len() > 0 => Some(arns[0].clone()), 91 | _ => None, 92 | } 93 | } 94 | 95 | /// Start new task and return its arn 96 | /// TODO better error management 97 | async fn start_task( 98 | &self, 99 | task_definition: String, 100 | cluster_name: String, 101 | subnets: Vec, 102 | security_group: String, 103 | ) -> Result { 104 | let input = RunTaskRequest { 105 | task_definition, 106 | count: Some(1), 107 | cluster: Some(cluster_name), 108 | group: None, 109 | network_configuration: Some(NetworkConfiguration { 110 | awsvpc_configuration: Some(AwsVpcConfiguration { 111 | assign_public_ip: Some("ENABLED".to_owned()), 112 | subnets, 113 | security_groups: Some(vec![security_group]), 114 | }), 115 | }), 116 | enable_ecs_managed_tags: None, 117 | capacity_provider_strategy: None, 118 | placement_constraints: None, 119 | placement_strategy: None, 120 | platform_version: None, 121 | launch_type: None, 122 | overrides: None, 123 | propagate_tags: None, 124 | reference_id: None, 125 | started_by: None, 126 | tags: None, 127 | }; 128 | let result = timeout(Duration::from_secs(5), self.client.run_task(input)) 129 | .await 130 | .map_err(|e| BuzzError::CloudClient(format!("{}", e)))? 131 | .map_err(|e| BuzzError::CloudClient(format!("{}", e)))?; 132 | if let Some(failures) = result.failures { 133 | if failures.len() > 0 { 134 | return Err(BuzzError::CloudClient(format!( 135 | "An error occured with AWS Fargate task creation: {:?}", 136 | failures 137 | ))); 138 | } 139 | } 140 | 141 | Ok(result 142 | .tasks 143 | .unwrap() 144 | .into_iter() 145 | .next() 146 | .unwrap() 147 | .task_arn 148 | .unwrap()) 149 | } 150 | 151 | /// Wait for the given task to be provisioned and attributed a private IP 152 | /// TODO better error management 153 | /// TODO fargate container lifecycle 154 | async fn wait_for_provisioning( 155 | &self, 156 | task_arn: String, 157 | hcomb_cluster_name: String, 158 | ) -> Result { 159 | loop { 160 | let input = DescribeTasksRequest { 161 | cluster: Some(hcomb_cluster_name.clone()), 162 | include: None, 163 | tasks: vec![task_arn.clone()], 164 | }; 165 | let description = self 166 | .client 167 | .describe_tasks(input) 168 | .await 169 | .map_err(|e| BuzzError::CloudClient(format!("{}", e)))? 170 | .tasks 171 | .unwrap(); 172 | 173 | let attachment_props = description[0].attachments.as_ref().unwrap()[0] 174 | .details 175 | .as_ref() 176 | .unwrap(); 177 | 178 | for prop in attachment_props { 179 | if let Some(ref key) = prop.name { 180 | if key == "privateIPv4Address" && prop.value.as_ref().is_some() { 181 | return Ok(prop.value.as_ref().unwrap().clone()); 182 | } 183 | } 184 | } 185 | 186 | tokio::time::sleep(Duration::from_millis(200)).await; 187 | } 188 | } 189 | } 190 | 191 | //// Lambda Client //// 192 | 193 | fn new_client(region: &str) -> Arc { 194 | let region = Region::from_str(region).unwrap(); 195 | Arc::new(EcsClient::new(region)) 196 | } 197 | -------------------------------------------------------------------------------- /code/src/clients/flight_client.rs: -------------------------------------------------------------------------------- 1 | use std::error::Error; 2 | use std::pin::Pin; 3 | 4 | use crate::datasource::HCombTableDesc; 5 | use crate::flight_utils; 6 | use crate::models::{actions, HCombAddress}; 7 | use crate::serde; 8 | use arrow::error::Result as ArrowResult; 9 | use arrow::record_batch::RecordBatch; 10 | use arrow_flight::flight_service_client::FlightServiceClient; 11 | use arrow_flight::Ticket; 12 | use futures::Stream; 13 | use futures::StreamExt; 14 | use prost::Message; 15 | 16 | /// Calls the hcomb do_get endpoint, expecting the first message to be the schema 17 | pub async fn call_do_get( 18 | address: &HCombAddress, 19 | hcomb_table: &HCombTableDesc, 20 | sql: String, 21 | source: String, 22 | ) -> Result>>>, Box> { 23 | // Create Flight client 24 | let mut client = FlightServiceClient::connect(address.clone()).await?; 25 | 26 | let proto_plan = serde::serialize_hcomb(hcomb_table, sql, source); 27 | 28 | let mut buf = vec![]; 29 | proto_plan.encode(&mut buf)?; 30 | 31 | // Call do_get to execute a SQL query and receive results 32 | let request = tonic::Request::new(Ticket { ticket: buf }); 33 | let stream = client.do_get(request).await?.into_inner(); 34 | let (_, record_batch_stream) = flight_utils::flight_to_batches(stream).await?; 35 | Ok(Box::pin(record_batch_stream)) 36 | } 37 | 38 | pub async fn call_do_put( 39 | query_id: String, 40 | address: &HCombAddress, 41 | results: Vec, 42 | ) -> Result<(), Box> { 43 | // Create Flight client after delay, to leave time for the server to boot 44 | let input = flight_utils::batch_vec_to_flight(&query_id, results).await?; 45 | 46 | let request = tonic::Request::new(input); 47 | let mut client = FlightServiceClient::connect(address.clone()).await?; 48 | // wait for the response to be complete but don't do anything with it 49 | client 50 | .do_put(request) 51 | .await? 52 | .into_inner() 53 | .collect::>() 54 | .await; 55 | 56 | Ok(()) 57 | } 58 | 59 | pub async fn call_fail_action( 60 | query_id: String, 61 | address: &HCombAddress, 62 | reason: String, 63 | ) -> Result<(), Box> { 64 | let action = arrow_flight::Action { 65 | body: serde_json::to_vec(&actions::Fail { query_id, reason }).unwrap(), 66 | r#type: actions::ActionType::Fail.to_string(), 67 | }; 68 | 69 | let request = tonic::Request::new(action); 70 | 71 | let mut client = FlightServiceClient::connect(address.clone()).await?; 72 | // wait for the response to be complete but don't do anything with it 73 | client 74 | .do_action(request) 75 | .await? 76 | .into_inner() 77 | .collect::>() 78 | .await; 79 | 80 | Ok(()) 81 | } 82 | 83 | pub async fn try_connect(address: &HCombAddress) -> Result<(), Box> { 84 | FlightServiceClient::connect(address.clone()).await?; 85 | Ok(()) 86 | } 87 | -------------------------------------------------------------------------------- /code/src/clients/lambda.rs: -------------------------------------------------------------------------------- 1 | use std::convert::Into; 2 | use std::str::FromStr; 3 | use std::sync::Arc; 4 | use std::time::Duration; 5 | 6 | use crate::error::{BuzzError, Result}; 7 | use crate::models::env; 8 | use rusoto_core::Region; 9 | use rusoto_lambda::{InvocationRequest, Lambda, LambdaClient}; 10 | use tokio::time::timeout; 11 | 12 | pub struct LambdaInvokeClient { 13 | client: Arc, 14 | lambda_name: String, 15 | } 16 | 17 | impl LambdaInvokeClient { 18 | pub fn try_new() -> Result { 19 | let config = env::get_lambda_config()?; 20 | Ok(Self { 21 | client: new_client(&config.aws_region), 22 | lambda_name: config.hbee_lambda_name, 23 | }) 24 | } 25 | } 26 | 27 | impl LambdaInvokeClient { 28 | /// Invoke an async lambda 29 | pub async fn invoke(&self, body: Vec) -> Result<()> { 30 | let input = InvocationRequest { 31 | client_context: None, 32 | function_name: self.lambda_name.clone(), 33 | invocation_type: Some("Event".to_owned()), 34 | log_type: None, 35 | payload: Some(body.into()), 36 | qualifier: None, 37 | }; 38 | let result = timeout(Duration::from_secs(5), self.client.invoke(input)) 39 | .await 40 | .map_err(|e| BuzzError::CloudClient(format!("{}", e)))? // first unwrap timeout 41 | .map_err(|e| BuzzError::CloudClient(format!("{}", e)))?; 42 | if let Some(is_handled) = result.function_error { 43 | Err(BuzzError::CloudClient(format!( 44 | "An error occured with AWS Lambda invokation: {}", 45 | is_handled 46 | ))) 47 | } else { 48 | Ok(()) 49 | } 50 | } 51 | } 52 | 53 | //// Lambda Client //// 54 | 55 | fn new_client(region: &str) -> Arc { 56 | let region = Region::from_str(region).unwrap(); 57 | Arc::new(LambdaClient::new(region)) 58 | } 59 | -------------------------------------------------------------------------------- /code/src/clients/mod.rs: -------------------------------------------------------------------------------- 1 | //! modules that help connecting to the outside world 2 | 3 | mod cached_file; 4 | pub mod fargate; 5 | pub mod flight_client; 6 | pub mod lambda; 7 | mod range_cache; 8 | pub mod s3; 9 | 10 | pub use cached_file::CachedFile; 11 | pub use range_cache::{Downloader, RangeCache}; 12 | -------------------------------------------------------------------------------- /code/src/clients/s3.rs: -------------------------------------------------------------------------------- 1 | use std::str::FromStr; 2 | use std::sync::Arc; 3 | 4 | use super::range_cache::Downloader; 5 | use crate::error::{BuzzError, Result}; 6 | use async_trait::async_trait; 7 | use rusoto_core::Region; 8 | use rusoto_s3::{GetObjectOutput, GetObjectRequest, S3Client, S3}; 9 | use tokio::io::AsyncReadExt; 10 | 11 | //// Implementation of the `download` function used by the range cache to fetch data 12 | 13 | #[derive(Clone)] 14 | struct S3Downloader { 15 | client: Arc, 16 | } 17 | 18 | #[async_trait] 19 | impl Downloader for S3Downloader { 20 | async fn download( 21 | &self, 22 | file_id: String, 23 | start: u64, 24 | length: usize, 25 | ) -> Result> { 26 | let mut file_id_split = file_id.split("/"); 27 | let range = format!("bytes={}-{}", start, start + length as u64 - 1); 28 | println!("file_id:{}", file_id); 29 | let get_obj_req = GetObjectRequest { 30 | bucket: file_id_split.next().unwrap().to_owned(), 31 | key: file_id_split.collect::>().join("/"), 32 | range: Some(range), 33 | ..Default::default() 34 | }; 35 | let obj: GetObjectOutput = self 36 | .client 37 | .get_object(get_obj_req) 38 | .await 39 | .map_err(|e| BuzzError::Download(format!("{}", e)))?; 40 | let mut reader = obj.body.unwrap().into_async_read(); 41 | let mut res = vec![]; 42 | res.reserve(length); 43 | let bytes_read = reader 44 | .read_to_end(&mut res) 45 | .await 46 | .map_err(|e| BuzzError::Download(format!("{}", e)))?; 47 | if bytes_read != length { 48 | Err(BuzzError::Download( 49 | "Not the expected number of bytes".to_owned(), 50 | )) 51 | } else { 52 | Ok(res) 53 | } 54 | } 55 | } 56 | 57 | pub fn downloader_creator( 58 | region: &str, 59 | ) -> (String, Box Arc>) { 60 | let region_clone = region.to_owned(); 61 | let creator: Box Arc> = Box::new(move || { 62 | Arc::new(S3Downloader { 63 | client: new_client(®ion_clone), 64 | }) 65 | }); 66 | 67 | (format!("s3::{}", region), creator) 68 | } 69 | 70 | pub fn file_id(bucket: &str, key: &str) -> String { 71 | format!("{}/{}", bucket, key) 72 | } 73 | 74 | //// S3 Client //// 75 | 76 | fn new_client(region: &str) -> Arc { 77 | let region = Region::from_str(region).unwrap(); 78 | Arc::new(S3Client::new(region)) 79 | } 80 | -------------------------------------------------------------------------------- /code/src/datasource/catalog/delta_catalog.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use super::{catalog_schema, SplittableTable}; 4 | use crate::datasource::{HBeeTableDesc, S3ParquetTable}; 5 | use crate::error::Result; 6 | use crate::internal_err; 7 | use crate::models::SizedFile; 8 | use arrow::array::*; 9 | use arrow::datatypes::*; 10 | use arrow::record_batch::RecordBatch; 11 | use async_trait::async_trait; 12 | use datafusion::datasource::{MemTable, TableProvider}; 13 | use deltalake::storage::{ 14 | file::FileStorageBackend, s3::S3StorageBackend, StorageBackend, 15 | }; 16 | use deltalake::{DeltaTable, Schema as DeltaSchema}; 17 | use rusoto_core::Region; 18 | use rusoto_s3::S3Client; 19 | use std::convert::{TryFrom, TryInto}; 20 | use std::str::FromStr; 21 | 22 | /// A catalog table that gets values from delta-rs. 23 | /// Only supports S3 parquet files for now and simply sends each file into a different hbee. 24 | pub struct DeltaCatalogTable { 25 | region: String, 26 | bucket: String, 27 | root: String, 28 | delta_table: DeltaTable, 29 | } 30 | 31 | impl DeltaCatalogTable { 32 | /// Create a catalog from a Delta S3 URI and a region. 33 | pub async fn try_new(uri: &str, region: String) -> Result { 34 | let region_enum = Region::from_str(®ion).unwrap(); 35 | let storage_backend: Box = 36 | Box::new(S3StorageBackend::new_with(S3Client::new(region_enum), None)); 37 | let mut delta_table = DeltaTable::new(&uri, storage_backend)?; 38 | delta_table.load().await?; 39 | let uri_split = uri.split("/").collect::>(); 40 | let bucket = uri_split[2].to_owned(); 41 | let root = uri_split[3..].join("/") + "/"; 42 | Ok(Self { 43 | delta_table, 44 | bucket, 45 | root, 46 | region, 47 | }) 48 | } 49 | 50 | /// Create a catalog from a local Delta path. 51 | pub async fn new_local(path: &str) -> Self { 52 | let mut delta_table = 53 | DeltaTable::new(path, Box::new(FileStorageBackend::new(path))) 54 | .expect("DeltaTable from local storage created"); 55 | delta_table 56 | .load() 57 | .await 58 | .expect("DeltaTable from local storage loaded"); 59 | 60 | Self { 61 | delta_table, 62 | bucket: "local".to_owned(), 63 | root: path.to_owned() + "/", 64 | region: "local".to_owned(), 65 | } 66 | } 67 | 68 | async fn to_table(&self) -> Result> { 69 | let actions = self.delta_table.get_actions(); 70 | let mut key_builder = StringBuilder::new(actions.len()); 71 | let mut length_builder = UInt64Builder::new(actions.len()); 72 | let delta_metadata = self.delta_table.get_metadata()?; 73 | let mut partition_builders = delta_metadata 74 | .partition_columns 75 | .iter() 76 | .map(|_| StringBuilder::new(actions.len())) 77 | .collect::>(); 78 | for catalog_action in actions { 79 | key_builder.append_value(&(self.root.clone() + &catalog_action.path))?; 80 | let catalog_file_size: u64 = catalog_action 81 | .size 82 | .try_into() 83 | .map_err(|_| internal_err!("Unexpected negative length from delta"))?; 84 | length_builder.append_value(catalog_file_size)?; 85 | for (i, part_col) in delta_metadata.partition_columns.iter().enumerate() { 86 | // TODO check whether partition value might be some kind of wildcard 87 | partition_builders[i] 88 | .append_value(catalog_action.partition_values[part_col].clone())?; 89 | } 90 | } 91 | 92 | // finish all builders 93 | let mut col_arrays: Vec = vec![ 94 | ArrayBuilder::finish(&mut key_builder), 95 | ArrayBuilder::finish(&mut length_builder), 96 | ]; 97 | for mut partition_builder in partition_builders { 98 | col_arrays.push(ArrayBuilder::finish(&mut partition_builder)); 99 | } 100 | 101 | let schema = catalog_schema(&delta_metadata.partition_columns); 102 | 103 | let record_batch = RecordBatch::try_new(Arc::clone(&schema), col_arrays)?; 104 | Ok(Arc::new(MemTable::try_new( 105 | schema, 106 | vec![vec![record_batch]], 107 | )?)) 108 | } 109 | } 110 | 111 | #[async_trait] 112 | impl SplittableTable for DeltaCatalogTable { 113 | fn split(&self, files: Vec) -> Vec { 114 | files 115 | .into_iter() 116 | .map(|file| { 117 | S3ParquetTable::new( 118 | self.region.clone(), 119 | self.bucket.clone(), 120 | vec![file], 121 | self.schema(), 122 | ) 123 | }) 124 | .collect() 125 | } 126 | fn partition_columns(&self) -> &[String] { 127 | &self 128 | .delta_table 129 | .get_metadata() 130 | .expect("TODO Handle Delta failure") 131 | .partition_columns 132 | } 133 | fn schema(&self) -> SchemaRef { 134 | Arc::new( 135 | >::try_from( 136 | DeltaTable::schema(&self.delta_table).unwrap(), 137 | ) 138 | .unwrap(), 139 | ) 140 | } 141 | async fn file_table(&self) -> Arc { 142 | self.to_table().await.unwrap() 143 | } 144 | } 145 | 146 | #[cfg(test)] 147 | mod tests { 148 | use super::*; 149 | 150 | #[tokio::test] 151 | async fn test_simple_delta_catalog() { 152 | let catalog = 153 | DeltaCatalogTable::new_local("./examples/delta-tbl-overwrite").await; 154 | let table = catalog.to_table().await.expect("Logical table created"); 155 | let table_phys = table 156 | .scan(&None, 100, &[], None) 157 | .expect("Physical table created"); 158 | let catalog_rgs = datafusion::physical_plan::collect(table_phys) 159 | .await 160 | .expect("Table collected"); 161 | 162 | let expected_batches = vec![RecordBatch::try_new( 163 | catalog_schema(&[]), 164 | vec![ 165 | Arc::new(StringArray::from(vec![ 166 | "./examples/delta-tbl-overwrite/part-00000-f4a247c9-a3bb-4b1e-adc7-7269808b8d73-c000.snappy.parquet", 167 | ])), 168 | Arc::new(UInt64Array::from(vec![1006])), 169 | ], 170 | ) 171 | .expect("Build target RecordBatch")]; 172 | 173 | assert_eq!( 174 | format!("{:?}", catalog_rgs), 175 | format!("{:?}", expected_batches) 176 | ); 177 | } 178 | 179 | #[tokio::test] 180 | async fn test_partitioned_delta_catalog() { 181 | let catalog = 182 | DeltaCatalogTable::new_local("./examples/delta-tbl-partition").await; 183 | let table = catalog.to_table().await.expect("Logical table created"); 184 | let table_phys = table 185 | .scan(&None, 100, &[], None) 186 | .expect("Physical table created"); 187 | let catalog_rgs = datafusion::physical_plan::collect(table_phys) 188 | .await 189 | .expect("Table collected"); 190 | 191 | let expected_batches = vec![RecordBatch::try_new( 192 | catalog_schema(&["year".to_owned()]), 193 | vec![ 194 | Arc::new(StringArray::from(vec![ 195 | "./examples/delta-tbl-partition/year=2020/part-00000-755eb925-59ad-4167-b46b-db694e7f3b2c.c000.snappy.parquet", 196 | "./examples/delta-tbl-partition/year=2021/part-00000-c22c8742-53c8-4ffa-b9c3-ba7705218ca2.c000.snappy.parquet", 197 | ])), 198 | Arc::new(UInt64Array::from(vec![744, 744])), 199 | Arc::new(StringArray::from(vec!["2020", "2021"])), 200 | ], 201 | ) 202 | .expect("Build target RecordBatch")]; 203 | 204 | assert_eq!( 205 | format!("{:?}", catalog_rgs), 206 | format!("{:?}", expected_batches) 207 | ); 208 | } 209 | } 210 | -------------------------------------------------------------------------------- /code/src/datasource/catalog/mod.rs: -------------------------------------------------------------------------------- 1 | use std::any::Any; 2 | use std::sync::Arc; 3 | 4 | use crate::datasource::HBeeTableDesc; 5 | use crate::error::{BuzzError, Result}; 6 | use crate::models::SizedFile; 7 | use arrow::array::*; 8 | use arrow::datatypes::*; 9 | use async_trait::async_trait; 10 | use datafusion::datasource::datasource::Statistics; 11 | use datafusion::datasource::TableProvider; 12 | use datafusion::error::{DataFusionError, Result as DataFusionResult}; 13 | use datafusion::execution::context::ExecutionContext; 14 | use datafusion::logical_plan::Expr; 15 | use datafusion::physical_plan::ExecutionPlan; 16 | 17 | /// A specific type of TableProvider that cannot be converted to a physical plan 18 | /// but can be splitted to be distributed to hbees 19 | #[async_trait] 20 | pub trait SplittableTable { 21 | fn split(&self, files: Vec) -> Vec; 22 | /// Get the names of the partitioning columns, in order of evaluation. 23 | fn partition_columns(&self) -> &[String]; 24 | /// schema including the partition columns 25 | fn schema(&self) -> SchemaRef; 26 | fn statistics(&self) -> Statistics { 27 | Statistics::default() 28 | } 29 | async fn file_table(&self) -> Arc; 30 | } 31 | 32 | /// A generic catalog table that wraps splittable tables 33 | pub struct CatalogTable { 34 | source_table: Box, 35 | } 36 | 37 | impl CatalogTable { 38 | pub fn new(source_table: Box) -> Self { 39 | Self { source_table } 40 | } 41 | 42 | /// Explore the catalog with the given `partition_filter` and generate the tables to be processed by each hbee. 43 | pub async fn split( 44 | &self, 45 | partition_filters: &Option, 46 | ) -> Result> { 47 | let files = self.filter_catalog(partition_filters).await?; 48 | Ok(self.source_table.split(files)) 49 | } 50 | 51 | /// Applies the given filters 52 | async fn filter_catalog( 53 | &self, 54 | partition_filters: &Option, 55 | ) -> Result> { 56 | let phys_plan; 57 | { 58 | let mut context = ExecutionContext::new(); 59 | context.register_table("catalog", self.source_table.file_table().await)?; 60 | let sql_pattern = "SELECT * FROM catalog"; 61 | let sql_statement = match partition_filters { 62 | Some(sql_where) => format!("{} WHERE {}", sql_pattern, sql_where), 63 | None => sql_pattern.to_owned(), 64 | }; 65 | let df = context.sql(&sql_statement)?; 66 | phys_plan = context.create_physical_plan(&df.to_logical_plan())?; 67 | } 68 | 69 | let file_rec = datafusion::physical_plan::collect(phys_plan).await?; 70 | 71 | file_rec 72 | .iter() 73 | .map(|rec_batch| { 74 | let key_array = rec_batch 75 | .column(0) 76 | .as_any() 77 | .downcast_ref::() 78 | .ok_or(BuzzError::Execution(format!( 79 | "Invalid type for catalog keys" 80 | )))?; 81 | let length_array = rec_batch 82 | .column(1) 83 | .as_any() 84 | .downcast_ref::() 85 | .ok_or(BuzzError::Execution(format!( 86 | "Invalid type for catalog lengths" 87 | )))?; 88 | let sized_files = (0..rec_batch.num_rows()).map(move |i| SizedFile { 89 | key: key_array.value(i).to_owned(), 90 | length: length_array.value(i), 91 | }); 92 | Ok(sized_files) 93 | }) 94 | .flat_map(|rec_iter_res| match rec_iter_res { 95 | Ok(rec_iter) => rec_iter.map(|rec| Ok(rec)).collect(), 96 | Err(er) => vec![Err(er)], 97 | }) 98 | .collect::>>() 99 | } 100 | } 101 | 102 | impl TableProvider for CatalogTable { 103 | fn as_any(&self) -> &dyn Any { 104 | self 105 | } 106 | 107 | fn schema(&self) -> SchemaRef { 108 | self.source_table.schema() 109 | } 110 | 111 | fn scan( 112 | &self, 113 | _projection: &Option>, 114 | _batch_size: usize, 115 | _filters: &[Expr], 116 | _limit: Option, 117 | ) -> DataFusionResult> { 118 | Err(DataFusionError::Plan( 119 | "Catalog table cannot generate an execution plan".to_owned(), 120 | )) 121 | } 122 | 123 | fn statistics(&self) -> Statistics { 124 | self.source_table.statistics() 125 | } 126 | } 127 | 128 | pub fn catalog_schema(partition_names: &[String]) -> Arc { 129 | let mut fields = vec![ 130 | Field::new("key", DataType::Utf8, false), 131 | Field::new("length", DataType::UInt64, false), 132 | ]; 133 | for col in partition_names { 134 | fields.push(Field::new(col, DataType::Utf8, false)); 135 | } 136 | Arc::new(Schema::new(fields)) 137 | } 138 | 139 | //// Implems //// 140 | 141 | pub mod delta_catalog; 142 | pub mod static_catalog; 143 | pub(crate) mod test_catalog; 144 | // pub(crate) mod utils; 145 | 146 | #[cfg(test)] 147 | mod tests { 148 | use super::*; 149 | use crate::datasource::CatalogTable; 150 | 151 | #[tokio::test] 152 | async fn test_filter_catalog() { 153 | let nb_split = 5; 154 | let catalog_table = CatalogTable::new(Box::new( 155 | test_catalog::MockSplittableTable::new(nb_split, 0), 156 | )); 157 | 158 | let result = catalog_table.filter_catalog(&None).await.unwrap(); 159 | assert_eq!(result.len(), 5); 160 | } 161 | 162 | #[tokio::test] 163 | async fn test_filter_partitioned_catalog() { 164 | let nb_split = 5; 165 | let catalog_table = CatalogTable::new(Box::new( 166 | test_catalog::MockSplittableTable::new(nb_split, 1), 167 | )); 168 | 169 | let result = catalog_table.filter_catalog(&None).await.unwrap(); 170 | assert_eq!(result.len(), 5); 171 | 172 | let result = catalog_table 173 | .filter_catalog(&Some("part_key_1='part_value_002'".to_owned())) 174 | .await 175 | .unwrap(); 176 | assert_eq!(result.len(), 1); 177 | assert_eq!(result[0].key, "file_2"); 178 | } 179 | } 180 | -------------------------------------------------------------------------------- /code/src/datasource/catalog/static_catalog.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use super::{catalog_schema, SplittableTable}; 4 | use crate::datasource::{HBeeTableDesc, S3ParquetTable}; 5 | use crate::error::Result; 6 | use crate::models::SizedFile; 7 | use arrow::array::*; 8 | use arrow::datatypes::*; 9 | use arrow::record_batch::RecordBatch; 10 | use async_trait::async_trait; 11 | use datafusion::datasource::{MemTable, TableProvider}; 12 | 13 | pub struct CatalogFile { 14 | sized_file: SizedFile, 15 | partitions: Vec, 16 | } 17 | 18 | impl CatalogFile { 19 | pub fn new(key: &str, length: u64, partitions: Vec) -> Self { 20 | CatalogFile { 21 | sized_file: SizedFile { 22 | key: key.to_owned(), 23 | length, 24 | }, 25 | partitions, 26 | } 27 | } 28 | } 29 | 30 | /// A catalog table that contains a static list of files. 31 | /// Only supports S3 parquet files for now and simply sends each file into a different hbee. 32 | pub struct StaticCatalogTable { 33 | schema: SchemaRef, 34 | region: String, 35 | bucket: String, 36 | files: Vec, 37 | partition_cols: Vec, 38 | } 39 | 40 | impl StaticCatalogTable { 41 | pub fn new( 42 | schema: SchemaRef, 43 | region: String, 44 | bucket: String, 45 | partition_cols: Vec, 46 | files: Vec, 47 | ) -> Self { 48 | Self { 49 | schema, 50 | region, 51 | bucket, 52 | files, 53 | partition_cols, 54 | } 55 | } 56 | 57 | fn to_table(&self) -> Result> { 58 | let mut key_builder = StringBuilder::new(self.files.len()); 59 | let mut length_builder = UInt64Builder::new(self.files.len()); 60 | let mut partition_builders = self 61 | .partition_cols 62 | .iter() 63 | .map(|_| StringBuilder::new(self.files.len())) 64 | .collect::>(); 65 | for catalog_file in &self.files { 66 | assert_eq!(catalog_file.partitions.len(), self.partition_cols.len(),"Each catalog entry should have as many partition values as partition cols."); 67 | key_builder.append_value(&catalog_file.sized_file.key)?; 68 | length_builder.append_value(catalog_file.sized_file.length)?; 69 | for (i, part_val) in catalog_file.partitions.iter().enumerate() { 70 | partition_builders[i].append_value(part_val)?; 71 | } 72 | } 73 | 74 | // finish all builders 75 | let mut col_arrays: Vec = vec![ 76 | ArrayBuilder::finish(&mut key_builder), 77 | ArrayBuilder::finish(&mut length_builder), 78 | ]; 79 | for mut partition_builder in partition_builders { 80 | col_arrays.push(ArrayBuilder::finish(&mut partition_builder)); 81 | } 82 | 83 | let schema = catalog_schema(&self.partition_cols); 84 | 85 | let record_batch = RecordBatch::try_new(Arc::clone(&schema), col_arrays)?; 86 | Ok(Arc::new(MemTable::try_new( 87 | schema, 88 | vec![vec![record_batch]], 89 | )?)) 90 | } 91 | } 92 | 93 | #[async_trait] 94 | impl SplittableTable for StaticCatalogTable { 95 | fn split(&self, files: Vec) -> Vec { 96 | files 97 | .into_iter() 98 | .map(|file| { 99 | S3ParquetTable::new( 100 | self.region.clone(), 101 | self.bucket.clone(), 102 | vec![file], 103 | Arc::clone(&self.schema), 104 | ) 105 | }) 106 | .collect() 107 | } 108 | fn partition_columns(&self) -> &[String] { 109 | &self.partition_cols 110 | } 111 | fn schema(&self) -> SchemaRef { 112 | let mut fields = self.schema.fields().clone(); 113 | for partition_col in &self.partition_cols { 114 | fields.push(Field::new(partition_col, DataType::Utf8, false)) 115 | } 116 | Arc::new(Schema::new_with_metadata( 117 | fields, 118 | self.schema.metadata().clone(), 119 | )) 120 | } 121 | async fn file_table(&self) -> Arc { 122 | self.to_table().unwrap() 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /code/src/datasource/catalog/test_catalog.rs: -------------------------------------------------------------------------------- 1 | //! Test Fixtures for Catalog Tables 2 | 3 | use super::*; 4 | use crate::datasource::{S3ParquetTable, SplittableTable}; 5 | use crate::models::SizedFile; 6 | use arrow::datatypes::DataType; 7 | use arrow::datatypes::{Field, Schema, SchemaRef}; 8 | use arrow::record_batch::RecordBatch; 9 | use async_trait::async_trait; 10 | use datafusion::datasource::{MemTable, TableProvider}; 11 | 12 | /// `pattern_vec(pattern, len)` creates a vector of String of length `len` 13 | macro_rules! pattern_vec { 14 | ($x:expr, $y:expr) => { 15 | (1..=$y).map(|i| format!($x, i)).collect::>() 16 | }; 17 | } 18 | 19 | /// A SplittableTable that splits into `nb_split` S3Parquet tables 20 | pub struct MockSplittableTable { 21 | nb_split: usize, 22 | partitions: Vec, 23 | } 24 | 25 | impl MockSplittableTable { 26 | /// File keys range from file_1 to file_{nb_split} 27 | /// Partition keys range from part_key_1 to part_key_{partition_keys} 28 | /// For each partition key, values range from part_value_001 to part_value_{nb_split} 29 | pub fn new(nb_split: usize, partition_keys: usize) -> Self { 30 | Self { 31 | nb_split, 32 | partitions: pattern_vec!("part_key_{}", partition_keys), 33 | } 34 | } 35 | } 36 | 37 | #[async_trait] 38 | impl SplittableTable for MockSplittableTable { 39 | fn split(&self, files: Vec) -> Vec { 40 | files 41 | .into_iter() 42 | .map(|file| { 43 | S3ParquetTable::new( 44 | "north-pole-1".to_owned(), 45 | "santas-bucket".to_owned(), 46 | vec![file], 47 | test_schema(), 48 | ) 49 | }) 50 | .collect::>() 51 | } 52 | fn partition_columns(&self) -> &[String] { 53 | &self.partitions 54 | } 55 | fn schema(&self) -> SchemaRef { 56 | let mut fields = test_schema().fields().clone(); 57 | for partition_col in &self.partitions { 58 | fields.push(Field::new(partition_col, DataType::Utf8, false)) 59 | } 60 | Arc::new(Schema::new_with_metadata( 61 | fields, 62 | test_schema().metadata().clone(), 63 | )) 64 | } 65 | async fn file_table(&self) -> Arc { 66 | let mut fields = vec![ 67 | Field::new("key", DataType::Utf8, false), 68 | Field::new("length", DataType::UInt64, false), 69 | ]; 70 | for partition in &self.partitions { 71 | fields.push(Field::new(partition, DataType::Utf8, false)); 72 | } 73 | 74 | let file_table_schema = Arc::new(Schema::new(fields)); 75 | 76 | let keys = pattern_vec!("file_{}", self.nb_split); 77 | let lengths = vec![999999999 as u64; self.nb_split]; 78 | let parts = 79 | vec![pattern_vec!("part_value_{:03}", self.nb_split); self.partitions.len()]; 80 | 81 | let mut arrays = vec![ 82 | Arc::new(StringArray::from(refvec(&keys))) as ArrayRef, 83 | Arc::new(UInt64Array::from(lengths)) as ArrayRef, 84 | ]; 85 | for i in 1..=parts.len() { 86 | arrays.push(Arc::new(StringArray::from(refvec(&parts[i - 1]))) as ArrayRef); 87 | } 88 | 89 | let batches = RecordBatch::try_new(Arc::clone(&file_table_schema), arrays) 90 | .expect("Invalid test data"); 91 | 92 | Arc::new( 93 | MemTable::try_new(file_table_schema, vec![vec![batches]]) 94 | .expect("invalid test table"), 95 | ) 96 | } 97 | } 98 | 99 | fn test_schema() -> SchemaRef { 100 | Arc::new(Schema::new(vec![Field::new( 101 | "data_col", 102 | DataType::Int64, 103 | true, 104 | )])) 105 | } 106 | 107 | fn refvec(vec: &Vec) -> Vec> { 108 | vec.iter() 109 | .map(|item| Some(item.as_ref())) 110 | .collect::>() 111 | } 112 | -------------------------------------------------------------------------------- /code/src/datasource/hbee/mod.rs: -------------------------------------------------------------------------------- 1 | pub mod s3_parquet; 2 | 3 | use std::any::Any; 4 | use std::sync::Arc; 5 | 6 | use crate::clients::RangeCache; 7 | use arrow::datatypes::*; 8 | use datafusion::datasource::datasource::Statistics; 9 | use datafusion::datasource::TableProvider; 10 | use datafusion::error::Result; 11 | use datafusion::logical_plan::Expr; 12 | use datafusion::physical_plan::ExecutionPlan; 13 | use s3_parquet::S3ParquetTable; 14 | 15 | /// Implemented as an enum because serialization must be mapped for new implems 16 | #[derive(Debug)] 17 | pub enum HBeeTableDesc { 18 | S3Parquet(S3ParquetTable), 19 | } 20 | 21 | impl HBeeTableDesc { 22 | pub fn schema(&self) -> SchemaRef { 23 | match self { 24 | HBeeTableDesc::S3Parquet(table) => table.schema(), 25 | } 26 | } 27 | } 28 | 29 | /// A table that can be distributed to hbees 30 | pub struct HBeeTable { 31 | desc: Arc, 32 | cache: Arc, 33 | } 34 | 35 | impl HBeeTable { 36 | pub fn new(desc: Arc, cache: Arc) -> Self { 37 | Self { desc, cache } 38 | } 39 | 40 | pub fn description(&self) -> Arc { 41 | Arc::clone(&self.desc) 42 | } 43 | } 44 | 45 | impl TableProvider for HBeeTable { 46 | fn as_any(&self) -> &dyn Any { 47 | self 48 | } 49 | 50 | fn schema(&self) -> SchemaRef { 51 | self.desc.schema() 52 | } 53 | 54 | fn scan( 55 | &self, 56 | projection: &Option>, 57 | batch_size: usize, 58 | filters: &[Expr], 59 | limit: Option, 60 | ) -> Result> { 61 | match self.desc.as_ref() { 62 | HBeeTableDesc::S3Parquet(table) => table.scan( 63 | Arc::clone(&self.cache), 64 | projection, 65 | batch_size, 66 | filters, 67 | limit, 68 | ), 69 | } 70 | } 71 | 72 | fn statistics(&self) -> Statistics { 73 | Statistics::default() 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /code/src/datasource/hbee/s3_parquet.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use super::HBeeTableDesc; 4 | use crate::clients::s3; 5 | use crate::clients::CachedFile; 6 | use crate::clients::RangeCache; 7 | use crate::execution_plan::ParquetExec; 8 | use crate::models::SizedFile; 9 | use arrow::datatypes::*; 10 | use datafusion::error::Result; 11 | use datafusion::logical_plan::Expr; 12 | use datafusion::physical_plan::ExecutionPlan; 13 | 14 | /// Table-based representation of a `ParquetFile` backed by S3. 15 | #[derive(Debug)] 16 | pub struct S3ParquetTable { 17 | region: String, 18 | bucket: String, 19 | files: Vec, 20 | schema: SchemaRef, 21 | } 22 | 23 | impl S3ParquetTable { 24 | /// Initialize a new `ParquetTable` from a list of s3 files and an expected schema. 25 | pub fn new( 26 | region: String, 27 | bucket: String, 28 | files: Vec, 29 | schema: SchemaRef, 30 | ) -> HBeeTableDesc { 31 | HBeeTableDesc::S3Parquet(Self { 32 | schema, 33 | region, 34 | bucket, 35 | files, 36 | }) 37 | } 38 | 39 | pub fn region(&self) -> &str { 40 | &self.region 41 | } 42 | 43 | pub fn bucket(&self) -> &str { 44 | &self.bucket 45 | } 46 | 47 | pub fn files(&self) -> &[SizedFile] { 48 | &self.files 49 | } 50 | 51 | pub fn schema(&self) -> SchemaRef { 52 | self.schema.clone() 53 | } 54 | 55 | pub fn scan( 56 | &self, 57 | cache: Arc, 58 | projection: &Option>, 59 | batch_size: usize, 60 | _filters: &[Expr], 61 | _limit: Option, 62 | ) -> Result> { 63 | let s3_files = self 64 | .files 65 | .iter() 66 | .map(|file| { 67 | let (dler_id, dler_creator) = s3::downloader_creator(&self.region); 68 | let file_id = s3::file_id(&self.bucket, &file.key); 69 | CachedFile::new( 70 | file_id, 71 | file.length, 72 | Arc::clone(&cache), 73 | dler_id, 74 | dler_creator, 75 | ) 76 | }) 77 | .collect::>(); 78 | Ok(Arc::new(ParquetExec::new( 79 | s3_files, 80 | projection.clone(), 81 | batch_size, 82 | Arc::clone(&self.schema), 83 | ))) 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /code/src/datasource/hcomb.rs: -------------------------------------------------------------------------------- 1 | use std::any::Any; 2 | use std::pin::Pin; 3 | use std::sync::{Arc, Mutex}; 4 | 5 | use crate::execution_plan::StreamExec; 6 | use arrow::datatypes::*; 7 | use arrow::error::Result as ArrowResult; 8 | use arrow::record_batch::RecordBatch; 9 | use datafusion::datasource::datasource::Statistics; 10 | use datafusion::datasource::TableProvider; 11 | use datafusion::error::Result; 12 | use datafusion::logical_plan::Expr; 13 | use datafusion::physical_plan::ExecutionPlan; 14 | use futures::Stream; 15 | 16 | #[derive(Debug, Clone)] 17 | pub struct HCombTableDesc { 18 | query_id: String, 19 | nb_hbee: usize, 20 | schema: SchemaRef, 21 | } 22 | 23 | impl HCombTableDesc { 24 | pub fn new(query_id: String, nb_hbee: usize, schema: SchemaRef) -> Self { 25 | Self { 26 | query_id, 27 | nb_hbee, 28 | schema, 29 | } 30 | } 31 | 32 | pub fn query_id(&self) -> &str { 33 | &self.query_id 34 | } 35 | 36 | pub fn nb_hbee(&self) -> usize { 37 | self.nb_hbee 38 | } 39 | 40 | pub fn schema(&self) -> SchemaRef { 41 | Arc::clone(&self.schema) 42 | } 43 | } 44 | 45 | /// A table from a stream of batches that can be executed only once 46 | pub struct HCombTable { 47 | stream: Mutex< 48 | Option> + Send + Sync>>>, 49 | >, 50 | desc: HCombTableDesc, 51 | } 52 | 53 | impl HCombTable { 54 | pub fn new( 55 | desc: HCombTableDesc, 56 | stream: Pin> + Send + Sync>>, 57 | ) -> Self { 58 | Self { 59 | stream: Mutex::new(Some(stream)), 60 | desc, 61 | } 62 | } 63 | 64 | pub fn new_empty(desc: HCombTableDesc) -> Self { 65 | Self::new(desc, Box::pin(futures::stream::iter(vec![]))) 66 | } 67 | 68 | pub fn query_id(&self) -> &str { 69 | &self.desc.query_id 70 | } 71 | } 72 | 73 | impl TableProvider for HCombTable { 74 | fn as_any(&self) -> &dyn Any { 75 | self 76 | } 77 | 78 | fn schema(&self) -> SchemaRef { 79 | self.desc.schema() 80 | } 81 | 82 | fn scan( 83 | &self, 84 | projection: &Option>, 85 | batch_size: usize, 86 | _filters: &[Expr], 87 | _limit: Option, 88 | ) -> Result> { 89 | match self.stream.lock().unwrap().take() { 90 | Some(stream) => Ok(Arc::new(StreamExec::new( 91 | stream, 92 | self.schema(), 93 | projection.clone(), 94 | batch_size, 95 | ))), 96 | None => Err(datafusion::error::DataFusionError::Execution( 97 | "Cannot scan stream source more than once".to_owned(), 98 | )), 99 | } 100 | } 101 | 102 | fn statistics(&self) -> Statistics { 103 | Statistics::default() 104 | } 105 | } 106 | 107 | #[cfg(test)] 108 | mod tests { 109 | use super::*; 110 | use arrow::array::Int32Array; 111 | use arrow::datatypes::{DataType, Field, Schema}; 112 | use datafusion::datasource::TableProvider; 113 | 114 | #[tokio::test] 115 | async fn test_not_empty() -> Result<()> { 116 | let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); 117 | let hcomb_table_desc = 118 | HCombTableDesc::new("mock_query_id".to_owned(), 1, schema.clone()); 119 | let batches = vec![RecordBatch::try_new( 120 | schema.clone(), 121 | vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], 122 | )?]; 123 | let hcomb_table = HCombTable::new( 124 | hcomb_table_desc, 125 | Box::pin(futures::stream::iter( 126 | batches.clone().into_iter().map(|b| Ok(b)), 127 | )), 128 | ); 129 | 130 | let exec_plan = hcomb_table.scan(&None, 1024, &[], None)?; 131 | 132 | let results = datafusion::physical_plan::collect(exec_plan).await?; 133 | assert_eq!(results.len(), 1); 134 | assert_eq!(format!("{:?}", results), format!("{:?}", batches)); 135 | Ok(()) 136 | } 137 | 138 | #[tokio::test] 139 | async fn test_empty() -> Result<()> { 140 | let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); 141 | let hcomb_table_desc = 142 | HCombTableDesc::new("mock_query_id".to_owned(), 1, schema.clone()); 143 | let hcomb_table = HCombTable::new_empty(hcomb_table_desc); 144 | 145 | let exec_plan = hcomb_table.scan(&Some(vec![0]), 2048, &[], None)?; 146 | 147 | let results = datafusion::physical_plan::collect(exec_plan).await?; 148 | assert_eq!(results.len(), 0); 149 | Ok(()) 150 | } 151 | } 152 | -------------------------------------------------------------------------------- /code/src/datasource/mod.rs: -------------------------------------------------------------------------------- 1 | //! Datasources are implementations of DataFusion's TableProvider trait 2 | 3 | mod catalog; 4 | mod hbee; 5 | mod hcomb; 6 | 7 | pub use catalog::delta_catalog::DeltaCatalogTable; 8 | pub use catalog::static_catalog::{CatalogFile, StaticCatalogTable}; 9 | pub use catalog::test_catalog::MockSplittableTable; 10 | pub use catalog::{CatalogTable, SplittableTable}; 11 | pub use hbee::{s3_parquet::S3ParquetTable, HBeeTable, HBeeTableDesc}; 12 | pub use hcomb::{HCombTable, HCombTableDesc}; 13 | -------------------------------------------------------------------------------- /code/src/error.rs: -------------------------------------------------------------------------------- 1 | use std::error; 2 | use std::fmt::{Display, Formatter}; 3 | use std::io; 4 | use std::result; 5 | 6 | use arrow::error::ArrowError; 7 | use arrow_parquet::errors::ParquetError; 8 | use datafusion::error::DataFusionError; 9 | use deltalake::DeltaTableError; 10 | 11 | /// Result type for operations that could result in an [BuzzError] 12 | pub type Result = result::Result; 13 | 14 | /// Buzz error 15 | #[derive(Debug)] 16 | pub enum BuzzError { 17 | /// Error returned by arrow. 18 | ArrowError(ArrowError), 19 | /// Error returned by DeltaLake. 20 | DeltaTableError(DeltaTableError), 21 | /// Wraps an error from the Parquet crate 22 | ParquetError(ParquetError), 23 | /// Wraps an error from the DataFusion crate 24 | DataFusionError(DataFusionError), 25 | /// A bad request was sent to Buzz 26 | BadRequest(String), 27 | /// Error associated to I/O operations and associated traits. 28 | IoError(io::Error), 29 | /// Error returned on a branch that we know it is possible 30 | /// but to which we still have no implementation for. 31 | /// Often, these errors are tracked in our issue tracker. 32 | NotImplemented(String), 33 | /// Error returned as a consequence of an error in Buzz. 34 | /// This error should not happen in normal usage of Buzz. 35 | // BuzzError has internal invariants that we are unable to ask the compiler to check for us. 36 | // This error is raised when one of those invariants is not verified during execution. 37 | Internal(String), 38 | /// This error happens whenever a plan is not valid. Examples include 39 | /// impossible casts, schema inference not possible and non-unique column names. 40 | Plan(String), 41 | /// Error returned during execution of the query. 42 | /// Examples include files not found, errors in parsing certain types. 43 | Execution(String), 44 | /// Error returned when hbee failed 45 | HBee(String), 46 | /// Client error 47 | CloudClient(String), 48 | /// Error when downloading data from an external source 49 | Download(String), 50 | } 51 | 52 | /// Creates an Internal error from a formatted string 53 | #[macro_export] 54 | macro_rules! internal_err { 55 | ($($arg:tt)*) => {{ 56 | let reason = format!($($arg)*); 57 | crate::error::BuzzError::Internal(reason) 58 | }} 59 | } 60 | 61 | /// Creates a NotImplemented error from a formatted string 62 | #[macro_export] 63 | macro_rules! not_impl_err { 64 | ($($arg:tt)*) => {{ 65 | let reason = format!($($arg)*); 66 | crate::error::BuzzError::NotImplemented(reason) 67 | }} 68 | } 69 | 70 | /// Creates a BadRequest error from a formatted string 71 | #[macro_export] 72 | macro_rules! bad_req_err { 73 | ($($arg:tt)*) => {{ 74 | let reason = format!($($arg)*); 75 | crate::error::BuzzError::BadRequest(reason) 76 | }} 77 | } 78 | 79 | /// Checks the predicate, if false return the formatted string 80 | #[macro_export] 81 | macro_rules! ensure { 82 | ($predicate:expr, $($arg:tt)*) => { 83 | if !$predicate { 84 | let reason = format!($($arg)*); 85 | return Err(crate::error::BuzzError::Internal(reason)); 86 | } 87 | }; 88 | } 89 | 90 | impl BuzzError { 91 | /// Wraps this [BuzzError] as an [Arrow::error::ArrowError]. 92 | pub fn into_arrow_external_error(self) -> ArrowError { 93 | ArrowError::from_external_error(Box::new(self)) 94 | } 95 | 96 | pub fn internal(reason: &'static str) -> Self { 97 | return Self::Internal(reason.to_owned()); 98 | } 99 | 100 | pub fn reason(&self) -> String { 101 | match *self { 102 | BuzzError::ArrowError(ref desc) => format!("{}", desc), 103 | BuzzError::DeltaTableError(ref desc) => format!("{}", desc), 104 | BuzzError::ParquetError(ref desc) => format!("{}", desc), 105 | BuzzError::DataFusionError(ref desc) => format!("{}", desc), 106 | BuzzError::BadRequest(ref desc) => format!("{}", desc), 107 | BuzzError::IoError(ref desc) => format!("{}", desc), 108 | BuzzError::NotImplemented(ref desc) => format!("{}", desc), 109 | BuzzError::Internal(ref desc) => format!("{}", desc), 110 | BuzzError::Plan(ref desc) => format!("{}", desc), 111 | BuzzError::Execution(ref desc) => format!("{}", desc), 112 | BuzzError::HBee(ref desc) => format!("{}", desc), 113 | BuzzError::Download(ref desc) => format!("{}", desc), 114 | BuzzError::CloudClient(ref desc) => format!("{}", desc), 115 | } 116 | } 117 | } 118 | 119 | impl From for BuzzError { 120 | fn from(e: io::Error) -> Self { 121 | BuzzError::IoError(e) 122 | } 123 | } 124 | 125 | impl From for BuzzError { 126 | fn from(e: ArrowError) -> Self { 127 | BuzzError::ArrowError(e) 128 | } 129 | } 130 | 131 | impl From for BuzzError { 132 | fn from(e: ParquetError) -> Self { 133 | BuzzError::ParquetError(e) 134 | } 135 | } 136 | 137 | impl From for BuzzError { 138 | fn from(e: DataFusionError) -> Self { 139 | BuzzError::DataFusionError(e) 140 | } 141 | } 142 | 143 | impl From for BuzzError { 144 | fn from(e: DeltaTableError) -> Self { 145 | BuzzError::DeltaTableError(e) 146 | } 147 | } 148 | 149 | impl Display for BuzzError { 150 | fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { 151 | match *self { 152 | BuzzError::ArrowError(ref desc) => write!(f, "Arrow error: {}", desc), 153 | BuzzError::DeltaTableError(ref desc) => write!(f, "Delta error: {}", desc), 154 | BuzzError::ParquetError(ref desc) => { 155 | write!(f, "Parquet error: {}", desc) 156 | } 157 | BuzzError::DataFusionError(ref desc) => { 158 | write!(f, "DataFusion error: {}", desc) 159 | } 160 | BuzzError::IoError(ref desc) => write!(f, "IO error: {}", desc), 161 | BuzzError::BadRequest(ref desc) => { 162 | write!(f, "Invalid request: {}", desc) 163 | } 164 | BuzzError::NotImplemented(ref desc) => { 165 | write!(f, "This feature is not implemented: {}", desc) 166 | } 167 | BuzzError::Internal(ref desc) => { 168 | write!(f, "Internal error: {}", desc) 169 | } 170 | BuzzError::Plan(ref desc) => { 171 | write!(f, "Error during planning: {}", desc) 172 | } 173 | BuzzError::Execution(ref desc) => { 174 | write!(f, "Execution error: {}", desc) 175 | } 176 | BuzzError::HBee(ref desc) => { 177 | write!(f, "HBee error: {}", desc) 178 | } 179 | BuzzError::Download(ref desc) => { 180 | write!(f, "Download error: {}", desc) 181 | } 182 | BuzzError::CloudClient(ref desc) => { 183 | write!(f, "Cloud client error: {}", desc) 184 | } 185 | } 186 | } 187 | } 188 | 189 | impl error::Error for BuzzError {} 190 | -------------------------------------------------------------------------------- /code/src/example_catalog.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use crate::datasource::{CatalogFile, CatalogTable, StaticCatalogTable}; 4 | use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; 5 | 6 | /// shortened nyc taxi, hosted by cloudfuse 7 | pub fn nyc_taxi_cloudfuse_sample() -> CatalogTable { 8 | CatalogTable::new(Box::new(StaticCatalogTable::new( 9 | nyc_taxi_v1_schema(TimeUnit::Microsecond), 10 | "us-east-2".to_owned(), 11 | "cloudfuse-taxi-data".to_owned(), 12 | vec!["month".to_owned()], 13 | vec![CatalogFile::new( 14 | "raw_small/2009/01/data.parquet", 15 | 27301328, 16 | vec!["2009/01".to_owned()], 17 | )], 18 | ))) 19 | } 20 | 21 | /// complete nyc taxi files with 5M rows per rowgroups, hosted by cloudfuse 22 | pub fn nyc_taxi_cloudfuse() -> CatalogTable { 23 | CatalogTable::new(Box::new(StaticCatalogTable::new( 24 | nyc_taxi_v1_schema(TimeUnit::Microsecond), 25 | "us-east-2".to_owned(), 26 | "cloudfuse-taxi-data".to_owned(), 27 | vec!["month".to_owned()], 28 | vec![ 29 | CatalogFile::new( 30 | "raw_5M/2009/01/data.parquet", 31 | 388070114, 32 | vec!["2009/01".to_owned()], 33 | ), 34 | CatalogFile::new( 35 | "raw_5M/2009/02/data.parquet", 36 | 368127982, 37 | vec!["2009/02".to_owned()], 38 | ), 39 | CatalogFile::new( 40 | "raw_5M/2009/03/data.parquet", 41 | 398600815, 42 | vec!["2009/03".to_owned()], 43 | ), 44 | CatalogFile::new( 45 | "raw_5M/2009/04/data.parquet", 46 | 396353841, 47 | vec!["2009/04".to_owned()], 48 | ), 49 | CatalogFile::new( 50 | "raw_5M/2009/05/data.parquet", 51 | 410283205, 52 | vec!["2009/05".to_owned()], 53 | ), 54 | ], 55 | ))) 56 | } 57 | 58 | /// A subset of the nyc taxi parquet files hosted by Ursa Labs 59 | /// Note that some nyc parquet files hosted by Ursa Labs have many small row groups which is inefficient 60 | pub fn nyc_taxi_ursa() -> CatalogTable { 61 | CatalogTable::new(Box::new(StaticCatalogTable::new( 62 | nyc_taxi_v1_schema(TimeUnit::Nanosecond), 63 | "us-east-2".to_owned(), 64 | "ursa-labs-taxi-data".to_owned(), 65 | vec!["month".to_owned()], 66 | vec![ 67 | CatalogFile::new( 68 | "2009/01/data.parquet", 69 | 461966527, 70 | vec!["2009/01".to_owned()], 71 | ), 72 | CatalogFile::new( 73 | "2009/02/data.parquet", 74 | 436405669, 75 | vec!["2009/02".to_owned()], 76 | ), 77 | CatalogFile::new( 78 | "2009/03/data.parquet", 79 | 474795751, 80 | vec!["2009/03".to_owned()], 81 | ), 82 | CatalogFile::new( 83 | "2009/04/data.parquet", 84 | 470914229, 85 | vec!["2009/04".to_owned()], 86 | ), 87 | CatalogFile::new( 88 | "2009/05/data.parquet", 89 | 489248585, 90 | vec!["2009/05".to_owned()], 91 | ), 92 | CatalogFile::new( 93 | "2009/06/data.parquet", 94 | 465578495, 95 | vec!["2009/06".to_owned()], 96 | ), 97 | CatalogFile::new( 98 | "2009/07/data.parquet", 99 | 448227037, 100 | vec!["2009/07".to_owned()], 101 | ), 102 | CatalogFile::new( 103 | "2009/08/data.parquet", 104 | 450774566, 105 | vec!["2009/08".to_owned()], 106 | ), 107 | CatalogFile::new( 108 | "2009/09/data.parquet", 109 | 460835784, 110 | vec!["2009/09".to_owned()], 111 | ), 112 | CatalogFile::new( 113 | "2009/10/data.parquet", 114 | 517609313, 115 | vec!["2009/10".to_owned()], 116 | ), 117 | CatalogFile::new( 118 | "2009/11/data.parquet", 119 | 471148697, 120 | vec!["2009/11".to_owned()], 121 | ), 122 | CatalogFile::new( 123 | "2009/12/data.parquet", 124 | 479899902, 125 | vec!["2009/12".to_owned()], 126 | ), 127 | ], 128 | ))) 129 | } 130 | 131 | /// schema found in earlier nyc taxi files (e.g 2009) 132 | fn nyc_taxi_v1_schema(time_unit: TimeUnit) -> Arc { 133 | Arc::new(Schema::new(vec![ 134 | Field::new("vendor_id", DataType::Utf8, true), 135 | Field::new( 136 | "pickup_at", 137 | DataType::Timestamp(time_unit.clone(), Option::None), 138 | true, 139 | ), 140 | Field::new( 141 | "dropoff_at", 142 | DataType::Timestamp(time_unit.clone(), Option::None), 143 | true, 144 | ), 145 | Field::new("passenger_count", DataType::Int8, true), 146 | Field::new("trip_distance", DataType::Float32, true), 147 | Field::new("pickup_longitude", DataType::Float32, true), 148 | Field::new("pickup_latitude", DataType::Float32, true), 149 | Field::new("rate_code_id", DataType::Null, true), 150 | Field::new("store_and_fwd_flag", DataType::Utf8, true), 151 | Field::new("dropoff_longitude", DataType::Float32, true), 152 | Field::new("dropoff_latitude", DataType::Float32, true), 153 | Field::new("payment_type", DataType::Utf8, true), 154 | Field::new("fare_amount", DataType::Float32, true), 155 | Field::new("extra", DataType::Float32, true), 156 | Field::new("mta_tax", DataType::Float32, true), 157 | Field::new("tip_amount", DataType::Float32, true), 158 | Field::new("tolls_amount", DataType::Float32, true), 159 | Field::new("total_amount", DataType::Float32, true), 160 | ])) 161 | } 162 | -------------------------------------------------------------------------------- /code/src/execution_plan/mod.rs: -------------------------------------------------------------------------------- 1 | //! Execution plans are implementations of DataFusion's ExecutionPlan trait 2 | 3 | mod parquet; 4 | mod stream; 5 | 6 | pub use parquet::ParquetExec; 7 | pub use stream::StreamExec; 8 | -------------------------------------------------------------------------------- /code/src/execution_plan/parquet.rs: -------------------------------------------------------------------------------- 1 | use fmt::Debug; 2 | use std::any::Any; 3 | use std::fmt; 4 | use std::sync::Arc; 5 | use std::task::{Context, Poll}; 6 | use tokio::{ 7 | sync::mpsc::{channel, Receiver, Sender}, 8 | task, 9 | }; 10 | use tokio_stream::wrappers::ReceiverStream; 11 | 12 | use crate::clients::CachedFile; 13 | use arrow::datatypes::{Schema, SchemaRef}; 14 | use arrow::error::{ArrowError, Result as ArrowResult}; 15 | use arrow::record_batch::RecordBatch; 16 | use arrow_parquet::arrow::{ArrowReader, ParquetFileArrowReader}; 17 | use arrow_parquet::file::reader::{FileReader, Length, SerializedFileReader}; 18 | use async_trait::async_trait; 19 | use datafusion::error::{DataFusionError, Result as DataFusionResult}; 20 | use datafusion::physical_plan::ExecutionPlan; 21 | use datafusion::physical_plan::Partitioning; 22 | use datafusion::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; 23 | use futures::stream::{Stream, StreamExt}; 24 | 25 | /// Execution plan for scanning a Parquet file 26 | #[derive(Debug, Clone)] 27 | pub struct ParquetExec { 28 | files: Vec, 29 | /// Schema before projection is applied 30 | file_schema: SchemaRef, 31 | /// Schema after projection is applied 32 | projected_schema: SchemaRef, 33 | /// Projection for which columns to load 34 | projection: Vec, 35 | /// Batch size 36 | batch_size: usize, 37 | } 38 | 39 | impl ParquetExec { 40 | /// Create a new Parquet reader execution plan 41 | pub fn new( 42 | files: Vec, 43 | projection: Option>, 44 | batch_size: usize, 45 | schema: SchemaRef, 46 | ) -> Self { 47 | let projection = match projection { 48 | Some(p) => p, 49 | None => (0..schema.fields().len()).collect(), 50 | }; 51 | let projected_schema = Schema::new( 52 | projection 53 | .iter() 54 | .map(|col| schema.field(*col).clone()) 55 | .collect(), 56 | ); 57 | Self { 58 | files, 59 | file_schema: schema, 60 | projected_schema: Arc::new(projected_schema), 61 | projection, 62 | batch_size, 63 | } 64 | } 65 | 66 | /// Read the footer and schedule the downloads of all the required chunks 67 | async fn init_file( 68 | &self, 69 | partition: usize, 70 | ) -> DataFusionResult>> { 71 | let end_dl_chunk_start = Self::download_footer(self.files[partition].clone()); 72 | let file_schema = self.file_schema.clone(); 73 | let file = self.files[partition].clone(); 74 | let projection = self.projection.clone(); 75 | 76 | // Reading the footer is blocking so it should be started on a specific thread 77 | tokio::task::spawn_blocking(move || { 78 | let file_reader = Arc::new( 79 | SerializedFileReader::new(file.clone()) 80 | .map_err(|e| DataFusionError::ParquetError(e))?, 81 | ); 82 | let mut arrow_reader = ParquetFileArrowReader::new(file_reader.clone()); 83 | 84 | let schema_eq = file_schema 85 | .fields() 86 | .iter() 87 | .zip(arrow_reader.get_schema()?.fields().iter()) 88 | .all(|x| { 89 | x.0.name() == x.1.name() 90 | && x.0.data_type() == x.1.data_type() 91 | && x.0.is_nullable() == x.1.is_nullable() 92 | }); 93 | 94 | if !schema_eq { 95 | return Err(DataFusionError::Plan(format!( 96 | "Expected and parsed schema fields are not equal: \n{:?} \n!= \n{:?}", 97 | file_schema.fields(), 98 | arrow_reader.get_schema()?.fields() 99 | ))); 100 | } 101 | // prefetch usefull byte ranges 102 | let metadata = file_reader.metadata(); 103 | for i in 0..metadata.num_row_groups() { 104 | for proj in &projection { 105 | let rg_metadata = metadata.row_group(i); 106 | let col_metadata = rg_metadata.column(*proj); 107 | let (start, length) = col_metadata.byte_range(); 108 | if start < end_dl_chunk_start { 109 | file.prefetch(start, length as usize); 110 | } 111 | } 112 | } 113 | Ok(file_reader) 114 | }) 115 | .await 116 | .unwrap() 117 | } 118 | 119 | // returns the start of the downloaded chunk 120 | fn download_footer(file: CachedFile) -> u64 { 121 | let end_length = 1024 * 1024; 122 | let (end_start, end_length) = match file.len().checked_sub(end_length) { 123 | Some(val) => (val, end_length), 124 | None => (0, file.len()), 125 | }; 126 | file.prefetch(end_start, end_length as usize); 127 | end_start 128 | } 129 | } 130 | 131 | #[async_trait] 132 | impl ExecutionPlan for ParquetExec { 133 | /// Return a reference to Any that can be used for downcasting 134 | fn as_any(&self) -> &dyn Any { 135 | self 136 | } 137 | 138 | fn schema(&self) -> SchemaRef { 139 | self.projected_schema.clone() 140 | } 141 | 142 | fn children(&self) -> Vec> { 143 | // this is a leaf node and has no children 144 | vec![] 145 | } 146 | 147 | /// Get the output partitioning of this plan 148 | fn output_partitioning(&self) -> Partitioning { 149 | Partitioning::UnknownPartitioning(self.files.len()) 150 | } 151 | 152 | fn with_new_children( 153 | &self, 154 | children: Vec>, 155 | ) -> DataFusionResult> { 156 | if children.is_empty() { 157 | Ok(Arc::new(self.clone())) 158 | } else { 159 | Err(DataFusionError::Internal(format!( 160 | "Children cannot be replaced in {:?}", 161 | self 162 | ))) 163 | } 164 | } 165 | 166 | async fn execute( 167 | &self, 168 | partition: usize, 169 | ) -> DataFusionResult { 170 | let parquet_reader = self.init_file(partition).await?; 171 | let (response_tx, response_rx): ( 172 | Sender>, 173 | Receiver>, 174 | ) = channel(2); 175 | 176 | let projection = self.projection.clone(); 177 | let batch_size = self.batch_size; 178 | 179 | task::spawn_blocking(move || { 180 | if let Err(e) = read_file(parquet_reader, projection, batch_size, response_tx) 181 | { 182 | println!("Parquet reader thread terminated due to error: {:?}", e); 183 | } 184 | }); 185 | 186 | Ok(Box::pin(ParquetStream { 187 | schema: self.projected_schema.clone(), 188 | response_rx: ReceiverStream::new(response_rx), 189 | })) 190 | } 191 | } 192 | 193 | fn send_result( 194 | response_tx: &Sender>, 195 | result: ArrowResult, 196 | ) -> DataFusionResult<()> { 197 | response_tx 198 | .blocking_send(result) 199 | .map_err(|e| DataFusionError::Execution(e.to_string()))?; 200 | Ok(()) 201 | } 202 | 203 | fn read_file( 204 | file_reader: Arc>, 205 | projection: Vec, 206 | batch_size: usize, 207 | response_tx: Sender>, 208 | ) -> DataFusionResult<()> { 209 | let mut arrow_reader = ParquetFileArrowReader::new(file_reader.clone()); 210 | let mut batch_reader = 211 | arrow_reader.get_record_reader_by_columns(projection.clone(), batch_size)?; 212 | loop { 213 | match batch_reader.next() { 214 | Some(Ok(batch)) => send_result(&response_tx, Ok(batch))?, 215 | None => { 216 | break; 217 | } 218 | Some(Err(e)) => { 219 | let err_msg = format!("Error reading batch from file: {}", e.to_string()); 220 | // send error to operator 221 | send_result( 222 | &response_tx, 223 | Err(ArrowError::ParquetError(err_msg.clone())), 224 | )?; 225 | // terminate thread with error 226 | return Err(DataFusionError::Execution(err_msg)); 227 | } 228 | } 229 | } 230 | Ok(()) 231 | } 232 | 233 | struct ParquetStream { 234 | schema: SchemaRef, 235 | response_rx: ReceiverStream>, 236 | } 237 | 238 | impl Stream for ParquetStream { 239 | type Item = ArrowResult; 240 | 241 | fn poll_next( 242 | mut self: std::pin::Pin<&mut Self>, 243 | cx: &mut Context<'_>, 244 | ) -> Poll> { 245 | self.response_rx.poll_next_unpin(cx) 246 | } 247 | } 248 | 249 | impl RecordBatchStream for ParquetStream { 250 | fn schema(&self) -> SchemaRef { 251 | self.schema.clone() 252 | } 253 | } 254 | 255 | #[cfg(test)] 256 | mod tests { 257 | use std::env; 258 | use std::fs; 259 | use std::io::SeekFrom; 260 | use std::path::PathBuf; 261 | 262 | use super::*; 263 | use crate::clients::Downloader; 264 | use crate::clients::RangeCache; 265 | use crate::error::Result as BuzzResult; 266 | use arrow::array::*; 267 | use arrow::datatypes::{DataType, Field, Schema}; 268 | use arrow_parquet::arrow::ArrowWriter; 269 | use async_trait::async_trait; 270 | use tokio::fs::File as TokioFile; 271 | use tokio::io::{AsyncReadExt, AsyncSeekExt}; 272 | 273 | #[tokio::test] 274 | async fn test_small_parquet() { 275 | let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); 276 | let rec_batch = RecordBatch::try_new( 277 | schema.clone(), 278 | vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], 279 | ) 280 | .unwrap(); 281 | 282 | let filename = "test_small_parquet.parquet"; 283 | let results = write_and_exec(&rec_batch, filename).await; 284 | assert_eq!(results.len(), 1); 285 | assert_eq!(format!("{:?}", results[0]), format!("{:?}", rec_batch)); 286 | } 287 | 288 | #[tokio::test] 289 | async fn test_parquet_two_columns() { 290 | let schema = Arc::new(Schema::new(vec![ 291 | Field::new("a", DataType::Int32, false), 292 | Field::new("b", DataType::Utf8, false), 293 | ])); 294 | let rec_batch = RecordBatch::try_new( 295 | schema.clone(), 296 | vec![ 297 | Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])), 298 | Arc::new(StringArray::from(vec![ 299 | "hello", "from", "the", "other", "side", 300 | ])), 301 | ], 302 | ) 303 | .unwrap(); 304 | 305 | let filename = "test_2col_parquet.parquet"; 306 | let results = write_and_exec(&rec_batch, filename).await; 307 | assert_eq!(results.len(), 1); 308 | assert_eq!(format!("{:?}", results[0]), format!("{:?}", rec_batch)); 309 | } 310 | 311 | // Without the threaded_scheduler the async machine gets stuck because of blocking call 312 | #[tokio::test(flavor = "multi_thread")] 313 | async fn test_larger_parquet() { 314 | let schema = 315 | Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)])); 316 | let rec_batch = RecordBatch::try_new( 317 | schema.clone(), 318 | vec![Arc::new(Float64Array::from( 319 | (0..200_000).map(|val| val as f64 * 0.1).collect::>(), 320 | ))], 321 | ) 322 | .unwrap(); 323 | 324 | let filename = "test_larger_parquet.parquet"; 325 | let results = write_and_exec(&rec_batch, filename).await; 326 | assert_eq!( 327 | results.iter().map(|rb| rb.num_rows()).sum::(), 328 | 200_000 329 | ); 330 | } 331 | 332 | /// Write the given `rec_batch` as a parquet file then make it into an exec plan 333 | async fn write_and_exec(rec_batch: &RecordBatch, filename: &str) -> Vec { 334 | let (tmp_file, path) = get_temp_file(filename); 335 | 336 | let mut writer = 337 | ArrowWriter::try_new(tmp_file.try_clone().unwrap(), rec_batch.schema(), None) 338 | .unwrap(); 339 | writer.write(&rec_batch).unwrap(); 340 | writer.close().unwrap(); 341 | 342 | let file = CachedFile::new( 343 | path.into_os_string().into_string().unwrap(), 344 | tmp_file.metadata().unwrap().len(), 345 | Arc::new(RangeCache::new().await), 346 | "file_downloader".to_owned(), 347 | || Arc::new(FileDownloader {}), 348 | ); 349 | 350 | let exec_plan = ParquetExec::new(vec![file], None, 2048, rec_batch.schema()); 351 | 352 | datafusion::physical_plan::collect(Arc::new(exec_plan)) 353 | .await 354 | .unwrap() 355 | } 356 | 357 | /// A downloader that simply reads from file system (file_id is the file path) 358 | #[derive(Clone)] 359 | struct FileDownloader {} 360 | 361 | #[async_trait] 362 | impl Downloader for FileDownloader { 363 | async fn download( 364 | &self, 365 | file_id: String, 366 | start: u64, 367 | length: usize, 368 | ) -> BuzzResult> { 369 | let mut result = vec![0; length]; 370 | let mut f = TokioFile::open(&file_id).await?; 371 | f.seek(SeekFrom::Start(start)).await?; 372 | f.read_exact(&mut result).await?; 373 | Ok(result) 374 | } 375 | } 376 | 377 | /// Returns file handle for a temp file in 'target' directory with an empty content 378 | fn get_temp_file(file_name: &str) -> (fs::File, PathBuf) { 379 | // build tmp path to a file in "target/debug/testdata" 380 | let mut path_buf = env::current_dir().unwrap(); 381 | path_buf.push("target"); 382 | path_buf.push("debug"); 383 | path_buf.push("testdata"); 384 | fs::create_dir_all(&path_buf).unwrap(); 385 | path_buf.push(file_name); 386 | 387 | // create empty file (truncate) 388 | fs::File::create(path_buf.as_path()).unwrap(); 389 | 390 | // return file handle for both read and write 391 | let file = fs::OpenOptions::new() 392 | .read(true) 393 | .write(true) 394 | .open(path_buf.as_path()); 395 | assert!(file.is_ok()); 396 | (file.unwrap(), path_buf) 397 | } 398 | } 399 | -------------------------------------------------------------------------------- /code/src/execution_plan/stream.rs: -------------------------------------------------------------------------------- 1 | use std::any::Any; 2 | use std::fmt; 3 | use std::pin::Pin; 4 | use std::sync::{Arc, Mutex}; 5 | use std::task::{Context, Poll}; 6 | 7 | use arrow::datatypes::{Schema, SchemaRef}; 8 | use arrow::error::Result as ArrowResult; 9 | use arrow::record_batch::RecordBatch; 10 | use datafusion::error::{DataFusionError, Result}; 11 | use datafusion::physical_plan::ExecutionPlan; 12 | use datafusion::physical_plan::Partitioning; 13 | use datafusion::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; 14 | 15 | use async_trait::async_trait; 16 | use futures::stream::Stream; 17 | use pin_project::pin_project; 18 | 19 | pub struct StreamExec { 20 | stream: Mutex< 21 | Option> + Send + Sync>>>, 22 | >, 23 | schema: SchemaRef, 24 | 25 | projection: Vec, 26 | batch_size: usize, 27 | } 28 | 29 | impl fmt::Debug for StreamExec { 30 | fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { 31 | f.debug_struct("StreamExec") 32 | .field("schema", &self.schema) 33 | .field("projection", &self.projection) 34 | .field("batch_size", &self.batch_size) 35 | .finish() 36 | } 37 | } 38 | 39 | impl StreamExec { 40 | pub fn new( 41 | stream: Pin> + Send + Sync>>, 42 | schema: SchemaRef, 43 | projection: Option>, 44 | batch_size: usize, 45 | ) -> Self { 46 | let projection = match projection { 47 | Some(p) => p, 48 | None => (0..schema.fields().len()).collect(), 49 | }; 50 | 51 | let projected_schema = Schema::new( 52 | projection 53 | .iter() 54 | .map(|i| schema.field(*i).clone()) 55 | .collect(), 56 | ); 57 | 58 | Self { 59 | stream: Mutex::new(Some(stream)), 60 | schema: Arc::new(projected_schema), 61 | projection, 62 | batch_size, 63 | } 64 | } 65 | } 66 | 67 | #[async_trait] 68 | impl ExecutionPlan for StreamExec { 69 | fn as_any(&self) -> &dyn Any { 70 | self 71 | } 72 | 73 | fn schema(&self) -> SchemaRef { 74 | self.schema.clone() 75 | } 76 | 77 | fn children(&self) -> Vec> { 78 | // this is a leaf node and has no children 79 | vec![] 80 | } 81 | 82 | fn output_partitioning(&self) -> Partitioning { 83 | Partitioning::UnknownPartitioning(1) 84 | } 85 | 86 | fn with_new_children( 87 | &self, 88 | _children: Vec>, 89 | ) -> Result> { 90 | Err(DataFusionError::Internal(format!( 91 | "Children cannot be replaced in {:?}", 92 | self 93 | ))) 94 | } 95 | 96 | async fn execute(&self, _partition: usize) -> Result { 97 | match self.stream.lock().unwrap().take() { 98 | Some(stream) => Ok(Box::pin(StreamStream { 99 | schema: self.schema.clone(), 100 | stream, 101 | })), 102 | None => Err(datafusion::error::DataFusionError::Execution( 103 | "Cannot execute stream execution plan more than once".to_owned(), 104 | )), 105 | } 106 | } 107 | } 108 | 109 | #[pin_project] 110 | struct StreamStream { 111 | schema: SchemaRef, 112 | #[pin] 113 | stream: St, 114 | } 115 | 116 | impl>> Stream for StreamStream { 117 | type Item = ArrowResult; 118 | 119 | fn poll_next( 120 | self: Pin<&mut Self>, 121 | ctx: &mut Context<'_>, 122 | ) -> Poll> { 123 | let mut this = self.project(); 124 | let inner_res = this.stream.as_mut().poll_next(ctx); 125 | inner_res 126 | } 127 | 128 | fn size_hint(&self) -> (usize, Option) { 129 | self.stream.size_hint() 130 | } 131 | } 132 | 133 | impl>> RecordBatchStream for StreamStream { 134 | fn schema(&self) -> SchemaRef { 135 | self.schema.clone() 136 | } 137 | } 138 | -------------------------------------------------------------------------------- /code/src/flight_utils.rs: -------------------------------------------------------------------------------- 1 | //! Utils to convert flight objects to and from record batches 2 | 3 | use std::convert::TryFrom; 4 | use std::error::Error; 5 | use std::sync::Arc; 6 | 7 | use crate::internal_err; 8 | use arrow::datatypes::Schema; 9 | use arrow::error::{ArrowError, Result as ArrowResult}; 10 | use arrow::ipc::writer::IpcWriteOptions; 11 | use arrow::record_batch::RecordBatch; 12 | use arrow_flight::utils::{ 13 | flight_data_from_arrow_batch, flight_data_from_arrow_schema, 14 | flight_data_to_arrow_batch, 15 | }; 16 | use arrow_flight::{flight_descriptor, FlightData, FlightDescriptor}; 17 | use datafusion::physical_plan::SendableRecordBatchStream; 18 | use futures::{Stream, StreamExt}; 19 | use tokio_stream::wrappers::UnboundedReceiverStream; 20 | use tonic::Status; 21 | 22 | /// Convert a flight stream to a tuple with the cmd in the first flight and a stream of RecordBatch 23 | pub async fn flight_to_batches( 24 | flights: tonic::Streaming, 25 | ) -> Result<(String, impl Stream>), Box> { 26 | let mut flights = Box::pin(flights); 27 | let flight_data = flights.next().await.unwrap()?; 28 | let schema = Arc::new(Schema::try_from(&flight_data)?); 29 | let cmd = descriptor_to_cmd(flight_data.flight_descriptor)?; 30 | 31 | // all the remaining stream messages should be dictionary and record batches 32 | let record_batch_stream = flights.map(move |flight_data_res| match flight_data_res { 33 | Ok(flight_data) => { 34 | flight_data_to_arrow_batch(&flight_data, Arc::clone(&schema), &[]) 35 | } 36 | Err(e) => Err(ArrowError::ExternalError(Box::new(e))), 37 | }); 38 | Ok((cmd, record_batch_stream)) 39 | } 40 | 41 | /// Convert a stream of RecordBatches and a cmd to a stream of flights 42 | pub async fn batch_stream_to_flight( 43 | cmd: &str, 44 | batches: SendableRecordBatchStream, 45 | ) -> Result> + Send + Sync, Box> 46 | { 47 | let (sender, result) = 48 | tokio::sync::mpsc::unbounded_channel::>(); 49 | 50 | let options = Arc::new(IpcWriteOptions::default()); 51 | let mut flight_schema = flight_data_from_arrow_schema(&batches.schema(), &options); 52 | flight_schema.flight_descriptor = cmd_to_descriptor(&cmd); 53 | sender.send(Ok(flight_schema))?; 54 | 55 | // use channels to make stream sync (required by tonic) 56 | // TODO what happens with errors (currently all unwrapped in spawned task) 57 | tokio::spawn(async move { 58 | // then stream the rest 59 | batches 60 | .for_each(|batch_res| async { 61 | match batch_res { 62 | Ok(batch) => { 63 | let (dicts, data) = 64 | flight_data_from_arrow_batch(&batch, &options); 65 | dicts 66 | .into_iter() 67 | .chain(std::iter::once(data)) 68 | .for_each(|flight| (&sender).send(Ok(flight)).unwrap()); 69 | } 70 | Err(err) => { 71 | (&sender) 72 | .send(Err(Status::aborted(format!("{}", err)))) 73 | .unwrap(); 74 | } 75 | }; 76 | }) 77 | .await; 78 | }); 79 | 80 | Ok(UnboundedReceiverStream::new(result)) 81 | } 82 | 83 | /// Convert a vector of RecordBatches and a cmd to a stream of flights 84 | /// If there are no batches (empty vec), a flight with an empty schema is sent 85 | pub async fn batch_vec_to_flight( 86 | cmd: &str, 87 | batches: Vec, 88 | ) -> Result, Box> { 89 | let schema; 90 | if batches.len() == 0 { 91 | schema = Arc::new(Schema::empty()); 92 | } else { 93 | schema = batches[0].schema(); 94 | } 95 | // create an initial FlightData message that sends schema 96 | let options = IpcWriteOptions::default(); 97 | let mut flight_schema = flight_data_from_arrow_schema(&schema, &options); 98 | flight_schema.flight_descriptor = cmd_to_descriptor(&cmd); 99 | 100 | let mut flight_vec = vec![flight_schema]; 101 | 102 | let mut batches: Vec = batches 103 | .iter() 104 | .flat_map(|batch| { 105 | let (dicts, data) = flight_data_from_arrow_batch(batch, &options); 106 | dicts.into_iter().chain(std::iter::once(data)) 107 | }) 108 | .collect(); 109 | 110 | // append batch vector to schema vector, so that the first message sent is the schema 111 | flight_vec.append(&mut batches); 112 | 113 | Ok(futures::stream::iter(flight_vec)) 114 | } 115 | 116 | fn cmd_to_descriptor(cmd: &str) -> Option { 117 | Some(FlightDescriptor { 118 | r#type: flight_descriptor::DescriptorType::Cmd as i32, 119 | cmd: cmd.as_bytes().to_owned(), 120 | path: vec![], 121 | }) 122 | } 123 | 124 | fn descriptor_to_cmd( 125 | descriptor: Option, 126 | ) -> Result> { 127 | let descriptor = descriptor.ok_or(Box::new(internal_err!( 128 | "Descriptor not found in first flight" 129 | )))?; 130 | if descriptor.r#type != flight_descriptor::DescriptorType::Cmd as i32 { 131 | Err(Box::new(internal_err!("Descriptor type should be cmd"))) 132 | } else { 133 | Ok(String::from_utf8(descriptor.cmd).unwrap()) 134 | } 135 | } 136 | -------------------------------------------------------------------------------- /code/src/lib.rs: -------------------------------------------------------------------------------- 1 | pub mod clients; 2 | pub mod datasource; 3 | pub mod error; 4 | pub mod example_catalog; 5 | pub mod execution_plan; 6 | pub mod flight_utils; 7 | pub mod models; 8 | pub mod plan_utils; 9 | pub mod serde; 10 | pub mod services; 11 | 12 | // include the generated protobuf source as a submodule 13 | #[allow(clippy::all)] 14 | pub mod protobuf { 15 | include!(concat!(env!("OUT_DIR"), "/buzz.protobuf.rs")); 16 | } 17 | -------------------------------------------------------------------------------- /code/src/models/actions.rs: -------------------------------------------------------------------------------- 1 | use serde::{Deserialize, Serialize}; 2 | 3 | pub enum ActionType { 4 | Fail, 5 | HealthCheck, 6 | Unknown, 7 | } 8 | 9 | impl ActionType { 10 | pub fn from_string(serialized: String) -> Self { 11 | match serialized.as_str() { 12 | "F" => ActionType::Fail, 13 | "H" => ActionType::HealthCheck, 14 | _ => ActionType::Unknown, 15 | } 16 | } 17 | 18 | pub fn to_string(&self) -> String { 19 | match self { 20 | ActionType::Fail => "F".to_owned(), 21 | ActionType::HealthCheck => "H".to_owned(), 22 | ActionType::Unknown => "U".to_owned(), 23 | } 24 | } 25 | } 26 | 27 | #[derive(Serialize, Deserialize)] 28 | pub struct Fail { 29 | #[serde(rename = "qid")] 30 | pub query_id: String, 31 | #[serde(rename = "r")] 32 | pub reason: String, 33 | } 34 | -------------------------------------------------------------------------------- /code/src/models/env.rs: -------------------------------------------------------------------------------- 1 | use crate::error::{BuzzError, Result}; 2 | use serde::Deserialize; 3 | 4 | #[derive(Deserialize, Debug)] 5 | pub struct FargateConfig { 6 | pub hcomb_cluster_name: String, 7 | pub hcomb_task_sg_id: String, 8 | pub public_subnets: Vec, 9 | pub hcomb_task_def_arn: String, 10 | pub aws_region: String, 11 | } 12 | 13 | pub fn get_fargate_config() -> Result { 14 | envy::from_env::().map_err(|e| BuzzError::Internal(format!("{}", e))) 15 | } 16 | 17 | #[derive(Deserialize, Debug)] 18 | pub struct LambdaConfig { 19 | pub hbee_lambda_name: String, 20 | pub aws_region: String, 21 | } 22 | 23 | pub fn get_lambda_config() -> Result { 24 | envy::from_env::().map_err(|e| BuzzError::Internal(format!("{}", e))) 25 | } 26 | -------------------------------------------------------------------------------- /code/src/models/hbee_event.rs: -------------------------------------------------------------------------------- 1 | use std::io::Cursor; 2 | 3 | use crate::datasource::HBeeTableDesc; 4 | use crate::error::Result; 5 | use crate::internal_err; 6 | use crate::models::HCombAddress; 7 | use crate::protobuf; 8 | use crate::serde as proto_serde; 9 | use base64; 10 | use prost::Message; 11 | use serde::{Deserialize, Serialize}; 12 | 13 | #[derive(Serialize, Deserialize)] 14 | /// Binary Base64 encoded representation of a logical plan 15 | /// TODO serialize this as json instead of base64 proto 16 | pub struct HBeePlanBytes { 17 | #[serde(rename = "b")] 18 | bytes: String, 19 | } 20 | 21 | #[derive(Serialize, Deserialize)] 22 | pub struct HBeeEvent { 23 | #[serde(rename = "id")] 24 | pub query_id: String, 25 | #[serde(rename = "a")] 26 | pub hcomb_address: HCombAddress, 27 | #[serde(rename = "p")] 28 | pub plan: HBeePlanBytes, 29 | } 30 | 31 | impl HBeePlanBytes { 32 | /// Serialize and encode the given logical plan 33 | pub fn try_new( 34 | table_desc: &HBeeTableDesc, 35 | sql: String, 36 | source: String, 37 | ) -> Result { 38 | let proto_plan = proto_serde::serialize_hbee(table_desc, sql, source); 39 | 40 | let mut buf = vec![]; 41 | proto_plan 42 | .encode(&mut buf) 43 | .map_err(|_| internal_err!("Could convert proto to bytes"))?; 44 | 45 | Ok(Self { 46 | bytes: base64::encode(&buf), 47 | }) 48 | } 49 | 50 | pub fn parse(&self) -> Result<(HBeeTableDesc, String, String)> { 51 | let buf = base64::decode(&self.bytes) 52 | .map_err(|_| internal_err!("Could convert parse Base64"))?; 53 | let proto_plan = protobuf::HBeeScanNode::decode(&mut Cursor::new(buf)) 54 | .map_err(|_| internal_err!("Could convert bytes to proto"))?; 55 | proto_serde::deserialize_hbee(proto_plan) 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /code/src/models/mod.rs: -------------------------------------------------------------------------------- 1 | //! Models are entities that are common to services 2 | 3 | pub mod actions; 4 | pub mod env; 5 | mod hbee_event; 6 | pub mod query; 7 | 8 | pub use hbee_event::{HBeeEvent, HBeePlanBytes}; 9 | 10 | pub type HCombAddress = String; 11 | 12 | #[derive(Clone, Debug)] 13 | pub struct SizedFile { 14 | pub key: String, 15 | pub length: u64, 16 | } 17 | -------------------------------------------------------------------------------- /code/src/models/query.rs: -------------------------------------------------------------------------------- 1 | use serde::Deserialize; 2 | 3 | #[derive(PartialEq, Deserialize)] 4 | pub enum BuzzStepType { 5 | HBee, 6 | HComb, 7 | } 8 | 9 | #[derive(Deserialize)] 10 | pub struct BuzzStep { 11 | pub sql: String, 12 | pub name: String, 13 | pub partition_filter: Option, 14 | pub step_type: BuzzStepType, 15 | } 16 | 17 | #[derive(Deserialize)] 18 | pub struct HCombCapacity { 19 | /// For now only 1 zone is supported (I know, I know... YAGNI! :) 20 | pub zones: i16, 21 | } 22 | 23 | #[derive(PartialEq, Deserialize)] 24 | pub enum BuzzCatalogType { 25 | DeltaLake, 26 | Static, 27 | } 28 | 29 | #[derive(Deserialize)] 30 | pub struct BuzzCatalog { 31 | pub name: String, 32 | pub uri: String, 33 | pub r#type: BuzzCatalogType, 34 | } 35 | 36 | #[derive(Deserialize)] 37 | pub struct BuzzQuery { 38 | pub steps: Vec, 39 | pub capacity: HCombCapacity, 40 | pub catalogs: Vec, 41 | } 42 | -------------------------------------------------------------------------------- /code/src/plan_utils.rs: -------------------------------------------------------------------------------- 1 | use datafusion::logical_plan::{self, Expr, Operator}; 2 | 3 | /// converts "A AND (B AND (C OR D))" => [A, B, C OR D] 4 | /// Copied from DataFusion filter pushdown 5 | pub fn split_expr<'a>(predicate: &'a Expr, predicates: &mut Vec<&'a Expr>) { 6 | match predicate { 7 | Expr::BinaryExpr { 8 | right, 9 | op: Operator::And, 10 | left, 11 | } => { 12 | split_expr(&left, predicates); 13 | split_expr(&right, predicates); 14 | } 15 | other => predicates.push(other), 16 | } 17 | } 18 | 19 | /// converts [A, B, C] => "(A AND B) AND C" 20 | pub fn merge_expr<'a>(predicates: &[Expr]) -> Expr { 21 | let mut predicates_iter = predicates.iter(); 22 | let mut merged_pred = predicates_iter 23 | .next() 24 | .expect("Merging requires at least one expr") 25 | .clone(); 26 | while let Some(expr) = predicates_iter.next() { 27 | merged_pred = logical_plan::and(merged_pred, expr.clone()); 28 | } 29 | merged_pred 30 | } 31 | -------------------------------------------------------------------------------- /code/src/serde/from_proto.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use crate::datasource::{HBeeTableDesc, HCombTableDesc, S3ParquetTable}; 4 | use crate::error::Result; 5 | use crate::internal_err; 6 | use crate::models::SizedFile; 7 | use crate::protobuf; 8 | use arrow::ipc::convert; 9 | 10 | pub fn deserialize_hbee( 11 | message: protobuf::HBeeScanNode, 12 | ) -> Result<(HBeeTableDesc, String, String)> { 13 | let schema = convert::schema_from_bytes(&message.schema)?; 14 | let scan = message 15 | .scan 16 | .ok_or(internal_err!("Scan field cannot be empty"))?; 17 | let provider = match scan { 18 | protobuf::h_bee_scan_node::Scan::S3Parquet(scan_node) => S3ParquetTable::new( 19 | scan_node.region.to_owned(), 20 | scan_node.bucket.to_owned(), 21 | scan_node 22 | .files 23 | .iter() 24 | .map(|sized_file| SizedFile { 25 | key: sized_file.key.to_owned(), 26 | length: sized_file.length, 27 | }) 28 | .collect(), 29 | Arc::new(schema), 30 | ), 31 | }; 32 | 33 | Ok((provider, message.sql, message.source)) 34 | } 35 | 36 | pub fn deserialize_hcomb( 37 | message: protobuf::HCombScanNode, 38 | ) -> Result<(HCombTableDesc, String, String)> { 39 | let schema = convert::schema_from_bytes(&message.schema)?; 40 | let provider = HCombTableDesc::new( 41 | message.query_id.to_owned(), 42 | message.nb_hbee as usize, 43 | Arc::new(schema), 44 | ); 45 | Ok((provider, message.sql, message.source)) 46 | } 47 | -------------------------------------------------------------------------------- /code/src/serde/mod.rs: -------------------------------------------------------------------------------- 1 | //! These serialization / deserialization methods allow the exchange of DataFusion logical plans between services 2 | 3 | mod from_proto; 4 | mod to_proto; 5 | 6 | pub use from_proto::*; 7 | pub use to_proto::*; 8 | 9 | #[cfg(test)] 10 | mod tests { 11 | use std::sync::Arc; 12 | 13 | use super::*; 14 | use crate::datasource::{HCombTableDesc, S3ParquetTable}; 15 | use crate::models::SizedFile; 16 | use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; 17 | 18 | #[test] 19 | fn roundtrip_parquet() { 20 | let parquet_table = S3ParquetTable::new( 21 | "south-pole-1".to_owned(), 22 | "santa".to_owned(), 23 | vec![SizedFile { 24 | key: "gift1".to_owned(), 25 | length: 1, 26 | }], 27 | Arc::new(test_schema()), 28 | ); 29 | let sql = "SELECT * FROM swag"; 30 | let source = "swag"; 31 | 32 | let proto = 33 | to_proto::serialize_hbee(&parquet_table, sql.to_owned(), source.to_owned()); 34 | 35 | let (transfered_table, transfered_sql, transfered_source) = 36 | from_proto::deserialize_hbee(proto).unwrap(); 37 | 38 | assert_eq!(sql, transfered_sql); 39 | assert_eq!(source, transfered_source); 40 | assert_eq!( 41 | format!("{:?}", parquet_table), 42 | format!("{:?}", transfered_table) 43 | ); 44 | } 45 | 46 | #[test] 47 | fn roundtrip_hcomb() { 48 | let hcomb_table = 49 | HCombTableDesc::new("test_query_id".to_owned(), 16, Arc::new(test_schema())); 50 | let sql = "SELECT * FROM swag"; 51 | let source = "swag"; 52 | 53 | let proto = 54 | to_proto::serialize_hcomb(&hcomb_table, sql.to_owned(), source.to_owned()); 55 | 56 | let (transfered_table, transfered_sql, transfered_source) = 57 | from_proto::deserialize_hcomb(proto).unwrap(); 58 | 59 | assert_eq!(sql, transfered_sql); 60 | assert_eq!(source, transfered_source); 61 | assert_eq!( 62 | format!("{:?}", hcomb_table), 63 | format!("{:?}", transfered_table) 64 | ); 65 | } 66 | 67 | fn test_schema() -> Schema { 68 | Schema::new(vec![ 69 | Field::new("id", DataType::Int32, false), 70 | Field::new("name", DataType::Utf8, false), 71 | Field::new("state", DataType::Utf8, false), 72 | Field::new("salary", DataType::Float64, false), 73 | Field::new( 74 | "last_login", 75 | DataType::Timestamp(TimeUnit::Millisecond, None), 76 | false, 77 | ), 78 | ]) 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /code/src/serde/to_proto.rs: -------------------------------------------------------------------------------- 1 | use crate::datasource::{HBeeTableDesc, HCombTableDesc}; 2 | use crate::protobuf; 3 | use arrow::datatypes::Schema; 4 | use arrow::ipc::{writer, writer::EncodedData, writer::IpcWriteOptions}; 5 | 6 | fn serialize_schema(schema: &Schema) -> EncodedData { 7 | let options = IpcWriteOptions::default(); 8 | let data_gen = writer::IpcDataGenerator::default(); 9 | data_gen.schema_to_bytes(schema, &options) 10 | } 11 | 12 | pub fn serialize_hbee( 13 | hbee_table: &HBeeTableDesc, 14 | sql: String, 15 | source: String, 16 | ) -> protobuf::HBeeScanNode { 17 | let schema = serialize_schema(&hbee_table.schema()); 18 | let scan = match hbee_table { 19 | HBeeTableDesc::S3Parquet(table) => Some( 20 | protobuf::h_bee_scan_node::Scan::S3Parquet(protobuf::S3ParquetScanNode { 21 | region: table.region().to_owned(), 22 | bucket: table.bucket().to_owned(), 23 | files: table 24 | .files() 25 | .iter() 26 | .map(|sized_file| protobuf::SizedFile { 27 | key: sized_file.key.to_owned(), 28 | length: sized_file.length, 29 | }) 30 | .collect(), 31 | }), 32 | ), 33 | }; 34 | protobuf::HBeeScanNode { 35 | scan, 36 | sql, 37 | schema: schema.ipc_message, 38 | source, 39 | } 40 | } 41 | 42 | pub fn serialize_hcomb( 43 | hcomb_table: &HCombTableDesc, 44 | sql: String, 45 | source: String, 46 | ) -> protobuf::HCombScanNode { 47 | let schema = serialize_schema(&hcomb_table.schema()); 48 | protobuf::HCombScanNode { 49 | query_id: hcomb_table.query_id().to_owned(), 50 | nb_hbee: hcomb_table.nb_hbee() as u32, 51 | schema: schema.ipc_message, 52 | sql, 53 | source, 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /code/src/services/fuse/fuse_service.rs: -------------------------------------------------------------------------------- 1 | use std::time::Instant; 2 | 3 | use super::hbee_scheduler::HBeeScheduler; 4 | use super::hcomb_manager::HCombManager; 5 | use super::hcomb_scheduler::HCombScheduler; 6 | use super::query_planner::QueryPlanner; 7 | use crate::bad_req_err; 8 | use crate::datasource::{CatalogTable, DeltaCatalogTable}; 9 | use crate::error::Result; 10 | use crate::example_catalog; 11 | use crate::models::query::{BuzzCatalog, BuzzCatalogType, BuzzQuery}; 12 | use arrow::record_batch::RecordBatch; 13 | use arrow::util::pretty; 14 | use chrono::Utc; 15 | use futures::{StreamExt, TryStreamExt}; 16 | use tokio::join; 17 | 18 | pub struct FuseService { 19 | hbee_scheduler: Box, 20 | hcomb_manager: Box, 21 | hcomb_scheduler: Box, 22 | query_planner: QueryPlanner, 23 | } 24 | 25 | impl FuseService { 26 | pub fn new( 27 | hbee_scheduler: Box, 28 | hcomb_manager: Box, 29 | hcomb_scheduler: Box, 30 | query_planner: QueryPlanner, 31 | ) -> Self { 32 | FuseService { 33 | hbee_scheduler, 34 | hcomb_manager, 35 | hcomb_scheduler, 36 | query_planner, 37 | } 38 | } 39 | 40 | pub async fn configure_catalog(&mut self, configs: &Vec) -> Result<()> { 41 | // TODO check definition validity 42 | for conf in configs { 43 | let catalog = match &conf.r#type { 44 | BuzzCatalogType::Static => match &conf.uri[..] { 45 | "nyc_taxi_ursa" => example_catalog::nyc_taxi_ursa(), 46 | "nyc_taxi_cloudfuse" => example_catalog::nyc_taxi_cloudfuse(), 47 | "nyc_taxi_cloudfuse_sample" => { 48 | example_catalog::nyc_taxi_cloudfuse_sample() 49 | } 50 | _ => return Err(bad_req_err!("Static catalog not found")), 51 | }, 52 | BuzzCatalogType::DeltaLake => CatalogTable::new(Box::new( 53 | DeltaCatalogTable::try_new(&conf.uri, "us-east-2".to_owned()).await?, 54 | )), 55 | }; 56 | self.query_planner.add_catalog(&conf.name, catalog)? 57 | } 58 | Ok(()) 59 | } 60 | 61 | pub async fn run(&mut self, query: BuzzQuery) -> Result<()> { 62 | self.configure_catalog(&query.catalogs).await?; 63 | let start_run = Instant::now(); 64 | let addresses_future = self.hcomb_manager.find_or_start(&query.capacity); 65 | let query_id = format!("query-{}", Utc::now().to_rfc3339()); 66 | let plan_future = 67 | self.query_planner 68 | .plan(query_id.clone(), query.steps, query.capacity.zones); 69 | let (addresses, plan) = join!(addresses_future, plan_future); 70 | let addresses = addresses?; 71 | let plan = plan?; 72 | 73 | assert!( 74 | addresses.len() >= plan.zones.len(), 75 | "Not enough hcombs (found {}) were started for plan (expected {})", 76 | addresses.len(), 77 | plan.zones.len() 78 | ); 79 | 80 | if plan.zones.len() == 0 { 81 | println!("no work scheduled, empty result"); 82 | return Ok(()); 83 | } 84 | 85 | // connect to the hcombs to init the query and get result handle 86 | println!("[fuse] schedule hcombs"); 87 | let future_hcombs = (0..plan.zones.len()).map(|i| { 88 | self.hcomb_scheduler.schedule( 89 | &addresses[i], 90 | &plan.zones[i].hcomb.table, 91 | plan.zones[i].hcomb.sql.clone(), 92 | plan.zones[i].hcomb.source.clone(), 93 | ) 94 | }); 95 | let hcomb_streams = futures::stream::iter(future_hcombs) 96 | .buffer_unordered(10) 97 | .try_collect::>() 98 | .await?; 99 | 100 | // when hcombs are ready, start hbees! 101 | // TODO start hbees for hcombs that are ready before the others? 102 | println!("[fuse] schedule {} hbees", plan.nb_hbee); 103 | let start_schedule = Instant::now(); 104 | let mut hcomb_hbee_idx_tuple = (0..plan.zones.len()) 105 | .flat_map(|i| (0..plan.zones[i].hbee.len()).map(move |j| (i, j))) 106 | .collect::>(); 107 | 108 | // sort by hbee index in order to alternate between hcombs 109 | hcomb_hbee_idx_tuple.sort_by(|a, b| a.1.partial_cmp(&b.1).unwrap()); 110 | 111 | let future_hbees = hcomb_hbee_idx_tuple.into_iter().map(|(i, j)| { 112 | self.hbee_scheduler.schedule( 113 | query_id.clone(), 114 | &addresses[i], 115 | &plan.zones[i].hbee[j].table, 116 | plan.zones[i].hbee[j].sql.clone(), 117 | plan.zones[i].hbee[j].source.clone(), 118 | ) 119 | }); 120 | futures::stream::iter(future_hbees) 121 | .buffer_unordered(10) 122 | .try_collect::>() 123 | .await?; 124 | 125 | println!( 126 | "[fuse] hbee scheduling duration: {}", 127 | start_schedule.elapsed().as_millis() 128 | ); 129 | 130 | // wait for hcombs to collect all the results and desplay them comb by comb 131 | println!("[fuse] collect hcombs"); 132 | for hcomb_stream in hcomb_streams { 133 | let result: Vec = hcomb_stream.try_collect::>().await?; 134 | pretty::print_batches(&result).unwrap(); 135 | } 136 | 137 | println!( 138 | "[fuse] hbee total duration: {}", 139 | start_schedule.elapsed().as_millis() 140 | ); 141 | println!( 142 | "[fuse] total run duration: {}", 143 | start_run.elapsed().as_millis() 144 | ); 145 | Ok(()) 146 | } 147 | } 148 | -------------------------------------------------------------------------------- /code/src/services/fuse/hbee_scheduler.rs: -------------------------------------------------------------------------------- 1 | use crate::clients::lambda::LambdaInvokeClient; 2 | use crate::datasource::HBeeTableDesc; 3 | use crate::error::Result; 4 | use crate::internal_err; 5 | use crate::models::{HBeeEvent, HBeePlanBytes, HCombAddress}; 6 | use async_trait::async_trait; 7 | use hyper::{Body, Client, Request}; 8 | 9 | #[async_trait] 10 | pub trait HBeeScheduler { 11 | async fn schedule( 12 | &self, 13 | query_id: String, 14 | address: &HCombAddress, 15 | table: &HBeeTableDesc, 16 | sql: String, 17 | source: String, 18 | ) -> Result<()>; 19 | } 20 | 21 | pub struct TestHBeeScheduler { 22 | pub domain: String, 23 | } 24 | 25 | #[async_trait] 26 | impl HBeeScheduler for TestHBeeScheduler { 27 | async fn schedule( 28 | &self, 29 | query_id: String, 30 | address: &HCombAddress, 31 | table: &HBeeTableDesc, 32 | sql: String, 33 | source: String, 34 | ) -> Result<()> { 35 | let client = Client::new(); 36 | 37 | let req_body = serde_json::to_string(&HBeeEvent { 38 | query_id, 39 | hcomb_address: address.clone(), 40 | plan: HBeePlanBytes::try_new(&table, sql, source)?, 41 | }) 42 | .map_err(|_| internal_err!("failed to serialize to json"))?; 43 | 44 | let req = Request::builder() 45 | .method("POST") 46 | .uri(format!("http://{}:3000", self.domain)) 47 | .body(Body::from(req_body)) 48 | .map_err(|_| internal_err!("failed to build hbee request"))?; 49 | 50 | client 51 | .request(req) 52 | .await 53 | .map_err(|e| internal_err!("hbee scheduling failed: {}", e))?; 54 | 55 | Ok(()) 56 | } 57 | } 58 | 59 | pub struct LambdaHBeeScheduler { 60 | client: LambdaInvokeClient, 61 | } 62 | 63 | impl LambdaHBeeScheduler { 64 | pub fn try_new() -> Result { 65 | Ok(Self { 66 | client: LambdaInvokeClient::try_new()?, 67 | }) 68 | } 69 | } 70 | 71 | #[async_trait] 72 | impl HBeeScheduler for LambdaHBeeScheduler { 73 | async fn schedule( 74 | &self, 75 | query_id: String, 76 | address: &HCombAddress, 77 | table: &HBeeTableDesc, 78 | sql: String, 79 | source: String, 80 | ) -> Result<()> { 81 | let req_body = serde_json::to_vec(&HBeeEvent { 82 | query_id, 83 | hcomb_address: address.clone(), 84 | plan: HBeePlanBytes::try_new(&table, sql, source)?, 85 | }) 86 | .map_err(|_| internal_err!("failed to serialize to json"))?; 87 | 88 | self.client.invoke(req_body).await?; 89 | 90 | Ok(()) 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /code/src/services/fuse/hcomb_manager.rs: -------------------------------------------------------------------------------- 1 | use std::time::{Duration, Instant}; 2 | 3 | use crate::clients::fargate::FargateCreationClient; 4 | use crate::clients::flight_client; 5 | use crate::error::Result; 6 | use crate::internal_err; 7 | use crate::models::{query::HCombCapacity, HCombAddress}; 8 | use async_trait::async_trait; 9 | use tokio::time::timeout; 10 | 11 | #[async_trait] 12 | pub trait HCombManager { 13 | /// Search for availaible combs or start new ones if none was found. 14 | async fn find_or_start(&self, capactity: &HCombCapacity) 15 | -> Result>; 16 | } 17 | 18 | pub struct TestHCombManager { 19 | pub domain: String, 20 | } 21 | 22 | #[async_trait] 23 | impl HCombManager for TestHCombManager { 24 | async fn find_or_start( 25 | &self, 26 | capactity: &HCombCapacity, 27 | ) -> Result> { 28 | assert_eq!(capactity.zones, 1, "Only single zone supported for now"); 29 | Ok(vec![format!("http://{}:3333", self.domain)]) 30 | } 31 | } 32 | 33 | pub struct FargateHCombManager { 34 | client: FargateCreationClient, 35 | } 36 | 37 | impl FargateHCombManager { 38 | pub fn try_new() -> Result { 39 | Ok(Self { 40 | client: FargateCreationClient::try_new()?, 41 | }) 42 | } 43 | } 44 | 45 | #[async_trait] 46 | impl HCombManager for FargateHCombManager { 47 | async fn find_or_start( 48 | &self, 49 | capactity: &HCombCapacity, 50 | ) -> Result> { 51 | assert_eq!(capactity.zones, 1, "Only single zone supported for now"); 52 | 53 | let private_ip = self.client.create_new().await?; 54 | 55 | let address = format!("http://{}:3333", private_ip); 56 | 57 | let start = Instant::now(); 58 | let timeout_sec = 20; 59 | loop { 60 | match timeout( 61 | Duration::from_millis(1), 62 | flight_client::try_connect(&address), 63 | ) 64 | .await 65 | { 66 | Ok(Ok(())) => break, 67 | Ok(Err(e)) if start.elapsed().as_secs() >= timeout_sec => { 68 | Err(internal_err!( 69 | "Couldn't connect to hcomb for more than {}s with: {}", 70 | timeout_sec, 71 | e 72 | ))? 73 | } 74 | Err(_) if start.elapsed().as_secs() >= timeout_sec => { 75 | Err(internal_err!( 76 | "Couldn't connect to hcomb for more than {}s because of timeouts", 77 | timeout_sec 78 | ))? 79 | } 80 | Ok(Err(_)) | Err(_) => continue, 81 | } 82 | } 83 | println!( 84 | "[fuse] took {}ms to connect to hcomb", 85 | start.elapsed().as_millis() 86 | ); 87 | Ok(vec![address]) 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /code/src/services/fuse/hcomb_scheduler.rs: -------------------------------------------------------------------------------- 1 | use std::pin::Pin; 2 | 3 | use crate::clients::flight_client; 4 | use crate::datasource::HCombTableDesc; 5 | use crate::error::Result; 6 | use crate::internal_err; 7 | use crate::models::HCombAddress; 8 | use arrow::error::Result as ArrowResult; 9 | use arrow::record_batch::RecordBatch; 10 | use async_trait::async_trait; 11 | use tokio_stream::Stream; 12 | 13 | #[async_trait] 14 | pub trait HCombScheduler { 15 | /// Notifies the hcomb that a query is starting and opens a stream of results. 16 | async fn schedule( 17 | &self, 18 | address: &HCombAddress, 19 | hcomb_table: &HCombTableDesc, 20 | sql: String, 21 | source: String, 22 | ) -> Result>>>>; 23 | } 24 | 25 | pub struct HttpHCombScheduler; 26 | 27 | #[async_trait] 28 | impl HCombScheduler for HttpHCombScheduler { 29 | async fn schedule( 30 | &self, 31 | address: &HCombAddress, 32 | hcomb_table: &HCombTableDesc, 33 | sql: String, 34 | source: String, 35 | ) -> Result>>>> { 36 | flight_client::call_do_get(address, hcomb_table, sql, source) 37 | .await 38 | .map_err(|e| internal_err!("Could not get result from HComb: {}", e)) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /code/src/services/fuse/mod.rs: -------------------------------------------------------------------------------- 1 | mod fuse_service; 2 | mod hbee_scheduler; 3 | mod hcomb_manager; 4 | mod hcomb_scheduler; 5 | mod query_planner; 6 | 7 | pub use fuse_service::FuseService; 8 | pub use hbee_scheduler::{HBeeScheduler, LambdaHBeeScheduler, TestHBeeScheduler}; 9 | pub use hcomb_manager::{FargateHCombManager, HCombManager, TestHCombManager}; 10 | pub use hcomb_scheduler::{HCombScheduler, HttpHCombScheduler}; 11 | pub use query_planner::{HBeePlan, QueryPlanner}; 12 | -------------------------------------------------------------------------------- /code/src/services/fuse/query_planner.rs: -------------------------------------------------------------------------------- 1 | use crate::datasource::{CatalogTable, HBeeTableDesc, HCombTable, HCombTableDesc}; 2 | use crate::error::{BuzzError, Result}; 3 | use crate::models::query::{BuzzStep, BuzzStepType}; 4 | use crate::not_impl_err; 5 | use crate::plan_utils; 6 | use crate::services::utils; 7 | use datafusion::execution::context::ExecutionContext; 8 | use datafusion::logical_plan::LogicalPlan; 9 | use futures::future::{BoxFuture, FutureExt}; 10 | use std::sync::Arc; 11 | 12 | pub struct QueryPlanner { 13 | /// This execution context is not meant to run queries but only to plan them. 14 | execution_context: ExecutionContext, 15 | } 16 | 17 | #[derive(Debug)] 18 | pub struct HBeePlan { 19 | pub sql: String, 20 | pub source: String, 21 | pub table: HBeeTableDesc, 22 | } 23 | 24 | #[derive(Debug)] 25 | pub struct HCombPlan { 26 | pub sql: String, 27 | pub source: String, 28 | pub table: HCombTableDesc, 29 | } 30 | 31 | #[derive(Debug)] 32 | pub struct ZonePlan { 33 | pub hbee: Vec, 34 | pub hcomb: HCombPlan, 35 | } 36 | 37 | /// The plans to be distributed among hbees and hcombs 38 | /// To transfer them over the wire, these logical plans should be serializable 39 | #[derive(Debug)] 40 | pub struct DistributedPlan { 41 | /// One hcomb/hbee combination of plan for each zone. 42 | pub zones: Vec, 43 | pub nb_hbee: usize, 44 | } 45 | 46 | impl QueryPlanner { 47 | pub fn new() -> Self { 48 | Self { 49 | execution_context: ExecutionContext::new(), 50 | } 51 | } 52 | 53 | pub fn add_catalog(&mut self, name: &str, table: CatalogTable) -> Result<()> { 54 | self.execution_context 55 | .register_table(name, Arc::new(table))?; 56 | Ok(()) 57 | } 58 | 59 | pub async fn plan( 60 | &mut self, 61 | query_id: String, 62 | query_steps: Vec, 63 | nb_hcomb: i16, 64 | ) -> Result { 65 | // TODO lift the limitation inforced by the following assert: 66 | assert!( 67 | query_steps.len() == 2 68 | && query_steps[0].step_type == BuzzStepType::HBee 69 | && query_steps[1].step_type == BuzzStepType::HComb, 70 | "You must have one exactly one HBee step followed by one HComb step for now" 71 | ); 72 | 73 | let hbee_step = &query_steps[0]; 74 | let hcomb_step = &query_steps[1]; 75 | 76 | let bee_df = self.execution_context.sql(&hbee_step.sql)?; 77 | let src_bee_plan = self.execution_context.optimize(&bee_df.to_logical_plan())?; 78 | let hbee_actual_src = utils::find_table_name::(&src_bee_plan)?; 79 | let bee_output_schema = src_bee_plan.schema().as_ref().clone(); 80 | let bee_plans = self 81 | .split(&src_bee_plan, &hbee_step.partition_filter) 82 | .await?; 83 | let nb_hbee = bee_plans.len(); 84 | 85 | if nb_hbee == 0 { 86 | return Ok(DistributedPlan { 87 | zones: vec![], 88 | nb_hbee, 89 | }); 90 | } 91 | 92 | // register a handle to the intermediate table on the context 93 | let hcomb_table_desc = 94 | HCombTableDesc::new(query_id, nb_hbee, bee_output_schema.into()); 95 | let hcomb_expected_src: &str = &hbee_step.name; 96 | 97 | // plan the hcomb part of the query, to check if it is valid 98 | let hcomb_table = HCombTable::new_empty(hcomb_table_desc.clone()); 99 | self.execution_context 100 | .register_table(hcomb_expected_src, Arc::new(hcomb_table))?; 101 | let hcomb_df = self.execution_context.sql(&hcomb_step.sql)?; 102 | let hcomb_plan = hcomb_df.to_logical_plan(); 103 | let hcomb_actual_src = utils::find_table_name::(&hcomb_plan)?; 104 | if hcomb_actual_src != hcomb_expected_src { 105 | return Err(BuzzError::BadRequest(format!( 106 | "The source table for the {} step is not an HBee", 107 | hcomb_step.name, 108 | ))); 109 | } 110 | 111 | // If they are less hbees than hcombs, don't use all hcombs 112 | let used_hcomb = std::cmp::min(nb_hcomb as usize, nb_hbee); 113 | 114 | // init plans for each zone 115 | let mut zones = (0..used_hcomb) 116 | .map(|_i| ZonePlan { 117 | hbee: vec![], 118 | hcomb: HCombPlan { 119 | table: hcomb_table_desc.clone(), 120 | sql: query_steps[1].sql.clone(), 121 | source: hcomb_actual_src.to_owned(), 122 | }, 123 | }) 124 | .collect::>(); 125 | // distribute hbee plans between zones 126 | bee_plans.into_iter().enumerate().for_each(|(i, bee_plan)| { 127 | zones[i % used_hcomb].hbee.push(HBeePlan { 128 | table: bee_plan, 129 | sql: hbee_step.sql.clone(), 130 | source: hbee_actual_src.to_owned(), 131 | }) 132 | }); 133 | 134 | Ok(DistributedPlan { 135 | zones: zones, 136 | nb_hbee, 137 | }) 138 | } 139 | 140 | /// Takes a plan and if the source is a catalog, distibutes the files accordingly 141 | /// Each resulting HBee table is a good workload for a given hbee 142 | /// Only works with linear plans (only one datasource) 143 | fn split<'a>( 144 | &'a mut self, 145 | plan: &'a LogicalPlan, 146 | partition_filters: &'a Option, 147 | ) -> BoxFuture<'a, Result>> { 148 | async move { 149 | let new_inputs = plan.inputs(); 150 | if new_inputs.len() > 1 { 151 | Err(not_impl_err!( 152 | "Operations with more than one inputs are not supported", 153 | )) 154 | } else if new_inputs.len() == 1 { 155 | let mut filter_exprs = vec![]; 156 | if let LogicalPlan::Filter { predicate, .. } = &plan { 157 | plan_utils::split_expr(predicate, &mut filter_exprs); 158 | } 159 | let table_descs = self.split(new_inputs[0], partition_filters).await?; 160 | Ok(table_descs) 161 | } else if let Some(catalog_table) = Self::as_catalog(&plan) { 162 | let table_descs = catalog_table.split(partition_filters).await?; 163 | Ok(table_descs) 164 | } else { 165 | Err(not_impl_err!("Split only works with catalog tables",)) 166 | } 167 | } 168 | .boxed() // recursion in an `async fn` requires boxing 169 | } 170 | 171 | fn as_catalog<'a>(plan: &'a LogicalPlan) -> Option<&'a CatalogTable> { 172 | if let LogicalPlan::TableScan { source: table, .. } = plan { 173 | table.as_any().downcast_ref::() 174 | } else { 175 | None 176 | } 177 | } 178 | } 179 | 180 | #[cfg(test)] 181 | mod tests { 182 | use super::*; 183 | use crate::datasource::CatalogTable; 184 | use crate::datasource::MockSplittableTable; 185 | 186 | #[tokio::test] 187 | async fn test_simple_query() { 188 | let mut planner = QueryPlanner::new(); 189 | let nb_split = 5; 190 | planner 191 | .add_catalog( 192 | "test", 193 | CatalogTable::new(Box::new(MockSplittableTable::new(nb_split, 0))), 194 | ) 195 | .expect("Catalog added"); 196 | 197 | let steps = vec![ 198 | BuzzStep { 199 | sql: "SELECT * FROM test".to_owned(), 200 | name: "mapper".to_owned(), 201 | step_type: BuzzStepType::HBee, 202 | partition_filter: None, 203 | }, 204 | BuzzStep { 205 | sql: "SELECT * FROM mapper".to_owned(), 206 | name: "reducer".to_owned(), 207 | step_type: BuzzStepType::HComb, 208 | partition_filter: None, 209 | }, 210 | ]; 211 | 212 | let plan_res = planner.plan("mock_query_id".to_owned(), steps, 1).await; 213 | let plan = plan_res.expect("The planner failed on a simple query"); 214 | assert_eq!(plan.zones.len(), 1); 215 | assert_eq!(plan.zones[0].hbee.len(), nb_split); 216 | } 217 | 218 | #[tokio::test] 219 | async fn test_query_unknown_table() { 220 | let mut planner = QueryPlanner::new(); 221 | let steps = vec![ 222 | BuzzStep { 223 | sql: "SELECT * FROM test".to_owned(), 224 | name: "mapper".to_owned(), 225 | step_type: BuzzStepType::HBee, 226 | partition_filter: None, 227 | }, 228 | BuzzStep { 229 | sql: "SELECT * FROM mapper".to_owned(), 230 | name: "reducer".to_owned(), 231 | step_type: BuzzStepType::HComb, 232 | partition_filter: None, 233 | }, 234 | ]; 235 | 236 | let plan_res = planner.plan("mock_query_id".to_owned(), steps, 1).await; 237 | assert!( 238 | plan_res.is_err(), 239 | "The planner should have failed as the 'test' table is not defined" 240 | ); 241 | } 242 | 243 | #[tokio::test] 244 | async fn test_query_with_condition() { 245 | let mut planner = QueryPlanner::new(); 246 | let nb_split = 5; 247 | planner 248 | .add_catalog( 249 | "test", 250 | CatalogTable::new(Box::new(MockSplittableTable::new(nb_split, 2))), 251 | ) 252 | .expect("Catalog added"); 253 | 254 | let steps = vec![ 255 | BuzzStep { 256 | sql: "SELECT * FROM test WHERE data_col=0".to_owned(), 257 | name: "mapper".to_owned(), 258 | step_type: BuzzStepType::HBee, 259 | partition_filter: Some( 260 | "part_key_2>='part_value_001' AND part_key_2<='part_value_003'" 261 | .to_owned(), 262 | ), 263 | }, 264 | BuzzStep { 265 | sql: "SELECT * FROM mapper".to_owned(), 266 | name: "reducer".to_owned(), 267 | step_type: BuzzStepType::HComb, 268 | partition_filter: None, 269 | }, 270 | ]; 271 | 272 | let plan_res = planner.plan("mock_query_id".to_owned(), steps, 1).await; 273 | let plan = plan_res.expect("The planner failed on a query with condition"); 274 | assert_eq!(plan.zones.len(), 1); 275 | assert_eq!(plan.zones[0].hbee.len(), 3); 276 | } 277 | 278 | #[tokio::test] 279 | async fn test_query_with_empty_catalog() { 280 | let mut planner = QueryPlanner::new(); 281 | let nb_split = 5; 282 | planner 283 | .add_catalog( 284 | "test", 285 | CatalogTable::new(Box::new(MockSplittableTable::new(nb_split, 1))), 286 | ) 287 | .expect("Catalog added"); 288 | 289 | let steps = vec![ 290 | BuzzStep { 291 | sql: "SELECT * FROM test".to_owned(), 292 | name: "mapper".to_owned(), 293 | step_type: BuzzStepType::HBee, 294 | partition_filter: Some("part_key_1='not_in_partition_value'".to_owned()), 295 | }, 296 | BuzzStep { 297 | sql: "SELECT * FROM mapper".to_owned(), 298 | name: "reducer".to_owned(), 299 | step_type: BuzzStepType::HComb, 300 | partition_filter: None, 301 | }, 302 | ]; 303 | 304 | let plan_res = planner.plan("mock_query_id".to_owned(), steps, 1).await; 305 | let plan = 306 | plan_res.expect("The planner failed on a query with no data in catalog"); 307 | assert_eq!(plan.zones.len(), 0); 308 | } 309 | 310 | #[tokio::test] 311 | async fn test_query_with_grouping() { 312 | let mut planner = QueryPlanner::new(); 313 | let nb_split = 5; 314 | planner 315 | .add_catalog( 316 | "test", 317 | CatalogTable::new(Box::new(MockSplittableTable::new(nb_split, 1))), 318 | ) 319 | .expect("Catalog added"); 320 | 321 | let steps = vec![ 322 | BuzzStep { 323 | sql: 324 | "SELECT data_col, count(data_col) as cnt FROM test GROUP BY data_col" 325 | .to_owned(), 326 | name: "mapper".to_owned(), 327 | step_type: BuzzStepType::HBee, 328 | partition_filter: None, 329 | }, 330 | BuzzStep { 331 | sql: "SELECT data_col, count(cnt) FROM mapper GROUP BY data_col" 332 | .to_owned(), 333 | name: "reducer".to_owned(), 334 | step_type: BuzzStepType::HComb, 335 | partition_filter: None, 336 | }, 337 | ]; 338 | 339 | let plan_res = planner.plan("mock_query_id".to_owned(), steps, 1).await; 340 | let plan = plan_res.expect("The planner failed on a query with condition"); 341 | assert_eq!(plan.zones.len(), 1); 342 | assert_eq!(plan.zones[0].hbee.len(), nb_split); 343 | } 344 | 345 | #[tokio::test] 346 | async fn test_bad_hcomb_table() { 347 | let mut planner = QueryPlanner::new(); 348 | let nb_split = 5; 349 | planner 350 | .add_catalog( 351 | "test", 352 | CatalogTable::new(Box::new(MockSplittableTable::new(nb_split, 0))), 353 | ) 354 | .expect("Catalog added"); 355 | 356 | let steps = vec![ 357 | BuzzStep { 358 | sql: "SELECT * FROM test".to_owned(), 359 | name: "mapper".to_owned(), 360 | step_type: BuzzStepType::HBee, 361 | partition_filter: None, 362 | }, 363 | BuzzStep { 364 | sql: "SELECT * FROM test".to_owned(), 365 | name: "reducer".to_owned(), 366 | step_type: BuzzStepType::HComb, 367 | partition_filter: None, 368 | }, 369 | ]; 370 | 371 | let plan_res = planner.plan("mock_query_id".to_owned(), steps, 1).await; 372 | plan_res.expect_err("The source table for the reducer step is not an HBee"); 373 | } 374 | } 375 | -------------------------------------------------------------------------------- /code/src/services/hbee/collector.rs: -------------------------------------------------------------------------------- 1 | use crate::clients::flight_client; 2 | use crate::error::Result; 3 | use crate::internal_err; 4 | use crate::models::HCombAddress; 5 | use arrow::record_batch::RecordBatch; 6 | use arrow::util::pretty; 7 | use async_trait::async_trait; 8 | 9 | #[async_trait] 10 | pub trait Collector: Send + Sync { 11 | /// send results back to hcomb 12 | async fn send_back( 13 | &self, 14 | query_id: String, 15 | data: Result>, 16 | address: HCombAddress, 17 | ) -> Result<()>; 18 | } 19 | 20 | pub struct NoopCollector {} 21 | 22 | #[async_trait] 23 | impl Collector for NoopCollector { 24 | async fn send_back( 25 | &self, 26 | _query_id: String, 27 | data: Result>, 28 | _address: HCombAddress, 29 | ) -> Result<()> { 30 | let result = data?; 31 | pretty::print_batches(&result).unwrap(); 32 | Ok(()) 33 | } 34 | } 35 | 36 | pub struct HttpCollector {} 37 | 38 | #[async_trait] 39 | impl Collector for HttpCollector { 40 | async fn send_back( 41 | &self, 42 | query_id: String, 43 | data: Result>, 44 | address: HCombAddress, 45 | ) -> Result<()> { 46 | match data { 47 | Ok(query_res) => flight_client::call_do_put(query_id, &address, query_res) 48 | .await 49 | .map_err(|e| internal_err!("Could not do_put hbee result: {}", e)), 50 | Err(query_err) => { 51 | flight_client::call_fail_action( 52 | query_id, 53 | &address, 54 | format!("{}", query_err), 55 | ) 56 | .await 57 | .map_err(|e| internal_err!("Could not do_action(FAIL) hbee: {}", e))?; 58 | Err(query_err) 59 | } 60 | } 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /code/src/services/hbee/hbee_service.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | use std::time::Instant; 3 | 4 | use super::Collector; 5 | use crate::clients::RangeCache; 6 | use crate::datasource::{HBeeTableDesc, HBeeTable}; 7 | use crate::error::Result; 8 | use crate::internal_err; 9 | use crate::models::HCombAddress; 10 | use arrow::record_batch::RecordBatch; 11 | use datafusion::execution::context::{ExecutionConfig, ExecutionContext}; 12 | use datafusion::physical_plan::{merge::MergeExec, ExecutionPlan}; 13 | 14 | pub struct HBeeService { 15 | execution_config: ExecutionConfig, 16 | range_cache: Arc, 17 | collector: Box, 18 | } 19 | 20 | impl HBeeService { 21 | pub async fn new(collector: Box) -> Self { 22 | let execution_config = ExecutionConfig::new() 23 | .with_batch_size(2048) 24 | .with_concurrency(1); 25 | Self { 26 | execution_config, 27 | range_cache: Arc::new(RangeCache::new().await), 28 | collector, 29 | } 30 | } 31 | } 32 | 33 | impl HBeeService { 34 | pub async fn execute_query( 35 | &self, 36 | query_id: String, 37 | table: HBeeTableDesc, 38 | sql: String, 39 | source: String, 40 | address: HCombAddress, 41 | ) -> Result<()> { 42 | println!("[hbee] execute query"); 43 | let start = Instant::now(); 44 | let query_res = self.query(table, sql, source).await; 45 | let cache_stats = self.range_cache.statistics(); 46 | println!("[hbee] query_duration={}, waiting_download_ms={}, downloaded_bytes={}, processed_bytes={}, download_count={}", 47 | start.elapsed().as_millis(), 48 | cache_stats.waiting_download_ms(), 49 | cache_stats.downloaded_bytes(), 50 | cache_stats.processed_bytes(), 51 | cache_stats.download_count(), 52 | ); 53 | let start = Instant::now(); 54 | let exec_res = self.collector.send_back(query_id, query_res, address).await; 55 | println!("[hbee] collector duration: {}", start.elapsed().as_millis()); 56 | exec_res 57 | } 58 | 59 | /// Execute the logical plan and collect the results 60 | /// Collecting the results might increase latency and mem consumption but: 61 | /// - reduces connection duration from hbee to hcomb, thus decreasing load on hcomb 62 | /// - allows to collect exec errors at once, effectively choosing between do_put and FAIL action 63 | async fn query(&self,table: HBeeTableDesc, sql: String, source: String) -> Result> { 64 | let mut execution_context = ExecutionContext::with_config(self.execution_config.clone()); 65 | let start = Instant::now(); 66 | let provider = HBeeTable::new(Arc::new(table), Arc::clone(&self.range_cache)); 67 | let source_ref: &str = &source; 68 | execution_context 69 | .register_table(source_ref, Arc::new(provider))?; 70 | let physical_plan; 71 | { 72 | let df = execution_context.sql(&sql)?; 73 | let plan = df.to_logical_plan(); 74 | let plan = execution_context.optimize(&plan)?; 75 | physical_plan = execution_context.create_physical_plan(&plan)?; 76 | } 77 | println!( 78 | "[hbee] planning duration: {}, partitions: {}", 79 | start.elapsed().as_millis(), 80 | physical_plan.output_partitioning().partition_count() 81 | ); 82 | // if necessary, merge the partitions 83 | let merged_plan = match physical_plan.output_partitioning().partition_count() { 84 | 0 => Err(internal_err!("Should have at least one partition"))?, 85 | 1 => physical_plan, 86 | _ => { 87 | // merge into a single partition 88 | let physical_plan = MergeExec::new(physical_plan.clone()); 89 | assert_eq!(1, physical_plan.output_partitioning().partition_count()); 90 | Arc::new(physical_plan) 91 | } 92 | }; 93 | datafusion::physical_plan::collect(merged_plan) 94 | .await 95 | .map_err(|e| e.into()) 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /code/src/services/hbee/mod.rs: -------------------------------------------------------------------------------- 1 | mod collector; 2 | mod hbee_service; 3 | 4 | pub use collector::{Collector, HttpCollector, NoopCollector}; 5 | pub use hbee_service::HBeeService; 6 | -------------------------------------------------------------------------------- /code/src/services/hcomb/flight_service.rs: -------------------------------------------------------------------------------- 1 | use std::io::Cursor; 2 | use std::pin::Pin; 3 | use std::sync::Arc; 4 | 5 | use super::hcomb_service::HCombService; 6 | use crate::error::BuzzError; 7 | use crate::flight_utils; 8 | use crate::models::actions; 9 | use crate::protobuf; 10 | use crate::serde; 11 | use arrow_flight::flight_service_server::FlightServiceServer; 12 | use arrow_flight::{ 13 | flight_service_server::FlightService, Action, ActionType, Criteria, Empty, 14 | FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse, 15 | PutResult, SchemaResult, Ticket, 16 | }; 17 | use futures::Stream; 18 | use prost::Message; 19 | use tonic::transport::Server; 20 | use tonic::{Request, Response, Status, Streaming}; 21 | 22 | #[derive(Clone)] 23 | pub struct FlightServiceImpl { 24 | hcomb_service: Arc, 25 | } 26 | 27 | impl FlightServiceImpl { 28 | pub fn new(hcomb_service: HCombService) -> Self { 29 | Self { 30 | hcomb_service: Arc::new(hcomb_service), 31 | } 32 | } 33 | 34 | pub async fn start(&self) -> tokio::task::JoinHandle<()> { 35 | let addr = "0.0.0.0:3333".parse().unwrap(); 36 | let svc = FlightServiceServer::new(self.clone()); 37 | tokio::spawn(async move { 38 | println!("[hcomb] Listening on {:?}", addr); 39 | Server::builder() 40 | .add_service(svc) 41 | .serve(addr) 42 | .await 43 | .unwrap(); 44 | }) 45 | } 46 | } 47 | 48 | #[tonic::async_trait] 49 | impl FlightService for FlightServiceImpl { 50 | type HandshakeStream = Pin< 51 | Box> + Send + Sync + 'static>, 52 | >; 53 | type ListFlightsStream = 54 | Pin> + Send + Sync + 'static>>; 55 | type DoGetStream = 56 | Pin> + Send + Sync + 'static>>; 57 | type DoPutStream = 58 | Pin> + Send + Sync + 'static>>; 59 | type DoActionStream = Pin< 60 | Box< 61 | dyn Stream> 62 | + Send 63 | + Sync 64 | + 'static, 65 | >, 66 | >; 67 | type ListActionsStream = 68 | Pin> + Send + Sync + 'static>>; 69 | type DoExchangeStream = 70 | Pin> + Send + Sync + 'static>>; 71 | 72 | async fn get_schema( 73 | &self, 74 | _request: Request, 75 | ) -> Result, Status> { 76 | Err(Status::unimplemented("Not yet implemented")) 77 | } 78 | 79 | async fn do_get( 80 | &self, 81 | request: Request, 82 | ) -> Result, Status> { 83 | // parse request 84 | let ticket = request.into_inner().ticket; 85 | let plan_node = protobuf::HCombScanNode::decode(&mut Cursor::new(ticket)) 86 | .map_err(|_| { 87 | Status::invalid_argument("Plan could not be parsed from bytes") 88 | })?; 89 | let (provider, sql, source) = 90 | serde::deserialize_hcomb(plan_node).map_err(|_| { 91 | Status::invalid_argument("Plan could not be converted from proto") 92 | })?; 93 | // execute query 94 | let results = self 95 | .hcomb_service 96 | .execute_query(provider, sql, source) 97 | .await 98 | .map_err(|e| Status::internal(format!("Query failed: {}", e)))?; 99 | // serialize response 100 | let flights = flight_utils::batch_stream_to_flight(&results.0, results.1) 101 | .await 102 | .map_err(|_| Status::internal("Plan could not be converted into flight"))?; 103 | Ok(Response::new(Box::pin(flights))) 104 | } 105 | 106 | async fn handshake( 107 | &self, 108 | _request: Request>, 109 | ) -> Result, Status> { 110 | Err(Status::unimplemented("Not yet implemented")) 111 | } 112 | 113 | async fn list_flights( 114 | &self, 115 | _request: Request, 116 | ) -> Result, Status> { 117 | Err(Status::unimplemented("Not yet implemented")) 118 | } 119 | 120 | async fn get_flight_info( 121 | &self, 122 | _request: Request, 123 | ) -> Result, Status> { 124 | Err(Status::unimplemented("Not yet implemented")) 125 | } 126 | 127 | async fn do_put( 128 | &self, 129 | request: Request>, 130 | ) -> Result, Status> { 131 | let (cmd, batches) = flight_utils::flight_to_batches(request.into_inner()) 132 | .await 133 | .map_err(|e| { 134 | Status::invalid_argument(format!("Invalid put request:{}", e)) 135 | })?; 136 | 137 | self.hcomb_service.add_results(&cmd, batches).await; 138 | let output = futures::stream::empty(); 139 | Ok(Response::new(Box::pin(output) as Self::DoPutStream)) 140 | } 141 | 142 | async fn do_action( 143 | &self, 144 | request: Request, 145 | ) -> Result, Status> { 146 | let action = request.into_inner(); 147 | match actions::ActionType::from_string(action.r#type) { 148 | actions::ActionType::Fail => { 149 | let fail_action: actions::Fail = 150 | serde_json::from_slice(&action.body).unwrap(); 151 | self.hcomb_service.fail( 152 | &fail_action.query_id, 153 | BuzzError::HBee(format!( 154 | "FAIL action called: {}", 155 | &fail_action.reason 156 | )), 157 | ); 158 | let output = futures::stream::empty(); 159 | Ok(Response::new(Box::pin(output) as Self::DoActionStream)) 160 | } 161 | actions::ActionType::HealthCheck => { 162 | let output = futures::stream::empty(); 163 | Ok(Response::new(Box::pin(output) as Self::DoActionStream)) 164 | } 165 | actions::ActionType::Unknown => { 166 | Err(Status::unimplemented("Not yet implemented")) 167 | } 168 | } 169 | } 170 | 171 | async fn list_actions( 172 | &self, 173 | _request: Request, 174 | ) -> Result, Status> { 175 | Err(Status::unimplemented("Not yet implemented")) 176 | } 177 | 178 | async fn do_exchange( 179 | &self, 180 | _request: Request>, 181 | ) -> Result, Status> { 182 | Err(Status::unimplemented("Not yet implemented")) 183 | } 184 | } 185 | 186 | // fn to_tonic_err(e: &datafusion::error::DataFusionError) -> Status { 187 | // Status::internal(format!("{:?}", e)) 188 | // } 189 | -------------------------------------------------------------------------------- /code/src/services/hcomb/hcomb_service.rs: -------------------------------------------------------------------------------- 1 | use std::process::exit; 2 | use std::sync::atomic::{AtomicI64, Ordering}; 3 | use std::sync::Arc; 4 | use std::time::Duration; 5 | 6 | use super::results_service::ResultsService; 7 | use crate::datasource::{HCombTable, HCombTableDesc}; 8 | use crate::error::{BuzzError, Result}; 9 | use crate::internal_err; 10 | use arrow::error::{ArrowError, Result as ArrowResult}; 11 | use arrow::record_batch::RecordBatch; 12 | use datafusion::execution::context::{ExecutionConfig, ExecutionContext}; 13 | use datafusion::physical_plan::{ 14 | merge::MergeExec, ExecutionPlan, SendableRecordBatchStream, 15 | }; 16 | use futures::{Stream, StreamExt}; 17 | 18 | pub struct HCombService { 19 | results_service: Arc, 20 | execution_config: ExecutionConfig, 21 | last_query: Arc, // timestamp of the last query in seconds 22 | } 23 | 24 | const TASK_EXPIRATION_SEC: i64 = 300; 25 | 26 | impl HCombService { 27 | pub fn new() -> Self { 28 | let execution_config = ExecutionConfig::new() 29 | .with_batch_size(2048) 30 | .with_concurrency(1); 31 | let last_query = Arc::new(AtomicI64::new(chrono::Utc::now().timestamp())); 32 | let last_query_ref = Arc::clone(&last_query); 33 | tokio::spawn(async move { 34 | let mut interval = tokio::time::interval(Duration::from_secs(1)); 35 | loop { 36 | interval.tick().await; 37 | let elapsed = chrono::Utc::now().timestamp() 38 | - last_query_ref.load(Ordering::Relaxed); 39 | if elapsed >= TASK_EXPIRATION_SEC { 40 | println!( 41 | "[hcomb] task expired after {}s of inactivity, shutting down...", 42 | elapsed 43 | ); 44 | exit(0); 45 | } 46 | } 47 | }); 48 | Self { 49 | results_service: Arc::new(ResultsService::new()), 50 | execution_config, 51 | last_query, 52 | } 53 | } 54 | 55 | /// Executes the hcomb plan 56 | /// Returns the query id and the result stream 57 | pub async fn execute_query( 58 | &self, 59 | provider_desc: HCombTableDesc, 60 | sql: String, 61 | source: String, 62 | ) -> Result<(String, SendableRecordBatchStream)> { 63 | println!("[hcomb] execute query..."); 64 | let mut execution_context = 65 | ExecutionContext::with_config(self.execution_config.clone()); 66 | self.last_query 67 | .store(chrono::Utc::now().timestamp(), Ordering::Relaxed); 68 | let query_id = provider_desc.query_id().to_owned(); 69 | let batch_stream = self 70 | .results_service 71 | .new_query(query_id.clone(), provider_desc.nb_hbee()); 72 | let provider = HCombTable::new(provider_desc, Box::pin(batch_stream)); 73 | let physical_plan; 74 | { 75 | // limit scope of df because not send so should not overlab await 76 | let source_ref: &str = &source; 77 | execution_context.register_table(source_ref, Arc::new(provider))?; 78 | let df = execution_context.sql(&sql)?; 79 | let plan = df.to_logical_plan(); 80 | physical_plan = execution_context.create_physical_plan(&plan)?; 81 | } 82 | 83 | // if necessary, merge the partitions 84 | let query_res = match physical_plan.output_partitioning().partition_count() { 85 | 0 => Err(internal_err!("Should have at least one partition")), 86 | 1 => physical_plan.execute(0).await.map_err(|e| e.into()), 87 | _ => { 88 | // merge into a single partition 89 | let physical_plan = MergeExec::new(physical_plan.clone()); 90 | assert_eq!(1, physical_plan.output_partitioning().partition_count()); 91 | physical_plan.execute(0).await.map_err(|e| e.into()) 92 | } 93 | }; 94 | query_res.map(|res| (query_id.clone(), res)) 95 | } 96 | 97 | pub async fn add_results( 98 | &self, 99 | query_id: &str, 100 | batches: impl Stream>, 101 | ) { 102 | let mut batches = Box::pin(batches); 103 | let mut has_err = false; 104 | while let Some(batch) = batches.next().await { 105 | if batch.is_err() { 106 | has_err = true; 107 | } 108 | self.results_service.add_result(&query_id, batch); 109 | } 110 | if !has_err { 111 | self.results_service.task_finished(&query_id); 112 | } 113 | } 114 | 115 | pub fn fail(&self, query_id: &str, err: BuzzError) { 116 | self.results_service.add_result( 117 | query_id, 118 | Err(ArrowError::from_external_error(Box::new(err))), 119 | ); 120 | } 121 | } 122 | -------------------------------------------------------------------------------- /code/src/services/hcomb/mod.rs: -------------------------------------------------------------------------------- 1 | mod flight_service; 2 | mod hcomb_service; 3 | mod results_service; 4 | 5 | pub use flight_service::FlightServiceImpl; 6 | pub use hcomb_service::HCombService; 7 | -------------------------------------------------------------------------------- /code/src/services/hcomb/results_service.rs: -------------------------------------------------------------------------------- 1 | use std::collections::HashMap; 2 | use std::sync::Mutex; 3 | 4 | use arrow::error::Result as ArrowResult; 5 | use arrow::record_batch::RecordBatch; 6 | use tokio::sync::mpsc::{self}; 7 | use tokio_stream::wrappers::UnboundedReceiverStream; 8 | use tokio_stream::Stream; 9 | 10 | struct IntermediateRes { 11 | tx: Option>>, 12 | remaining_tasks: usize, 13 | } 14 | 15 | pub struct ResultsService { 16 | tx_map: Mutex>, 17 | } 18 | 19 | impl ResultsService { 20 | pub fn new() -> Self { 21 | Self { 22 | tx_map: Mutex::new(HashMap::new()), 23 | } 24 | } 25 | 26 | pub fn new_query( 27 | &self, 28 | query_id: String, 29 | nb_hbees: usize, 30 | ) -> impl Stream> { 31 | let (tx, rx) = mpsc::unbounded_channel(); 32 | { 33 | let mut sender_map_guard = self.tx_map.lock().unwrap(); 34 | sender_map_guard.insert( 35 | query_id, 36 | IntermediateRes { 37 | tx: Some(tx), 38 | remaining_tasks: nb_hbees, 39 | }, 40 | ); 41 | } 42 | UnboundedReceiverStream::new(rx) 43 | } 44 | 45 | pub fn add_result(&self, query_id: &str, data: ArrowResult) { 46 | let sender_map = self.tx_map.lock().unwrap(); 47 | let res_opt = sender_map.get(query_id); 48 | match res_opt { 49 | Some(res) => { 50 | let send_res = res.tx.as_ref().unwrap().send(data); 51 | if send_res.is_err() { 52 | println!("[hcomb] Result chan closed because query '{}' failed, ignoring result", query_id); 53 | } 54 | } 55 | None => { 56 | println!( 57 | "[hcomb] Query '{}' not registered in IntermediateResults", 58 | query_id 59 | ); 60 | } 61 | } 62 | } 63 | 64 | pub fn task_finished(&self, query_id: &str) { 65 | let mut sender_map = self.tx_map.lock().unwrap(); 66 | let res_opt = sender_map.get_mut(query_id); 67 | match res_opt { 68 | Some(res) => { 69 | res.remaining_tasks -= 1; 70 | if res.remaining_tasks == 0 { 71 | res.tx = None; 72 | } 73 | } 74 | None => { 75 | println!( 76 | "[hcomb] Query '{}' not registered in IntermediateResults", 77 | query_id 78 | ); 79 | } 80 | } 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /code/src/services/mod.rs: -------------------------------------------------------------------------------- 1 | //! Services are organized according to where they are running 2 | 3 | pub mod fuse; 4 | pub mod hbee; 5 | pub mod hcomb; 6 | mod utils; 7 | -------------------------------------------------------------------------------- /code/src/services/utils.rs: -------------------------------------------------------------------------------- 1 | use crate::error::Result; 2 | use crate::not_impl_err; 3 | use datafusion::datasource::TableProvider; 4 | use datafusion::logical_plan::LogicalPlan; 5 | 6 | /// Search a TableProvider of the given type in the plan. 7 | /// Only works with linear plans (only one datasource). 8 | pub fn find_table_name<'a, T: TableProvider + 'static>( 9 | plan: &'a LogicalPlan, 10 | ) -> Result<&'a str> { 11 | let new_inputs = plan.inputs(); 12 | if new_inputs.len() > 1 { 13 | Err(not_impl_err!( 14 | "Operations with more than one inputs are not supported", 15 | )) 16 | } else if new_inputs.len() == 1 { 17 | // recurse 18 | find_table_name::(new_inputs[0]) 19 | } else { 20 | if let Some(result_table) = as_table_name::(&plan) { 21 | Ok(result_table) 22 | } else { 23 | Err(not_impl_err!( 24 | "Expected root to be a {}", 25 | std::any::type_name::() 26 | )) 27 | } 28 | } 29 | } 30 | 31 | fn as_table_name<'a, T: TableProvider + 'static>( 32 | plan: &'a LogicalPlan, 33 | ) -> Option<&'a str> { 34 | if let LogicalPlan::TableScan { 35 | source, table_name, .. 36 | } = plan 37 | { 38 | source 39 | .as_any() 40 | .downcast_ref::() 41 | .map(|_| table_name.as_ref()) 42 | } else { 43 | None 44 | } 45 | } 46 | 47 | #[cfg(test)] 48 | mod tests { 49 | use std::sync::Arc; 50 | 51 | use super::*; 52 | use crate::datasource::CatalogTable; 53 | use arrow::datatypes::Schema; 54 | use datafusion::datasource::empty::EmptyTable; 55 | use datafusion::execution::context::ExecutionContext; 56 | use datafusion::logical_plan::{sum, Expr}; 57 | use datafusion::scalar::ScalarValue; 58 | 59 | #[test] 60 | fn search_table_df_plan() -> Result<()> { 61 | let mut ctx = ExecutionContext::new(); 62 | let empty_table = Arc::new(EmptyTable::new(Arc::new(Schema::empty()))); 63 | let scalar_expr = Expr::Literal(ScalarValue::from(10)); 64 | 65 | let source_df = ctx.read_table(empty_table.clone())?; 66 | let log_plan = &source_df.to_logical_plan(); 67 | find_table_name::(log_plan)?; 68 | 69 | let filtered_df = 70 | source_df.filter(scalar_expr.clone().eq(scalar_expr.clone()))?; 71 | let log_plan = &filtered_df.to_logical_plan(); 72 | find_table_name::(log_plan)?; 73 | 74 | let grouped_df = filtered_df 75 | .aggregate(vec![scalar_expr.clone()], vec![sum(scalar_expr.clone())])?; 76 | let log_plan = &grouped_df.to_logical_plan(); 77 | find_table_name::(log_plan)?; 78 | 79 | // search but not found 80 | find_table_name::(&grouped_df.to_logical_plan()) 81 | .expect_err("Catalog table should not have been found"); 82 | 83 | Ok(()) 84 | } 85 | 86 | #[test] 87 | fn search_table_sql_plan() -> Result<()> { 88 | let mut ctx = ExecutionContext::new(); 89 | let empty_table = Arc::new(EmptyTable::new(Arc::new(Schema::empty()))); 90 | ctx.register_table("test_tbl", empty_table)?; 91 | let df = ctx.sql("SELECT * FROM test_tbl")?; 92 | let log_plan = df.to_logical_plan(); 93 | let found_name = find_table_name::(&log_plan).unwrap(); 94 | assert_eq!(found_name, "test_tbl"); 95 | 96 | Ok(()) 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /docker/Dockerfile: -------------------------------------------------------------------------------- 1 | # syntax=docker/dockerfile:experimental 2 | 3 | # This docker file contains different build targets of the application 4 | # Each target is represented by a stage 5 | 6 | ## BUILD STAGE ## 7 | # Create a static binary (with musl) for one of the binary targets. 8 | 9 | FROM rust:1.53.0-buster as build-stage 10 | ARG BIN_NAME 11 | 12 | # install environment 13 | 14 | RUN apt-get update 15 | RUN apt-get install musl-tools zip -y 16 | 17 | WORKDIR /buildspace 18 | 19 | RUN rustup target add x86_64-unknown-linux-musl && rustup component add rustfmt 20 | 21 | COPY ./code . 22 | 23 | # use BuildKit experimental cache mount to speed up builds 24 | RUN --mount=type=cache,target=./target \ 25 | --mount=type=cache,target=/usr/local/cargo/git \ 26 | --mount=type=cache,target=/usr/local/cargo/registry \ 27 | cargo build --bin ${BIN_NAME} --release --target=x86_64-unknown-linux-musl && \ 28 | cp ./target/x86_64-unknown-linux-musl/release/${BIN_NAME} ./exec-static 29 | 30 | ## LAMBDA PACKAGE STAGE ## 31 | # Create a zip archive to be deployed to AWS Lambda 32 | 33 | FROM build-stage as package-stage 34 | ARG BIN_NAME 35 | # the exec name inside a lambda archive should be `bootstrap` 36 | RUN cp ./exec-static ./bootstrap 37 | RUN zip ${BIN_NAME}.zip bootstrap 38 | 39 | ## RUNTIME STAGE ## 40 | # A runtime container 41 | 42 | FROM scratch as runtime-stage 43 | ARG PORT 44 | EXPOSE ${PORT} 45 | COPY --from=build-stage /buildspace/exec-static /app 46 | COPY --from=build-stage /etc/ssl/certs/ca-certificates.crt /etc/ssl/certs/ 47 | 48 | ENTRYPOINT ["./app"] 49 | 50 | ## EXPORT STAGE ## 51 | # Isolate the archive so it can be exported with `docker build -o` 52 | 53 | FROM scratch as export-stage 54 | ARG BIN_NAME 55 | COPY --from=package-stage /buildspace/${BIN_NAME}.zip / -------------------------------------------------------------------------------- /docker/docker-compose.yml: -------------------------------------------------------------------------------- 1 | version: '2.4' 2 | 3 | services: 4 | fuse: 5 | build: 6 | context: ../ 7 | dockerfile: ./docker/Dockerfile 8 | target: runtime-stage 9 | args: 10 | BIN_NAME: fuse_local 11 | environment: 12 | - RUST_BACKTRACE=1 13 | depends_on: 14 | - hcomb 15 | - hbee 16 | 17 | hcomb: 18 | build: 19 | context: ../ 20 | dockerfile: ./docker/Dockerfile 21 | target: runtime-stage 22 | args: 23 | BIN_NAME: hcomb 24 | PORT: 3333 25 | environment: 26 | - RUST_BACKTRACE=1 27 | mem_limit: 4096M 28 | cpus: 4 29 | ports: 30 | - 3333 31 | # scale: 1 32 | 33 | hbee: 34 | build: 35 | context: ../ 36 | dockerfile: ./docker/Dockerfile 37 | target: runtime-stage 38 | args: 39 | BIN_NAME: hbee_local 40 | PORT: 3000 41 | volumes: 42 | - ~/.aws/credentials:/.aws/credentials 43 | environment: 44 | - AWS_SHARED_CREDENTIALS_FILE=/.aws/credentials 45 | - AWS_PROFILE 46 | - RUST_BACKTRACE=1 47 | mem_limit: 2048M 48 | cpus: 2 49 | ports: 50 | - 3000 51 | # scale: 1 52 | 53 | -------------------------------------------------------------------------------- /infra/.terraform.lock.hcl: -------------------------------------------------------------------------------- 1 | # This file is maintained automatically by "terraform init". 2 | # Manual edits may be lost in future updates. 3 | 4 | provider "registry.terraform.io/hashicorp/aws" { 5 | version = "3.23.0" 6 | constraints = "~> 3.0, >= 3.10.0" 7 | hashes = [ 8 | "h1:GugGr7igctZkUUt0im9b0CbdinTRxb4dNXvmGuN2gZ8=", 9 | "zh:30b0733027c00472618da998bc77967c692e238ae117c07e046fdd7336b83fa3", 10 | "zh:3677550a8bef8e01c67cb615407dc8a69d32f4e36017033cd6f71a831c99d5de", 11 | "zh:3c2fb4c14bfd43cf20ee25d0068ce09f1d48758408b8f1c88a096cea243612b3", 12 | "zh:5577543322003693c4fe24a69ed0d47e58f867426fd704fac94cf5c16d3d6153", 13 | "zh:6771f09d76ad01ffc04baa3bce7a3eed09f6a8a949274ffbd9d11756a58a4329", 14 | "zh:7a57b79d304d17cf52ee3ddce91679f6b4289c5bdda2e31b763bf7d512e542d9", 15 | "zh:815fb027e17bfe754b05367d20bd0694726a95a99b81e8d939ddd44e2b1f05a9", 16 | "zh:a3d67db5ec0f4e9750eb19676a9a1aff36b0721e276a4ba789f42b991bf5951c", 17 | "zh:cd67ff33860ad578172c19412ce608ba818e7590083197df2b793f870d6f50a3", 18 | "zh:fbe0835055d1260fb77ad19a32a8726248ba7ac187f6c463ded90737b4cea8e6", 19 | ] 20 | } 21 | 22 | provider "registry.terraform.io/hashicorp/http" { 23 | version = "2.0.0" 24 | constraints = "~> 2.0" 25 | hashes = [ 26 | "h1:PZBhWKq5UZDP0sv8lbR7D29dYcpnPrQXiXzunA/+zI4=", 27 | "zh:066b5f4f80013dcc7d94d7d4b5537607c74f93e8a414ac993709d138a36d7db6", 28 | "zh:41c1916d44c263828d39eeb12ab0bea8507e4e41af6f3376cb3f2b93158e35a6", 29 | "zh:78a429752ae9c3587a4ccf106ea17a89918732511f45f99ec905d014f88a3e88", 30 | "zh:8be68e5e4f095a090534594d1e0e08a8ff76638fc94cb38e5e38a683b8cb62ec", 31 | "zh:b00a3d86b86ac07347cd3d89f8bbb966686d897a7c37a18a69b715e8e01a4728", 32 | "zh:b9aab522396439716fb2e2cacf97c4083835fd0f8ea15b95d0a754578253c66c", 33 | "zh:c475bfacb7da999ff5a652680e7a7969b63c1e994462936a1ac7b98708e4ea1f", 34 | "zh:d85f228d5fb4d4197972939d27a00cc6f48309a3d562de44eade347c70f02f83", 35 | "zh:d91ece204cd2aec195dc7e0d1a620ceb19dcc2a64072af134fc9e49f233abb3e", 36 | "zh:faa5c9801b6686872908fb55d5feb4952168dd53e0dadec0962768e24b93116a", 37 | ] 38 | } 39 | 40 | provider "registry.terraform.io/hashicorp/null" { 41 | version = "3.0.0" 42 | constraints = "~> 3.0" 43 | hashes = [ 44 | "h1:ysHGBhBNkIiJLEpthB/IVCLpA1Qoncp3KbCTFGFZTO0=", 45 | "zh:05fb7eab469324c97e9b73a61d2ece6f91de4e9b493e573bfeda0f2077bc3a4c", 46 | "zh:1688aa91885a395c4ae67636d411475d0b831e422e005dcf02eedacaafac3bb4", 47 | "zh:24a0b1292e3a474f57c483a7a4512d797e041bc9c2fbaac42fe12e86a7fb5a3c", 48 | "zh:2fc951bd0d1b9b23427acc93be09b6909d72871e464088171da60fbee4fdde03", 49 | "zh:6db825759425599a326385a68acc6be2d9ba0d7d6ef587191d0cdc6daef9ac63", 50 | "zh:85985763d02618993c32c294072cc6ec51f1692b803cb506fcfedca9d40eaec9", 51 | "zh:a53186599c57058be1509f904da512342cfdc5d808efdaf02dec15f0f3cb039a", 52 | "zh:c2e07b49b6efa676bdc7b00c06333ea1792a983a5720f9e2233db27323d2707c", 53 | "zh:cdc8fe1096103cf5374751e2e8408ec4abd2eb67d5a1c5151fe2c7ecfd525bef", 54 | "zh:dbdef21df0c012b0d08776f3d4f34eb0f2f229adfde07ff252a119e52c0f65b7", 55 | ] 56 | } 57 | -------------------------------------------------------------------------------- /infra/common.tf: -------------------------------------------------------------------------------- 1 | resource "aws_security_group" "service_endpoint" { 2 | name = "${module.env.module_name}_service_endpoint_${module.env.stage}" 3 | description = "allow inbound access to endpoint from local network" 4 | vpc_id = module.vpc.vpc_id 5 | 6 | ingress { 7 | protocol = "tcp" 8 | from_port = 80 9 | to_port = 80 10 | cidr_blocks = [module.env.vpc_cidr] 11 | } 12 | 13 | ingress { 14 | protocol = "tcp" 15 | from_port = 443 16 | to_port = 443 17 | cidr_blocks = [module.env.vpc_cidr] 18 | } 19 | 20 | tags = module.env.tags 21 | } 22 | 23 | data "aws_availability_zones" "available" {} 24 | 25 | module "vpc" { 26 | source = "terraform-aws-modules/vpc/aws" 27 | version = "2.70.0" 28 | 29 | name = "${module.env.module_name}-vpc-${module.env.stage}" 30 | cidr = module.env.vpc_cidr 31 | azs = slice(data.aws_availability_zones.available.names, 0, length(module.env.subnet_cidrs)) 32 | public_subnets = module.env.subnet_cidrs 33 | 34 | enable_nat_gateway = false 35 | enable_vpn_gateway = false 36 | enable_s3_endpoint = true 37 | 38 | ## enable access to the ECS and Lambda APIs. This costs ~ $8 / month / endpoint 39 | # dns settings allow to access services transparently 40 | enable_dns_support = true 41 | enable_dns_hostnames = true 42 | # activate ecs access 43 | enable_ecs_endpoint = true 44 | ecs_endpoint_subnet_ids = [module.vpc.public_subnets[0]] 45 | ecs_endpoint_security_group_ids = [aws_security_group.service_endpoint.id] 46 | ecs_endpoint_private_dns_enabled = true 47 | # activate lambda access 48 | enable_lambda_endpoint = true 49 | lambda_endpoint_subnet_ids = [module.vpc.public_subnets[0]] 50 | lambda_endpoint_security_group_ids = [aws_security_group.service_endpoint.id] 51 | lambda_endpoint_private_dns_enabled = true 52 | 53 | 54 | tags = module.env.tags 55 | } 56 | 57 | # TODO move all below this into ./fargate ? 58 | 59 | resource "aws_ecs_cluster" "hcomb_cluster" { 60 | name = "${module.env.module_name}-cluster-${module.env.stage}" 61 | capacity_providers = ["FARGATE"] 62 | default_capacity_provider_strategy { 63 | capacity_provider = "FARGATE" 64 | } 65 | setting { 66 | name = "containerInsights" 67 | value = "disabled" 68 | } 69 | tags = module.env.tags 70 | } 71 | 72 | resource "aws_iam_role" "ecs_task_execution_role" { 73 | name = "${module.env.module_name}_task_execution_${module.env.stage}_${var.region_name}" 74 | 75 | assume_role_policy = <