Compare commits

..

20 Commits

Author SHA1 Message Date
Yingwen
28124abbb7 feat: update dashboard to v0.11.9 (#7364) (#7371)
Signed-off-by: evenyag <realevenyag@gmail.com>
Co-authored-by: ZonaHe <zonahe@qq.com>
Co-authored-by: sunchanglong <sunchanglong@users.noreply.github.com>
2025-12-09 17:34:42 +08:00
Weny Xu
40bd6ef79e chore: pick #7199 and #7266 to release/v0.15 (#7267)
* fix: correct leader state reset and region migration locking consistency (#7199)

* fix(meta): remove table route cache in region migration ctx

Signed-off-by: WenyXu <wenymedia@gmail.com>

* fix: fix unit tests

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: fix clippy

Signed-off-by: WenyXu <wenymedia@gmail.com>

* fix: fix campaign reset not clearing leader state-s

Signed-off-by: WenyXu <wenymedia@gmail.com>

* feat: gracefully handle region lease renewal errors

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: add tests for election reset and region lease failure handling (#7266)

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-11-21 11:25:44 +08:00
Yingwen
50eaa3c80a chore: cherry pick #7157, #7229, #7239 to 0.15 branch (#7256)
* fix: cache estimate methods (#7157)

* fix: cache estimate methods

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* revert page value change

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* Apply suggestion from @evenyag

Co-authored-by: Yingwen <realevenyag@gmail.com>

* update test

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Co-authored-by: Yingwen <realevenyag@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>

* fix: clone the page before putting into the index cache (#7229)

* fix: clone the page before putting into the index cache

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: fix warnings

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: evenyag <realevenyag@gmail.com>

* fix: allow compacting L1 files under append mode (#7239)

* fix: allow compacting L1 files under append mode

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: limit the number of compaction input files

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
Co-authored-by: Ruihang Xia <waynestxia@gmail.com>
2025-11-19 14:41:35 +08:00
Yingwen
b08bdcb465 fix(mito): avoid shortcut in picking multi window files (#7174) (#7224)
* fix(mito): avoid shortcut in picking multi window files (#7174)

* fix/pick-continue:
 ### Add Tests for TWCS Compaction Logic

 - **`twcs.rs`**:
   - Modified the logic in `TwcsPicker` to handle cases with zero runs by using `continue` instead of `return`.
   - Added two new test cases: `test_build_output_multiple_windows_with_zero_runs` and `test_build_output_single_window_zero_runs` to verify the behavior of the compaction logic when there are zero runs in
 the windows.

 - **`memtable_util.rs`**:
   - Removed unused import `PredicateGroup`.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix: clippy

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix/pick-continue:

* refactor/progressive-compaction:
 **Enhance Compaction Task Error Handling**

 - Updated `task.rs` to conditionally execute the removal of expired SST files only when they exist, improving error handling and performance.
 - Added a check for non-empty `expired_ssts` before initiating the removal process, ensuring unnecessary operations are avoided.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* refactor/progressive-compaction:
 ### Add Max Background Compaction Tasks Configuration

 - **`compaction.rs`**: Added `max_background_compactions` to the compaction scheduler to limit background tasks.
 - **`compaction/compactor.rs`**: Removed immediate manifest update logic after task completion.
 - **`compaction/picker.rs`**: Introduced `max_background_tasks` parameter in `new_picker` to control task limits.
 - **`compaction/twcs.rs`**: Updated `TwcsPicker` to include `max_background_tasks` and truncate inputs exceeding this limit. Added related test cases to ensure functionality.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix/pick-continue:
 ### Add Unit Tests for Compaction Task and TWCS Picker

 - **`twcs.rs`**: Introduced tests for `TwcsPicker` to ensure correct handling of `max_background_tasks` during compaction, including scenarios with and without task truncation.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

---------

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: fix typos

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: update bitnami config (#6847)

* chore: update bitnami config

Signed-off-by: liyang <daviderli614@gmail.com>

* update postgresql chart version

Signed-off-by: liyang <daviderli614@gmail.com>

* fix ci

Signed-off-by: liyang <daviderli614@gmail.com>

* refactor: add pull-test-deps-images.sh to pull images one by one to avoid rate limit

Signed-off-by: zyy17 <zyylsxm@gmail.com>

---------

Signed-off-by: liyang <daviderli614@gmail.com>
Signed-off-by: zyy17 <zyylsxm@gmail.com>
Co-authored-by: zyy17 <zyylsxm@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: use greptime dockerhub image (#6865)

Signed-off-by: liyang <daviderli614@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>

* ci: remove etcd-tls in fixtures

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
Signed-off-by: liyang <daviderli614@gmail.com>
Signed-off-by: zyy17 <zyylsxm@gmail.com>
Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com>
Co-authored-by: liyang <daviderli614@gmail.com>
Co-authored-by: zyy17 <zyylsxm@gmail.com>
2025-11-14 15:06:51 +08:00
Yingwen
1048339b06 chore: update dev-builder for 0.15 (#7124)
chore: update dev-builder to 2025-05-19-f55023f3-20250829091211

The old image was gone

Signed-off-by: evenyag <realevenyag@gmail.com>
2025-10-21 20:02:06 +08:00
Lei, HUANG
02af2cb3cd fix: limit compaction input num 0.15 (#7117)
* fix/limit-compaction-input-num-0.15:
 **Add File Count Limit and Debug Logging in Compaction Process**

 - **`run.rs`**: Introduced a new method `num_files` in `FileGroup` to count files. This aids in managing file limits during compaction.
 - **`twcs.rs`**: Implemented an environment variable `TWCS_MAX_INPUT_FILE_NUM` to limit the number of input files during compaction. Added debug logging to track the maximum compaction file number and info logging to enforce the file limit. Enhanced logic to skip large files in append mode and adjusted the
 compaction process to respect the new file count limit.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix/limit-compaction-input-num-0.15:
 **Enhancements in `twcs.rs`**

 - Introduced a default value for `max_input_file_num` with `DEFAULT_MAX_INPUT_FILE_NUM` constant set to 32.
 - Added error handling for environment variable `TWCS_MAX_INPUT_FILE_NUM` using `warn` to log unrecognized values.
 - Improved logging in `TwcsPicker` to include the current total input files when enforcing the max input file number limit.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

* fix: typo

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>

---------

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
2025-10-20 20:28:14 +08:00
Yingwen
53fc32b0da chore: cherry pick #6461 to v0.15 (#6936)
refactor: stores the http server builder in Metasrv instance (#6461)

* refactor: stores the http server builder in Metasrv instance



* resolve PR comments



* fix ci



---------

Signed-off-by: luofucong <luofc@foxmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
Co-authored-by: LFC <990479+MichaelScofield@users.noreply.github.com>
2025-09-09 15:39:18 +08:00
Lei, HUANG
8380ae13c7 chore: refine metrics tracking the flush/compaction cost time (#6630)
chore: refine metrics tracking the per-stage cost time during flush and compaction

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-08 16:39:00 +08:00
Weny Xu
e2393d27b2 feat: add written_bytes_since_open column to region_statistics table (#6904)
* feat: add `write_bytes` column to `region_statistics` table

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: update comments

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: rename `write_bytes` to `written_bytes`

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: rename `written_bytes` to `written_bytes_since_open`

Signed-off-by: WenyXu <wenymedia@gmail.com>

* chore: apply suggestions

Signed-off-by: WenyXu <wenymedia@gmail.com>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
2025-09-08 11:33:07 +08:00
Weny Xu
e6831704d8 chore: fix typo (#6887)
Signed-off-by: WenyXu <wenymedia@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Ruihang Xia
f1043bb4cc chore: fix typo (#6885)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Ruihang Xia
8f997e731a feat: skip compaction on large file on append only mode (#6838)
* feat: skip compaction on large file on append only mode

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* log ignored files

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* format

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* only ignore level 1 files

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* early exit

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* fix typo

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Zhenchi
240061771d fix: move prune_region_dir to region drop (#6891)
* fix: move prune_region_dir to region drop

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* address comments

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Zhenchi
0b7b47fdef fix: prune intermediate dirs on index finish and region pruge (#6878)
* fix: prune intermediate dirs on index finish and region pruge

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* address comments

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Yingwen
02d9245516 fix: use actual buf size as cache page value size (#6829)
* feat: cache the cloned page bytes

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: cache the whole row group pages

The opendal reader may merge IO requests so the pages of different
columns can share the same Bytes.
When we use a per-column page cache, the page cache may still referencing
the whole Bytes after eviction if there are other columns in the cache that
share the same Bytes.

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: check possible max byte range and copy pages if needed

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: always copy pages

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: returns the copied pages

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: compute cache size by MERGE_GAP

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: align to buf size

Signed-off-by: evenyag <realevenyag@gmail.com>

* feat: aligh to 2MB

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: remove unused code

Signed-off-by: evenyag <realevenyag@gmail.com>

* style: fix clippy

Signed-off-by: evenyag <realevenyag@gmail.com>

* chore: fix typo

Signed-off-by: evenyag <realevenyag@gmail.com>

* test: fix parquet read with cache test

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Ruihang Xia
beb3447938 perf: improve bloom filter reader's byte reading logic (#6658)
* perf: improve bloom filter reader's byte reading logic

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* revert toml change

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* clearify comment

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* benchmark

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* update lock file

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* pub util fn

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* note endian

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Ruihang Xia
a5d58b525d feat: count underscore in English tokenizer and improve performance (#6660)
* feat: count underscore in English tokenizer and improve performance

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* update lock file

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* update test results

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* assert lookup table

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* handle utf8 alphanumeric

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

* finalize

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>

---------

Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Zhenchi
772bc21b65 feat: MatchesConstTerm displays probes (#6518)
* feat: `MatchesConstTerm` displays probes

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* fix fmt

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Yingwen
a7c22c253c perf: Reduce fulltext bloom load time (#6651)
* perf: cached reader do not get page concurrently

Otherwise they will all fetch the same pages in parallel

Signed-off-by: evenyag <realevenyag@gmail.com>

* perf: always disable zstd for bloom

Signed-off-by: evenyag <realevenyag@gmail.com>

---------

Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
Weny Xu
bfbd7f608a fix: correct heartbeat stream handling logic (#6821)
* fix: correct heartbeat stream handling logic

Signed-off-by: WenyXu <wenymedia@gmail.com>

* Update src/meta-srv/src/service/heartbeat.rs

Co-authored-by: jeremyhi <jiachun_feng@proton.me>

---------

Signed-off-by: WenyXu <wenymedia@gmail.com>
Co-authored-by: jeremyhi <jiachun_feng@proton.me>
Signed-off-by: evenyag <realevenyag@gmail.com>
2025-09-05 20:09:44 +08:00
101 changed files with 1908 additions and 888 deletions

View File

@@ -12,7 +12,7 @@ runs:
steps:
- name: Install Etcd cluster
shell: bash
run: |
run: |
helm upgrade \
--install etcd oci://registry-1.docker.io/bitnamicharts/etcd \
--set replicaCount=${{ inputs.etcd-replicas }} \
@@ -24,4 +24,9 @@ runs:
--set auth.rbac.token.enabled=false \
--set persistence.size=2Gi \
--create-namespace \
--set global.security.allowInsecureImages=true \
--set image.registry=docker.io \
--set image.repository=greptime/etcd \
--set image.tag=3.6.1-debian-12-r3 \
--version 12.0.8 \
-n ${{ inputs.namespace }}

View File

@@ -12,7 +12,7 @@ runs:
steps:
- name: Install Kafka cluster
shell: bash
run: |
run: |
helm upgrade \
--install kafka oci://registry-1.docker.io/bitnamicharts/kafka \
--set controller.replicaCount=${{ inputs.controller-replicas }} \
@@ -23,4 +23,8 @@ runs:
--set listeners.controller.protocol=PLAINTEXT \
--set listeners.client.protocol=PLAINTEXT \
--create-namespace \
--set image.registry=docker.io \
--set image.repository=greptime/kafka \
--set image.tag=3.9.0-debian-12-r1 \
--version 31.0.0 \
-n ${{ inputs.namespace }}

View File

@@ -6,9 +6,7 @@ inputs:
description: "Number of PostgreSQL replicas"
namespace:
default: "postgres-namespace"
postgres-version:
default: "14.2"
description: "PostgreSQL version"
description: "The PostgreSQL namespace"
storage-size:
default: "1Gi"
description: "Storage size for PostgreSQL"
@@ -22,7 +20,11 @@ runs:
helm upgrade \
--install postgresql oci://registry-1.docker.io/bitnamicharts/postgresql \
--set replicaCount=${{ inputs.postgres-replicas }} \
--set image.tag=${{ inputs.postgres-version }} \
--set global.security.allowInsecureImages=true \
--set image.registry=docker.io \
--set image.repository=greptime/postgresql \
--set image.tag=17.5.0-debian-12-r3 \
--version 16.7.4 \
--set persistence.size=${{ inputs.storage-size }} \
--set postgresql.username=greptimedb \
--set postgresql.password=admin \

34
.github/scripts/pull-test-deps-images.sh vendored Executable file
View File

@@ -0,0 +1,34 @@
#!/bin/bash
# This script is used to pull the test dependency images that are stored in public ECR one by one to avoid rate limiting.
set -e
MAX_RETRIES=3
IMAGES=(
"greptime/zookeeper:3.7"
"greptime/kafka:3.9.0-debian-12-r1"
"greptime/etcd:3.6.1-debian-12-r3"
"greptime/minio:2024"
"greptime/mysql:5.7"
)
for image in "${IMAGES[@]}"; do
for ((attempt=1; attempt<=MAX_RETRIES; attempt++)); do
if docker pull "$image"; then
# Successfully pulled the image.
break
else
# Use some simple exponential backoff to avoid rate limiting.
if [ $attempt -lt $MAX_RETRIES ]; then
sleep_seconds=$((attempt * 5))
echo "Attempt $attempt failed for $image, waiting $sleep_seconds seconds"
sleep $sleep_seconds # 5s, 10s delays
else
echo "Failed to pull $image after $MAX_RETRIES attempts"
exit 1
fi
fi
done
done

View File

@@ -719,6 +719,10 @@ jobs:
save-if: ${{ github.ref == 'refs/heads/main' }}
- name: Install latest nextest release
uses: taiki-e/install-action@nextest
- name: Pull test dependencies images
run: ./.github/scripts/pull-test-deps-images.sh
- name: Setup external services
working-directory: tests-integration/fixtures
run: docker compose up -d --wait

17
Cargo.lock generated
View File

@@ -1988,7 +1988,6 @@ dependencies = [
"common-version",
"common-wal",
"datatypes",
"either",
"etcd-client",
"futures",
"humantime",
@@ -2116,7 +2115,6 @@ dependencies = [
"common-wal",
"datanode",
"datatypes",
"either",
"etcd-client",
"file-engine",
"flow",
@@ -4216,9 +4214,9 @@ dependencies = [
[[package]]
name = "either"
version = "1.13.0"
version = "1.15.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0"
checksum = "48c757948c5ede0e46177b7add2e67155f70e33c07fea8284df6576da70b3719"
dependencies = [
"serde",
]
@@ -5946,6 +5944,7 @@ dependencies = [
"common-runtime",
"common-telemetry",
"common-test-util",
"criterion 0.4.0",
"fastbloom",
"fst",
"futures",
@@ -5958,6 +5957,7 @@ dependencies = [
"prost 0.13.5",
"puffin",
"rand 0.9.0",
"rand_chacha 0.9.0",
"regex",
"regex-automata 0.4.8",
"roaring",
@@ -6714,7 +6714,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "4979f22fdb869068da03c9f7528f8297c6fd2606bc3a4affe42e6a823fdb8da4"
dependencies = [
"cfg-if",
"windows-targets 0.48.5",
"windows-targets 0.52.6",
]
[[package]]
@@ -7190,6 +7190,7 @@ dependencies = [
"deadpool",
"deadpool-postgres",
"derive_builder 0.20.1",
"either",
"etcd-client",
"futures",
"h2 0.3.26",
@@ -9594,7 +9595,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "be769465445e8c1474e9c5dac2018218498557af32d9ed057325ec9a41ae81bf"
dependencies = [
"heck 0.5.0",
"itertools 0.11.0",
"itertools 0.14.0",
"log",
"multimap",
"once_cell",
@@ -9640,7 +9641,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8a56d757972c98b346a9b766e3f02746cde6dd1cd1d1d563472929fdd74bec4d"
dependencies = [
"anyhow",
"itertools 0.11.0",
"itertools 0.14.0",
"proc-macro2",
"quote",
"syn 2.0.100",
@@ -14211,7 +14212,7 @@ version = "0.1.9"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb"
dependencies = [
"windows-sys 0.48.0",
"windows-sys 0.59.0",
]
[[package]]

View File

@@ -130,6 +130,7 @@ deadpool = "0.12"
deadpool-postgres = "0.14"
derive_builder = "0.20"
dotenv = "0.15"
either = "1.15"
etcd-client = "0.14"
fst = "0.4.7"
futures = "0.3"

View File

@@ -8,7 +8,7 @@ CARGO_BUILD_OPTS := --locked
IMAGE_REGISTRY ?= docker.io
IMAGE_NAMESPACE ?= greptime
IMAGE_TAG ?= latest
DEV_BUILDER_IMAGE_TAG ?= 2025-05-19-b2377d4b-20250520045554
DEV_BUILDER_IMAGE_TAG ?= 2025-05-19-f55023f3-20250829091211
BUILDX_MULTI_PLATFORM_BUILD ?= false
BUILDX_BUILDER_NAME ?= gtbuilder
BASE_IMAGE ?= ubuntu

View File

@@ -40,6 +40,7 @@ const REGION_ID: &str = "region_id";
const TABLE_ID: &str = "table_id";
const REGION_NUMBER: &str = "region_number";
const REGION_ROWS: &str = "region_rows";
const WRITTEN_BYTES: &str = "written_bytes_since_open";
const DISK_SIZE: &str = "disk_size";
const MEMTABLE_SIZE: &str = "memtable_size";
const MANIFEST_SIZE: &str = "manifest_size";
@@ -56,6 +57,7 @@ const INIT_CAPACITY: usize = 42;
/// - `table_id`: The table id.
/// - `region_number`: The region number.
/// - `region_rows`: The number of rows in region.
/// - `written_bytes_since_open`: The total bytes written of the region since region opened.
/// - `memtable_size`: The memtable size in bytes.
/// - `disk_size`: The approximate disk size in bytes.
/// - `manifest_size`: The manifest size in bytes.
@@ -83,6 +85,7 @@ impl InformationSchemaRegionStatistics {
ColumnSchema::new(TABLE_ID, ConcreteDataType::uint32_datatype(), false),
ColumnSchema::new(REGION_NUMBER, ConcreteDataType::uint32_datatype(), false),
ColumnSchema::new(REGION_ROWS, ConcreteDataType::uint64_datatype(), true),
ColumnSchema::new(WRITTEN_BYTES, ConcreteDataType::uint64_datatype(), true),
ColumnSchema::new(DISK_SIZE, ConcreteDataType::uint64_datatype(), true),
ColumnSchema::new(MEMTABLE_SIZE, ConcreteDataType::uint64_datatype(), true),
ColumnSchema::new(MANIFEST_SIZE, ConcreteDataType::uint64_datatype(), true),
@@ -145,6 +148,7 @@ struct InformationSchemaRegionStatisticsBuilder {
table_ids: UInt32VectorBuilder,
region_numbers: UInt32VectorBuilder,
region_rows: UInt64VectorBuilder,
written_bytes: UInt64VectorBuilder,
disk_sizes: UInt64VectorBuilder,
memtable_sizes: UInt64VectorBuilder,
manifest_sizes: UInt64VectorBuilder,
@@ -163,6 +167,7 @@ impl InformationSchemaRegionStatisticsBuilder {
table_ids: UInt32VectorBuilder::with_capacity(INIT_CAPACITY),
region_numbers: UInt32VectorBuilder::with_capacity(INIT_CAPACITY),
region_rows: UInt64VectorBuilder::with_capacity(INIT_CAPACITY),
written_bytes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY),
disk_sizes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY),
memtable_sizes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY),
manifest_sizes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY),
@@ -193,6 +198,7 @@ impl InformationSchemaRegionStatisticsBuilder {
(TABLE_ID, &Value::from(region_stat.id.table_id())),
(REGION_NUMBER, &Value::from(region_stat.id.region_number())),
(REGION_ROWS, &Value::from(region_stat.num_rows)),
(WRITTEN_BYTES, &Value::from(region_stat.written_bytes)),
(DISK_SIZE, &Value::from(region_stat.approximate_bytes)),
(MEMTABLE_SIZE, &Value::from(region_stat.memtable_size)),
(MANIFEST_SIZE, &Value::from(region_stat.manifest_size)),
@@ -211,6 +217,7 @@ impl InformationSchemaRegionStatisticsBuilder {
self.region_numbers
.push(Some(region_stat.id.region_number()));
self.region_rows.push(Some(region_stat.num_rows));
self.written_bytes.push(Some(region_stat.written_bytes));
self.disk_sizes.push(Some(region_stat.approximate_bytes));
self.memtable_sizes.push(Some(region_stat.memtable_size));
self.manifest_sizes.push(Some(region_stat.manifest_size));
@@ -226,6 +233,7 @@ impl InformationSchemaRegionStatisticsBuilder {
Arc::new(self.table_ids.finish()),
Arc::new(self.region_numbers.finish()),
Arc::new(self.region_rows.finish()),
Arc::new(self.written_bytes.finish()),
Arc::new(self.disk_sizes.finish()),
Arc::new(self.memtable_sizes.finish()),
Arc::new(self.manifest_sizes.finish()),

View File

@@ -43,7 +43,6 @@ common-time.workspace = true
common-version.workspace = true
common-wal.workspace = true
datatypes.workspace = true
either = "1.8"
etcd-client.workspace = true
futures.workspace = true
humantime.workspace = true

View File

@@ -29,7 +29,7 @@ use futures::TryStreamExt;
use crate::error::InvalidArgumentsSnafu;
use crate::metadata::common::StoreConfig;
use crate::metadata::control::utils::{decode_key_value, get_table_id_by_name, json_fromatter};
use crate::metadata::control::utils::{decode_key_value, get_table_id_by_name, json_formatter};
use crate::Tool;
/// Getting metadata from metadata store.
@@ -206,7 +206,7 @@ impl Tool for GetTableTool {
println!(
"{}\n{}",
TableInfoKey::new(table_id),
json_fromatter(self.pretty, &*table_info)
json_formatter(self.pretty, &*table_info)
);
} else {
println!("Table info not found");
@@ -221,7 +221,7 @@ impl Tool for GetTableTool {
println!(
"{}\n{}",
TableRouteKey::new(table_id),
json_fromatter(self.pretty, &table_route)
json_formatter(self.pretty, &table_route)
);
} else {
println!("Table route not found");

View File

@@ -27,7 +27,7 @@ pub fn decode_key_value(kv: KeyValue) -> CommonMetaResult<(String, String)> {
}
/// Formats a value as a JSON string.
pub fn json_fromatter<T>(pretty: bool, value: &T) -> String
pub fn json_formatter<T>(pretty: bool, value: &T) -> String
where
T: Serialize,
{

View File

@@ -52,7 +52,6 @@ common-version.workspace = true
common-wal.workspace = true
datanode.workspace = true
datatypes.workspace = true
either = "1.8"
etcd-client.workspace = true
file-engine.workspace = true
flow.workspace = true

View File

@@ -54,6 +54,10 @@ impl Instance {
pub fn get_inner(&self) -> &MetasrvInstance {
&self.instance
}
pub fn mut_inner(&mut self) -> &mut MetasrvInstance {
&mut self.instance
}
}
#[async_trait]

View File

@@ -792,6 +792,7 @@ impl InformationExtension for StandaloneInformationExtension {
region_manifest: region_stat.manifest.into(),
data_topic_latest_entry_id: region_stat.data_topic_latest_entry_id,
metadata_topic_latest_entry_id: region_stat.metadata_topic_latest_entry_id,
written_bytes: region_stat.written_bytes,
}
})
.collect::<Vec<_>>();

View File

@@ -97,6 +97,8 @@ pub struct RegionStat {
pub index_size: u64,
/// The manifest infoof the region.
pub region_manifest: RegionManifestInfo,
/// The total bytes written of the region since region opened.
pub written_bytes: u64,
/// The latest entry id of topic used by data.
/// **Only used by remote WAL prune.**
pub data_topic_latest_entry_id: u64,
@@ -277,6 +279,7 @@ impl From<&api::v1::meta::RegionStat> for RegionStat {
sst_size: region_stat.sst_size,
index_size: region_stat.index_size,
region_manifest: region_stat.manifest.into(),
written_bytes: region_stat.written_bytes,
data_topic_latest_entry_id: region_stat.data_topic_latest_entry_id,
metadata_topic_latest_entry_id: region_stat.metadata_topic_latest_entry_id,
}

View File

@@ -19,7 +19,7 @@ use opentelemetry::propagation::TextMapPropagator;
use opentelemetry_sdk::propagation::TraceContextPropagator;
use tracing_opentelemetry::OpenTelemetrySpanExt;
// An wapper for `Futures` that provides tracing instrument adapters.
// A wrapper for `Futures` that provides tracing instrument adapters.
pub trait FutureExt: std::future::Future + Sized {
fn trace(self, span: tracing::span::Span) -> tracing::instrument::Instrumented<Self>;
}

View File

@@ -218,6 +218,7 @@ impl HeartbeatTask {
if let Some(message) = message {
Self::new_heartbeat_request(&heartbeat_request, Some(message), &latest_report)
} else {
warn!("Sender has been dropped, exiting the heartbeat loop");
// Receives None that means Sender was dropped, we need to break the current loop
break
}
@@ -259,7 +260,11 @@ impl HeartbeatTask {
error!(e; "Error while handling heartbeat response");
}
}
Ok(None) => break,
Ok(None) => {
warn!("Heartbeat response stream closed");
capture_self.start_with_retry(retry_interval).await;
break;
}
Err(e) => {
error!(e; "Occur error while reading heartbeat response");
capture_self.start_with_retry(retry_interval).await;

View File

@@ -71,6 +71,6 @@ pub struct LinearStagePlan {
/// The key expressions to use for the lookup relation.
pub lookup_key: Vec<ScalarExpr>,
/// The closure to apply to the concatenation of the key columns,
/// the stream value columns, and the lookup value colunms.
/// the stream value columns, and the lookup value columns.
pub closure: JoinFilter,
}

View File

@@ -23,7 +23,7 @@ use common_meta::heartbeat::handler::{
};
use common_meta::heartbeat::mailbox::{HeartbeatMailbox, MailboxRef, OutgoingMessage};
use common_meta::heartbeat::utils::outgoing_message_to_mailbox_message;
use common_telemetry::{debug, error, info};
use common_telemetry::{debug, error, info, warn};
use meta_client::client::{HeartbeatSender, HeartbeatStream, MetaClient};
use servers::addrs;
use servers::heartbeat_options::HeartbeatOptions;
@@ -42,8 +42,8 @@ use crate::metrics::{HEARTBEAT_RECV_COUNT, HEARTBEAT_SENT_COUNT};
pub struct HeartbeatTask {
peer_addr: String,
meta_client: Arc<MetaClient>,
report_interval: u64,
retry_interval: u64,
report_interval: Duration,
retry_interval: Duration,
resp_handler_executor: HeartbeatResponseHandlerExecutorRef,
start_time_ms: u64,
}
@@ -58,8 +58,8 @@ impl HeartbeatTask {
HeartbeatTask {
peer_addr: addrs::resolve_addr(&opts.grpc.bind_addr, Some(&opts.grpc.server_addr)),
meta_client,
report_interval: heartbeat_opts.interval.as_millis() as u64,
retry_interval: heartbeat_opts.retry_interval.as_millis() as u64,
report_interval: heartbeat_opts.interval,
retry_interval: heartbeat_opts.retry_interval,
resp_handler_executor,
start_time_ms: common_time::util::current_time_millis() as u64,
}
@@ -103,13 +103,15 @@ impl HeartbeatTask {
HEARTBEAT_RECV_COUNT.with_label_values(&["success"]).inc();
}
}
Ok(None) => break,
Ok(None) => {
warn!("Heartbeat response stream closed");
capture_self.start_with_retry(retry_interval).await;
break;
}
Err(e) => {
HEARTBEAT_RECV_COUNT.with_label_values(&["error"]).inc();
error!(e; "Occur error while reading heartbeat response");
capture_self
.start_with_retry(Duration::from_millis(retry_interval))
.await;
capture_self.start_with_retry(retry_interval).await;
break;
}
@@ -177,12 +179,13 @@ impl HeartbeatTask {
if let Some(message) = message {
Self::new_heartbeat_request(&heartbeat_request, Some(message))
} else {
warn!("Sender has been dropped, exiting the heartbeat loop");
// Receives None that means Sender was dropped, we need to break the current loop
break
}
}
_ = &mut sleep => {
sleep.as_mut().reset(Instant::now() + Duration::from_millis(report_interval));
sleep.as_mut().reset(Instant::now() + report_interval);
Self::new_heartbeat_request(&heartbeat_request, None)
}
};

View File

@@ -42,7 +42,17 @@ uuid.workspace = true
[dev-dependencies]
common-test-util.workspace = true
criterion = "0.4"
rand.workspace = true
rand_chacha = "0.9"
tempfile.workspace = true
tokio.workspace = true
tokio-util.workspace = true
[[bench]]
name = "tokenizer_bench"
harness = false
[[bench]]
name = "bytes_to_u64_vec"
harness = false

View File

@@ -0,0 +1,99 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::hint::black_box;
use bytes::Bytes;
use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion, Throughput};
use index::bloom_filter::reader::bytes_to_u64_vec;
use rand::{Rng, SeedableRng};
use rand_chacha::ChaCha8Rng;
/// Generate test data that is guaranteed to be aligned to 8-byte boundary
fn generate_aligned_data(size: usize) -> Bytes {
let mut rng = ChaCha8Rng::seed_from_u64(42);
let u64_count = size / 8; // Number of u64 values
// Generate random u64 values directly - this guarantees alignment
let mut u64_data: Vec<u64> = Vec::with_capacity(u64_count);
for _ in 0..u64_count {
u64_data.push(rng.random::<u64>());
}
// Transmute Vec<u64> to Vec<u8> while preserving alignment
let byte_vec = unsafe {
let ptr = u64_data.as_mut_ptr() as *mut u8;
let len = u64_data.len() * std::mem::size_of::<u64>();
let cap = u64_data.capacity() * std::mem::size_of::<u64>();
std::mem::forget(u64_data); // Prevent dropping the original Vec
Vec::from_raw_parts(ptr, len, cap)
};
Bytes::from(byte_vec)
}
/// Generate test data that is guaranteed to be unaligned
fn generate_unaligned_data(size: usize) -> Bytes {
let mut rng = ChaCha8Rng::seed_from_u64(42);
let u64_count = size / 8; // Number of u64 values
// Generate random u64 values - start with aligned data
let mut u64_data: Vec<u64> = Vec::with_capacity(u64_count);
for _ in 0..u64_count {
u64_data.push(rng.random::<u64>());
}
// Transmute Vec<u64> to Vec<u8>
let byte_vec = unsafe {
let ptr = u64_data.as_mut_ptr() as *mut u8;
let len = u64_data.len() * std::mem::size_of::<u64>();
let cap = u64_data.capacity() * std::mem::size_of::<u64>();
std::mem::forget(u64_data); // Prevent dropping the original Vec
Vec::from_raw_parts(ptr, len, cap)
};
let unaligned_bytes = Bytes::from(byte_vec);
unaligned_bytes.slice(1..)
}
fn benchmark_convert(c: &mut Criterion) {
let sizes = vec![1024, 16384, 262144, 1048576]; // 1KB to 1MB
let mut group = c.benchmark_group("bytes_to_u64_vec");
for size in sizes {
let data = generate_aligned_data(size);
group.throughput(Throughput::Bytes(data.len() as u64));
group.bench_with_input(BenchmarkId::new("aligned", size), &data, |b, data| {
b.iter(|| {
let result = bytes_to_u64_vec(black_box(data));
black_box(result);
});
});
let data = generate_unaligned_data(size);
group.throughput(Throughput::Bytes(data.len() as u64));
group.bench_with_input(BenchmarkId::new("unaligned", size), &data, |b, data| {
b.iter(|| {
let result = bytes_to_u64_vec(black_box(data));
black_box(result);
});
});
}
group.finish();
}
criterion_group!(benches, benchmark_convert);
criterion_main!(benches);

View File

@@ -0,0 +1,66 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion};
use index::fulltext_index::tokenizer::{EnglishTokenizer, Tokenizer};
fn bench_english_tokenizer(c: &mut Criterion) {
let tokenizer = EnglishTokenizer;
let texts = vec![
("short", "Hello, world! This is a test."),
("medium", "The quick brown fox jumps over the lazy dog. Lorem ipsum dolor sit amet, consectetur adipiscing elit. Sed do eiusmod tempor incididunt ut labore et dolore magna aliqua."),
("long", "Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do eiusmod tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident, sunt in culpa qui officia deserunt mollit anim id est laborum. Sed ut perspiciatis unde omnis iste natus error sit voluptatem accusantium doloremque laudantium, totam rem aperiam, eaque ipsa quae ab illo inventore veritatis et quasi architecto beatae vitae dicta sunt explicabo. Nemo enim ipsam voluptatem quia voluptas sit aspernatur aut odit aut fugit, sed quia consequuntur magni dolores eos qui ratione voluptatem sequi nesciunt."),
("mixed_chars", "Hello123 world!!! This-is_a.test@example.com with various: punctuation; and [brackets] {curly} (parentheses) & symbols* + numbers456."),
("numbers_heavy", "test123 456test test789 abc123def 999888777 hello42world 123 456 789 mix1ng l3tt3rs 4nd numb3rs"),
("punctuation_heavy", "Hello!!! World??? This...is...a...test... With lots of!!! punctuation??? marks!!! And... ellipses???"),
("postgres log", "2025-08-01 21:09:28.928 UTC [27] LOG: checkpoint complete: wrote 0 buffers (0.0%); 0 WAL file(s) added, 0 removed, 0 recycled; write=0.001 s, sync=0.001 s, total=0.003 s; sync files=0, longest=0.000 s, average=0.000 s; distance=0 kB, estimate=5 kB; lsn=0/1992868, redo lsn=0/1992868"),
("many_short_words", "a b c d e f g h i j k l m n o p q r s t u v w x y z"),
("with_unicode", "这是,一个测试。🈶一些 Unicøde 字符比如 café and naïve words."),
];
let mut group = c.benchmark_group("english_tokenizer");
for (size, text) in texts {
group.bench_with_input(BenchmarkId::new("tokenize", size), &text, |b, text| {
b.iter(|| tokenizer.tokenize(text))
});
}
group.finish();
// Benchmark with repeated tokenization to simulate real-world usage
let mut repeat_group = c.benchmark_group("english_tokenizer_repeated");
let sample_text = "The quick brown fox jumps over the lazy dog. This sentence contains most letters of the alphabet.";
for repeat_count in [10, 100, 1000] {
repeat_group.bench_with_input(
BenchmarkId::new("repeated_tokenize", repeat_count),
&repeat_count,
|b, &repeat_count| {
b.iter(|| {
for _ in 0..repeat_count {
tokenizer.tokenize(sample_text);
}
})
},
);
}
repeat_group.finish();
}
criterion_group!(benches, bench_english_tokenizer);
criterion_main!(benches);

View File

@@ -12,9 +12,10 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::ops::Range;
use std::ops::{Range, Rem};
use async_trait::async_trait;
use bytemuck::try_cast_slice;
use bytes::Bytes;
use common_base::range_read::RangeReader;
use fastbloom::BloomFilter;
@@ -33,6 +34,47 @@ const BLOOM_META_LEN_SIZE: u64 = 4;
/// Default prefetch size of bloom filter meta.
pub const DEFAULT_PREFETCH_SIZE: u64 = 8192; // 8KiB
/// Safely converts bytes to Vec<u64> using bytemuck for optimal performance.
/// Faster than chunking and converting each piece individually.
///
/// The input bytes are a sequence of little-endian u64s.
pub fn bytes_to_u64_vec(bytes: &Bytes) -> Vec<u64> {
// drop tailing things, this keeps the same behavior with `chunks_exact`.
let aligned_length = bytes.len() - bytes.len().rem(std::mem::size_of::<u64>());
let byte_slice = &bytes[..aligned_length];
// Try fast path first: direct cast if aligned
let u64_vec = if let Ok(u64_slice) = try_cast_slice::<u8, u64>(byte_slice) {
u64_slice.to_vec()
} else {
// Slow path: create aligned Vec<u64> and copy data
let u64_count = byte_slice.len() / std::mem::size_of::<u64>();
let mut u64_vec = Vec::<u64>::with_capacity(u64_count);
// SAFETY: We're creating a properly sized slice from uninitialized but allocated memory
// to copy bytes into. The slice has exactly the right size for the byte data.
let dest_slice = unsafe {
std::slice::from_raw_parts_mut(u64_vec.as_mut_ptr() as *mut u8, byte_slice.len())
};
dest_slice.copy_from_slice(byte_slice);
// SAFETY: We've just initialized exactly u64_count elements worth of bytes
unsafe { u64_vec.set_len(u64_count) };
u64_vec
};
// Convert from platform endianness to little endian if needed
// Just in case.
#[cfg(target_endian = "little")]
{
u64_vec
}
#[cfg(target_endian = "big")]
{
u64_vec.into_iter().map(|x| x.swap_bytes()).collect()
}
}
/// `BloomFilterReader` reads the bloom filter from the file.
#[async_trait]
pub trait BloomFilterReader: Sync {
@@ -56,10 +98,7 @@ pub trait BloomFilterReader: Sync {
/// Reads a bloom filter with the given location.
async fn bloom_filter(&self, loc: &BloomFilterLoc) -> Result<BloomFilter> {
let bytes = self.range_read(loc.offset, loc.size as _).await?;
let vec = bytes
.chunks_exact(std::mem::size_of::<u64>())
.map(|chunk| u64::from_le_bytes(chunk.try_into().unwrap()))
.collect();
let vec = bytes_to_u64_vec(&bytes);
let bm = BloomFilter::from_vec(vec)
.seed(&SEED)
.expected_items(loc.element_count as _);
@@ -75,10 +114,7 @@ pub trait BloomFilterReader: Sync {
let mut result = Vec::with_capacity(bss.len());
for (bs, loc) in bss.into_iter().zip(locs.iter()) {
let vec = bs
.chunks_exact(std::mem::size_of::<u64>())
.map(|chunk| u64::from_le_bytes(chunk.try_into().unwrap()))
.collect();
let vec = bytes_to_u64_vec(&bs);
let bm = BloomFilter::from_vec(vec)
.seed(&SEED)
.expected_items(loc.element_count as _);

View File

@@ -89,8 +89,12 @@ impl FulltextIndexCreator for BloomFilterFulltextIndexCreator {
&mut self,
puffin_writer: &mut (impl PuffinWriter + Send),
blob_key: &str,
put_options: PutOptions,
mut put_options: PutOptions,
) -> Result<u64> {
// Compressing the bloom filter doesn't reduce the size but hurts read performance.
// Always disable compression here.
put_options.compression = None;
let creator = self.inner.as_mut().context(AbortedSnafu)?;
let (tx, rx) = tokio::io::duplex(PIPE_BUFFER_SIZE_FOR_SENDING_BLOB);

View File

@@ -19,6 +19,29 @@ lazy_static::lazy_static! {
static ref JIEBA: jieba_rs::Jieba = jieba_rs::Jieba::new();
}
/// A-Z, a-z, 0-9, and '_' are true
const VALID_ASCII_TOKEN: [bool; 256] = [
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, true, true, true, true, true,
true, true, true, true, true, false, false, false, false, false, false, false, true, true,
true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true,
true, true, true, true, true, true, true, true, false, false, false, false, true, false, true,
true, true, true, true, true, true, true, true, true, true, true, true, true, true, true, true,
true, true, true, true, true, true, true, true, true, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false, false, false, false,
false, false, false, false, false, false, false, false, false, false,
];
/// `Tokenizer` tokenizes a text into a list of tokens.
pub trait Tokenizer: Send {
fn tokenize<'a>(&self, text: &'a str) -> Vec<&'a str>;
@@ -32,9 +55,28 @@ pub struct EnglishTokenizer;
impl Tokenizer for EnglishTokenizer {
fn tokenize<'a>(&self, text: &'a str) -> Vec<&'a str> {
text.split(|c: char| !c.is_alphanumeric())
.filter(|s| !s.is_empty())
.collect()
if text.is_ascii() {
let mut tokens = Vec::new();
let mut start = 0;
for (i, &byte) in text.as_bytes().iter().enumerate() {
if !VALID_ASCII_TOKEN[byte as usize] {
if start < i {
tokens.push(&text[start..i]);
}
start = i + 1;
}
}
if start < text.len() {
tokens.push(&text[start..]);
}
tokens
} else {
text.split(|c: char| !c.is_alphanumeric() && c != '_')
.filter(|s| !s.is_empty())
.collect()
}
}
}
@@ -96,9 +138,25 @@ mod tests {
#[test]
fn test_english_tokenizer() {
let tokenizer = EnglishTokenizer;
let text = "Hello, world! This is a test0.";
let text = "Hello, world!!! This is a----++ test012_345+67890";
let tokens = tokenizer.tokenize(text);
assert_eq!(tokens, vec!["Hello", "world", "This", "is", "a", "test0"]);
assert_eq!(
tokens,
vec!["Hello", "world", "This", "is", "a", "test012_345", "67890"]
);
}
#[test]
fn test_english_tokenizer_with_utf8() {
let tokenizer = EnglishTokenizer;
let text = "💸unfold the 纸巾😣and gently 清洁表😭面";
let tokens = tokenizer.tokenize(text);
assert_eq!(
tokens,
// Don't care what happens to non-ASCII characters.
// It's kind of a misconfiguration to use EnglishTokenizer on non-ASCII text.
vec!["unfold", "the", "纸巾", "and", "gently", "清洁表", ""]
);
}
#[test]
@@ -109,6 +167,29 @@ mod tests {
assert_eq!(tokens, vec!["", "喜欢", "苹果"]);
}
#[test]
fn test_valid_ascii_token_lookup_table() {
// Test all ASCII values in a single loop
for c in 0u8..=255u8 {
let is_valid = VALID_ASCII_TOKEN[c as usize];
let should_be_valid = (c as char).is_ascii_alphanumeric() || c == b'_';
assert_eq!(
is_valid,
should_be_valid,
"Character '{}' (byte {}) validity mismatch: expected {}, got {}",
if c.is_ascii() && !c.is_ascii_control() {
c as char
} else {
'?'
},
c,
should_be_valid,
is_valid
);
}
}
#[test]
fn test_analyzer() {
let tokenizer = EnglishTokenizer;

View File

@@ -525,7 +525,7 @@ impl MetaClient {
self.heartbeat_client()?.ask_leader().await
}
/// Returns a heartbeat bidirectional streaming: (sender, recever), the
/// Returns a heartbeat bidirectional streaming: (sender, receiver), the
/// other end is the leader of `metasrv`.
///
/// The `datanode` needs to use the sender to continuously send heartbeat

View File

@@ -48,6 +48,7 @@ datatypes.workspace = true
deadpool = { workspace = true, optional = true }
deadpool-postgres = { workspace = true, optional = true }
derive_builder.workspace = true
either.workspace = true
etcd-client.workspace = true
futures.workspace = true
h2 = "0.3"

View File

@@ -34,6 +34,7 @@ use common_meta::kv_backend::{KvBackendRef, ResettableKvBackendRef};
use common_telemetry::info;
#[cfg(feature = "pg_kvbackend")]
use deadpool_postgres::{Config, Runtime};
use either::Either;
use etcd_client::Client;
use servers::configurator::ConfiguratorRef;
use servers::export_metrics::ExportMetricsTask;
@@ -77,7 +78,7 @@ use crate::{error, Result};
pub struct MetasrvInstance {
metasrv: Arc<Metasrv>,
http_server: HttpServer,
http_server: Either<Option<HttpServerBuilder>, HttpServer>,
opts: MetasrvOptions,
@@ -103,11 +104,10 @@ impl MetasrvInstance {
// Wire up the admin_axum_router as an extra router
let extra_routers = admin_axum_router(metasrv.clone());
let http_server = HttpServerBuilder::new(opts.http.clone())
let builder = HttpServerBuilder::new(opts.http.clone())
.with_metrics_handler(MetricsHandler)
.with_greptime_config_options(opts.to_toml().context(error::TomlFormatSnafu)?)
.with_extra_router(extra_routers)
.build();
.with_extra_router(extra_routers);
// put metasrv into plugins for later use
plugins.insert::<Arc<Metasrv>>(metasrv.clone());
@@ -115,7 +115,7 @@ impl MetasrvInstance {
.context(error::InitExportMetricsTaskSnafu)?;
Ok(MetasrvInstance {
metasrv,
http_server,
http_server: Either::Left(Some(builder)),
opts,
signal_sender: None,
plugins,
@@ -126,6 +126,25 @@ impl MetasrvInstance {
}
pub async fn start(&mut self) -> Result<()> {
if let Some(builder) = self.http_server.as_mut().left()
&& let Some(builder) = builder.take()
{
let mut server = builder.build();
let addr = self.opts.http.addr.parse().context(error::ParseAddrSnafu {
addr: &self.opts.http.addr,
})?;
info!("starting http server at {}", addr);
server.start(addr).await.context(error::StartHttpSnafu)?;
self.http_server = Either::Right(server);
} else {
// If the http server builder is not present, the Metasrv has to be called "start"
// already, regardless of the startup was successful or not. Return an `Ok` here for
// simplicity.
return Ok(());
};
self.metasrv.try_start().await?;
if let Some(t) = self.export_metrics_task.as_ref() {
@@ -149,14 +168,6 @@ impl MetasrvInstance {
.await?;
self.bind_addr = Some(socket_addr);
let addr = self.opts.http.addr.parse().context(error::ParseAddrSnafu {
addr: &self.opts.http.addr,
})?;
self.http_server
.start(addr)
.await
.context(error::StartHttpSnafu)?;
*self.serve_state.lock().await = Some(serve_state_rx);
Ok(())
}
@@ -174,12 +185,15 @@ impl MetasrvInstance {
.context(error::SendShutdownSignalSnafu)?;
}
self.metasrv.shutdown().await?;
self.http_server
.shutdown()
.await
.context(error::ShutdownServerSnafu {
server: self.http_server.name(),
})?;
if let Some(http_server) = self.http_server.as_ref().right() {
http_server
.shutdown()
.await
.context(error::ShutdownServerSnafu {
server: http_server.name(),
})?;
}
Ok(())
}
@@ -193,6 +207,14 @@ impl MetasrvInstance {
pub fn bind_addr(&self) -> &Option<SocketAddr> {
&self.bind_addr
}
pub fn mut_http_server(&mut self) -> &mut Either<Option<HttpServerBuilder>, HttpServer> {
&mut self.http_server
}
pub fn http_server(&self) -> Option<&HttpServer> {
self.http_server.as_ref().right()
}
}
pub async fn bootstrap_metasrv_with_router(

View File

@@ -17,7 +17,7 @@ use std::sync::Arc;
use std::time::Duration;
use common_meta::key::{CANDIDATES_ROOT, ELECTION_KEY};
use common_telemetry::{error, warn};
use common_telemetry::{error, info, warn};
use common_time::Timestamp;
use snafu::{ensure, OptionExt, ResultExt};
use sqlx::mysql::{MySqlArguments, MySqlRow};
@@ -645,6 +645,13 @@ impl Election for MySqlElection {
}
async fn reset_campaign(&self) {
info!("Resetting campaign");
if self.is_leader.load(Ordering::Relaxed) {
if let Err(err) = self.step_down_without_lock().await {
error!(err; "Failed to step down without lock");
}
info!("Step down without lock successfully, due to reset campaign");
}
if let Err(err) = self.client.lock().await.reset_client().await {
error!(err; "Failed to reset client");
}
@@ -1190,7 +1197,7 @@ mod tests {
));
handles.push(handle);
}
// Wait for candidates to registrate themselves and renew their leases at least once.
// Wait for candidates to register themselves and renew their leases at least once.
tokio::time::sleep(candidate_lease_ttl / 2 + Duration::from_secs(1)).await;
let (tx, _) = broadcast::channel(100);
@@ -1639,6 +1646,41 @@ mod tests {
drop_table(&leader_mysql_election.client, table_name).await;
}
#[tokio::test]
async fn test_reset_campaign() {
maybe_skip_mysql_integration_test!();
common_telemetry::init_default_ut_logging();
let leader_value = "test_leader".to_string();
let uuid = uuid::Uuid::new_v4().to_string();
let table_name = "test_reset_campaign_greptime_metakv";
let candidate_lease_ttl = Duration::from_secs(5);
let meta_lease_ttl = Duration::from_secs(2);
let execution_timeout = Duration::from_secs(10);
let idle_session_timeout = Duration::from_secs(0);
let client = create_mysql_client(Some(table_name), execution_timeout, idle_session_timeout)
.await
.unwrap();
let (tx, _) = broadcast::channel(100);
let leader_mysql_election = MySqlElection {
leader_value,
client,
is_leader: AtomicBool::new(false),
leader_infancy: AtomicBool::new(true),
leader_watcher: tx,
store_key_prefix: uuid,
candidate_lease_ttl,
meta_lease_ttl,
sql_set: ElectionSqlFactory::new(table_name).build(),
};
leader_mysql_election
.is_leader
.store(true, Ordering::Relaxed);
leader_mysql_election.reset_campaign().await;
assert!(!leader_mysql_election.is_leader());
drop_table(&leader_mysql_election.client, table_name).await;
}
#[tokio::test]
async fn test_follower_action() {
maybe_skip_mysql_integration_test!();

View File

@@ -17,7 +17,7 @@ use std::sync::Arc;
use std::time::Duration;
use common_meta::key::{CANDIDATES_ROOT, ELECTION_KEY};
use common_telemetry::{error, warn};
use common_telemetry::{error, info, warn};
use common_time::Timestamp;
use deadpool_postgres::{Manager, Pool};
use snafu::{ensure, OptionExt, ResultExt};
@@ -454,6 +454,13 @@ impl Election for PgElection {
}
async fn reset_campaign(&self) {
info!("Resetting campaign");
if self.is_leader.load(Ordering::Relaxed) {
if let Err(err) = self.step_down_without_lock().await {
error!(err; "Failed to step down without lock");
}
info!("Step down without lock successfully, due to reset campaign");
}
if let Err(err) = self.pg_client.write().await.reset_client().await {
error!(err; "Failed to reset client");
}
@@ -749,18 +756,12 @@ impl PgElection {
key: key.clone(),
..Default::default()
};
if self
.is_leader
.compare_exchange(true, false, Ordering::AcqRel, Ordering::Acquire)
.is_ok()
{
if let Err(e) = self
.leader_watcher
.send(LeaderChangeMessage::StepDown(Arc::new(leader_key)))
{
error!(e; "Failed to send leader change message");
}
}
send_leader_change_and_set_flags(
&self.is_leader,
&self.leader_infancy,
&self.leader_watcher,
LeaderChangeMessage::StepDown(Arc::new(leader_key)),
);
Ok(())
}
@@ -1012,7 +1013,7 @@ mod tests {
));
handles.push(handle);
}
// Wait for candidates to registrate themselves and renew their leases at least once.
// Wait for candidates to register themselves and renew their leases at least once.
tokio::time::sleep(Duration::from_secs(3)).await;
let (tx, _) = broadcast::channel(100);
@@ -1551,6 +1552,44 @@ mod tests {
drop_table(&follower_pg_election, table_name).await;
}
#[tokio::test]
async fn test_reset_campaign() {
maybe_skip_postgres_integration_test!();
let leader_value = "test_leader".to_string();
let uuid = uuid::Uuid::new_v4().to_string();
let table_name = "test_reset_campaign_greptime_metakv";
let candidate_lease_ttl = Duration::from_secs(5);
let execution_timeout = Duration::from_secs(10);
let statement_timeout = Duration::from_secs(10);
let meta_lease_ttl = Duration::from_secs(2);
let idle_session_timeout = Duration::from_secs(0);
let client = create_postgres_client(
Some(table_name),
execution_timeout,
idle_session_timeout,
statement_timeout,
)
.await
.unwrap();
let (tx, _) = broadcast::channel(100);
let leader_pg_election = PgElection {
leader_value,
pg_client: RwLock::new(client),
is_leader: AtomicBool::new(false),
leader_infancy: AtomicBool::new(true),
leader_watcher: tx,
store_key_prefix: uuid,
candidate_lease_ttl,
meta_lease_ttl,
sql_set: ElectionSqlFactory::new(28321, table_name).build(),
};
leader_pg_election.is_leader.store(true, Ordering::Relaxed);
leader_pg_election.reset_campaign().await;
assert!(!leader_pg_election.is_leader());
drop_table(&leader_pg_election, table_name).await;
}
#[tokio::test]
async fn test_idle_session_timeout() {
maybe_skip_postgres_integration_test!();

View File

@@ -124,6 +124,7 @@ mod tests {
index_size: 0,
data_topic_latest_entry_id: 0,
metadata_topic_latest_entry_id: 0,
written_bytes: 0,
}
}

View File

@@ -104,6 +104,7 @@ mod tests {
},
data_topic_latest_entry_id: 0,
metadata_topic_latest_entry_id: 0,
written_bytes: 0,
}
}
acc.stat = Some(Stat {

View File

@@ -19,6 +19,7 @@ use api::v1::meta::{HeartbeatRequest, RegionLease, Role};
use async_trait::async_trait;
use common_meta::key::TableMetadataManagerRef;
use common_meta::region_keeper::MemoryRegionKeeperRef;
use common_telemetry::error;
use store_api::region_engine::GrantedRegion;
use store_api::storage::RegionId;
@@ -83,36 +84,44 @@ impl HeartbeatHandler for RegionLeaseHandler {
let regions = stat.regions();
let datanode_id = stat.id;
let RenewRegionLeasesResponse {
non_exists,
renewed,
} = self
match self
.region_lease_keeper
.renew_region_leases(datanode_id, &regions)
.await?;
.await
{
Ok(RenewRegionLeasesResponse {
non_exists,
renewed,
}) => {
let renewed = if let Some(renewer) = &self.customized_region_lease_renewer {
renewer
.renew(ctx, renewed)
.into_iter()
.map(|region| region.into())
.collect()
} else {
renewed
.into_iter()
.map(|(region_id, region_lease_info)| {
GrantedRegion::new(region_id, region_lease_info.role).into()
})
.collect::<Vec<_>>()
};
let renewed = if let Some(renewer) = &self.customized_region_lease_renewer {
renewer
.renew(ctx, renewed)
.into_iter()
.map(|region| region.into())
.collect()
} else {
renewed
.into_iter()
.map(|(region_id, region_lease_info)| {
GrantedRegion::new(region_id, region_lease_info.role).into()
})
.collect::<Vec<_>>()
};
acc.region_lease = Some(RegionLease {
regions: renewed,
duration_since_epoch: req.duration_since_epoch,
lease_seconds: self.region_lease_seconds,
closeable_region_ids: non_exists.iter().map(|region| region.as_u64()).collect(),
});
acc.inactive_region_ids = non_exists;
acc.region_lease = Some(RegionLease {
regions: renewed,
duration_since_epoch: req.duration_since_epoch,
lease_seconds: self.region_lease_seconds,
closeable_region_ids: non_exists.iter().map(|region| region.as_u64()).collect(),
});
acc.inactive_region_ids = non_exists;
}
Err(e) => {
error!(e; "Failed to renew region leases for datanode: {datanode_id:?}, regions: {:?}", regions);
// If we throw error here, the datanode will be marked as failure by region failure handler.
// So we only log the error and continue.
}
}
Ok(HandleControl::Continue)
}
@@ -120,18 +129,27 @@ impl HeartbeatHandler for RegionLeaseHandler {
#[cfg(test)]
mod test {
use std::any::Any;
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use common_meta::datanode::{RegionManifestInfo, RegionStat, Stat};
use common_meta::distributed_time_constants;
use common_meta::error::Result as MetaResult;
use common_meta::key::table_route::TableRouteValue;
use common_meta::key::test_utils::new_test_table_info;
use common_meta::key::TableMetadataManager;
use common_meta::kv_backend::memory::MemoryKvBackend;
use common_meta::kv_backend::txn::{Txn, TxnResponse};
use common_meta::kv_backend::{KvBackend, TxnService};
use common_meta::peer::Peer;
use common_meta::region_keeper::MemoryRegionKeeper;
use common_meta::rpc::router::{LeaderState, Region, RegionRoute};
use common_meta::rpc::store::{
BatchDeleteRequest, BatchDeleteResponse, BatchGetRequest, BatchGetResponse,
BatchPutRequest, BatchPutResponse, DeleteRangeRequest, DeleteRangeResponse, PutRequest,
PutResponse, RangeRequest, RangeResponse,
};
use store_api::region_engine::RegionRole;
use store_api::storage::RegionId;
@@ -166,6 +184,7 @@ mod test {
},
data_topic_latest_entry_id: 0,
metadata_topic_latest_entry_id: 0,
written_bytes: 0,
}
}
@@ -403,4 +422,102 @@ mod test {
assert_eq!(granted, expected);
}
struct MockKvBackend;
#[async_trait::async_trait]
impl TxnService for MockKvBackend {
type Error = common_meta::error::Error;
async fn txn(&self, _txn: Txn) -> MetaResult<TxnResponse> {
unimplemented!()
}
fn max_txn_ops(&self) -> usize {
unimplemented!()
}
}
#[async_trait::async_trait]
impl KvBackend for MockKvBackend {
fn name(&self) -> &str {
"mock_kv_backend"
}
fn as_any(&self) -> &dyn Any {
self
}
async fn range(&self, _req: RangeRequest) -> MetaResult<RangeResponse> {
unimplemented!()
}
async fn put(&self, _req: PutRequest) -> MetaResult<PutResponse> {
unimplemented!()
}
async fn batch_put(&self, _req: BatchPutRequest) -> MetaResult<BatchPutResponse> {
unimplemented!()
}
async fn batch_get(&self, _req: BatchGetRequest) -> MetaResult<BatchGetResponse> {
common_meta::error::UnexpectedSnafu {
err_msg: "mock err",
}
.fail()
}
async fn delete_range(&self, _req: DeleteRangeRequest) -> MetaResult<DeleteRangeResponse> {
unimplemented!()
}
async fn batch_delete(&self, _req: BatchDeleteRequest) -> MetaResult<BatchDeleteResponse> {
unimplemented!()
}
}
#[tokio::test]
async fn test_handle_renew_region_lease_failure() {
common_telemetry::init_default_ut_logging();
let kvbackend = Arc::new(MockKvBackend);
let table_metadata_manager = Arc::new(TableMetadataManager::new(kvbackend));
let datanode_id = 1;
let region_number = 1u32;
let table_id = 10;
let region_id = RegionId::new(table_id, region_number);
let another_region_id = RegionId::new(table_id, region_number + 1);
let no_exist_region_id = RegionId::new(table_id, region_number + 2);
let peer = Peer::empty(datanode_id);
let builder = MetasrvBuilder::new();
let metasrv = builder.build().await.unwrap();
let ctx = &mut metasrv.new_ctx();
let req = HeartbeatRequest {
duration_since_epoch: 1234,
..Default::default()
};
let acc = &mut HeartbeatAccumulator::default();
acc.stat = Some(Stat {
id: peer.id,
region_stats: vec![
new_empty_region_stat(region_id, RegionRole::Leader),
new_empty_region_stat(another_region_id, RegionRole::Leader),
new_empty_region_stat(no_exist_region_id, RegionRole::Leader),
],
..Default::default()
});
let handler = RegionLeaseHandler::new(
distributed_time_constants::REGION_LEASE_SECS,
table_metadata_manager.clone(),
Default::default(),
None,
);
handler.handle(&req, ctx, acc).await.unwrap();
assert!(acc.region_lease.is_none());
assert!(acc.inactive_region_ids.is_empty());
}
}

View File

@@ -15,6 +15,7 @@
#![feature(result_flattening)]
#![feature(assert_matches)]
#![feature(hash_set_entry)]
#![feature(let_chains)]
pub mod bootstrap;
pub mod cache_invalidator;

View File

@@ -345,7 +345,8 @@ impl MetasrvBuilder {
region_migration_manager.clone(),
runtime_switch_manager.clone(),
peer_lookup_service.clone(),
);
)
.with_state(state.clone());
Some(RegionFailureHandler::new(
region_supervisor,

View File

@@ -38,7 +38,7 @@ use common_meta::key::table_info::TableInfoValue;
use common_meta::key::table_route::TableRouteValue;
use common_meta::key::{DeserializedValueWithBytes, TableMetadataManagerRef};
use common_meta::kv_backend::ResettableKvBackendRef;
use common_meta::lock_key::{CatalogLock, RegionLock, SchemaLock};
use common_meta::lock_key::{CatalogLock, RegionLock, SchemaLock, TableLock};
use common_meta::peer::Peer;
use common_meta::region_keeper::{MemoryRegionKeeperRef, OperatingRegionGuard};
use common_procedure::error::{
@@ -215,8 +215,6 @@ pub struct VolatileContext {
/// the corresponding [RegionRoute](common_meta::rpc::router::RegionRoute) of the opening region
/// was written into [TableRouteValue](common_meta::key::table_route::TableRouteValue).
opening_region_guard: Option<OperatingRegionGuard>,
/// `table_route` is stored via previous steps for future use.
table_route: Option<DeserializedValueWithBytes<TableRouteValue>>,
/// `datanode_table` is stored via previous steps for future use.
from_peer_datanode_table: Option<DatanodeTableValue>,
/// `table_info` is stored via previous steps for future use.
@@ -383,29 +381,23 @@ impl Context {
/// Retry:
/// - Failed to retrieve the metadata of table.
pub async fn get_table_route_value(
&mut self,
) -> Result<&DeserializedValueWithBytes<TableRouteValue>> {
let table_route_value = &mut self.volatile_ctx.table_route;
&self,
) -> Result<DeserializedValueWithBytes<TableRouteValue>> {
let table_id = self.persistent_ctx.region_id.table_id();
let table_route = self
.table_metadata_manager
.table_route_manager()
.table_route_storage()
.get_with_raw_bytes(table_id)
.await
.context(error::TableMetadataManagerSnafu)
.map_err(BoxedError::new)
.with_context(|_| error::RetryLaterWithSourceSnafu {
reason: format!("Failed to get TableRoute: {table_id}"),
})?
.context(error::TableRouteNotFoundSnafu { table_id })?;
if table_route_value.is_none() {
let table_id = self.persistent_ctx.region_id.table_id();
let table_route = self
.table_metadata_manager
.table_route_manager()
.table_route_storage()
.get_with_raw_bytes(table_id)
.await
.context(error::TableMetadataManagerSnafu)
.map_err(BoxedError::new)
.with_context(|_| error::RetryLaterWithSourceSnafu {
reason: format!("Failed to get TableRoute: {table_id}"),
})?
.context(error::TableRouteNotFoundSnafu { table_id })?;
*table_route_value = Some(table_route);
}
Ok(table_route_value.as_ref().unwrap())
Ok(table_route)
}
/// Notifies the RegionSupervisor to register failure detectors of failed region.
@@ -447,12 +439,6 @@ impl Context {
.await;
}
/// Removes the `table_route` of [VolatileContext], returns true if any.
pub fn remove_table_route_value(&mut self) -> bool {
let value = self.volatile_ctx.table_route.take();
value.is_some()
}
/// Returns the `table_info` of [VolatileContext] if any.
/// Otherwise, returns the value retrieved from remote.
///
@@ -627,14 +613,13 @@ impl RegionMigrationProcedure {
})
}
async fn rollback_inner(&mut self) -> Result<()> {
async fn rollback_inner(&mut self, procedure_ctx: &ProcedureContext) -> Result<()> {
let _timer = METRIC_META_REGION_MIGRATION_EXECUTE
.with_label_values(&["rollback"])
.start_timer();
let table_id = self.context.region_id().table_id();
let region_id = self.context.region_id();
self.context.remove_table_route_value();
let table_metadata_manager = self.context.table_metadata_manager.clone();
let table_route = self.context.get_table_route_value().await?;
@@ -647,9 +632,11 @@ impl RegionMigrationProcedure {
.any(|route| route.is_leader_downgrading());
if downgraded {
let table_lock = TableLock::Write(region_id.table_id()).into();
let _guard = procedure_ctx.provider.acquire_lock(&table_lock).await;
info!("Rollbacking downgraded region leader table route, region: {region_id}");
table_metadata_manager
.update_leader_region_status(table_id, table_route, |route| {
.update_leader_region_status(table_id, &table_route, |route| {
if route.region.id == region_id {
Some(None)
} else {
@@ -676,8 +663,8 @@ impl Procedure for RegionMigrationProcedure {
Self::TYPE_NAME
}
async fn rollback(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<()> {
self.rollback_inner()
async fn rollback(&mut self, ctx: &ProcedureContext) -> ProcedureResult<()> {
self.rollback_inner(ctx)
.await
.map_err(ProcedureError::external)
}

View File

@@ -46,7 +46,7 @@ impl UpdateMetadata {
// TODO(weny): ensures the leader region peer is the `from_peer`.
if let Err(err) = table_metadata_manager
.update_leader_region_status(table_id, current_table_route_value, |route| {
.update_leader_region_status(table_id, &current_table_route_value, |route| {
if route.region.id == region_id
&& route
.leader_peer
@@ -61,7 +61,6 @@ impl UpdateMetadata {
.await
.context(error::TableMetadataManagerSnafu)
{
ctx.remove_table_route_value();
return Err(BoxedError::new(err)).context(error::RetryLaterWithSourceSnafu {
reason: format!(
"Failed to update the table route during the downgrading leader region, region_id: {region_id}, from_peer_id: {from_peer_id}"
@@ -69,8 +68,6 @@ impl UpdateMetadata {
});
}
ctx.remove_table_route_value();
Ok(())
}
}
@@ -81,7 +78,7 @@ mod tests {
use common_meta::key::test_utils::new_test_table_info;
use common_meta::peer::Peer;
use common_meta::rpc::router::{LeaderState, Region, RegionRoute};
use common_meta::rpc::router::{Region, RegionRoute};
use store_api::storage::RegionId;
use crate::error::Error;
@@ -115,63 +112,6 @@ mod tests {
assert!(!err.is_retryable());
}
#[tokio::test]
async fn test_failed_to_update_table_route_error() {
let state = UpdateMetadata::Downgrade;
let persistent_context = new_persistent_context();
let from_peer = persistent_context.from_peer.clone();
let env = TestingEnv::new();
let mut ctx = env.context_factory().new_context(persistent_context);
let table_id = ctx.region_id().table_id();
let table_info = new_test_table_info(1024, vec![1, 2]).into();
let region_routes = vec![
RegionRoute {
region: Region::new_test(RegionId::new(1024, 1)),
leader_peer: Some(from_peer.clone()),
..Default::default()
},
RegionRoute {
region: Region::new_test(RegionId::new(1024, 2)),
leader_peer: Some(Peer::empty(4)),
..Default::default()
},
];
env.create_physical_table_metadata(table_info, region_routes)
.await;
let table_metadata_manager = env.table_metadata_manager();
let original_table_route = table_metadata_manager
.table_route_manager()
.table_route_storage()
.get_with_raw_bytes(table_id)
.await
.unwrap()
.unwrap();
// modifies the table route.
table_metadata_manager
.update_leader_region_status(table_id, &original_table_route, |route| {
if route.region.id == RegionId::new(1024, 2) {
Some(Some(LeaderState::Downgrading))
} else {
None
}
})
.await
.unwrap();
// sets the old table route.
ctx.volatile_ctx.table_route = Some(original_table_route);
let err = state.downgrade_leader_region(&mut ctx).await.unwrap_err();
assert!(ctx.volatile_ctx.table_route.is_none());
assert!(err.is_retryable());
assert!(format!("{err:?}").contains("Failed to update the table route"));
}
#[tokio::test]
async fn test_only_downgrade_from_peer() {
let mut state = Box::new(UpdateMetadata::Downgrade);
@@ -212,7 +152,6 @@ mod tests {
// It should remain unchanged.
assert_eq!(latest_table_route.version().unwrap(), 0);
assert!(!latest_table_route.region_routes().unwrap()[0].is_leader_downgrading());
assert!(ctx.volatile_ctx.table_route.is_none());
}
#[tokio::test]
@@ -254,6 +193,5 @@ mod tests {
.unwrap();
assert!(latest_table_route.region_routes().unwrap()[0].is_leader_downgrading());
assert!(ctx.volatile_ctx.table_route.is_none());
}
}

View File

@@ -35,7 +35,7 @@ impl UpdateMetadata {
let current_table_route_value = ctx.get_table_route_value().await?;
if let Err(err) = table_metadata_manager
.update_leader_region_status(table_id, current_table_route_value, |route| {
.update_leader_region_status(table_id, &current_table_route_value, |route| {
if route.region.id == region_id {
Some(None)
} else {
@@ -45,14 +45,12 @@ impl UpdateMetadata {
.await
.context(error::TableMetadataManagerSnafu)
{
ctx.remove_table_route_value();
return Err(BoxedError::new(err)).context(error::RetryLaterWithSourceSnafu {
reason: format!("Failed to update the table route during the rollback downgraded leader region: {region_id}"),
});
}
ctx.register_failure_detectors().await;
ctx.remove_table_route_value();
Ok(())
}
@@ -61,7 +59,6 @@ impl UpdateMetadata {
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use std::sync::Arc;
use common_meta::key::test_utils::new_test_table_info;
use common_meta::peer::Peer;
@@ -73,7 +70,6 @@ mod tests {
use crate::procedure::region_migration::test_util::{self, new_procedure_context, TestingEnv};
use crate::procedure::region_migration::update_metadata::UpdateMetadata;
use crate::procedure::region_migration::{ContextFactory, PersistentContext, State};
use crate::region::supervisor::RegionFailureDetectorControl;
fn new_persistent_context() -> PersistentContext {
test_util::new_persistent_context(1, 2, RegionId::new(1024, 1))
@@ -93,101 +89,6 @@ mod tests {
assert!(!err.is_retryable());
}
#[tokio::test]
async fn test_update_table_route_with_retry() {
let state = UpdateMetadata::Rollback;
let persistent_context = new_persistent_context();
let from_peer = persistent_context.from_peer.clone();
let env = TestingEnv::new();
let mut ctx = env.context_factory().new_context(persistent_context);
let (tx, mut rx) = tokio::sync::mpsc::channel(8);
ctx.region_failure_detector_controller = Arc::new(RegionFailureDetectorControl::new(tx));
let table_id = ctx.region_id().table_id();
let table_info = new_test_table_info(1024, vec![1, 2, 3]).into();
let region_routes = vec![
RegionRoute {
region: Region::new_test(RegionId::new(1024, 1)),
leader_peer: Some(from_peer.clone()),
leader_state: Some(LeaderState::Downgrading),
..Default::default()
},
RegionRoute {
region: Region::new_test(RegionId::new(1024, 2)),
leader_peer: Some(Peer::empty(4)),
leader_state: Some(LeaderState::Downgrading),
..Default::default()
},
RegionRoute {
region: Region::new_test(RegionId::new(1024, 3)),
leader_peer: Some(Peer::empty(5)),
..Default::default()
},
];
let expected_region_routes = {
let mut region_routes = region_routes.clone();
region_routes[0].leader_state = None;
region_routes[1].leader_state = None;
region_routes
};
env.create_physical_table_metadata(table_info, region_routes)
.await;
let table_metadata_manager = env.table_metadata_manager();
let old_table_route = table_metadata_manager
.table_route_manager()
.table_route_storage()
.get_with_raw_bytes(table_id)
.await
.unwrap()
.unwrap();
// modifies the table route.
table_metadata_manager
.update_leader_region_status(table_id, &old_table_route, |route| {
if route.region.id == RegionId::new(1024, 2) {
Some(None)
} else {
None
}
})
.await
.unwrap();
ctx.volatile_ctx.table_route = Some(old_table_route);
let err = state
.rollback_downgraded_region(&mut ctx)
.await
.unwrap_err();
assert!(ctx.volatile_ctx.table_route.is_none());
assert!(err.is_retryable());
assert!(format!("{err:?}").contains("Failed to update the table route"));
assert_eq!(rx.len(), 0);
state.rollback_downgraded_region(&mut ctx).await.unwrap();
let event = rx.try_recv().unwrap();
let detecting_regions = event.into_region_failure_detectors();
assert_eq!(
detecting_regions,
vec![(from_peer.id, ctx.persistent_ctx.region_id)]
);
let table_route = table_metadata_manager
.table_route_manager()
.table_route_storage()
.get(table_id)
.await
.unwrap()
.unwrap();
assert_eq!(
&expected_region_routes,
table_route.region_routes().unwrap()
);
}
#[tokio::test]
async fn test_next_migration_end_state() {
let mut state = Box::new(UpdateMetadata::Rollback);
@@ -238,8 +139,6 @@ mod tests {
.downcast_ref::<RegionMigrationAbort>()
.unwrap();
assert!(ctx.volatile_ctx.table_route.is_none());
let table_route = table_metadata_manager
.table_route_manager()
.table_route_storage()

View File

@@ -160,7 +160,7 @@ impl UpdateMetadata {
region_options: region_options.clone(),
region_wal_options: region_wal_options.clone(),
},
table_route_value,
&table_route_value,
region_routes,
&region_options,
&region_wal_options,
@@ -168,13 +168,11 @@ impl UpdateMetadata {
.await
.context(error::TableMetadataManagerSnafu)
{
ctx.remove_table_route_value();
return Err(BoxedError::new(err)).context(error::RetryLaterWithSourceSnafu {
reason: format!("Failed to update the table route during the upgrading candidate region: {region_id}"),
});
};
ctx.remove_table_route_value();
ctx.deregister_failure_detectors().await;
// Consumes the guard.
ctx.volatile_ctx.opening_region_guard.take();
@@ -304,71 +302,6 @@ mod tests {
assert_eq!(new_region_routes[0].leader_peer.as_ref().unwrap().id, 2);
}
#[tokio::test]
async fn test_failed_to_update_table_route_error() {
let state = UpdateMetadata::Upgrade;
let env = TestingEnv::new();
let persistent_context = new_persistent_context();
let mut ctx = env.context_factory().new_context(persistent_context);
let opening_keeper = MemoryRegionKeeper::default();
let table_id = 1024;
let table_info = new_test_table_info(table_id, vec![1]).into();
let region_routes = vec![
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 1)),
leader_peer: Some(Peer::empty(1)),
follower_peers: vec![Peer::empty(5), Peer::empty(3)],
leader_state: Some(LeaderState::Downgrading),
leader_down_since: Some(current_time_millis()),
},
RegionRoute {
region: Region::new_test(RegionId::new(table_id, 2)),
leader_peer: Some(Peer::empty(4)),
leader_state: Some(LeaderState::Downgrading),
..Default::default()
},
];
env.create_physical_table_metadata(table_info, region_routes)
.await;
let table_metadata_manager = env.table_metadata_manager();
let original_table_route = table_metadata_manager
.table_route_manager()
.table_route_storage()
.get_with_raw_bytes(table_id)
.await
.unwrap()
.unwrap();
// modifies the table route.
table_metadata_manager
.update_leader_region_status(table_id, &original_table_route, |route| {
if route.region.id == RegionId::new(1024, 2) {
// Removes the status.
Some(None)
} else {
None
}
})
.await
.unwrap();
// sets the old table route.
ctx.volatile_ctx.table_route = Some(original_table_route);
let guard = opening_keeper
.register(2, RegionId::new(table_id, 1))
.unwrap();
ctx.volatile_ctx.opening_region_guard = Some(guard);
let err = state.upgrade_candidate_region(&mut ctx).await.unwrap_err();
assert!(ctx.volatile_ctx.table_route.is_none());
assert!(ctx.volatile_ctx.opening_region_guard.is_some());
assert!(err.is_retryable());
assert!(format!("{err:?}").contains("Failed to update the table route"));
}
#[tokio::test]
async fn test_check_metadata() {
let state = UpdateMetadata::Upgrade;
@@ -486,7 +419,6 @@ mod tests {
.unwrap();
let region_routes = table_route.region_routes().unwrap();
assert!(ctx.volatile_ctx.table_route.is_none());
assert!(ctx.volatile_ctx.opening_region_guard.is_none());
assert_eq!(region_routes.len(), 1);
assert!(!region_routes[0].is_leader_downgrading());

View File

@@ -42,6 +42,7 @@ use crate::procedure::region_migration::{
};
use crate::region::failure_detector::RegionFailureDetector;
use crate::selector::SelectorOptions;
use crate::state::StateRef;
/// `DatanodeHeartbeat` represents the heartbeat signal sent from a datanode.
/// It includes identifiers for the cluster and datanode, a list of regions being monitored,
@@ -86,16 +87,6 @@ pub(crate) enum Event {
Dump(tokio::sync::oneshot::Sender<RegionFailureDetector>),
}
#[cfg(test)]
impl Event {
pub(crate) fn into_region_failure_detectors(self) -> Vec<DetectingRegion> {
match self {
Self::RegisterFailureDetectors(detecting_regions) => detecting_regions,
_ => unreachable!(),
}
}
}
impl Debug for Event {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
@@ -228,6 +219,8 @@ pub struct RegionSupervisor {
runtime_switch_manager: RuntimeSwitchManagerRef,
/// Peer lookup service
peer_lookup: PeerLookupServiceRef,
/// The meta state, used to check if the current metasrv is the leader.
state: Option<StateRef>,
}
/// Controller for managing failure detectors for regions.
@@ -308,12 +301,29 @@ impl RegionSupervisor {
region_migration_manager,
runtime_switch_manager,
peer_lookup,
state: None,
}
}
/// Sets the meta state.
pub(crate) fn with_state(mut self, state: StateRef) -> Self {
self.state = Some(state);
self
}
/// Runs the main loop.
pub(crate) async fn run(&mut self) {
while let Some(event) = self.receiver.recv().await {
if let Some(state) = self.state.as_ref()
&& !state.read().unwrap().is_leader()
{
warn!(
"The current metasrv is not the leader, ignore {:?} event",
event
);
continue;
}
match event {
Event::Tick => {
let regions = self.detect_region_failure();
@@ -326,7 +336,10 @@ impl RegionSupervisor {
self.deregister_failure_detectors(detecting_regions).await
}
Event::HeartbeatArrived(heartbeat) => self.on_heartbeat_arrived(heartbeat),
Event::Clear => self.clear(),
Event::Clear => {
self.clear();
info!("Region supervisor is initialized.");
}
#[cfg(test)]
Event::Dump(sender) => {
let _ = sender.send(self.failure_detector.dump());
@@ -759,6 +772,7 @@ pub(crate) mod tests {
while let Ok(event) = rx.try_recv() {
assert_matches!(event, Event::Tick | Event::Clear);
}
assert!(ticker.tick_handle.lock().unwrap().is_none());
}
}

View File

@@ -197,6 +197,7 @@ mod tests {
},
data_topic_latest_entry_id: 0,
metadata_topic_latest_entry_id: 0,
written_bytes: 0,
}],
..Default::default()
}
@@ -224,6 +225,7 @@ mod tests {
},
data_topic_latest_entry_id: 0,
metadata_topic_latest_entry_id: 0,
written_bytes: 0,
}],
..Default::default()
}
@@ -251,6 +253,7 @@ mod tests {
},
data_topic_latest_entry_id: 0,
metadata_topic_latest_entry_id: 0,
written_bytes: 0,
}],
..Default::default()
}

View File

@@ -27,10 +27,9 @@ use snafu::OptionExt;
use tokio::sync::mpsc;
use tokio::sync::mpsc::Sender;
use tokio_stream::wrappers::ReceiverStream;
use tonic::{Request, Response, Streaming};
use tonic::{Request, Response, Status, Streaming};
use crate::error;
use crate::error::Result;
use crate::error::{self, Result};
use crate::handler::{HeartbeatHandlerGroup, Pusher, PusherId};
use crate::metasrv::{Context, Metasrv};
use crate::metrics::METRIC_META_HEARTBEAT_RECV;
@@ -80,6 +79,7 @@ impl heartbeat_server::Heartbeat for Metasrv {
let res = handler_group
.handle(req, ctx.clone())
.await
.inspect_err(|e| warn!(e; "Failed to handle heartbeat request, pusher: {pusher_id:?}", ))
.map_err(|e| e.into());
is_not_leader = res.as_ref().is_ok_and(|r| r.is_not_leader());
@@ -109,6 +109,12 @@ impl heartbeat_server::Heartbeat for Metasrv {
if is_not_leader {
warn!("Quit because it is no longer the leader");
let _ = tx
.send(Err(Status::aborted(format!(
"The requested metasrv node is not leader, node addr: {}",
ctx.server_addr
))))
.await;
break;
}
}

View File

@@ -97,7 +97,7 @@ impl store_server::Store for Metasrv {
let req = req.into_inner();
let _timer = METRIC_META_KV_REQUEST_ELAPSED
.with_label_values(&[self.kv_backend().name(), "batch_pub"])
.with_label_values(&[self.kv_backend().name(), "batch_put"])
.start_timer();
let req: BatchPutRequest = req.into();

View File

@@ -75,6 +75,12 @@ impl State {
})
}
/// Returns true if the current state is a leader.
pub fn is_leader(&self) -> bool {
matches!(self, State::Leader(_))
}
/// Returns true if the leader cache is enabled.
pub fn enable_leader_cache(&self) -> bool {
match &self {
State::Leader(leader) => leader.enable_leader_cache,

View File

@@ -61,6 +61,7 @@ pub fn get_region_statistic(mito: &MitoEngine, region_id: RegionId) -> Option<Re
metadata_flushed_entry_id: metadata_stat.manifest.data_flushed_entry_id(),
metadata_manifest_version: metadata_stat.manifest.data_manifest_version(),
},
written_bytes: metadata_stat.written_bytes + data_stat.written_bytes,
data_topic_latest_entry_id: data_stat.data_topic_latest_entry_id,
metadata_topic_latest_entry_id: metadata_stat.metadata_topic_latest_entry_id,
}),

View File

@@ -13,6 +13,7 @@
// limitations under the License.
use std::sync::Arc;
use std::time::Duration;
use object_store::services::Fs;
use object_store::util::{join_dir, with_instrument_layers};
@@ -27,6 +28,7 @@ use crate::cache::write_cache::SstUploadRequest;
use crate::cache::CacheManagerRef;
use crate::config::{BloomFilterConfig, FulltextIndexConfig, InvertedIndexConfig};
use crate::error::{CleanDirSnafu, DeleteIndexSnafu, DeleteSstSnafu, OpenDalSnafu, Result};
use crate::metrics::{COMPACTION_STAGE_ELAPSED, FLUSH_ELAPSED};
use crate::read::Source;
use crate::region::options::IndexOptions;
use crate::sst::file::{FileHandle, FileId, FileMeta};
@@ -46,6 +48,87 @@ pub const ATOMIC_WRITE_DIR: &str = "tmp/";
/// For compatibility. Remove this after a major version release.
pub const OLD_ATOMIC_WRITE_DIR: &str = ".tmp/";
/// Write operation type.
#[derive(Eq, PartialEq, Debug)]
pub enum WriteType {
/// Writes from flush
Flush,
/// Writes from compaction.
Compaction,
}
#[derive(Debug)]
pub struct Metrics {
pub(crate) write_type: WriteType,
pub(crate) iter_source: Duration,
pub(crate) write_batch: Duration,
pub(crate) update_index: Duration,
pub(crate) upload_parquet: Duration,
pub(crate) upload_puffin: Duration,
}
impl Metrics {
pub(crate) fn new(write_type: WriteType) -> Self {
Self {
write_type,
iter_source: Default::default(),
write_batch: Default::default(),
update_index: Default::default(),
upload_parquet: Default::default(),
upload_puffin: Default::default(),
}
}
pub(crate) fn merge(mut self, other: Self) -> Self {
assert_eq!(self.write_type, other.write_type);
self.iter_source += other.iter_source;
self.write_batch += other.write_batch;
self.update_index += other.update_index;
self.upload_parquet += other.upload_parquet;
self.upload_puffin += other.upload_puffin;
self
}
pub(crate) fn observe(self) {
match self.write_type {
WriteType::Flush => {
FLUSH_ELAPSED
.with_label_values(&["iter_source"])
.observe(self.iter_source.as_secs_f64());
FLUSH_ELAPSED
.with_label_values(&["write_batch"])
.observe(self.write_batch.as_secs_f64());
FLUSH_ELAPSED
.with_label_values(&["update_index"])
.observe(self.update_index.as_secs_f64());
FLUSH_ELAPSED
.with_label_values(&["upload_parquet"])
.observe(self.upload_parquet.as_secs_f64());
FLUSH_ELAPSED
.with_label_values(&["upload_puffin"])
.observe(self.upload_puffin.as_secs_f64());
}
WriteType::Compaction => {
COMPACTION_STAGE_ELAPSED
.with_label_values(&["iter_source"])
.observe(self.iter_source.as_secs_f64());
COMPACTION_STAGE_ELAPSED
.with_label_values(&["write_batch"])
.observe(self.write_batch.as_secs_f64());
COMPACTION_STAGE_ELAPSED
.with_label_values(&["update_index"])
.observe(self.update_index.as_secs_f64());
COMPACTION_STAGE_ELAPSED
.with_label_values(&["upload_parquet"])
.observe(self.upload_parquet.as_secs_f64());
COMPACTION_STAGE_ELAPSED
.with_label_values(&["upload_puffin"])
.observe(self.upload_puffin.as_secs_f64());
}
};
}
}
/// A layer to access SST files under the same directory.
pub struct AccessLayer {
region_dir: String,
@@ -96,6 +179,11 @@ impl AccessLayer {
&self.puffin_manager_factory
}
/// Returns the intermediate manager.
pub fn intermediate_manager(&self) -> &IntermediateManager {
&self.intermediate_manager
}
/// Deletes a SST file (and its index file if it has one) with given file id.
pub(crate) async fn delete_sst(&self, file_meta: &FileMeta) -> Result<()> {
let path = location::sst_file_path(&self.region_dir, file_meta.file_id);
@@ -129,11 +217,12 @@ impl AccessLayer {
&self,
request: SstWriteRequest,
write_opts: &WriteOptions,
) -> Result<SstInfoArray> {
write_type: WriteType,
) -> Result<(SstInfoArray, Metrics)> {
let region_id = request.metadata.region_id;
let cache_manager = request.cache_manager.clone();
let sst_info = if let Some(write_cache) = cache_manager.write_cache() {
let (sst_info, metrics) = if let Some(write_cache) = cache_manager.write_cache() {
// Write to the write cache.
write_cache
.write_and_upload_sst(
@@ -145,6 +234,7 @@ impl AccessLayer {
remote_store: self.object_store.clone(),
},
write_opts,
write_type,
)
.await?
} else {
@@ -173,12 +263,15 @@ impl AccessLayer {
request.metadata,
indexer_builder,
path_provider,
Metrics::new(write_type),
)
.await
.with_file_cleaner(cleaner);
writer
let ssts = writer
.write_all(request.source, request.max_sequence, write_opts)
.await?
.await?;
let metrics = writer.into_metrics();
(ssts, metrics)
};
// Put parquet metadata to cache manager.
@@ -194,7 +287,7 @@ impl AccessLayer {
}
}
Ok(sst_info)
Ok((sst_info, metrics))
}
}

View File

@@ -23,6 +23,7 @@ pub(crate) mod test_util;
pub(crate) mod write_cache;
use std::mem;
use std::ops::Range;
use std::sync::Arc;
use bytes::Bytes;
@@ -32,7 +33,6 @@ use index::bloom_filter_index::{BloomFilterIndexCache, BloomFilterIndexCacheRef}
use index::result_cache::IndexResultCache;
use moka::notification::RemovalCause;
use moka::sync::Cache;
use parquet::column::page::Page;
use parquet::file::metadata::ParquetMetaData;
use puffin::puffin_manager::cache::{PuffinMetadataCache, PuffinMetadataCacheRef};
use store_api::storage::{ConcreteDataType, RegionId, TimeSeriesRowSelector};
@@ -674,49 +674,33 @@ pub struct ColumnPagePath {
column_idx: usize,
}
/// Cache key for pages of a SST row group.
/// Cache key to pages in a row group (after projection).
///
/// Different projections will have different cache keys.
/// We cache all ranges together because they may refer to the same `Bytes`.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub enum PageKey {
/// Cache key for a compressed page in a row group.
Compressed(ColumnPagePath),
/// Cache key for all uncompressed pages in a row group.
Uncompressed(ColumnPagePath),
pub struct PageKey {
/// Id of the SST file to cache.
file_id: FileId,
/// Index of the row group.
row_group_idx: usize,
/// Byte ranges of the pages to cache.
ranges: Vec<Range<u64>>,
}
impl PageKey {
/// Creates a key for a compressed page.
pub fn new_compressed(
region_id: RegionId,
file_id: FileId,
row_group_idx: usize,
column_idx: usize,
) -> PageKey {
PageKey::Compressed(ColumnPagePath {
region_id,
/// Creates a key for a list of pages.
pub fn new(file_id: FileId, row_group_idx: usize, ranges: Vec<Range<u64>>) -> PageKey {
PageKey {
file_id,
row_group_idx,
column_idx,
})
}
/// Creates a key for all uncompressed pages in a row group.
pub fn new_uncompressed(
region_id: RegionId,
file_id: FileId,
row_group_idx: usize,
column_idx: usize,
) -> PageKey {
PageKey::Uncompressed(ColumnPagePath {
region_id,
file_id,
row_group_idx,
column_idx,
})
ranges,
}
}
/// Returns memory used by the key (estimated).
fn estimated_size(&self) -> usize {
mem::size_of::<Self>()
mem::size_of::<Self>() + mem::size_of_val(self.ranges.as_slice())
}
}
@@ -724,38 +708,26 @@ impl PageKey {
// We don't use enum here to make it easier to mock and use the struct.
#[derive(Default)]
pub struct PageValue {
/// Compressed page of the column in the row group.
pub compressed: Bytes,
/// All pages of the column in the row group.
pub row_group: Vec<Page>,
/// Compressed page in the row group.
pub compressed: Vec<Bytes>,
/// Total size of the pages (may be larger than sum of compressed bytes due to gaps).
pub page_size: u64,
}
impl PageValue {
/// Creates a new value from a compressed page.
pub fn new_compressed(bytes: Bytes) -> PageValue {
/// Creates a new value from a range of compressed pages.
pub fn new(bytes: Vec<Bytes>, page_size: u64) -> PageValue {
PageValue {
compressed: bytes,
row_group: vec![],
}
}
/// Creates a new value from all pages in a row group.
pub fn new_row_group(pages: Vec<Page>) -> PageValue {
PageValue {
compressed: Bytes::new(),
row_group: pages,
page_size,
}
}
/// Returns memory used by the value (estimated).
fn estimated_size(&self) -> usize {
mem::size_of::<Self>()
+ self.compressed.len()
+ self
.row_group
.iter()
.map(|page| page.buffer().len())
.sum::<usize>()
+ self.page_size as usize
+ self.compressed.iter().map(mem::size_of_val).sum::<usize>()
}
}
@@ -834,7 +806,7 @@ mod tests {
.get_repeated_vector(&ConcreteDataType::int64_datatype(), &value)
.is_none());
let key = PageKey::new_uncompressed(region_id, file_id, 0, 0);
let key = PageKey::new(file_id, 1, vec![Range { start: 0, end: 5 }]);
let pages = Arc::new(PageValue::default());
cache.put_pages(key.clone(), pages);
assert!(cache.get_pages(&key).is_none());
@@ -882,9 +854,8 @@ mod tests {
#[test]
fn test_page_cache() {
let cache = CacheManager::builder().page_cache_size(1000).build();
let region_id = RegionId::new(1, 1);
let file_id = FileId::random();
let key = PageKey::new_compressed(region_id, file_id, 0, 0);
let key = PageKey::new(file_id, 0, vec![(0..10), (10..20)]);
assert!(cache.get_pages(&key).is_none());
let pages = Arc::new(PageValue::default());
cache.put_pages(key.clone(), pages);

View File

@@ -212,6 +212,8 @@ where
}
fn put_page(&self, key: K, page_key: PageKey, value: Bytes) {
// Clones the value to ensure it doesn't reference a larger buffer.
let value = Bytes::from(value.to_vec());
CACHE_BYTES
.with_label_values(&[INDEX_CONTENT_TYPE])
.add((self.weight_of_content)(&(key, page_key), &value).into());

View File

@@ -15,10 +15,9 @@
use std::ops::Range;
use std::sync::Arc;
use api::v1::index::BloomFilterMeta;
use api::v1::index::{BloomFilterLoc, BloomFilterMeta};
use async_trait::async_trait;
use bytes::Bytes;
use futures::future::try_join_all;
use index::bloom_filter::error::Result;
use index::bloom_filter::reader::BloomFilterReader;
use store_api::storage::ColumnId;
@@ -57,11 +56,17 @@ impl BloomFilterIndexCache {
/// Calculates weight for bloom filter index metadata.
fn bloom_filter_index_metadata_weight(
k: &(FileId, ColumnId, Tag),
_: &Arc<BloomFilterMeta>,
meta: &Arc<BloomFilterMeta>,
) -> u32 {
(k.0.as_bytes().len()
let base = k.0.as_bytes().len()
+ std::mem::size_of::<ColumnId>()
+ std::mem::size_of::<BloomFilterMeta>()) as u32
+ std::mem::size_of::<Tag>()
+ std::mem::size_of::<BloomFilterMeta>();
let vec_estimated = meta.segment_loc_indices.len() * std::mem::size_of::<u64>()
+ meta.bloom_filter_locs.len() * std::mem::size_of::<BloomFilterLoc>();
(base + vec_estimated) as u32
}
/// Calculates weight for bloom filter index content.
@@ -120,21 +125,24 @@ impl<R: BloomFilterReader + Send> BloomFilterReader for CachedBloomFilterIndexBl
}
async fn read_vec(&self, ranges: &[Range<u64>]) -> Result<Vec<Bytes>> {
let fetch = ranges.iter().map(|range| {
let mut pages = Vec::with_capacity(ranges.len());
for range in ranges {
let inner = &self.inner;
self.cache.get_or_load(
(self.file_id, self.column_id, self.tag),
self.blob_size,
range.start,
(range.end - range.start) as u32,
move |ranges| async move { inner.read_vec(&ranges).await },
)
});
Ok(try_join_all(fetch)
.await?
.into_iter()
.map(Bytes::from)
.collect::<Vec<_>>())
let page = self
.cache
.get_or_load(
(self.file_id, self.column_id, self.tag),
self.blob_size,
range.start,
(range.end - range.start) as u32,
move |ranges| async move { inner.read_vec(&ranges).await },
)
.await?;
pages.push(Bytes::from(page));
}
Ok(pages)
}
/// Reads the meta information of the bloom filter.
@@ -165,6 +173,45 @@ mod test {
const FUZZ_REPEAT_TIMES: usize = 100;
#[test]
fn bloom_filter_metadata_weight_counts_vec_contents() {
let file_id = FileId::parse_str("00000000-0000-0000-0000-000000000001").unwrap();
let column_id: ColumnId = 42;
let tag = Tag::Skipping;
let meta = BloomFilterMeta {
rows_per_segment: 128,
segment_count: 2,
row_count: 256,
bloom_filter_size: 1024,
segment_loc_indices: vec![0, 64, 128, 192],
bloom_filter_locs: vec![
BloomFilterLoc {
offset: 0,
size: 512,
element_count: 1000,
},
BloomFilterLoc {
offset: 512,
size: 512,
element_count: 1000,
},
],
};
let weight =
bloom_filter_index_metadata_weight(&(file_id, column_id, tag), &Arc::new(meta.clone()));
let base = file_id.as_bytes().len()
+ std::mem::size_of::<ColumnId>()
+ std::mem::size_of::<Tag>()
+ std::mem::size_of::<BloomFilterMeta>();
let expected_dynamic = meta.segment_loc_indices.len() * std::mem::size_of::<u64>()
+ meta.bloom_filter_locs.len() * std::mem::size_of::<BloomFilterLoc>();
assert_eq!(weight as usize, base + expected_dynamic);
}
#[test]
fn fuzz_index_calculation() {
let mut rng = rand::rng();

View File

@@ -18,7 +18,6 @@ use std::sync::Arc;
use api::v1::index::InvertedIndexMetas;
use async_trait::async_trait;
use bytes::Bytes;
use futures::future::try_join_all;
use index::inverted_index::error::Result;
use index::inverted_index::format::reader::InvertedIndexReader;
use prost::Message;
@@ -93,21 +92,24 @@ impl<R: InvertedIndexReader> InvertedIndexReader for CachedInvertedIndexBlobRead
}
async fn read_vec(&self, ranges: &[Range<u64>]) -> Result<Vec<Bytes>> {
let fetch = ranges.iter().map(|range| {
let mut pages = Vec::with_capacity(ranges.len());
for range in ranges {
let inner = &self.inner;
self.cache.get_or_load(
self.file_id,
self.blob_size,
range.start,
(range.end - range.start) as u32,
move |ranges| async move { inner.read_vec(&ranges).await },
)
});
Ok(try_join_all(fetch)
.await?
.into_iter()
.map(Bytes::from)
.collect::<Vec<_>>())
let page = self
.cache
.get_or_load(
self.file_id,
self.blob_size,
range.start,
(range.end - range.start) as u32,
move |ranges| async move { inner.read_vec(&ranges).await },
)
.await?;
pages.push(Bytes::from(page));
}
Ok(pages)
}
async fn metadata(&self) -> Result<Arc<InvertedIndexMetas>> {

View File

@@ -15,7 +15,7 @@
//! A write-through cache for remote object stores.
use std::sync::Arc;
use std::time::Duration;
use std::time::{Duration, Instant};
use common_base::readable_size::ReadableSize;
use common_telemetry::{debug, info};
@@ -25,14 +25,13 @@ use snafu::ResultExt;
use store_api::storage::RegionId;
use crate::access_layer::{
new_fs_cache_store, FilePathProvider, RegionFilePathFactory, SstInfoArray, SstWriteRequest,
TempFileCleaner, WriteCachePathProvider,
new_fs_cache_store, FilePathProvider, Metrics, RegionFilePathFactory, SstInfoArray,
SstWriteRequest, TempFileCleaner, WriteCachePathProvider, WriteType,
};
use crate::cache::file_cache::{FileCache, FileCacheRef, FileType, IndexKey, IndexValue};
use crate::error::{self, Result};
use crate::metrics::{
FLUSH_ELAPSED, UPLOAD_BYTES_TOTAL, WRITE_CACHE_DOWNLOAD_BYTES_TOTAL,
WRITE_CACHE_DOWNLOAD_ELAPSED,
UPLOAD_BYTES_TOTAL, WRITE_CACHE_DOWNLOAD_BYTES_TOTAL, WRITE_CACHE_DOWNLOAD_ELAPSED,
};
use crate::sst::index::intermediate::IntermediateManager;
use crate::sst::index::puffin_manager::PuffinManagerFactory;
@@ -107,11 +106,8 @@ impl WriteCache {
write_request: SstWriteRequest,
upload_request: SstUploadRequest,
write_opts: &WriteOptions,
) -> Result<SstInfoArray> {
let timer = FLUSH_ELAPSED
.with_label_values(&["write_sst"])
.start_timer();
write_type: WriteType,
) -> Result<(SstInfoArray, Metrics)> {
let region_id = write_request.metadata.region_id;
let store = self.file_cache.local_store();
@@ -137,6 +133,7 @@ impl WriteCache {
write_request.metadata,
indexer,
path_provider.clone(),
Metrics::new(write_type),
)
.await
.with_file_cleaner(cleaner);
@@ -144,12 +141,11 @@ impl WriteCache {
let sst_info = writer
.write_all(write_request.source, write_request.max_sequence, write_opts)
.await?;
timer.stop_and_record();
let mut metrics = writer.into_metrics();
// Upload sst file to remote object store.
if sst_info.is_empty() {
return Ok(sst_info);
return Ok((sst_info, metrics));
}
let mut upload_tracker = UploadTracker::new(region_id);
@@ -160,10 +156,12 @@ impl WriteCache {
let parquet_path = upload_request
.dest_path_provider
.build_sst_file_path(sst.file_id);
let start = Instant::now();
if let Err(e) = self.upload(parquet_key, &parquet_path, remote_store).await {
err = Some(e);
break;
}
metrics.upload_parquet += start.elapsed();
upload_tracker.push_uploaded_file(parquet_path);
if sst.index_metadata.file_size > 0 {
@@ -171,10 +169,12 @@ impl WriteCache {
let puffin_path = upload_request
.dest_path_provider
.build_index_file_path(sst.file_id);
let start = Instant::now();
if let Err(e) = self.upload(puffin_key, &puffin_path, remote_store).await {
err = Some(e);
break;
}
metrics.upload_puffin += start.elapsed();
upload_tracker.push_uploaded_file(puffin_path);
}
}
@@ -187,7 +187,7 @@ impl WriteCache {
return Err(err);
}
Ok(sst_info)
Ok((sst_info, metrics))
}
/// Removes a file from the cache by `index_key`.
@@ -297,13 +297,7 @@ impl WriteCache {
let file_type = index_key.file_type;
let cache_path = self.file_cache.cache_file_path(index_key);
let timer = FLUSH_ELAPSED
.with_label_values(&[match file_type {
FileType::Parquet => "upload_parquet",
FileType::Puffin => "upload_puffin",
}])
.start_timer();
let start = Instant::now();
let cached_value = self
.file_cache
.local_store()
@@ -347,11 +341,11 @@ impl WriteCache {
UPLOAD_BYTES_TOTAL.inc_by(bytes_written);
debug!(
"Successfully upload file to remote, region: {}, file: {}, upload_path: {}, cost: {:?}s",
"Successfully upload file to remote, region: {}, file: {}, upload_path: {}, cost: {:?}",
region_id,
file_id,
upload_path,
timer.stop_and_record()
start.elapsed(),
);
let index_value = IndexValue {
@@ -493,11 +487,11 @@ mod tests {
};
// Write to cache and upload sst to mock remote store
let sst_info = write_cache
.write_and_upload_sst(write_request, upload_request, &write_opts)
let (mut sst_infos, _) = write_cache
.write_and_upload_sst(write_request, upload_request, &write_opts, WriteType::Flush)
.await
.unwrap()
.remove(0); //todo(hl): we assume it only creates one file.
.unwrap();
let sst_info = sst_infos.remove(0);
let file_id = sst_info.file_id;
let sst_upload_path = path_provider.build_sst_file_path(file_id);
@@ -586,11 +580,11 @@ mod tests {
remote_store: mock_store.clone(),
};
let sst_info = write_cache
.write_and_upload_sst(write_request, upload_request, &write_opts)
let (mut sst_infos, _) = write_cache
.write_and_upload_sst(write_request, upload_request, &write_opts, WriteType::Flush)
.await
.unwrap()
.remove(0);
.unwrap();
let sst_info = sst_infos.remove(0);
let write_parquet_metadata = sst_info.file_metadata.unwrap();
// Read metadata from write cache
@@ -661,7 +655,7 @@ mod tests {
};
write_cache
.write_and_upload_sst(write_request, upload_request, &write_opts)
.write_and_upload_sst(write_request, upload_request, &write_opts, WriteType::Flush)
.await
.unwrap_err();
let atomic_write_dir = write_cache_dir.path().join(ATOMIC_WRITE_DIR);

View File

@@ -294,6 +294,7 @@ impl CompactionScheduler {
&options,
&request.current_version.options.compaction,
request.current_version.options.append_mode,
Some(self.engine_config.max_background_compactions),
);
let region_id = request.region_id();
let CompactionRequest {

View File

@@ -27,7 +27,7 @@ use snafu::{OptionExt, ResultExt};
use store_api::metadata::RegionMetadataRef;
use store_api::storage::RegionId;
use crate::access_layer::{AccessLayer, AccessLayerRef, OperationType, SstWriteRequest};
use crate::access_layer::{AccessLayer, AccessLayerRef, OperationType, SstWriteRequest, WriteType};
use crate::cache::{CacheManager, CacheManagerRef};
use crate::compaction::picker::{new_picker, PickerOutput};
use crate::compaction::{find_ttl, CompactionSstReaderBuilder};
@@ -339,7 +339,7 @@ impl Compactor for DefaultCompactor {
}
.build_sst_reader()
.await?;
let output_files = sst_layer
let (sst_infos, metrics) = sst_layer
.write_sst(
SstWriteRequest {
op_type: OperationType::Compact,
@@ -354,8 +354,10 @@ impl Compactor for DefaultCompactor {
bloom_filter_index_config,
},
&write_opts,
WriteType::Compaction,
)
.await?
.await?;
let output_files = sst_infos
.into_iter()
.map(|sst_info| FileMeta {
region_id,
@@ -373,9 +375,10 @@ impl Compactor for DefaultCompactor {
let output_file_names =
output_files.iter().map(|f| f.file_id.to_string()).join(",");
info!(
"Region {} compaction inputs: [{}], outputs: [{}]",
region_id, input_file_names, output_file_names
"Region {} compaction inputs: [{}], outputs: [{}], metrics: {:?}",
region_id, input_file_names, output_file_names, metrics
);
metrics.observe();
Ok(output_files)
});
}
@@ -448,6 +451,7 @@ impl Compactor for DefaultCompactor {
&compact_request_options,
&compaction_region.region_options.compaction,
compaction_region.region_options.append_mode,
None,
)
.pick(compaction_region);

View File

@@ -125,6 +125,7 @@ pub fn new_picker(
compact_request_options: &compact_request::Options,
compaction_options: &CompactionOptions,
append_mode: bool,
max_background_tasks: Option<usize>,
) -> Arc<dyn Picker> {
if let compact_request::Options::StrictWindow(window) = compact_request_options {
let window = if window.window_seconds == 0 {
@@ -140,6 +141,7 @@ pub fn new_picker(
time_window_seconds: twcs_opts.time_window_seconds(),
max_output_file_size: twcs_opts.max_output_file_size.map(|r| r.as_bytes()),
append_mode,
max_background_tasks,
}) as Arc<_>,
}
}

View File

@@ -163,6 +163,10 @@ impl FileGroup {
self.files.push(file);
}
pub(crate) fn num_files(&self) -> usize {
self.files.len()
}
#[cfg(test)]
pub(crate) fn files(&self) -> &[FileHandle] {
&self.files[..]

View File

@@ -42,6 +42,25 @@ pub fn new_file_handle_with_sequence(
end_ts_millis: i64,
level: Level,
sequence: u64,
) -> FileHandle {
new_file_handle_with_size_and_sequence(
file_id,
start_ts_millis,
end_ts_millis,
level,
sequence,
0,
)
}
/// Test util to create file handles with custom size.
pub fn new_file_handle_with_size_and_sequence(
file_id: FileId,
start_ts_millis: i64,
end_ts_millis: i64,
level: Level,
sequence: u64,
file_size: u64,
) -> FileHandle {
let file_purger = new_noop_file_purger();
FileHandle::new(
@@ -53,7 +72,7 @@ pub fn new_file_handle_with_sequence(
Timestamp::new_millisecond(end_ts_millis),
),
level,
file_size: 0,
file_size,
available_indexes: Default::default(),
index_file_size: 0,
num_rows: 0,

View File

@@ -16,9 +16,11 @@ use std::collections::hash_map::Entry;
use std::collections::{BTreeMap, HashMap};
use std::fmt::Debug;
use std::num::NonZeroU64;
use std::str::FromStr;
use common_base::readable_size::ReadableSize;
use common_telemetry::info;
use common_telemetry::tracing::warn;
use common_telemetry::{debug, info};
use common_time::timestamp::TimeUnit;
use common_time::timestamp_millis::BucketAligned;
use common_time::Timestamp;
@@ -36,6 +38,9 @@ use crate::sst::version::LevelMeta;
const LEVEL_COMPACTED: Level = 1;
/// Default value for max compaction input file num.
const DEFAULT_MAX_INPUT_FILE_NUM: usize = 32;
/// `TwcsPicker` picks files of which the max timestamp are in the same time window as compaction
/// candidates.
#[derive(Debug)]
@@ -48,6 +53,8 @@ pub struct TwcsPicker {
pub max_output_file_size: Option<u64>,
/// Whether the target region is in append mode.
pub append_mode: bool,
/// Max background compaction tasks.
pub max_background_tasks: Option<usize>,
}
impl TwcsPicker {
@@ -58,19 +65,52 @@ impl TwcsPicker {
time_windows: &mut BTreeMap<i64, Window>,
active_window: Option<i64>,
) -> Vec<CompactionOutput> {
let max_input_file_num = std::env::var("TWCS_MAX_INPUT_FILE_NUM")
.ok()
.and_then(|v| {
usize::from_str(&v)
.inspect_err(|_| {
warn!("Cannot recognize value for TWCS_MAX_INPUT_FILE_NUM: `{v}`");
})
.ok()
})
.unwrap_or(DEFAULT_MAX_INPUT_FILE_NUM);
debug!("Max compaction file num from env: {:?}", max_input_file_num);
let mut output = vec![];
for (window, files) in time_windows {
if files.files.is_empty() {
continue;
}
let mut files_to_merge: Vec<_> = files.files().cloned().collect();
// Filter out large files in append mode - they won't benefit from compaction
if self.append_mode
&& let Some(max_size) = self.max_output_file_size
{
let (kept_files, ignored_files) = files_to_merge
.into_iter()
.partition(|fg| fg.size() <= max_size as usize);
files_to_merge = kept_files;
info!(
"Compaction for {} skipped {} large files in append mode for region {}, window {}, max_size: {}",
region_id,
ignored_files.len(),
region_id,
window,
max_size
);
}
let sorted_runs = find_sorted_runs(&mut files_to_merge);
let found_runs = sorted_runs.len();
// We only remove deletion markers if we found less than 2 runs and not in append mode.
// because after compaction there will be no overlapping files.
let filter_deleted = !files.overlapping && found_runs <= 2 && !self.append_mode;
if found_runs == 0 {
continue;
}
let inputs = if found_runs > 1 {
let mut inputs = if found_runs > 1 {
reduce_runs(sorted_runs)
} else {
let run = sorted_runs.last().unwrap();
@@ -80,8 +120,30 @@ impl TwcsPicker {
// no overlapping files, try merge small files
merge_seq_files(run.items(), self.max_output_file_size)
};
if !inputs.is_empty() {
let total_input_files: usize = inputs.iter().map(|g| g.num_files()).sum();
if total_input_files > max_input_file_num {
// Sorts file groups by size first.
inputs.sort_unstable_by_key(|fg| fg.size());
let mut num_picked_files = 0;
inputs = inputs
.into_iter()
.take_while(|fg| {
let current_group_file_num = fg.num_files();
if current_group_file_num + num_picked_files <= max_input_file_num {
num_picked_files += current_group_file_num;
true
} else {
false
}
})
.collect::<Vec<_>>();
info!(
"Compaction for region {} enforces max input file num limit: {}, current total: {}, input: {:?}",
region_id, max_input_file_num, total_input_files, inputs
);
}
if inputs.len() > 1 {
// If we have more than one group to compact.
log_pick_result(
region_id,
*window,
@@ -98,6 +160,16 @@ impl TwcsPicker {
filter_deleted,
output_time_range: None, // we do not enforce output time range in twcs compactions.
});
if let Some(max_background_tasks) = self.max_background_tasks
&& output.len() >= max_background_tasks
{
debug!(
"Region ({:?}) compaction task size larger than max background tasks({}), remaining tasks discarded",
region_id, max_background_tasks
);
break;
}
}
}
output
@@ -330,7 +402,9 @@ mod tests {
use std::collections::HashSet;
use super::*;
use crate::compaction::test_util::{new_file_handle, new_file_handle_with_sequence};
use crate::compaction::test_util::{
new_file_handle, new_file_handle_with_sequence, new_file_handle_with_size_and_sequence,
};
use crate::sst::file::{FileId, Level};
#[test]
@@ -645,6 +719,7 @@ mod tests {
time_window_seconds: None,
max_output_file_size: None,
append_mode: false,
max_background_tasks: None,
}
.build_output(RegionId::from_u64(0), &mut windows, active_window);
@@ -756,5 +831,305 @@ mod tests {
.check();
}
#[test]
fn test_append_mode_filter_large_files() {
let file_ids = (0..4).map(|_| FileId::random()).collect::<Vec<_>>();
let max_output_file_size = 1000u64;
// Create files with different sizes
let small_file_1 = new_file_handle_with_size_and_sequence(file_ids[0], 0, 999, 0, 1, 500);
let large_file_1 = new_file_handle_with_size_and_sequence(file_ids[1], 0, 999, 0, 2, 1500);
let small_file_2 = new_file_handle_with_size_and_sequence(file_ids[2], 0, 999, 0, 3, 800);
let large_file_2 = new_file_handle_with_size_and_sequence(file_ids[3], 0, 999, 0, 4, 2000);
// Create file groups (each file is in its own group due to different sequences)
let mut files_to_merge = vec![
FileGroup::new_with_file(small_file_1),
FileGroup::new_with_file(large_file_1),
FileGroup::new_with_file(small_file_2),
FileGroup::new_with_file(large_file_2),
];
// Test filtering logic directly
let original_count = files_to_merge.len();
// Apply append mode filtering
files_to_merge.retain(|fg| fg.size() <= max_output_file_size as usize);
// Should have filtered out 2 large files, leaving 2 small files
assert_eq!(files_to_merge.len(), 2);
assert_eq!(original_count, 4);
// Verify the remaining files are the small ones
for fg in &files_to_merge {
assert!(
fg.size() <= max_output_file_size as usize,
"File size {} should be <= {}",
fg.size(),
max_output_file_size
);
}
}
#[test]
fn test_build_output_multiple_windows_with_zero_runs() {
let file_ids = (0..6).map(|_| FileId::random()).collect::<Vec<_>>();
let files = [
// Window 0: Contains 3 files but not forming any runs (not enough files in sequence to reach trigger_file_num)
new_file_handle_with_sequence(file_ids[0], 0, 999, 0, 1),
new_file_handle_with_sequence(file_ids[1], 0, 999, 0, 2),
new_file_handle_with_sequence(file_ids[2], 0, 999, 0, 3),
// Window 3: Contains files that will form 2 runs
new_file_handle_with_sequence(file_ids[3], 3000, 3999, 0, 4),
new_file_handle_with_sequence(file_ids[4], 3000, 3999, 0, 5),
new_file_handle_with_sequence(file_ids[5], 3000, 3999, 0, 6),
];
let mut windows = assign_to_windows(files.iter(), 3);
// Create picker with trigger_file_num of 4 so single files won't form runs in first window
let picker = TwcsPicker {
trigger_file_num: 4, // High enough to prevent runs in first window
time_window_seconds: Some(3),
max_output_file_size: None,
append_mode: false,
max_background_tasks: None,
};
let active_window = find_latest_window_in_seconds(files.iter(), 3);
let output = picker.build_output(RegionId::from_u64(123), &mut windows, active_window);
assert!(
!output.is_empty(),
"Should have output from windows with runs, even when one window has 0 runs"
);
let all_output_files: Vec<_> = output
.iter()
.flat_map(|o| o.inputs.iter())
.map(|f| f.file_id())
.collect();
assert!(
all_output_files.contains(&file_ids[3])
|| all_output_files.contains(&file_ids[4])
|| all_output_files.contains(&file_ids[5]),
"Output should contain files from the window with runs"
);
}
#[test]
fn test_build_output_single_window_zero_runs() {
let file_ids = (0..2).map(|_| FileId::random()).collect::<Vec<_>>();
let large_file_1 = new_file_handle_with_size_and_sequence(file_ids[0], 0, 999, 0, 1, 2000); // 2000 bytes
let large_file_2 = new_file_handle_with_size_and_sequence(file_ids[1], 0, 999, 0, 2, 2500); // 2500 bytes
let files = [large_file_1, large_file_2];
let mut windows = assign_to_windows(files.iter(), 3);
let picker = TwcsPicker {
trigger_file_num: 2,
time_window_seconds: Some(3),
max_output_file_size: Some(1000),
append_mode: true,
max_background_tasks: None,
};
let active_window = find_latest_window_in_seconds(files.iter(), 3);
let output = picker.build_output(RegionId::from_u64(456), &mut windows, active_window);
// Should return empty output (no compaction needed)
assert!(
output.is_empty(),
"Should return empty output when no runs are found after filtering"
);
}
#[test]
fn test_max_background_tasks_truncation() {
let file_ids = (0..10).map(|_| FileId::random()).collect::<Vec<_>>();
let max_background_tasks = 3;
// Create files across multiple windows that will generate multiple compaction outputs
let files = [
// Window 0: 4 files that will form a run
new_file_handle_with_sequence(file_ids[0], 0, 999, 0, 1),
new_file_handle_with_sequence(file_ids[1], 0, 999, 0, 2),
new_file_handle_with_sequence(file_ids[2], 0, 999, 0, 3),
new_file_handle_with_sequence(file_ids[3], 0, 999, 0, 4),
// Window 3: 4 files that will form another run
new_file_handle_with_sequence(file_ids[4], 3000, 3999, 0, 5),
new_file_handle_with_sequence(file_ids[5], 3000, 3999, 0, 6),
new_file_handle_with_sequence(file_ids[6], 3000, 3999, 0, 7),
new_file_handle_with_sequence(file_ids[7], 3000, 3999, 0, 8),
// Window 6: 4 files that will form another run
new_file_handle_with_sequence(file_ids[8], 6000, 6999, 0, 9),
new_file_handle_with_sequence(file_ids[9], 6000, 6999, 0, 10),
];
let mut windows = assign_to_windows(files.iter(), 3);
let picker = TwcsPicker {
trigger_file_num: 4,
time_window_seconds: Some(3),
max_output_file_size: None,
append_mode: false,
max_background_tasks: Some(max_background_tasks),
};
let active_window = find_latest_window_in_seconds(files.iter(), 3);
let output = picker.build_output(RegionId::from_u64(123), &mut windows, active_window);
// Should have at most max_background_tasks outputs
assert!(
output.len() <= max_background_tasks,
"Output should be truncated to max_background_tasks: expected <= {}, got {}",
max_background_tasks,
output.len()
);
// Without max_background_tasks, should have more outputs
let picker_no_limit = TwcsPicker {
trigger_file_num: 4,
time_window_seconds: Some(3),
max_output_file_size: None,
append_mode: false,
max_background_tasks: None,
};
let mut windows_no_limit = assign_to_windows(files.iter(), 3);
let output_no_limit = picker_no_limit.build_output(
RegionId::from_u64(123),
&mut windows_no_limit,
active_window,
);
// Without limit, should have more outputs (if there are enough windows)
if output_no_limit.len() > max_background_tasks {
assert!(
output_no_limit.len() > output.len(),
"Without limit should have more outputs than with limit"
);
}
}
#[test]
fn test_max_background_tasks_no_truncation_when_under_limit() {
let file_ids = (0..4).map(|_| FileId::random()).collect::<Vec<_>>();
let max_background_tasks = 10; // Larger than expected outputs
// Create files in one window that will generate one compaction output
let files = [
new_file_handle_with_sequence(file_ids[0], 0, 999, 0, 1),
new_file_handle_with_sequence(file_ids[1], 0, 999, 0, 2),
new_file_handle_with_sequence(file_ids[2], 0, 999, 0, 3),
new_file_handle_with_sequence(file_ids[3], 0, 999, 0, 4),
];
let mut windows = assign_to_windows(files.iter(), 3);
let picker = TwcsPicker {
trigger_file_num: 4,
time_window_seconds: Some(3),
max_output_file_size: None,
append_mode: false,
max_background_tasks: Some(max_background_tasks),
};
let active_window = find_latest_window_in_seconds(files.iter(), 3);
let output = picker.build_output(RegionId::from_u64(123), &mut windows, active_window);
// Should have all outputs since we're under the limit
assert!(
output.len() <= max_background_tasks,
"Output should be within limit"
);
// Should have at least one output
assert!(!output.is_empty(), "Should have at least one output");
}
#[test]
fn test_pick_multiple_runs() {
common_telemetry::init_default_ut_logging();
let num_files = 8;
let file_ids = (0..num_files).map(|_| FileId::random()).collect::<Vec<_>>();
// Create files with different sequences so they form multiple runs
let files: Vec<_> = file_ids
.iter()
.enumerate()
.map(|(idx, file_id)| {
new_file_handle_with_size_and_sequence(
*file_id,
0,
999,
0,
(idx + 1) as u64,
1024 * 1024,
)
})
.collect();
let mut windows = assign_to_windows(files.iter(), 3);
let picker = TwcsPicker {
trigger_file_num: 4,
time_window_seconds: Some(3),
max_output_file_size: None,
append_mode: false,
max_background_tasks: None,
};
let active_window = find_latest_window_in_seconds(files.iter(), 3);
let output = picker.build_output(RegionId::from_u64(123), &mut windows, active_window);
assert_eq!(1, output.len());
assert_eq!(output[0].inputs.len(), 2);
}
#[test]
fn test_limit_max_input_files() {
common_telemetry::init_default_ut_logging();
let num_files = 50;
let file_ids = (0..num_files).map(|_| FileId::random()).collect::<Vec<_>>();
// Create files with different sequences so they form 2 runs
let files: Vec<_> = file_ids
.iter()
.enumerate()
.map(|(idx, file_id)| {
new_file_handle_with_size_and_sequence(
*file_id,
(idx / 2 * 10) as i64,
(idx / 2 * 10 + 5) as i64,
0,
(idx + 1) as u64,
1024 * 1024,
)
})
.collect();
let mut windows = assign_to_windows(files.iter(), 3);
let picker = TwcsPicker {
trigger_file_num: 4,
time_window_seconds: Some(3),
max_output_file_size: None,
append_mode: false,
max_background_tasks: None,
};
let active_window = find_latest_window_in_seconds(files.iter(), 3);
let output = picker.build_output(RegionId::from_u64(123), &mut windows, active_window);
assert_eq!(1, output.len());
assert_eq!(output[0].inputs.len(), 32);
}
// TODO(hl): TTL tester that checks if get_expired_ssts function works as expected.
}

View File

@@ -15,6 +15,7 @@
//! Basic tests for mito engine.
use std::collections::HashMap;
use std::sync::atomic::Ordering;
use api::v1::value::ValueData;
use api::v1::{Rows, SemanticType};
@@ -86,11 +87,15 @@ async fn test_write_to_region() {
rows: build_rows(0, 42),
};
put_rows(&engine, region_id, rows).await;
let region = engine.get_region(region_id).unwrap();
assert!(region.written_bytes.load(Ordering::Relaxed) > 0);
}
#[apply(multiple_log_store_factories)]
async fn test_region_replay(factory: Option<LogStoreFactory>) {
use std::sync::atomic::Ordering;
use common_wal::options::{KafkaWalOptions, WalOptions};
common_telemetry::init_default_ut_logging();
@@ -155,6 +160,10 @@ async fn test_region_replay(factory: Option<LogStoreFactory>) {
.unwrap();
assert_eq!(0, result.affected_rows);
// The replay won't update the write bytes rate meter.
let region = engine.get_region(region_id).unwrap();
assert_eq!(region.written_bytes.load(Ordering::Relaxed), 0);
let request = ScanRequest::default();
let stream = engine.scan_to_stream(region_id, request).await.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();

View File

@@ -25,7 +25,7 @@ use store_api::storage::RegionId;
use strum::IntoStaticStr;
use tokio::sync::{mpsc, watch};
use crate::access_layer::{AccessLayerRef, OperationType, SstWriteRequest};
use crate::access_layer::{AccessLayerRef, Metrics, OperationType, SstWriteRequest, WriteType};
use crate::cache::CacheManagerRef;
use crate::config::MitoConfig;
use crate::error::{
@@ -341,6 +341,7 @@ impl RegionFlushTask {
let mut file_metas = Vec::with_capacity(memtables.len());
let mut flushed_bytes = 0;
let mut series_count = 0;
let mut flush_metrics = Metrics::new(WriteType::Flush);
for mem in memtables {
if mem.is_empty() {
// Skip empty memtables.
@@ -367,14 +368,15 @@ impl RegionFlushTask {
bloom_filter_index_config: self.engine_config.bloom_filter_index.clone(),
};
let ssts_written = self
let (ssts_written, metrics) = self
.access_layer
.write_sst(write_request, &write_opts)
.write_sst(write_request, &write_opts, WriteType::Flush)
.await?;
if ssts_written.is_empty() {
// No data written.
continue;
}
flush_metrics = flush_metrics.merge(metrics);
file_metas.extend(ssts_written.into_iter().map(|sst_info| {
flushed_bytes += sst_info.file_size;
@@ -399,13 +401,15 @@ impl RegionFlushTask {
let file_ids: Vec<_> = file_metas.iter().map(|f| f.file_id).collect();
info!(
"Successfully flush memtables, region: {}, reason: {}, files: {:?}, series count: {}, cost: {:?}s",
"Successfully flush memtables, region: {}, reason: {}, files: {:?}, series count: {}, cost: {:?}, metrics: {:?}",
self.region_id,
self.reason.as_str(),
file_ids,
series_count,
timer.stop_and_record(),
flush_metrics,
);
flush_metrics.observe();
let edit = RegionEdit {
files_to_add: file_metas,

View File

@@ -76,7 +76,7 @@ pub struct RegionManifestOptions {
/// -RegionMetadataRef metadata
/// }
/// class RegionEdit {
/// -VersionNumber regoin_version
/// -VersionNumber region_version
/// -Vec~FileMeta~ files_to_add
/// -Vec~FileMeta~ files_to_remove
/// -SequenceNumber flushed_sequence

View File

@@ -119,6 +119,14 @@ lazy_static! {
// Compaction metrics
/// Timer of different stages in compaction.
/// - pick
/// - merge (in parallel)
/// - iter_source
/// - write_batch
/// - update_index
/// - upload_parquet
/// - upload puffin
/// - write_manifest
pub static ref COMPACTION_STAGE_ELAPSED: HistogramVec = register_histogram_vec!(
"greptime_mito_compaction_stage_elapsed",
"mito compaction stage elapsed",

View File

@@ -129,6 +129,8 @@ pub struct MitoRegion {
/// There are no WAL entries in range [flushed_entry_id, topic_latest_entry_id] for current region,
/// which means these WAL entries maybe able to be pruned up to `topic_latest_entry_id`.
pub(crate) topic_latest_entry_id: AtomicU64,
/// The total bytes written to the region.
pub(crate) written_bytes: Arc<AtomicU64>,
/// Memtable builder for the region.
pub(crate) memtable_builder: MemtableBuilderRef,
/// manifest stats
@@ -313,6 +315,7 @@ impl MitoRegion {
let manifest_version = self.stats.manifest_version();
let topic_latest_entry_id = self.topic_latest_entry_id.load(Ordering::Relaxed);
let written_bytes = self.written_bytes.load(Ordering::Relaxed);
RegionStatistic {
num_rows,
@@ -327,6 +330,7 @@ impl MitoRegion {
},
data_topic_latest_entry_id: topic_latest_entry_id,
metadata_topic_latest_entry_id: topic_latest_entry_id,
written_bytes,
}
}

View File

@@ -277,6 +277,7 @@ impl RegionOpener {
time_provider: self.time_provider.clone(),
topic_latest_entry_id: AtomicU64::new(0),
memtable_builder,
written_bytes: Arc::new(AtomicU64::new(0)),
stats: self.stats,
})
}
@@ -455,6 +456,7 @@ impl RegionOpener {
last_compaction_millis: AtomicI64::new(now),
time_provider: self.time_provider.clone(),
topic_latest_entry_id: AtomicU64::new(0),
written_bytes: Arc::new(AtomicU64::new(0)),
memtable_builder,
stats: self.stats.clone(),
};
@@ -634,7 +636,7 @@ where
last_entry_id = last_entry_id.max(entry_id);
let mut region_write_ctx =
RegionWriteCtx::new(region_id, version_control, provider.clone());
RegionWriteCtx::new(region_id, version_control, provider.clone(), None);
for mutation in entry.mutations {
rows_replayed += mutation
.rows

View File

@@ -354,7 +354,7 @@ impl VersionBuilder {
self
}
/// Sets truncated entty id.
/// Sets truncated entry id.
pub(crate) fn truncated_entry_id(mut self, entry_id: Option<EntryId>) -> Self {
self.truncated_entry_id = entry_id;
self

View File

@@ -13,6 +13,7 @@
// limitations under the License.
use std::mem;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;
use api::v1::{BulkWalEntry, Mutation, OpType, Rows, WalEntry, WriteHint};
@@ -106,6 +107,8 @@ pub(crate) struct RegionWriteCtx {
pub(crate) put_num: usize,
/// Rows to delete.
pub(crate) delete_num: usize,
/// The total bytes written to the region.
pub(crate) written_bytes: Option<Arc<AtomicU64>>,
}
impl RegionWriteCtx {
@@ -114,6 +117,7 @@ impl RegionWriteCtx {
region_id: RegionId,
version_control: &VersionControlRef,
provider: Provider,
written_bytes: Option<Arc<AtomicU64>>,
) -> RegionWriteCtx {
let VersionControlData {
version,
@@ -136,6 +140,7 @@ impl RegionWriteCtx {
put_num: 0,
delete_num: 0,
bulk_parts: vec![],
written_bytes,
}
}
@@ -214,6 +219,12 @@ impl RegionWriteCtx {
}
let mutable = self.version.memtables.mutable.clone();
let prev_memory_usage = if self.written_bytes.is_some() {
Some(mutable.memory_usage())
} else {
None
};
let mutations = mem::take(&mut self.wal_entry.mutations)
.into_iter()
.enumerate()
@@ -246,6 +257,11 @@ impl RegionWriteCtx {
}
}
if let Some(written_bytes) = &self.written_bytes {
let new_memory_usage = mutable.memory_usage();
let bytes = new_memory_usage.saturating_sub(prev_memory_usage.unwrap_or_default());
written_bytes.fetch_add(bytes as u64, Ordering::Relaxed);
}
// Updates region sequence and entry id. Since we stores last sequence and entry id in region, we need
// to decrease `next_sequence` and `next_entry_id` by 1.
self.version_control
@@ -271,6 +287,13 @@ impl RegionWriteCtx {
.with_label_values(&["write_bulk"])
.start_timer();
let mutable_memtable = &self.version.memtables.mutable;
let prev_memory_usage = if self.written_bytes.is_some() {
Some(mutable_memtable.memory_usage())
} else {
None
};
if self.bulk_parts.len() == 1 {
let part = self.bulk_parts.swap_remove(0);
let num_rows = part.num_rows();
@@ -300,6 +323,11 @@ impl RegionWriteCtx {
}
}
if let Some(written_bytes) = &self.written_bytes {
let new_memory_usage = mutable_memtable.memory_usage();
let bytes = new_memory_usage.saturating_sub(prev_memory_usage.unwrap_or_default());
written_bytes.fetch_add(bytes as u64, Ordering::Relaxed);
}
self.version_control
.set_sequence_and_entry_id(self.next_sequence - 1, self.next_entry_id - 1);
}

View File

@@ -289,6 +289,10 @@ impl FileHandle {
pub fn num_rows(&self) -> usize {
self.inner.meta.num_rows as usize
}
pub fn level(&self) -> Level {
self.inner.meta.level
}
}
/// Inner data of [FileHandle].

View File

@@ -131,6 +131,14 @@ impl FilePurger for LocalFilePurger {
error!(e; "Failed to purge stager with index file, file_id: {}, region: {}",
file_meta.file_id, file_meta.region_id);
}
let file_id = file_meta.file_id;
if let Err(e) = sst_layer
.intermediate_manager()
.prune_sst_dir(&file_meta.region_id, &file_id)
.await
{
error!(e; "Failed to prune intermediate sst directory, region_id: {}, file_id: {}", file_meta.region_id, file_id);
}
})) {
error!(e; "Failed to schedule the file purge request");
}

View File

@@ -110,6 +110,7 @@ pub struct Indexer {
last_mem_fulltext_index: usize,
bloom_filter_indexer: Option<BloomFilterIndexer>,
last_mem_bloom_filter: usize,
intermediate_manager: Option<IntermediateManager>,
}
impl Indexer {
@@ -196,6 +197,7 @@ impl IndexerBuilder for IndexerBuilderImpl {
indexer.inverted_indexer = self.build_inverted_indexer(file_id);
indexer.fulltext_indexer = self.build_fulltext_indexer(file_id).await;
indexer.bloom_filter_indexer = self.build_bloom_filter_indexer(file_id);
indexer.intermediate_manager = Some(self.intermediate_manager.clone());
if indexer.inverted_indexer.is_none()
&& indexer.fulltext_indexer.is_none()
&& indexer.bloom_filter_indexer.is_none()

View File

@@ -22,7 +22,8 @@ use common_telemetry::warn;
use index::bloom_filter::applier::{BloomFilterApplier, InListPredicate};
use index::bloom_filter::reader::BloomFilterReaderImpl;
use index::fulltext_index::search::{FulltextIndexSearcher, RowId, TantivyFulltextIndexSearcher};
use index::fulltext_index::Config;
use index::fulltext_index::tokenizer::{ChineseTokenizer, EnglishTokenizer, Tokenizer};
use index::fulltext_index::{Analyzer, Config};
use object_store::ObjectStore;
use puffin::puffin_manager::cache::PuffinMetadataCacheRef;
use puffin::puffin_manager::{GuardWithMetadata, PuffinManager, PuffinReader};
@@ -393,21 +394,7 @@ impl FulltextIndexApplier {
// lowercased terms are not indexed
continue;
}
let ts = term
.term
.split(|c: char| !c.is_alphanumeric())
.filter(|&t| !t.is_empty())
.map(|t| {
if !config.case_sensitive {
t.to_lowercase()
} else {
t.to_string()
}
.into_bytes()
});
probes.extend(ts);
probes.extend(Self::term_to_probes(&term.term, config));
}
probes
@@ -417,6 +404,22 @@ impl FulltextIndexApplier {
})
.collect::<Vec<_>>()
}
fn term_to_probes<'a>(term: &'a str, config: &'a Config) -> impl Iterator<Item = Vec<u8>> + 'a {
let tokens = match config.analyzer {
Analyzer::English => EnglishTokenizer {}.tokenize(term),
Analyzer::Chinese => ChineseTokenizer {}.tokenize(term),
};
tokens.into_iter().map(|t| {
if !config.case_sensitive {
t.to_lowercase()
} else {
t.to_string()
}
.into_bytes()
})
}
}
/// The source of the index.

View File

@@ -21,6 +21,7 @@ impl Indexer {
self.do_abort_inverted_index().await;
self.do_abort_fulltext_index().await;
self.do_abort_bloom_filter().await;
self.do_prune_intm_sst_dir().await;
self.puffin_manager = None;
}

View File

@@ -53,6 +53,7 @@ impl Indexer {
return IndexOutput::default();
}
self.do_prune_intm_sst_dir().await;
output.file_size = self.do_finish_puffin_writer(writer).await;
output
}
@@ -266,4 +267,12 @@ impl Indexer {
output.row_count = row_count;
output.columns = column_ids;
}
pub(crate) async fn do_prune_intm_sst_dir(&mut self) {
if let Some(manager) = self.intermediate_manager.take() {
if let Err(e) = manager.prune_sst_dir(&self.region_id, &self.file_id).await {
warn!(e; "Failed to prune intermediate SST directory, region_id: {}, file_id: {}", self.region_id, self.file_id);
}
}
}
}

View File

@@ -54,14 +54,22 @@ impl IntermediateManager {
aux_path.as_ref()
);
// Remove the intermediate directory on bankground
let aux_pb = PathBuf::from(aux_path.as_ref());
let intm_dir = aux_pb.join(INTERMEDIATE_DIR);
let deleted_dir = intm_dir.with_extension(format!("deleted-{}", Uuid::new_v4()));
if let Err(err) = tokio::fs::rename(&intm_dir, &deleted_dir).await {
warn!(err; "Failed to rename intermediate directory");
}
tokio::spawn(async move {
if let Err(err) = tokio::fs::remove_dir_all(deleted_dir).await {
warn!(err; "Failed to remove intermediate directory");
}
});
let store = new_fs_cache_store(&normalize_dir(aux_path.as_ref())).await?;
let store = InstrumentedStore::new(store);
// Remove all garbage intermediate files from previous runs.
if let Err(err) = store.remove_all(INTERMEDIATE_DIR).await {
warn!(err; "Failed to remove garbage intermediate files");
}
Ok(Self {
base_dir: PathBuf::from(aux_path.as_ref()),
store,
@@ -94,6 +102,24 @@ impl IntermediateManager {
.join(sst_file_id.to_string())
.join(format!("fulltext-{column_id}-{uuid}"))
}
/// Prunes the intermediate directory for SST files.
pub(crate) async fn prune_sst_dir(
&self,
region_id: &RegionId,
sst_file_id: &FileId,
) -> Result<()> {
let region_id = region_id.as_u64();
let sst_dir = format!("{INTERMEDIATE_DIR}/{region_id}/{sst_file_id}/");
self.store.remove_all(&sst_dir).await
}
/// Prunes the intermediate directory for region files.
pub(crate) async fn prune_region_dir(&self, region_id: &RegionId) -> Result<()> {
let region_id = region_id.as_u64();
let region_dir = format!("{INTERMEDIATE_DIR}/{region_id}/");
self.store.remove_all(&region_dir).await
}
}
/// `IntermediateLocation` produces paths for intermediate files
@@ -268,6 +294,60 @@ mod tests {
.unwrap());
}
#[tokio::test]
async fn test_cleanup_dir() {
let temp_dir = temp_dir::create_temp_dir("test_cleanup_dir_");
let region_id = RegionId::new(0, 0);
let sst_file_id = FileId::random();
let region_dir = temp_dir
.path()
.join(INTERMEDIATE_DIR)
.join(region_id.as_u64().to_string());
let sst_dir = region_dir.join(sst_file_id.to_string());
let path = temp_dir.path().to_str().unwrap();
let manager = IntermediateManager::init_fs(path).await.unwrap();
let location = IntermediateLocation::new(&region_id, &sst_file_id);
let temp_file_provider = TempFileProvider::new(location, manager.clone());
let mut f1 = temp_file_provider
.create("sky", "000000000000")
.await
.unwrap();
f1.write_all(b"hello").await.unwrap();
f1.flush().await.unwrap();
f1.close().await.unwrap();
let mut f2 = temp_file_provider
.create("sky", "000000000001")
.await
.unwrap();
f2.write_all(b"world").await.unwrap();
f2.flush().await.unwrap();
f2.close().await.unwrap();
temp_file_provider.cleanup().await.unwrap();
// sst_dir and region_dir still exists
assert!(tokio::fs::try_exists(&sst_dir).await.unwrap());
assert!(tokio::fs::try_exists(&region_dir).await.unwrap());
// sst_dir should be deleted, region_dir still exists
manager
.prune_sst_dir(&region_id, &sst_file_id)
.await
.unwrap();
assert!(tokio::fs::try_exists(&region_dir).await.unwrap());
assert!(!tokio::fs::try_exists(&sst_dir).await.unwrap());
// sst_dir, region_dir should be deleted
manager.prune_region_dir(&region_id).await.unwrap();
assert!(!tokio::fs::try_exists(&sst_dir).await.unwrap());
assert!(!tokio::fs::try_exists(&region_dir).await.unwrap());
}
#[test]
fn test_intermediate_location() {
let sst_file_id = FileId::random();

View File

@@ -338,7 +338,7 @@ mod tests {
#[test]
fn test_collect_and_basic() {
let (_d, facotry) = PuffinManagerFactory::new_for_test_block("test_collect_and_basic_");
let (_d, factory) = PuffinManagerFactory::new_for_test_block("test_collect_and_basic_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -346,7 +346,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let expr = Expr::BinaryExpr(BinaryExpr {

View File

@@ -70,14 +70,14 @@ mod tests {
#[test]
fn test_collect_between_basic() {
let (_d, facotry) = PuffinManagerFactory::new_for_test_block("test_collect_between_basic_");
let (_d, factory) = PuffinManagerFactory::new_for_test_block("test_collect_between_basic_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
"test".to_string(),
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let between = Between {
@@ -110,7 +110,7 @@ mod tests {
#[test]
fn test_collect_between_negated() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_between_negated_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -118,7 +118,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let between = Between {
@@ -134,7 +134,7 @@ mod tests {
#[test]
fn test_collect_between_field_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_between_field_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -142,7 +142,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let between = Between {
@@ -175,7 +175,7 @@ mod tests {
#[test]
fn test_collect_between_type_mismatch() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_between_type_mismatch_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -183,7 +183,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let between = Between {
@@ -200,7 +200,7 @@ mod tests {
#[test]
fn test_collect_between_nonexistent_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_between_nonexistent_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -208,7 +208,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let between = Between {

View File

@@ -225,7 +225,7 @@ mod tests {
),
];
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_comparison_basic_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -233,7 +233,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
for ((left, op, right), _) in &cases {
@@ -252,7 +252,7 @@ mod tests {
#[test]
fn test_collect_comparison_type_mismatch() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_comparison_type_mismatch_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -260,7 +260,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let res = builder.collect_comparison_expr(&tag_column(), &Operator::Lt, &int64_lit(10));
@@ -270,7 +270,7 @@ mod tests {
#[test]
fn test_collect_comparison_field_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_comparison_field_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -278,7 +278,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
builder
@@ -303,7 +303,7 @@ mod tests {
#[test]
fn test_collect_comparison_nonexistent_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_comparison_nonexistent_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -311,7 +311,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let res = builder.collect_comparison_expr(

View File

@@ -134,14 +134,14 @@ mod tests {
#[test]
fn test_collect_eq_basic() {
let (_d, facotry) = PuffinManagerFactory::new_for_test_block("test_collect_eq_basic_");
let (_d, factory) = PuffinManagerFactory::new_for_test_block("test_collect_eq_basic_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
"test".to_string(),
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
builder
@@ -169,7 +169,7 @@ mod tests {
#[test]
fn test_collect_eq_field_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_eq_field_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -177,7 +177,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
builder
@@ -196,7 +196,7 @@ mod tests {
#[test]
fn test_collect_eq_nonexistent_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_eq_nonexistent_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -204,7 +204,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let res = builder.collect_eq(&nonexistent_column(), &string_lit("abc"));
@@ -214,7 +214,7 @@ mod tests {
#[test]
fn test_collect_eq_type_mismatch() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_eq_type_mismatch_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -222,7 +222,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let res = builder.collect_eq(&tag_column(), &int64_lit(1));
@@ -232,7 +232,7 @@ mod tests {
#[test]
fn test_collect_or_eq_list_basic() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_or_eq_list_basic_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -240,7 +240,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let eq_expr = DfExpr::BinaryExpr(BinaryExpr {
@@ -289,7 +289,7 @@ mod tests {
#[test]
fn test_collect_or_eq_list_invalid_op() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_or_eq_list_invalid_op_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -297,7 +297,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let eq_expr = DfExpr::BinaryExpr(BinaryExpr {
@@ -325,7 +325,7 @@ mod tests {
#[test]
fn test_collect_or_eq_list_multiple_columns() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_or_eq_list_multiple_columns_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -333,7 +333,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let eq_expr = DfExpr::BinaryExpr(BinaryExpr {

View File

@@ -65,14 +65,14 @@ mod tests {
#[test]
fn test_collect_in_list_basic() {
let (_d, facotry) = PuffinManagerFactory::new_for_test_block("test_collect_in_list_basic_");
let (_d, factory) = PuffinManagerFactory::new_for_test_block("test_collect_in_list_basic_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
"test".to_string(),
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let in_list = InList {
@@ -95,7 +95,7 @@ mod tests {
#[test]
fn test_collect_in_list_negated() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_in_list_negated_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -103,7 +103,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let in_list = InList {
@@ -118,7 +118,7 @@ mod tests {
#[test]
fn test_collect_in_list_field_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_in_list_field_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -126,7 +126,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let in_list = InList {
@@ -149,7 +149,7 @@ mod tests {
#[test]
fn test_collect_in_list_type_mismatch() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_in_list_type_mismatch_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -157,7 +157,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let in_list = InList {
@@ -173,7 +173,7 @@ mod tests {
#[test]
fn test_collect_in_list_nonexistent_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_collect_in_list_nonexistent_column_");
let metadata = test_region_metadata();
@@ -182,7 +182,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let in_list = InList {

View File

@@ -57,14 +57,14 @@ mod tests {
#[test]
fn test_regex_match_basic() {
let (_d, facotry) = PuffinManagerFactory::new_for_test_block("test_regex_match_basic_");
let (_d, factory) = PuffinManagerFactory::new_for_test_block("test_regex_match_basic_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
"test".to_string(),
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
builder
@@ -83,7 +83,7 @@ mod tests {
#[test]
fn test_regex_match_field_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_regex_match_field_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -91,7 +91,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
builder
@@ -110,7 +110,7 @@ mod tests {
#[test]
fn test_regex_match_type_mismatch() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_regex_match_type_mismatch_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -118,7 +118,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
builder
@@ -130,7 +130,7 @@ mod tests {
#[test]
fn test_regex_match_type_nonexist_column() {
let (_d, facotry) =
let (_d, factory) =
PuffinManagerFactory::new_for_test_block("test_regex_match_type_nonexist_column_");
let metadata = test_region_metadata();
let mut builder = InvertedIndexApplierBuilder::new(
@@ -138,7 +138,7 @@ mod tests {
test_object_store(),
&metadata,
HashSet::from_iter([1, 2, 3]),
facotry,
factory,
);
let res = builder.collect_regex_match(&nonexistent_column(), &string_lit("abc"));

View File

@@ -27,7 +27,6 @@ pub(crate) mod file_range;
pub mod format;
pub(crate) mod helper;
pub(crate) mod metadata;
pub(crate) mod page_reader;
pub mod plain_format;
pub mod reader;
pub mod row_group;
@@ -105,7 +104,9 @@ mod tests {
use tokio_util::compat::FuturesAsyncWriteCompatExt;
use super::*;
use crate::access_layer::{FilePathProvider, OperationType, RegionFilePathFactory};
use crate::access_layer::{
FilePathProvider, Metrics, OperationType, RegionFilePathFactory, WriteType,
};
use crate::cache::{CacheManager, CacheStrategy, PageKey};
use crate::read::BatchReader;
use crate::region::options::{IndexOptions, InvertedIndexOptions};
@@ -176,6 +177,7 @@ mod tests {
metadata.clone(),
NoopIndexBuilder,
file_path,
Metrics::new(WriteType::Flush),
)
.await;
@@ -233,10 +235,11 @@ mod tests {
FixedPathProvider {
file_id: handle.file_id(),
},
Metrics::new(WriteType::Flush),
)
.await;
writer
let sst_info = writer
.write_all(source, None, &write_opts)
.await
.unwrap()
@@ -265,16 +268,24 @@ mod tests {
.await;
}
// Doesn't have compressed page cached.
let page_key = PageKey::new_compressed(metadata.region_id, handle.file_id(), 0, 0);
assert!(cache.get_pages(&page_key).is_none());
let parquet_meta = sst_info.file_metadata.unwrap();
let get_ranges = |row_group_idx: usize| {
let row_group = parquet_meta.row_group(row_group_idx);
let mut ranges = Vec::with_capacity(row_group.num_columns());
for i in 0..row_group.num_columns() {
let (start, length) = row_group.column(i).byte_range();
ranges.push(start..start + length);
}
ranges
};
// Cache 4 row groups.
for i in 0..4 {
let page_key = PageKey::new_uncompressed(metadata.region_id, handle.file_id(), i, 0);
let page_key = PageKey::new(handle.file_id(), i, get_ranges(i));
assert!(cache.get_pages(&page_key).is_some());
}
let page_key = PageKey::new_uncompressed(metadata.region_id, handle.file_id(), 5, 0);
let page_key = PageKey::new(handle.file_id(), 5, vec![]);
assert!(cache.get_pages(&page_key).is_none());
}
@@ -304,6 +315,7 @@ mod tests {
FixedPathProvider {
file_id: handle.file_id(),
},
Metrics::new(WriteType::Flush),
)
.await;
@@ -346,6 +358,7 @@ mod tests {
FixedPathProvider {
file_id: handle.file_id(),
},
Metrics::new(WriteType::Flush),
)
.await;
writer
@@ -398,6 +411,7 @@ mod tests {
FixedPathProvider {
file_id: handle.file_id(),
},
Metrics::new(WriteType::Flush),
)
.await;
writer
@@ -435,6 +449,7 @@ mod tests {
FixedPathProvider {
file_id: handle.file_id(),
},
Metrics::new(WriteType::Flush),
)
.await;
@@ -577,6 +592,7 @@ mod tests {
metadata.clone(),
NoopIndexBuilder,
path_provider,
Metrics::new(WriteType::Flush),
)
.await;
@@ -648,6 +664,7 @@ mod tests {
metadata.clone(),
indexer_builder,
file_path.clone(),
Metrics::new(WriteType::Flush),
)
.await;

View File

@@ -89,7 +89,7 @@ fn parse_column_orders(
}
const FETCH_PARALLELISM: usize = 8;
const MERGE_GAP: usize = 512 * 1024;
pub(crate) const MERGE_GAP: usize = 512 * 1024;
/// Asynchronously fetches byte ranges from an object store.
///

View File

@@ -1,91 +0,0 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Parquet page reader.
use std::collections::VecDeque;
use parquet::column::page::{Page, PageMetadata, PageReader};
use parquet::errors::Result;
/// A reader that reads all pages from a cache.
pub(crate) struct RowGroupCachedReader {
/// Cached pages.
pages: VecDeque<Page>,
}
impl RowGroupCachedReader {
/// Returns a new reader from pages of a column in a row group.
pub(crate) fn new(pages: &[Page]) -> Self {
Self {
pages: pages.iter().cloned().collect(),
}
}
}
impl PageReader for RowGroupCachedReader {
fn get_next_page(&mut self) -> Result<Option<Page>> {
Ok(self.pages.pop_front())
}
fn peek_next_page(&mut self) -> Result<Option<PageMetadata>> {
Ok(self.pages.front().map(page_to_page_meta))
}
fn skip_next_page(&mut self) -> Result<()> {
// When the `SerializedPageReader` is in `SerializedPageReaderState::Pages` state, it never pops
// the dictionary page. So it always return the dictionary page as the first page. See:
// https://github.com/apache/arrow-rs/blob/1d6feeacebb8d0d659d493b783ba381940973745/parquet/src/file/serialized_reader.rs#L766-L770
// But the `GenericColumnReader` will read the dictionary page before skipping records so it won't skip dictionary page.
// So we don't need to handle the dictionary page specifically in this method.
// https://github.com/apache/arrow-rs/blob/65f7be856099d389b0d0eafa9be47fad25215ee6/parquet/src/column/reader.rs#L322-L331
self.pages.pop_front();
Ok(())
}
}
impl Iterator for RowGroupCachedReader {
type Item = Result<Page>;
fn next(&mut self) -> Option<Self::Item> {
self.get_next_page().transpose()
}
}
/// Get [PageMetadata] from `page`.
///
/// The conversion is based on [decode_page()](https://github.com/apache/arrow-rs/blob/1d6feeacebb8d0d659d493b783ba381940973745/parquet/src/file/serialized_reader.rs#L438-L481)
/// and [PageMetadata](https://github.com/apache/arrow-rs/blob/65f7be856099d389b0d0eafa9be47fad25215ee6/parquet/src/column/page.rs#L279-L301).
fn page_to_page_meta(page: &Page) -> PageMetadata {
match page {
Page::DataPage { num_values, .. } => PageMetadata {
num_rows: None,
num_levels: Some(*num_values as usize),
is_dict: false,
},
Page::DataPageV2 {
num_values,
num_rows,
..
} => PageMetadata {
num_rows: Some(*num_rows as usize),
num_levels: Some(*num_values as usize),
is_dict: false,
},
Page::DictionaryPage { .. } => PageMetadata {
num_rows: None,
num_levels: None,
is_dict: true,
},
}
}

View File

@@ -23,9 +23,8 @@ use parquet::arrow::arrow_reader::{RowGroups, RowSelection};
use parquet::arrow::ProjectionMask;
use parquet::column::page::{PageIterator, PageReader};
use parquet::errors::{ParquetError, Result};
use parquet::file::metadata::{ColumnChunkMetaData, ParquetMetaData, RowGroupMetaData};
use parquet::file::metadata::{ParquetMetaData, RowGroupMetaData};
use parquet::file::page_index::offset_index::OffsetIndexMetaData;
use parquet::file::properties::DEFAULT_PAGE_SIZE;
use parquet::file::reader::{ChunkReader, Length};
use parquet::file::serialized_reader::SerializedPageReader;
use store_api::storage::RegionId;
@@ -35,8 +34,7 @@ use crate::cache::file_cache::{FileType, IndexKey};
use crate::cache::{CacheStrategy, PageKey, PageValue};
use crate::metrics::{READ_STAGE_ELAPSED, READ_STAGE_FETCH_PAGES};
use crate::sst::file::FileId;
use crate::sst::parquet::helper::fetch_byte_ranges;
use crate::sst::parquet::page_reader::RowGroupCachedReader;
use crate::sst::parquet::helper::{fetch_byte_ranges, MERGE_GAP};
pub(crate) struct RowGroupBase<'a> {
metadata: &'a RowGroupMetaData,
@@ -44,11 +42,6 @@ pub(crate) struct RowGroupBase<'a> {
/// Compressed page of each column.
column_chunks: Vec<Option<Arc<ColumnChunkData>>>,
pub(crate) row_count: usize,
/// Row group level cached pages for each column.
///
/// These pages are uncompressed pages of a row group.
/// `column_uncompressed_pages.len()` equals to `column_chunks.len()`.
column_uncompressed_pages: Vec<Option<Arc<PageValue>>>,
}
impl<'a> RowGroupBase<'a> {
@@ -68,7 +61,6 @@ impl<'a> RowGroupBase<'a> {
offset_index,
column_chunks: vec![None; metadata.columns().len()],
row_count: metadata.num_rows() as usize,
column_uncompressed_pages: vec![None; metadata.columns().len()],
}
}
@@ -144,13 +136,9 @@ impl<'a> RowGroupBase<'a> {
pub(crate) fn calc_dense_read_ranges(&self, projection: &ProjectionMask) -> Vec<Range<u64>> {
self.column_chunks
.iter()
.zip(&self.column_uncompressed_pages)
.enumerate()
.filter(|&(idx, (chunk, uncompressed_pages))| {
// Don't need to fetch column data if we already cache the column's pages.
chunk.is_none() && projection.leaf_included(idx) && uncompressed_pages.is_none()
})
.map(|(idx, (_chunk, _pages))| {
.filter(|&(idx, chunk)| chunk.is_none() && projection.leaf_included(idx))
.map(|(idx, _chunk)| {
let column = self.metadata.column(idx);
let (start, length) = column.byte_range();
start..(start + length)
@@ -158,23 +146,17 @@ impl<'a> RowGroupBase<'a> {
.collect::<Vec<_>>()
}
/// Assigns uncompressed chunk binary data to [RowGroupBase::column_chunks]
/// Assigns compressed chunk binary data to [RowGroupBase::column_chunks]
/// and returns the chunk offset and binary data assigned.
pub(crate) fn assign_dense_chunk(
&mut self,
projection: &ProjectionMask,
chunk_data: Vec<Bytes>,
) -> Vec<(usize, Bytes)> {
) {
let mut chunk_data = chunk_data.into_iter();
let mut res = vec![];
for (idx, (chunk, row_group_pages)) in self
.column_chunks
.iter_mut()
.zip(&self.column_uncompressed_pages)
.enumerate()
{
if chunk.is_some() || !projection.leaf_included(idx) || row_group_pages.is_some() {
for (idx, chunk) in self.column_chunks.iter_mut().enumerate() {
if chunk.is_some() || !projection.leaf_included(idx) {
continue;
}
@@ -184,13 +166,11 @@ impl<'a> RowGroupBase<'a> {
};
let column = self.metadata.column(idx);
res.push((idx, data.clone()));
*chunk = Some(Arc::new(ColumnChunkData::Dense {
offset: column.byte_range().0 as usize,
data,
}));
}
res
}
/// Create [PageReader] from [RowGroupBase::column_chunks]
@@ -219,7 +199,6 @@ impl<'a> RowGroupBase<'a> {
}
};
// This column don't cache uncompressed pages.
Ok(page_reader)
}
}
@@ -277,9 +256,6 @@ impl<'a> InMemoryRowGroup<'a> {
self.base
.assign_sparse_chunk(projection, chunk_data, page_start_offsets);
} else {
// Now we only use cache in dense chunk data.
self.fetch_pages_from_cache(projection);
// Release the CPU to avoid blocking the runtime. Since `fetch_pages_from_cache`
// is a synchronous, CPU-bound operation.
yield_now().await;
@@ -296,75 +272,25 @@ impl<'a> InMemoryRowGroup<'a> {
let chunk_data = self.fetch_bytes(&fetch_ranges).await?;
// Assigns fetched data to base.
let assigned_columns = self.base.assign_dense_chunk(projection, chunk_data);
// Put fetched data to cache if necessary.
for (col_idx, data) in assigned_columns {
let column = self.base.metadata.column(col_idx);
if !cache_uncompressed_pages(column) {
// For columns that have multiple uncompressed pages, we only cache the compressed page
// to save memory.
let page_key = PageKey::new_compressed(
self.region_id,
self.file_id,
self.row_group_idx,
col_idx,
);
self.cache_strategy
.put_pages(page_key, Arc::new(PageValue::new_compressed(data.clone())));
}
}
self.base.assign_dense_chunk(projection, chunk_data);
}
Ok(())
}
/// Fetches pages for columns if cache is enabled.
/// If the page is in the cache, sets the column chunk or `column_uncompressed_pages` for the column.
fn fetch_pages_from_cache(&mut self, projection: &ProjectionMask) {
let _timer = READ_STAGE_FETCH_PAGES.start_timer();
self.base
.column_chunks
.iter_mut()
.enumerate()
.filter(|(idx, chunk)| chunk.is_none() && projection.leaf_included(*idx))
.for_each(|(idx, chunk)| {
let column = self.base.metadata.column(idx);
if cache_uncompressed_pages(column) {
// Fetches uncompressed pages for the row group.
let page_key = PageKey::new_uncompressed(
self.region_id,
self.file_id,
self.row_group_idx,
idx,
);
self.base.column_uncompressed_pages[idx] =
self.cache_strategy.get_pages(&page_key);
} else {
// Fetches the compressed page from the cache.
let page_key = PageKey::new_compressed(
self.region_id,
self.file_id,
self.row_group_idx,
idx,
);
*chunk = self.cache_strategy.get_pages(&page_key).map(|page_value| {
Arc::new(ColumnChunkData::Dense {
offset: column.byte_range().0 as usize,
data: page_value.compressed.clone(),
})
});
}
});
}
/// Try to fetch data from WriteCache,
/// Try to fetch data from the memory cache or the WriteCache,
/// if not in WriteCache, fetch data from object store directly.
async fn fetch_bytes(&self, ranges: &[Range<u64>]) -> Result<Vec<Bytes>> {
// Now fetch page timer includes the whole time to read pages.
let _timer = READ_STAGE_FETCH_PAGES.start_timer();
let page_key = PageKey::new(self.file_id, self.row_group_idx, ranges.to_vec());
if let Some(pages) = self.cache_strategy.get_pages(&page_key) {
return Ok(pages.compressed.clone());
}
let key = IndexKey::new(self.region_id, self.file_id, FileType::Parquet);
match self.fetch_ranges_from_write_cache(key, ranges).await {
Some(data) => Ok(data),
let pages = match self.fetch_ranges_from_write_cache(key, ranges).await {
Some(data) => data,
None => {
// Fetch data from object store.
let _timer = READ_STAGE_ELAPSED
@@ -373,9 +299,17 @@ impl<'a> InMemoryRowGroup<'a> {
let data = fetch_byte_ranges(self.file_path, self.object_store.clone(), ranges)
.await
.map_err(|e| ParquetError::External(Box::new(e)))?;
Ok(data)
data
}
}
};
// Put pages back to the cache.
let total_range_size = compute_total_range_size(ranges);
let page_value = PageValue::new(pages.clone(), total_range_size);
self.cache_strategy
.put_pages(page_key, Arc::new(page_value));
Ok(pages)
}
/// Fetches data from write cache.
@@ -390,40 +324,46 @@ impl<'a> InMemoryRowGroup<'a> {
}
None
}
/// Creates a page reader to read column at `i`.
fn column_page_reader(&self, i: usize) -> Result<Box<dyn PageReader>> {
if let Some(cached_pages) = &self.base.column_uncompressed_pages[i] {
debug_assert!(!cached_pages.row_group.is_empty());
// Hits the row group level page cache.
return Ok(Box::new(RowGroupCachedReader::new(&cached_pages.row_group)));
}
let page_reader = self.base.column_reader(i)?;
let column = self.base.metadata.column(i);
if cache_uncompressed_pages(column) {
// This column use row group level page cache.
// We collect all pages and put them into the cache.
let pages = page_reader.collect::<Result<Vec<_>>>()?;
let page_value = Arc::new(PageValue::new_row_group(pages));
let page_key =
PageKey::new_uncompressed(self.region_id, self.file_id, self.row_group_idx, i);
self.cache_strategy.put_pages(page_key, page_value.clone());
return Ok(Box::new(RowGroupCachedReader::new(&page_value.row_group)));
}
// This column don't cache uncompressed pages.
Ok(Box::new(page_reader))
}
}
/// Returns whether we cache uncompressed pages for the column.
fn cache_uncompressed_pages(column: &ColumnChunkMetaData) -> bool {
// If the row group only has a data page, cache the whole row group as
// it might be faster than caching a compressed page.
column.uncompressed_size() as usize <= DEFAULT_PAGE_SIZE
/// Computes the max possible buffer size to read the given `ranges`.
// See https://github.com/apache/opendal/blob/v0.54.0/core/src/types/read/reader.rs#L166-L192
fn compute_total_range_size(ranges: &[Range<u64>]) -> u64 {
if ranges.is_empty() {
return 0;
}
let gap = MERGE_GAP as u64;
let mut sorted_ranges = ranges.to_vec();
sorted_ranges.sort_unstable_by(|a, b| a.start.cmp(&b.start));
let mut total_size = 0;
let mut cur = sorted_ranges[0].clone();
for range in sorted_ranges.into_iter().skip(1) {
if range.start <= cur.end + gap {
// There is an overlap or the gap is small enough to merge
cur.end = cur.end.max(range.end);
} else {
// No overlap and the gap is too large, add current range to total and start a new one
total_size += align_to_pooled_buf_size(cur.end - cur.start);
cur = range;
}
}
// Add the last range
total_size += align_to_pooled_buf_size(cur.end - cur.start);
total_size
}
/// Aligns the given size to the multiple of the pooled buffer size.
// See:
// - https://github.com/apache/opendal/blob/v0.54.0/core/src/services/fs/backend.rs#L178
// - https://github.com/apache/opendal/blob/v0.54.0/core/src/services/fs/reader.rs#L36-L46
fn align_to_pooled_buf_size(size: u64) -> u64 {
const POOLED_BUF_SIZE: u64 = 2 * 1024 * 1024;
size.div_ceil(POOLED_BUF_SIZE) * POOLED_BUF_SIZE
}
impl RowGroups for InMemoryRowGroup<'_> {
@@ -432,10 +372,11 @@ impl RowGroups for InMemoryRowGroup<'_> {
}
fn column_chunks(&self, i: usize) -> Result<Box<dyn PageIterator>> {
let page_reader = self.column_page_reader(i)?;
// Creates a page reader to read column at `i`.
let page_reader = self.base.column_reader(i)?;
Ok(Box::new(ColumnChunkIterator {
reader: Some(Ok(page_reader)),
reader: Some(Ok(Box::new(page_reader))),
}))
}
}

View File

@@ -20,6 +20,7 @@ use std::pin::Pin;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;
use std::task::{Context, Poll};
use std::time::Instant;
use common_telemetry::debug;
use common_time::Timestamp;
@@ -38,7 +39,7 @@ use store_api::storage::SequenceNumber;
use tokio::io::AsyncWrite;
use tokio_util::compat::{Compat, FuturesAsyncWriteCompatExt};
use crate::access_layer::{FilePathProvider, SstInfoArray, TempFileCleaner};
use crate::access_layer::{FilePathProvider, Metrics, SstInfoArray, TempFileCleaner};
use crate::error::{InvalidMetadataSnafu, OpenDalSnafu, Result, WriteParquetSnafu};
use crate::read::{Batch, Source};
use crate::sst::file::FileId;
@@ -65,6 +66,8 @@ pub struct ParquetWriter<F: WriterFactory, I: IndexerBuilder, P: FilePathProvide
bytes_written: Arc<AtomicUsize>,
/// Cleaner to remove temp files on failure.
file_cleaner: Option<TempFileCleaner>,
/// Write metrics
metrics: Metrics,
}
pub trait WriterFactory {
@@ -100,12 +103,14 @@ where
metadata: RegionMetadataRef,
indexer_builder: I,
path_provider: P,
metrics: Metrics,
) -> ParquetWriter<ObjectStoreWriterFactory, I, P> {
ParquetWriter::new(
ObjectStoreWriterFactory { object_store },
metadata,
indexer_builder,
path_provider,
metrics,
)
.await
}
@@ -128,6 +133,7 @@ where
metadata: RegionMetadataRef,
indexer_builder: I,
path_provider: P,
metrics: Metrics,
) -> ParquetWriter<F, I, P> {
let init_file = FileId::random();
let indexer = indexer_builder.build(init_file).await;
@@ -142,6 +148,7 @@ where
current_indexer: Some(indexer),
bytes_written: Arc::new(AtomicUsize::new(0)),
file_cleaner: None,
metrics,
}
}
@@ -234,12 +241,14 @@ where
match res {
Ok(mut batch) => {
stats.update(&batch);
let start = Instant::now();
// safety: self.current_indexer must be set when first batch has been written.
self.current_indexer
.as_mut()
.unwrap()
.update(&mut batch)
.await;
self.metrics.update_index += start.elapsed();
if let Some(max_file_size) = opts.max_file_size
&& self.bytes_written.load(Ordering::Relaxed) > max_file_size
{
@@ -286,16 +295,21 @@ where
write_format: &WriteFormat,
opts: &WriteOptions,
) -> Result<Option<Batch>> {
let start = Instant::now();
let Some(batch) = source.next_batch().await? else {
return Ok(None);
};
self.metrics.iter_source += start.elapsed();
let arrow_batch = write_format.convert_batch(&batch)?;
let start = Instant::now();
self.maybe_init_writer(write_format.arrow_schema(), opts)
.await?
.write(&arrow_batch)
.await
.context(WriteParquetSnafu)?;
self.metrics.write_batch += start.elapsed();
Ok(Some(batch))
}
@@ -337,6 +351,11 @@ where
Ok(self.writer.as_mut().unwrap())
}
}
/// Consumes write and return the collected metrics.
pub fn into_metrics(self) -> Metrics {
self.metrics
}
}
#[derive(Default)]

View File

@@ -99,6 +99,7 @@ where
let object_store = region.access_layer.object_store().clone();
let dropping_regions = self.dropping_regions.clone();
let listener = self.listener.clone();
let intm_manager = self.intermediate_manager.clone();
common_runtime::spawn_global(async move {
let gc_duration = listener
.on_later_drop_begin(region_id)
@@ -111,6 +112,9 @@ where
gc_duration,
)
.await;
if let Err(err) = intm_manager.prune_region_dir(&region_id).await {
warn!(err; "Failed to prune intermediate region directory, region_id: {}", region_id);
}
listener.on_later_drop_end(region_id, removed);
});

View File

@@ -247,6 +247,7 @@ impl<S> RegionWorkerLoop<S> {
region.region_id,
&region.version_control,
region.provider.clone(),
Some(region.written_bytes.clone()),
);
e.insert(region_ctx);
@@ -350,6 +351,7 @@ impl<S> RegionWorkerLoop<S> {
region.region_id,
&region.version_control,
region.provider.clone(),
Some(region.written_bytes.clone()),
);
e.insert(region_ctx);

View File

@@ -70,7 +70,7 @@ impl ScalarCalculate {
interval: Millisecond,
input: LogicalPlan,
time_index: &str,
tag_colunms: &[String],
tag_columns: &[String],
field_column: &str,
table_name: Option<&str>,
) -> Result<Self> {
@@ -97,7 +97,7 @@ impl ScalarCalculate {
end,
interval,
time_index: time_index.to_string(),
tag_columns: tag_colunms.to_vec(),
tag_columns: tag_columns.to_vec(),
field_column: field_column.to_string(),
input,
output_schema: Arc::new(schema),

View File

@@ -369,6 +369,9 @@ impl<H> BoundedStager<H> {
/// Note: It can't recover the mapping between puffin files and keys, so TTL
/// is configured to purge the dangling files and directories.
async fn recover(&self) -> Result<()> {
let timer = std::time::Instant::now();
info!("Recovering the staging area, base_dir: {:?}", self.base_dir);
let mut read_dir = fs::read_dir(&self.base_dir).await.context(ReadSnafu)?;
let mut elems = HashMap::new();
@@ -430,6 +433,7 @@ impl<H> BoundedStager<H> {
}
let mut size = 0;
let num_elems = elems.len();
for (key, value) in elems {
size += value.size();
self.cache.insert(key, value).await;
@@ -440,6 +444,12 @@ impl<H> BoundedStager<H> {
self.cache.run_pending_tasks().await;
info!(
"Recovered the staging area, num_entries: {}, num_bytes: {}, cost: {:?}",
num_elems,
size,
timer.elapsed()
);
Ok(())
}

View File

@@ -82,7 +82,7 @@ impl ExtensionPlanner for MergeSortExtensionPlanner {
// and we only need to do a merge sort, otherwise fallback to quick sort
let can_merge_sort = partition_cnt >= region_cnt;
if can_merge_sort {
// TODO(discord9): use `SortPreversingMergeExec here`
// TODO(discord9): use `SortPreservingMergeExec here`
}
// for now merge sort only exist in logical plan, and have the same effect as `Sort`
// doesn't change the execution plan, this will change in the future

View File

@@ -42,11 +42,19 @@ pub struct PreCompiledMatchesTermExpr {
term: String,
/// The pre-compiled term finder
finder: MatchesTermFinder,
/// No used but show how index tokenizes the term basically.
/// Not precise due to column options is unknown but for debugging purpose in most cases it's enough.
probes: Vec<String>,
}
impl fmt::Display for PreCompiledMatchesTermExpr {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "MatchesConstTerm({}, \"{}\")", self.text, self.term)
write!(
f,
"MatchesConstTerm({}, term: \"{}\", probes: {:?})",
self.text, self.term, self.probes
)
}
}
@@ -118,6 +126,7 @@ impl PhysicalExpr for PreCompiledMatchesTermExpr {
text: children[0].clone(),
term: self.term.clone(),
finder: self.finder.clone(),
probes: self.probes.clone(),
}))
}
}
@@ -167,10 +176,19 @@ impl PhysicalOptimizerRule for MatchesConstantTermOptimizer {
if let Some(lit) = args[1].as_any().downcast_ref::<Literal>() {
if let ScalarValue::Utf8(Some(term)) = lit.value() {
let finder = MatchesTermFinder::new(term);
// For debugging purpose. Not really precise but enough for most cases.
let probes = term
.split(|c: char| !c.is_alphanumeric() && c != '_')
.filter(|s| !s.is_empty())
.map(|s| s.to_string())
.collect();
let expr = PreCompiledMatchesTermExpr {
text: args[0].clone(),
term: term.to_string(),
finder,
probes,
};
return Ok(Transformed::yes(Arc::new(expr)));
@@ -390,7 +408,7 @@ mod tests {
async fn test_matches_term_optimization_from_sql() {
let sql = "WITH base AS (
SELECT text, timestamp FROM test
WHERE MATCHES_TERM(text, 'hello')
WHERE MATCHES_TERM(text, 'hello wo_rld')
AND timestamp > '2025-01-01 00:00:00'
),
subquery1 AS (
@@ -448,7 +466,15 @@ mod tests {
.unwrap();
let plan_str = get_plan_string(&physical_plan).join("\n");
assert!(plan_str.contains("MatchesConstTerm"));
assert!(plan_str.contains("MatchesConstTerm(text@0, term: \"foo\", probes: [\"foo\"]"));
assert!(plan_str.contains(
"MatchesConstTerm(text@0, term: \"hello wo_rld\", probes: [\"hello\", \"wo_rld\"]"
));
assert!(plan_str.contains("MatchesConstTerm(text@0, term: \"world\", probes: [\"world\"]"));
assert!(plan_str
.contains("MatchesConstTerm(text@0, term: \"greeting\", probes: [\"greeting\"]"));
assert!(plan_str.contains("MatchesConstTerm(text@0, term: \"there\", probes: [\"there\"]"));
assert!(plan_str.contains("MatchesConstTerm(text@0, term: \"42\", probes: [\"42\"]"));
assert!(!plan_str.contains("matches_term"))
}
}

View File

@@ -1 +1,2 @@
v0.10.1
v0.11.9

View File

@@ -352,7 +352,7 @@ async fn dryrun_pipeline_inner(
)
.await?;
let colume_type_key = "colume_type";
let column_type_key = "column_type";
let data_type_key = "data_type";
let name_key = "name";
@@ -376,7 +376,7 @@ async fn dryrun_pipeline_inner(
JsonValue::String(cs.datatype().as_str_name().to_string()),
);
map.insert(
colume_type_key.to_string(),
column_type_key.to_string(),
JsonValue::String(cs.semantic_type().as_str_name().to_string()),
);
map.insert(
@@ -409,7 +409,7 @@ async fn dryrun_pipeline_inner(
);
map.insert(
"semantic_type".to_string(),
schema[idx][colume_type_key].clone(),
schema[idx][column_type_key].clone(),
);
map.insert(
"data_type".to_string(),

View File

@@ -51,7 +51,7 @@ mod tests {
use crate::statements::statement::Statement;
#[test]
fn test_display_for_tuncate_table() {
fn test_display_for_truncate_table() {
let sql = r"truncate table t1;";
let stmts: Vec<Statement> =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())

View File

@@ -450,6 +450,9 @@ pub struct RegionStatistic {
/// The details of the region.
#[serde(default)]
pub manifest: RegionManifestInfo,
#[serde(default)]
/// The total bytes written of the region since region opened.
pub written_bytes: u64,
/// The latest entry id of the region's remote WAL since last flush.
/// For metric engine, there're two latest entry ids, one for data and one for metadata.
/// TODO(weny): remove this two fields and use single instead.

View File

@@ -1,14 +1,14 @@
services:
zookeeper:
image: docker.io/bitnami/zookeeper:3.7
image: greptime/zookeeper:3.7
ports:
- '2181:2181'
environment:
- ALLOW_ANONYMOUS_LOGIN=yes
kafka:
image: docker.io/bitnami/kafka:3.9.0
image: greptime/kafka:3.9.0-debian-12-r1
container_name: kafka
ports:
- 9092:9092
@@ -32,7 +32,7 @@ services:
condition: service_started
etcd:
image: docker.io/bitnami/etcd:3.5
image: greptime/etcd:3.6.1-debian-12-r3
ports:
- "2379:2379"
- "2380:2380"
@@ -43,8 +43,35 @@ services:
ETCD_ADVERTISE_CLIENT_URLS: http://etcd:2379
ETCD_MAX_REQUEST_BYTES: 10485760
# etcd-tls:
# image: greptime/etcd:3.6.1-debian-12-r3
# ports:
# - "2378:2378"
# - "2381:2381"
# environment:
# ALLOW_NONE_AUTHENTICATION: "yes"
# ETCD_NAME: etcd-tls
# ETCD_LISTEN_CLIENT_URLS: https://0.0.0.0:2378
# ETCD_ADVERTISE_CLIENT_URLS: https://etcd-tls:2378
# ETCD_LISTEN_PEER_URLS: https://0.0.0.0:2381
# ETCD_INITIAL_ADVERTISE_PEER_URLS: https://etcd-tls:2381
# ETCD_INITIAL_CLUSTER: etcd-tls=https://etcd-tls:2381
# ETCD_INITIAL_CLUSTER_TOKEN: etcd-tls-cluster
# ETCD_INITIAL_CLUSTER_STATE: new
# ETCD_CERT_FILE: /certs/server.crt
# ETCD_KEY_FILE: /certs/server-key.pem
# ETCD_TRUSTED_CA_FILE: /certs/ca.crt
# ETCD_PEER_CERT_FILE: /certs/server.crt
# ETCD_PEER_KEY_FILE: /certs/server-key.pem
# ETCD_PEER_TRUSTED_CA_FILE: /certs/ca.crt
# ETCD_CLIENT_CERT_AUTH: "true"
# ETCD_PEER_CLIENT_CERT_AUTH: "true"
# ETCD_MAX_REQUEST_BYTES: 10485760
# volumes:
# - ./etcd-tls-certs:/certs:ro
minio:
image: docker.io/bitnami/minio:2024
image: greptime/minio:2024
ports:
- '9000:9000'
- '9001:9001'
@@ -68,7 +95,7 @@ services:
- POSTGRES_PASSWORD=admin
mysql:
image: bitnami/mysql:5.7
image: greptime/mysql:5.7
ports:
- 3306:3306
volumes:

View File

@@ -3174,37 +3174,37 @@ transform:
let dryrun_schema = json!([
{
"colume_type": "FIELD",
"column_type": "FIELD",
"data_type": "INT32",
"fulltext": false,
"name": "id1"
},
{
"colume_type": "FIELD",
"column_type": "FIELD",
"data_type": "INT32",
"fulltext": false,
"name": "id2"
},
{
"colume_type": "FIELD",
"column_type": "FIELD",
"data_type": "STRING",
"fulltext": false,
"name": "type"
},
{
"colume_type": "FIELD",
"column_type": "FIELD",
"data_type": "STRING",
"fulltext": false,
"name": "log"
},
{
"colume_type": "FIELD",
"column_type": "FIELD",
"data_type": "STRING",
"fulltext": false,
"name": "logger"
},
{
"colume_type": "TIMESTAMP",
"column_type": "TIMESTAMP",
"data_type": "TIMESTAMP_NANOSECOND",
"fulltext": false,
"name": "time"

View File

@@ -22,15 +22,17 @@ INSERT INTO test VALUES
Affected Rows: 3
-- SQLNESS SLEEP 3s
SELECT SUM(region_rows), SUM(disk_size), SUM(sst_size), SUM(index_size)
-- For regions using different WAL implementations, the manifest size may vary.
-- The remote WAL implementation additionally stores a flushed entry ID when creating the manifest.
SELECT SUM(region_rows),SUM(written_bytes_since_open), SUM(memtable_size), SUM(sst_size), SUM(index_size)
FROM INFORMATION_SCHEMA.REGION_STATISTICS WHERE table_id
IN (SELECT TABLE_ID FROM INFORMATION_SCHEMA.TABLES WHERE table_name = 'test' and table_schema = 'public');
+-------------------------------------------------------+-----------------------------------------------------+----------------------------------------------------+------------------------------------------------------+
| sum(information_schema.region_statistics.region_rows) | sum(information_schema.region_statistics.disk_size) | sum(information_schema.region_statistics.sst_size) | sum(information_schema.region_statistics.index_size) |
+-------------------------------------------------------+-----------------------------------------------------+----------------------------------------------------+------------------------------------------------------+
| 3 | 2238 | 0 | 0 |
+-------------------------------------------------------+-----------------------------------------------------+----------------------------------------------------+------------------------------------------------------+
+-------------------------------------------------------+--------------------------------------------------------------------+---------------------------------------------------------+----------------------------------------------------+------------------------------------------------------+
| sum(information_schema.region_statistics.region_rows) | sum(information_schema.region_statistics.written_bytes_since_open) | sum(information_schema.region_statistics.memtable_size) | sum(information_schema.region_statistics.sst_size) | sum(information_schema.region_statistics.index_size) |
+-------------------------------------------------------+--------------------------------------------------------------------+---------------------------------------------------------+----------------------------------------------------+------------------------------------------------------+
| 3 | 78 | 78 | 0 | 0 |
+-------------------------------------------------------+--------------------------------------------------------------------+---------------------------------------------------------+----------------------------------------------------+------------------------------------------------------+
SELECT data_length, index_length, avg_row_length, table_rows FROM INFORMATION_SCHEMA.TABLES WHERE table_name = 'test';

View File

@@ -17,7 +17,9 @@ INSERT INTO test VALUES
(21, 'c', 21);
-- SQLNESS SLEEP 3s
SELECT SUM(region_rows), SUM(disk_size), SUM(sst_size), SUM(index_size)
-- For regions using different WAL implementations, the manifest size may vary.
-- The remote WAL implementation additionally stores a flushed entry ID when creating the manifest.
SELECT SUM(region_rows),SUM(written_bytes_since_open), SUM(memtable_size), SUM(sst_size), SUM(index_size)
FROM INFORMATION_SCHEMA.REGION_STATISTICS WHERE table_id
IN (SELECT TABLE_ID FROM INFORMATION_SCHEMA.TABLES WHERE table_name = 'test' and table_schema = 'public');

View File

@@ -317,17 +317,18 @@ select * from information_schema.columns order by table_schema, table_name, colu
| greptime | information_schema | region_peers | table_catalog | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | region_peers | table_name | 3 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | region_peers | table_schema | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | region_statistics | disk_size | 5 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | engine | 10 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | region_statistics | index_size | 9 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | manifest_size | 7 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | memtable_size | 6 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | disk_size | 6 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | engine | 11 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | region_statistics | index_size | 10 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | manifest_size | 8 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | memtable_size | 7 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | region_id | 1 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | region_statistics | region_number | 3 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | region_statistics | region_role | 11 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | region_statistics | region_role | 12 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | region_statistics | region_rows | 4 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | sst_size | 8 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | sst_size | 9 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | region_statistics | table_id | 2 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | region_statistics | written_bytes_since_open | 5 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | routines | character_maximum_length | 7 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | |
| greptime | information_schema | routines | character_octet_length | 8 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | |
| greptime | information_schema | routines | character_set_client | 29 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |

View File

@@ -2,7 +2,7 @@
version: '3.8'
services:
kafka:
image: bitnami/kafka:3.6.0
image: greptime/kafka:3.9.0-debian-12-r1
container_name: kafka
ports:
- 9092:9092

Some files were not shown because too many files have changed in this diff Show More