mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-04 12:22:55 +00:00
Compare commits
11 Commits
v0.1.0-alp
...
v0.1.0-alp
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
0ffe640f7d | ||
|
|
0d660e45cf | ||
|
|
a640872cda | ||
|
|
7e3c59fb51 | ||
|
|
7bbc679c76 | ||
|
|
0b3a2cbcda | ||
|
|
53ee85cdad | ||
|
|
bc9a46dbb7 | ||
|
|
a61e96477b | ||
|
|
f8500e54c1 | ||
|
|
e85780b5e4 |
164
Cargo.lock
generated
164
Cargo.lock
generated
@@ -2488,6 +2488,17 @@ version = "1.1.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "de853764b47027c2e862a995c34978ffa63c1501f2e15f987ba11bd4f9bba193"
|
||||
|
||||
[[package]]
|
||||
name = "fail"
|
||||
version = "0.5.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "fe5e43d0f78a42ad591453aedb1d7ae631ce7ee445c7643691055a9ed8d3b01c"
|
||||
dependencies = [
|
||||
"log",
|
||||
"once_cell",
|
||||
"rand 0.8.5",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "fallible-iterator"
|
||||
version = "0.2.0"
|
||||
@@ -2698,6 +2709,16 @@ dependencies = [
|
||||
"syn",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "fs2"
|
||||
version = "0.4.3"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "9564fc758e15025b46aa6643b1b77d047d1a56a1aea6e01002ac0c7026876213"
|
||||
dependencies = [
|
||||
"libc",
|
||||
"winapi",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "fuchsia-cprng"
|
||||
version = "0.1.1"
|
||||
@@ -3165,6 +3186,12 @@ dependencies = [
|
||||
"unicode-normalization",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "if_chain"
|
||||
version = "1.0.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "cb56e1aa765b4b4f3aadfab769793b7087bb03a4ea4920644a6d238e2df5b9ed"
|
||||
|
||||
[[package]]
|
||||
name = "indexmap"
|
||||
version = "1.9.2"
|
||||
@@ -3532,6 +3559,9 @@ dependencies = [
|
||||
"futures",
|
||||
"futures-util",
|
||||
"hex",
|
||||
"protobuf",
|
||||
"protobuf-build",
|
||||
"raft-engine",
|
||||
"rand 0.8.5",
|
||||
"snafu",
|
||||
"store-api",
|
||||
@@ -4133,6 +4163,20 @@ dependencies = [
|
||||
"memoffset 0.6.5",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "nix"
|
||||
version = "0.25.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "f346ff70e7dbfd675fe90590b92d59ef2de15a8779ae305ebcbfd3f0caf59be4"
|
||||
dependencies = [
|
||||
"autocfg",
|
||||
"bitflags",
|
||||
"cfg-if 1.0.0",
|
||||
"libc",
|
||||
"memoffset 0.6.5",
|
||||
"pin-utils",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "nom"
|
||||
version = "7.1.1"
|
||||
@@ -4189,6 +4233,17 @@ dependencies = [
|
||||
"serde",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "num-derive"
|
||||
version = "0.3.3"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "876a53fff98e03a936a674b29568b0e605f06b29372c2489ff4de23f1949743d"
|
||||
dependencies = [
|
||||
"proc-macro2",
|
||||
"quote",
|
||||
"syn",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "num-integer"
|
||||
version = "0.1.45"
|
||||
@@ -4313,9 +4368,9 @@ dependencies = [
|
||||
|
||||
[[package]]
|
||||
name = "once_cell"
|
||||
version = "1.16.0"
|
||||
version = "1.17.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "86f0b0d4bf799edbc74508c1e8bf170ff5f41238e5f8225603ca7caaae2b7860"
|
||||
checksum = "6f61fba1741ea2b3d6a1e3178721804bb716a68a6aeba1149b5d52e3d464ea66"
|
||||
|
||||
[[package]]
|
||||
name = "oorandom"
|
||||
@@ -5037,6 +5092,33 @@ dependencies = [
|
||||
"unicode-ident",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "prometheus"
|
||||
version = "0.13.3"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "449811d15fbdf5ceb5c1144416066429cf82316e2ec8ce0c1f6f8a02e7bbcf8c"
|
||||
dependencies = [
|
||||
"cfg-if 1.0.0",
|
||||
"fnv",
|
||||
"lazy_static",
|
||||
"memchr",
|
||||
"parking_lot",
|
||||
"protobuf",
|
||||
"thiserror",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "prometheus-static-metric"
|
||||
version = "0.5.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "f8f30cdb09c39930b8fa5e0f23cbb895ab3f766b187403a0ba0956fc1ef4f0e5"
|
||||
dependencies = [
|
||||
"lazy_static",
|
||||
"proc-macro2",
|
||||
"quote",
|
||||
"syn",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "promql"
|
||||
version = "0.1.0"
|
||||
@@ -5166,6 +5248,36 @@ dependencies = [
|
||||
"prost 0.11.3",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "protobuf"
|
||||
version = "2.28.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "106dd99e98437432fed6519dedecfade6a06a73bb7b2a1e019fdd2bee5778d94"
|
||||
dependencies = [
|
||||
"bytes",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "protobuf-build"
|
||||
version = "0.14.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "6fb3c02f54ecaf12572c1a60dbdb36b1f8f713a16105881143f2be84cca5bbe3"
|
||||
dependencies = [
|
||||
"bitflags",
|
||||
"protobuf",
|
||||
"protobuf-codegen",
|
||||
"regex",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "protobuf-codegen"
|
||||
version = "2.28.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "033460afb75cf755fcfc16dfaed20b86468082a2ea24e05ac35ab4a099a017d6"
|
||||
dependencies = [
|
||||
"protobuf",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "ptr_meta"
|
||||
version = "0.1.4"
|
||||
@@ -5299,6 +5411,39 @@ dependencies = [
|
||||
"nibble_vec",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "raft-engine"
|
||||
version = "0.3.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "67b66e735395b7ff12f3ebbb4794006aecb365c4c9a82141279b58b227ac3a8b"
|
||||
dependencies = [
|
||||
"byteorder",
|
||||
"crc32fast",
|
||||
"crossbeam",
|
||||
"fail",
|
||||
"fs2",
|
||||
"hashbrown 0.12.3",
|
||||
"hex",
|
||||
"if_chain",
|
||||
"lazy_static",
|
||||
"libc",
|
||||
"log",
|
||||
"lz4-sys",
|
||||
"nix 0.25.1",
|
||||
"num-derive",
|
||||
"num-traits",
|
||||
"parking_lot",
|
||||
"prometheus",
|
||||
"prometheus-static-metric",
|
||||
"protobuf",
|
||||
"rayon",
|
||||
"scopeguard",
|
||||
"serde",
|
||||
"serde_repr",
|
||||
"strum",
|
||||
"thiserror",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "rand"
|
||||
version = "0.4.6"
|
||||
@@ -6230,6 +6375,7 @@ dependencies = [
|
||||
"snafu",
|
||||
"sql",
|
||||
"storage",
|
||||
"store-api",
|
||||
"table",
|
||||
"tempdir",
|
||||
"tokio",
|
||||
@@ -6369,6 +6515,17 @@ dependencies = [
|
||||
"serde",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "serde_repr"
|
||||
version = "0.1.10"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "9a5ec9fa74a20ebbe5d9ac23dac1fc96ba0ecfe9f50f2843b52e537b10fbcb4e"
|
||||
dependencies = [
|
||||
"proc-macro2",
|
||||
"quote",
|
||||
"syn",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "serde_urlencoded"
|
||||
version = "0.7.1"
|
||||
@@ -6684,10 +6841,12 @@ version = "0.1.0"
|
||||
dependencies = [
|
||||
"api",
|
||||
"catalog",
|
||||
"common-base",
|
||||
"common-catalog",
|
||||
"common-error",
|
||||
"common-time",
|
||||
"datatypes",
|
||||
"hex",
|
||||
"itertools",
|
||||
"mito",
|
||||
"once_cell",
|
||||
@@ -6784,6 +6943,7 @@ dependencies = [
|
||||
"common-base",
|
||||
"common-error",
|
||||
"common-query",
|
||||
"common-recordbatch",
|
||||
"common-runtime",
|
||||
"common-telemetry",
|
||||
"common-time",
|
||||
|
||||
@@ -7,6 +7,14 @@ mysql_addr = '127.0.0.1:4406'
|
||||
mysql_runtime_size = 4
|
||||
enable_memory_catalog = false
|
||||
|
||||
[wal]
|
||||
dir = "/tmp/greptimedb/wal"
|
||||
file_size = 1073741824
|
||||
purge_interval = 600
|
||||
purge_threshold = 53687091200
|
||||
read_batch_size = 128
|
||||
sync_write = false
|
||||
|
||||
[storage]
|
||||
type = 'File'
|
||||
data_dir = '/tmp/greptimedb/data/'
|
||||
|
||||
@@ -1,12 +1,20 @@
|
||||
node_id = 0
|
||||
mode = 'standalone'
|
||||
wal_dir = '/tmp/greptimedb/wal/'
|
||||
enable_memory_catalog = false
|
||||
|
||||
[http_options]
|
||||
addr = '127.0.0.1:4000'
|
||||
timeout = "30s"
|
||||
|
||||
[wal]
|
||||
dir = "/tmp/greptimedb/wal"
|
||||
file_size = 1073741824
|
||||
purge_interval = 600
|
||||
purge_threshold = 53687091200
|
||||
read_batch_size = 128
|
||||
sync_write = false
|
||||
|
||||
|
||||
[storage]
|
||||
type = 'File'
|
||||
data_dir = '/tmp/greptimedb/data/'
|
||||
|
||||
@@ -24,6 +24,8 @@ RUN cargo build --release
|
||||
# TODO(zyy17): Maybe should use the more secure container image.
|
||||
FROM ubuntu:22.04 as base
|
||||
|
||||
RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install ca-certificates
|
||||
|
||||
WORKDIR /greptime
|
||||
COPY --from=builder /greptimedb/target/release/greptime /greptime/bin/
|
||||
ENV PATH /greptime/bin/:$PATH
|
||||
|
||||
@@ -1,5 +1,7 @@
|
||||
FROM ubuntu:22.04
|
||||
|
||||
RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install ca-certificates
|
||||
|
||||
ARG TARGETARCH
|
||||
|
||||
ADD $TARGETARCH/greptime /greptime/bin/
|
||||
|
||||
BIN
docs/rfcs/2022-12-20-promql-in-rust/example.png
Normal file
BIN
docs/rfcs/2022-12-20-promql-in-rust/example.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 34 KiB |
BIN
docs/rfcs/2022-12-20-promql-in-rust/instant-and-vector.png
Normal file
BIN
docs/rfcs/2022-12-20-promql-in-rust/instant-and-vector.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 58 KiB |
BIN
docs/rfcs/2022-12-20-promql-in-rust/matrix-from-array.png
Normal file
BIN
docs/rfcs/2022-12-20-promql-in-rust/matrix-from-array.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 35 KiB |
BIN
docs/rfcs/2022-12-20-promql-in-rust/range-vector-with-matrix.png
Normal file
BIN
docs/rfcs/2022-12-20-promql-in-rust/range-vector-with-matrix.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 46 KiB |
175
docs/rfcs/2022-12-20-promql-in-rust/rfc.md
Normal file
175
docs/rfcs/2022-12-20-promql-in-rust/rfc.md
Normal file
@@ -0,0 +1,175 @@
|
||||
---
|
||||
Feature Name: "promql-in-rust"
|
||||
Tracking Issue: https://github.com/GreptimeTeam/greptimedb/issues/596
|
||||
Date: 2022-12-20
|
||||
Author: "Ruihang Xia <waynestxia@gmail.com>"
|
||||
---
|
||||
|
||||
Rewrite PromQL in Rust
|
||||
----------------------
|
||||
|
||||
# Summary
|
||||
A Rust native implementation of PromQL, for GreptimeDB.
|
||||
|
||||
# Motivation
|
||||
Prometheus and its query language PromQL prevails in the cloud-native observability area, which is an important scenario for time series database like GreptimeDB. We already have support for its remote read and write protocols. Users can now integrate GreptimeDB as the storage backend to existing Prometheus deployment, but cannot run PromQL query directly on GreptimeDB like SQL.
|
||||
|
||||
This RFC proposes to add support for PromQL. Because it was created in Go, we can't use the existing code easily. For interoperability, performance and extendability, porting its logic to Rust is a good choice.
|
||||
|
||||
# Details
|
||||
|
||||
## Overview
|
||||
One of the goals is to make use of our existing basic operators, execution model and runtime to reduce the work. So the entire proposal is built on top of Apache Arrow DataFusion. The rewrote PromQL logic is manifested as `Expr` or `Execution Plan` in DataFusion. And both the intermediate data structure and the result is in the format of `Arrow`'s `RecordBatch`.
|
||||
|
||||
The following sections are organized in a top-down manner. Starts with evaluation procedure. Then introduces the building blocks of our new PromQL operation. Follows by an explanation of data model. And end with an example logic plan.
|
||||
|
||||
*This RFC is heavily related to Prometheus and PromQL. It won't repeat some basic concepts of them.*
|
||||
|
||||
## Evaluation
|
||||
|
||||
The original implementation is like an interpreter of parsed PromQL AST. It has two characteristics: (1) Operations are evaluated in place after they are parsed to AST. And some key parameters are separated from the AST because they do not present in the query, but come from other places like another field in the HTTP payload. (2) calculation is performed per timestamp. You can see this pattern many times:
|
||||
```go
|
||||
for ts := ev.startTimestamp; ts <= ev.endTimestamp; ts += ev.interval {}
|
||||
```
|
||||
|
||||
These bring out two differences in the proposed implementation. First, to make it more general and clear, the evaluation procedure is reorganized into serval phases (and is the same as DataFusion's). And second, data are evaluated by time series (corresponding to "columnar calculation", if think timestamp as row number).
|
||||
|
||||
```
|
||||
Logic
|
||||
Query AST Plan
|
||||
─────────► Parser ───────► Logical ────────► Physical ────┐
|
||||
Planner Planner │
|
||||
│
|
||||
◄───────────────────────────── Executor ◄────────────────┘
|
||||
Evaluation Result Execution
|
||||
Plan
|
||||
```
|
||||
|
||||
- Parser
|
||||
|
||||
Provided by [`promql-parser`](https://github.com/GreptimeTeam/promql-parser) crate. Same as the original implementation.
|
||||
|
||||
- Logical Planner
|
||||
|
||||
Generates a logical plan with all the needed parameters. It should accept something like `EvalStmt` in Go's implementation, which contains query time range, evaluation interval and lookback range.
|
||||
|
||||
Another important thing done here is assembling the logic plan, with all the operations baked into logically. Like what's the filter and time range to read, how the data then flows through a selector into a binary operation, etc. Or what's the output schema of every single step. The generated logic plan is deterministic without variables, and can be `EXPLAIN`ed clearly.
|
||||
|
||||
- Physical Planner
|
||||
|
||||
This step converts a logic plan into evaluatable execution plan. There are not many special things like the previous step. Except when a query is going to be executed distributedly. In this case, a logic plan will be divided into serval parts and sent to serval nodes. One physical planner only sees its own part.
|
||||
|
||||
- Executor
|
||||
|
||||
As its name shows, this step calculates data to result. And all new calculation logic, the implementation of PromQL in rust, is placed here. And the rewrote functions are using `RecordBatch` and `Array` from `Arrow` as the intermediate data structure.
|
||||
|
||||
Each "batch" contains only data from single time series. This is from the underlying storage implementation. Though it's not a requirement of this RFC, having this property can simplify some functions.
|
||||
|
||||
Another thing to mention is the rewrote functions don't aware of timestamp or value columns, they are defined only based on the input data types. For example, `increase()` function in PromQL calculates the unbiased delta of data, its implementation here only does this single thing. Let's compare the signature of two implementations:
|
||||
|
||||
- Go
|
||||
```go
|
||||
func funcIncrease(vals []parser.Value, args parser.Expressions) Vector {}
|
||||
```
|
||||
- Rust
|
||||
```rust
|
||||
fn prom_increase(input: Array) -> Array {}
|
||||
```
|
||||
|
||||
Some unimportant parameters are omitted. The original Go version only writes the logic for `Point`'s value, either float or histogram. But the proposed rewritten one accepts a generic `Array` as input, which can be any type that suits, from `i8` to `u64` to `TimestampNanosecond`.
|
||||
|
||||
## Plan and Expression
|
||||
|
||||
They are structures to express logic from PromQL. The proposed implementation is built on top of DataFusion, thus our plan and expression are in form of `ExtensionPlan` and `ScalarUDF`. The only difference between them in this context is the return type: plan returns a record batch while expression returns a single column.
|
||||
|
||||
This RFC proposes to add four new plans, they are fundamental building blocks that mainly handle data selection logic in PromQL, for the following calculation expressions.
|
||||
|
||||
- `SeriesNormalize`
|
||||
|
||||
Sort data inside one series on the timestamp column, and bias "offset" if has. This plan usually comes after `TableScan` (or `TableScan` and `Filter`) plan.
|
||||
|
||||
- `VectorManipulator` and `MatrixManipulator`
|
||||
|
||||
Corresponding to `InstantSelector` and `RangeSelector`. We don't calculate timestamp by timestamp, thus use "vector" instead of "instant", this image shows the difference. And "matrix" is another name for "range vector", for not confused with our "vector". The following section will detail how they are implemented using Arrow.
|
||||
|
||||

|
||||
|
||||
Due to "interval" parameter in PromQL, data after "selector" (or "manipulator" here) are usually shorter than input. And we have to modify the entire record batch to shorten both timestamp, value and tag columns. So they are formed as plan.
|
||||
|
||||
- `PromAggregator`
|
||||
|
||||
The carrier of aggregator expressions. This should not be very different from the DataFusion built-in `Aggregate` plan, except PromQL can use "group without" to do reverse selection.
|
||||
|
||||
PromQL has around 70 expressions and functions. But luckily we can reuse lots of them from DataFusion. Like unary expression, binary expression and aggregator. We only need to implement those PromQL-specific expressions, like `rate` or `percentile`. The following table lists some typical functions in PromQL, and their signature in the proposed implementation. Other function should be the same.
|
||||
|
||||
| Name | In Param(s) | Out Param(s) | Explain |
|
||||
|-------------------- |------------------------------------------------------ |-------------- |-------------------- |
|
||||
| instant_delta | Matrix T | Array T | idelta in PromQL |
|
||||
| increase | Matrix T | Array T | increase in PromQL |
|
||||
| extrapolate_factor | - Matrix T<br>- Array Timestamp<br>- Array Timestamp | Array T | * |
|
||||
|
||||
*: *`extrapolate_factor` is one of the "dark sides" in PromQL. In short it's a translation of this [paragraph](https://github.com/prometheus/prometheus/blob/0372e259baf014bbade3134fd79bcdfd8cbdef2c/promql/functions.go#L134-L159)*
|
||||
|
||||
To reuse those common calculation logic, we can break them into serval expressions, and assemble in the logic planning phase. Like `rate()` in PromQL can be represented as `increase / extrapolate_factor`.
|
||||
|
||||
## Data Model
|
||||
|
||||
This part explains how data is represented. Following the data model in GreptimeDB, all the data are stored as table, with tag columns, timestamp column and value column. Table to record batch is very straightforward. So an instant vector can be thought of as a row (though as said before, we don't use instant vectors) in the table. Given four basic types in PromQL: scalar, string, instant vector and range vector, only the last "range vector" need some tricks to adapt our columnar calculation.
|
||||
|
||||
Range vector is some sort of matrix, it's consisted of small one-dimension vectors, with each being an input of range function. And, applying range function to a range vector can be thought of kind of convolution.
|
||||
|
||||

|
||||
|
||||
(Left is an illustration of range vector. Notice the Y-axis has no meaning, it's just put different pieces separately. The right side is an imagined "matrix" as range function. Multiplying the left side to it can get a one-dimension "matrix" with four elements. That's the evaluation result of a range vector.)
|
||||
|
||||
To adapt this range vector to record batch, it should be represented by a column. This RFC proposes to use `DictionaryArray` from Arrow to represent range vector, or `Matrix`. This is "misusing" `DictionaryArray` to ship some additional information about an array. Because the range vector is sliding over one series, we only need to know the `offset` and `length` of each slides to reconstruct the matrix from an array:
|
||||
|
||||

|
||||
|
||||
The length is not fixed, it depends on the input's timestamp. An PoC implementation of `Matrix` and `increase()` can be found in [this repo](https://github.com/waynexia/corroding-prometheus).
|
||||
|
||||
## Example
|
||||
|
||||
The logic plan of PromQL query
|
||||
```promql
|
||||
# start: 2022-12-20T10:00:00
|
||||
# end: 2022-12-21T10:00:00
|
||||
# interval: 1m
|
||||
# lookback: 30s
|
||||
sum (rate(request_duration[5m])) by (idc)
|
||||
```
|
||||
looks like
|
||||
|
||||
<!-- title: 'PromAggregator: \naggr = sum, column = idc'
|
||||
operator: prom
|
||||
inputs:
|
||||
- title: 'Matrix Manipulator: \ninterval = 1m, range = 5m, expr = div(increase(value), extrapolate_factor(timestamp))'
|
||||
operator: prom
|
||||
inputs:
|
||||
- title: 'Series Normalize: \noffset = 0'
|
||||
operator: prom
|
||||
inputs:
|
||||
- title: 'Filter: \ntimetamp > 2022-12-20T10:00:00 && timestamp < 2022-12-21T10:00:00'
|
||||
operator: filter
|
||||
inputs:
|
||||
- title: 'Table Scan: \ntable = request_duration, timetamp > 2022-12-20T10:00:00 && timestamp < 2022-12-21T10:00:00'
|
||||
operator: scan -->
|
||||
|
||||

|
||||
|
||||
# Drawbacks
|
||||
|
||||
Human-being is always error-prone. It's harder to endeavor to rewrite from the ground and requires more attention to ensure correctness, than translate line-by-line. And, since the evaluator's architecture are different, it might be painful to catch up with PromQL's breaking update (if any) in the future.
|
||||
|
||||
Misusing Arrow's DictionaryVector as Matrix is another point. This hack needs some `unsafe` function call to bypass Arrow's check. And though Arrow's API is stable, this is still an undocumented behavior.
|
||||
|
||||
# Alternatives
|
||||
|
||||
There are a few alternatives we've considered:
|
||||
- Wrap the existing PromQL's implementation via FFI, and import it to GreptimeDB.
|
||||
- Translate its evaluator engine line-by-line, rather than rewrite one.
|
||||
- Integrate the Prometheus server into GreptimeDB via RPC, making it a detached execution engine for PromQL.
|
||||
|
||||
The first and second options are making a separate execution engine in GreptimeDB, they may alleviate the pain during rewriting, but will have negative impacts to afterward evolve like resource management. And introduce another deploy component in the last option will bring a complex deploy architecture.
|
||||
|
||||
And all of them are more or less redundant in data transportation that affects performance and resources. The proposed built-in executing procedure is also easy to integrate and expose to the existing SQL interface GreptimeDB currently provides. Some concepts in PromQL like sliding windows (range vector in PromQL) are very convenient and ergonomic in analyzing series data. This makes it not only a PromQL evaluator, but also an enhancement to our query system.
|
||||
@@ -367,7 +367,7 @@ pub struct TableEntryValue {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use log_store::fs::noop::NoopLogStore;
|
||||
use log_store::NoopLogStore;
|
||||
use mito::config::EngineConfig;
|
||||
use mito::engine::MitoEngine;
|
||||
use object_store::ObjectStore;
|
||||
|
||||
@@ -125,7 +125,7 @@ impl TryFrom<StartCommand> for DatanodeOptions {
|
||||
}
|
||||
|
||||
if let Some(wal_dir) = cmd.wal_dir {
|
||||
opts.wal_dir = wal_dir;
|
||||
opts.wal.dir = wal_dir;
|
||||
}
|
||||
Ok(opts)
|
||||
}
|
||||
@@ -151,7 +151,7 @@ mod tests {
|
||||
};
|
||||
let options: DatanodeOptions = cmd.try_into().unwrap();
|
||||
assert_eq!("127.0.0.1:3001".to_string(), options.rpc_addr);
|
||||
assert_eq!("/tmp/greptimedb/wal".to_string(), options.wal_dir);
|
||||
assert_eq!("/tmp/greptimedb/wal".to_string(), options.wal.dir);
|
||||
assert_eq!("127.0.0.1:4406".to_string(), options.mysql_addr);
|
||||
assert_eq!(4, options.mysql_runtime_size);
|
||||
let MetaClientOpts {
|
||||
|
||||
@@ -16,7 +16,7 @@ use std::sync::Arc;
|
||||
|
||||
use clap::Parser;
|
||||
use common_telemetry::info;
|
||||
use datanode::datanode::{Datanode, DatanodeOptions, ObjectStoreConfig};
|
||||
use datanode::datanode::{Datanode, DatanodeOptions, ObjectStoreConfig, WalConfig};
|
||||
use datanode::instance::InstanceRef;
|
||||
use frontend::frontend::{Frontend, FrontendOptions};
|
||||
use frontend::grpc::GrpcOptions;
|
||||
@@ -72,7 +72,7 @@ pub struct StandaloneOptions {
|
||||
pub influxdb_options: Option<InfluxdbOptions>,
|
||||
pub prometheus_options: Option<PrometheusOptions>,
|
||||
pub mode: Mode,
|
||||
pub wal_dir: String,
|
||||
pub wal: WalConfig,
|
||||
pub storage: ObjectStoreConfig,
|
||||
pub enable_memory_catalog: bool,
|
||||
}
|
||||
@@ -88,7 +88,7 @@ impl Default for StandaloneOptions {
|
||||
influxdb_options: Some(InfluxdbOptions::default()),
|
||||
prometheus_options: Some(PrometheusOptions::default()),
|
||||
mode: Mode::Standalone,
|
||||
wal_dir: "/tmp/greptimedb/wal".to_string(),
|
||||
wal: WalConfig::default(),
|
||||
storage: ObjectStoreConfig::default(),
|
||||
enable_memory_catalog: false,
|
||||
}
|
||||
@@ -112,7 +112,7 @@ impl StandaloneOptions {
|
||||
|
||||
fn datanode_options(self) -> DatanodeOptions {
|
||||
DatanodeOptions {
|
||||
wal_dir: self.wal_dir,
|
||||
wal: self.wal,
|
||||
storage: self.storage,
|
||||
enable_memory_catalog: self.enable_memory_catalog,
|
||||
..Default::default()
|
||||
|
||||
@@ -45,6 +45,35 @@ impl Default for ObjectStoreConfig {
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||
pub struct WalConfig {
|
||||
// wal directory
|
||||
pub dir: String,
|
||||
// wal file size in bytes
|
||||
pub file_size: usize,
|
||||
// wal purge threshold in bytes
|
||||
pub purge_threshold: usize,
|
||||
// purge interval in seconds
|
||||
pub purge_interval: u64,
|
||||
// read batch size
|
||||
pub read_batch_size: usize,
|
||||
// whether to sync log file after every write
|
||||
pub sync_write: bool,
|
||||
}
|
||||
|
||||
impl Default for WalConfig {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
dir: "/tmp/greptimedb/wal".to_string(),
|
||||
file_size: 1024 * 1024 * 1024, // log file size 1G
|
||||
purge_threshold: 1024 * 1024 * 1024 * 50, // purge threshold 50G
|
||||
purge_interval: 600,
|
||||
read_batch_size: 128,
|
||||
sync_write: false,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct DatanodeOptions {
|
||||
pub node_id: Option<u64>,
|
||||
@@ -53,7 +82,7 @@ pub struct DatanodeOptions {
|
||||
pub mysql_addr: String,
|
||||
pub mysql_runtime_size: usize,
|
||||
pub meta_client_opts: Option<MetaClientOpts>,
|
||||
pub wal_dir: String,
|
||||
pub wal: WalConfig,
|
||||
pub storage: ObjectStoreConfig,
|
||||
pub enable_memory_catalog: bool,
|
||||
pub mode: Mode,
|
||||
@@ -68,7 +97,7 @@ impl Default for DatanodeOptions {
|
||||
mysql_addr: "127.0.0.1:4406".to_string(),
|
||||
mysql_runtime_size: 2,
|
||||
meta_client_opts: None,
|
||||
wal_dir: "/tmp/greptimedb/wal".to_string(),
|
||||
wal: WalConfig::default(),
|
||||
storage: ObjectStoreConfig::default(),
|
||||
enable_memory_catalog: false,
|
||||
mode: Mode::Standalone,
|
||||
|
||||
@@ -1,10 +1,10 @@
|
||||
// Copyright 2022 Greptime Team
|
||||
// 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
|
||||
// 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,
|
||||
@@ -144,12 +144,6 @@ pub enum Error {
|
||||
source: log_store::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to star log store gc task, source: {}", source))]
|
||||
StartLogStore {
|
||||
#[snafu(backtrace)]
|
||||
source: log_store::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to storage engine, source: {}", source))]
|
||||
OpenStorageEngine { source: StorageError },
|
||||
|
||||
@@ -367,7 +361,6 @@ impl ErrorExt for Error {
|
||||
Error::BumpTableId { source, .. } => source.status_code(),
|
||||
Error::MissingNodeId { .. } => StatusCode::InvalidArguments,
|
||||
Error::MissingMetasrvOpts { .. } => StatusCode::InvalidArguments,
|
||||
Error::StartLogStore { source, .. } => source.status_code(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -21,8 +21,8 @@ use catalog::remote::MetaKvBackend;
|
||||
use catalog::CatalogManagerRef;
|
||||
use common_grpc::channel_manager::{ChannelConfig, ChannelManager};
|
||||
use common_telemetry::logging::info;
|
||||
use log_store::fs::config::LogConfig;
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use log_store::LogConfig;
|
||||
use meta_client::client::{MetaClient, MetaClientBuilder};
|
||||
use meta_client::MetaClientOpts;
|
||||
use mito::config::EngineConfig as TableEngineConfig;
|
||||
@@ -36,13 +36,12 @@ use servers::Mode;
|
||||
use snafu::prelude::*;
|
||||
use storage::config::EngineConfig as StorageEngineConfig;
|
||||
use storage::EngineImpl;
|
||||
use store_api::logstore::LogStore;
|
||||
use table::table::TableIdProviderRef;
|
||||
|
||||
use crate::datanode::{DatanodeOptions, ObjectStoreConfig};
|
||||
use crate::datanode::{DatanodeOptions, ObjectStoreConfig, WalConfig};
|
||||
use crate::error::{
|
||||
self, CatalogSnafu, MetaClientInitSnafu, MissingMetasrvOptsSnafu, MissingNodeIdSnafu,
|
||||
NewCatalogSnafu, Result, StartLogStoreSnafu,
|
||||
NewCatalogSnafu, OpenLogStoreSnafu, Result,
|
||||
};
|
||||
use crate::heartbeat::HeartbeatTask;
|
||||
use crate::script::ScriptExecutor;
|
||||
@@ -52,7 +51,7 @@ mod grpc;
|
||||
mod script;
|
||||
mod sql;
|
||||
|
||||
pub(crate) type DefaultEngine = MitoEngine<EngineImpl<LocalFileLogStore>>;
|
||||
pub(crate) type DefaultEngine = MitoEngine<EngineImpl<RaftEngineLogStore>>;
|
||||
|
||||
// An abstraction to read/write services.
|
||||
pub struct Instance {
|
||||
@@ -62,7 +61,6 @@ pub struct Instance {
|
||||
pub(crate) script_executor: ScriptExecutor,
|
||||
pub(crate) table_id_provider: Option<TableIdProviderRef>,
|
||||
pub(crate) heartbeat_task: Option<HeartbeatTask>,
|
||||
pub(crate) logstore: Arc<LocalFileLogStore>,
|
||||
}
|
||||
|
||||
pub type InstanceRef = Arc<Instance>;
|
||||
@@ -70,7 +68,7 @@ pub type InstanceRef = Arc<Instance>;
|
||||
impl Instance {
|
||||
pub async fn new(opts: &DatanodeOptions) -> Result<Self> {
|
||||
let object_store = new_object_store(&opts.storage).await?;
|
||||
let logstore = Arc::new(create_local_file_log_store(&opts.wal_dir).await?);
|
||||
let logstore = Arc::new(create_log_store(&opts.wal).await?);
|
||||
|
||||
let meta_client = match opts.mode {
|
||||
Mode::Standalone => None,
|
||||
@@ -160,7 +158,6 @@ impl Instance {
|
||||
script_executor,
|
||||
heartbeat_task,
|
||||
table_id_provider,
|
||||
logstore,
|
||||
})
|
||||
}
|
||||
|
||||
@@ -169,7 +166,6 @@ impl Instance {
|
||||
.start()
|
||||
.await
|
||||
.context(NewCatalogSnafu)?;
|
||||
self.logstore.start().await.context(StartLogStoreSnafu)?;
|
||||
if let Some(task) = &self.heartbeat_task {
|
||||
task.start().await?;
|
||||
}
|
||||
@@ -275,23 +271,23 @@ async fn new_metasrv_client(node_id: u64, meta_config: &MetaClientOpts) -> Resul
|
||||
Ok(meta_client)
|
||||
}
|
||||
|
||||
pub(crate) async fn create_local_file_log_store(
|
||||
path: impl AsRef<str>,
|
||||
) -> Result<LocalFileLogStore> {
|
||||
let path = path.as_ref();
|
||||
pub(crate) async fn create_log_store(wal_config: &WalConfig) -> Result<RaftEngineLogStore> {
|
||||
// create WAL directory
|
||||
fs::create_dir_all(path::Path::new(path)).context(error::CreateDirSnafu { dir: path })?;
|
||||
|
||||
info!("The WAL directory is: {}", path);
|
||||
|
||||
fs::create_dir_all(path::Path::new(&wal_config.dir)).context(error::CreateDirSnafu {
|
||||
dir: &wal_config.dir,
|
||||
})?;
|
||||
info!("Creating logstore with config: {:?}", wal_config);
|
||||
let log_config = LogConfig {
|
||||
log_file_dir: path.to_string(),
|
||||
..Default::default()
|
||||
file_size: wal_config.file_size,
|
||||
log_file_dir: wal_config.dir.clone(),
|
||||
purge_interval: Duration::from_secs(wal_config.purge_interval),
|
||||
purge_threshold: wal_config.purge_threshold,
|
||||
read_batch_size: wal_config.read_batch_size,
|
||||
sync_write: wal_config.sync_write,
|
||||
};
|
||||
|
||||
let log_store = LocalFileLogStore::open(&log_config)
|
||||
let logstore = RaftEngineLogStore::try_new(log_config)
|
||||
.await
|
||||
.context(error::OpenLogStoreSnafu)?;
|
||||
|
||||
Ok(log_store)
|
||||
.context(OpenLogStoreSnafu)?;
|
||||
Ok(logstore)
|
||||
}
|
||||
|
||||
@@ -29,7 +29,7 @@ use table::table::TableIdProvider;
|
||||
use crate::datanode::DatanodeOptions;
|
||||
use crate::error::Result;
|
||||
use crate::heartbeat::HeartbeatTask;
|
||||
use crate::instance::{create_local_file_log_store, new_object_store, DefaultEngine, Instance};
|
||||
use crate::instance::{create_log_store, new_object_store, DefaultEngine, Instance};
|
||||
use crate::script::ScriptExecutor;
|
||||
use crate::sql::SqlHandler;
|
||||
|
||||
@@ -41,7 +41,7 @@ impl Instance {
|
||||
|
||||
pub async fn with_mock_meta_server(opts: &DatanodeOptions, meta_srv: MockInfo) -> Result<Self> {
|
||||
let object_store = new_object_store(&opts.storage).await?;
|
||||
let logstore = Arc::new(create_local_file_log_store(&opts.wal_dir).await?);
|
||||
let logstore = Arc::new(create_log_store(&opts.wal).await?);
|
||||
let meta_client = Arc::new(mock_meta_client(meta_srv, opts.node_id.unwrap_or(42)).await);
|
||||
let table_engine = Arc::new(DefaultEngine::new(
|
||||
TableEngineConfig::default(),
|
||||
@@ -83,7 +83,6 @@ impl Instance {
|
||||
script_executor,
|
||||
table_id_provider: Some(Arc::new(LocalTableIdProvider::default())),
|
||||
heartbeat_task: Some(heartbeat_task),
|
||||
logstore,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -125,7 +125,7 @@ mod tests {
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnSchema, SchemaBuilder, SchemaRef};
|
||||
use datatypes::value::Value;
|
||||
use log_store::fs::noop::NoopLogStore;
|
||||
use log_store::NoopLogStore;
|
||||
use mito::config::EngineConfig as TableEngineConfig;
|
||||
use mito::engine::MitoEngine;
|
||||
use object_store::services::fs::Builder;
|
||||
|
||||
@@ -28,7 +28,7 @@ use table::engine::{EngineContext, TableEngineRef};
|
||||
use table::requests::CreateTableRequest;
|
||||
use tempdir::TempDir;
|
||||
|
||||
use crate::datanode::{DatanodeOptions, ObjectStoreConfig};
|
||||
use crate::datanode::{DatanodeOptions, ObjectStoreConfig, WalConfig};
|
||||
use crate::error::{CreateTableSnafu, Result};
|
||||
use crate::sql::SqlHandler;
|
||||
|
||||
@@ -43,7 +43,10 @@ pub fn create_tmp_dir_and_datanode_opts(name: &str) -> (DatanodeOptions, TestGua
|
||||
let wal_tmp_dir = TempDir::new(&format!("gt_wal_{name}")).unwrap();
|
||||
let data_tmp_dir = TempDir::new(&format!("gt_data_{name}")).unwrap();
|
||||
let opts = DatanodeOptions {
|
||||
wal_dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
wal: WalConfig {
|
||||
dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
},
|
||||
storage: ObjectStoreConfig::File {
|
||||
data_dir: data_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
},
|
||||
|
||||
@@ -20,7 +20,7 @@ use catalog::remote::MetaKvBackend;
|
||||
use client::Client;
|
||||
use common_grpc::channel_manager::ChannelManager;
|
||||
use common_runtime::Builder as RuntimeBuilder;
|
||||
use datanode::datanode::{DatanodeOptions, ObjectStoreConfig};
|
||||
use datanode::datanode::{DatanodeOptions, ObjectStoreConfig, WalConfig};
|
||||
use datanode::instance::Instance as DatanodeInstance;
|
||||
use meta_client::client::MetaClientBuilder;
|
||||
use meta_client::rpc::Peer;
|
||||
@@ -62,7 +62,10 @@ fn create_tmp_dir_and_datanode_opts(name: &str) -> (DatanodeOptions, TestGuard)
|
||||
let wal_tmp_dir = TempDir::new(&format!("gt_wal_{name}")).unwrap();
|
||||
let data_tmp_dir = TempDir::new(&format!("gt_data_{name}")).unwrap();
|
||||
let opts = DatanodeOptions {
|
||||
wal_dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
wal: WalConfig {
|
||||
dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
},
|
||||
storage: ObjectStoreConfig::File {
|
||||
data_dir: data_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
},
|
||||
@@ -142,7 +145,10 @@ async fn create_dist_datanode_instance(
|
||||
let data_tmp_dir = TempDir::new_in("/tmp", &format!("dist_datanode-data-{current}")).unwrap();
|
||||
let opts = DatanodeOptions {
|
||||
node_id: Some(datanode_id),
|
||||
wal_dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
wal: WalConfig {
|
||||
dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
},
|
||||
storage: ObjectStoreConfig::File {
|
||||
data_dir: data_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
},
|
||||
|
||||
@@ -4,10 +4,15 @@ version.workspace = true
|
||||
edition.workspace = true
|
||||
license.workspace = true
|
||||
|
||||
[build-dependencies]
|
||||
protobuf-build = { version = "0.14", default-features = false, features = [
|
||||
"protobuf-codec",
|
||||
] }
|
||||
|
||||
[dependencies]
|
||||
arc-swap = "1.5"
|
||||
async-stream = "0.3"
|
||||
async-trait = "0.1"
|
||||
async-stream="0.3"
|
||||
async-trait="0.1"
|
||||
base64 = "0.13"
|
||||
byteorder = "1.4"
|
||||
bytes = "1.1"
|
||||
@@ -16,9 +21,11 @@ common-error = { path = "../common/error" }
|
||||
common-runtime = { path = "../common/runtime" }
|
||||
common-telemetry = { path = "../common/telemetry" }
|
||||
crc = "3.0"
|
||||
futures = "0.3"
|
||||
futures-util = "0.3"
|
||||
futures="0.3"
|
||||
hex = "0.4"
|
||||
protobuf = { version = "2", features = ["bytes"] }
|
||||
raft-engine = "0.3"
|
||||
snafu = { version = "0.7", features = ["backtraces"] }
|
||||
store-api = { path = "../store-api" }
|
||||
tempdir = "0.3"
|
||||
|
||||
@@ -12,13 +12,13 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#[cfg(all(unix, not(miri)))]
|
||||
mod unix;
|
||||
// todo(hl): maybe support windows seek_write/seek_read
|
||||
#[cfg(any(not(unix), miri))]
|
||||
mod fallback;
|
||||
use protobuf_build::Builder;
|
||||
|
||||
#[cfg(any(all(not(unix), not(windows)), miri))]
|
||||
pub use fallback::{pread_exact, pwrite_all};
|
||||
#[cfg(all(unix, not(miri)))]
|
||||
pub use unix::{pread_exact, pwrite_all};
|
||||
fn main() {
|
||||
let base = std::env::var("CARGO_MANIFEST_DIR").unwrap_or_else(|_| ".".to_string());
|
||||
Builder::new()
|
||||
.search_dir_for_protos(&format!("{base}/proto"))
|
||||
.includes(&[format!("{base}/include"), format!("{base}/proto")])
|
||||
.include_google_protos()
|
||||
.generate()
|
||||
}
|
||||
17
src/log-store/proto/logstore.proto
Normal file
17
src/log-store/proto/logstore.proto
Normal file
@@ -0,0 +1,17 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package logstore;
|
||||
|
||||
message EntryImpl {
|
||||
uint64 id = 1;
|
||||
uint64 namespace_id = 2;
|
||||
bytes data = 3;
|
||||
}
|
||||
|
||||
message LogStoreState {
|
||||
uint64 last_index = 1;
|
||||
}
|
||||
|
||||
message NamespaceImpl {
|
||||
uint64 id = 1;
|
||||
}
|
||||
@@ -16,10 +16,12 @@ use std::time::Duration;
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct LogConfig {
|
||||
pub append_buffer_size: usize,
|
||||
pub max_log_file_size: usize,
|
||||
pub file_size: usize,
|
||||
pub log_file_dir: String,
|
||||
pub gc_interval: Duration,
|
||||
pub purge_interval: Duration,
|
||||
pub purge_threshold: usize,
|
||||
pub read_batch_size: usize,
|
||||
pub sync_write: bool,
|
||||
}
|
||||
|
||||
impl Default for LogConfig {
|
||||
@@ -27,10 +29,12 @@ impl Default for LogConfig {
|
||||
/// in tests.
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 1024 * 1024 * 1024,
|
||||
file_size: 1024 * 1024 * 1024,
|
||||
log_file_dir: "/tmp/greptimedb".to_string(),
|
||||
gc_interval: Duration::from_secs(10 * 60),
|
||||
purge_interval: Duration::from_secs(10 * 60),
|
||||
purge_threshold: 1024 * 1024 * 1024 * 50,
|
||||
read_batch_size: 128,
|
||||
sync_write: false,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -46,8 +50,10 @@ mod tests {
|
||||
common_telemetry::logging::init_default_ut_logging();
|
||||
let default = LogConfig::default();
|
||||
info!("LogConfig::default(): {:?}", default);
|
||||
assert_eq!(1024 * 1024 * 1024, default.max_log_file_size);
|
||||
assert_eq!(128, default.append_buffer_size);
|
||||
assert_eq!(Duration::from_secs(600), default.gc_interval);
|
||||
assert_eq!(1024 * 1024 * 1024, default.file_size);
|
||||
assert_eq!(Duration::from_secs(600), default.purge_interval);
|
||||
assert_eq!(1024 * 1024 * 1024 * 50, default.purge_threshold);
|
||||
assert_eq!(128, default.read_batch_size);
|
||||
assert!(!default.sync_write);
|
||||
}
|
||||
}
|
||||
@@ -14,7 +14,6 @@
|
||||
|
||||
use std::any::Any;
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_error::prelude::{ErrorExt, Snafu};
|
||||
use snafu::{Backtrace, ErrorCompat};
|
||||
use tokio::task::JoinError;
|
||||
@@ -22,83 +21,46 @@ use tokio::task::JoinError;
|
||||
#[derive(Debug, Snafu)]
|
||||
#[snafu(visibility(pub))]
|
||||
pub enum Error {
|
||||
#[snafu(display("Failed to encode entry, source: {}", source))]
|
||||
Encode { source: common_base::buffer::Error },
|
||||
|
||||
#[snafu(display("Failed to decode entry, remain size: {}", size))]
|
||||
Decode { size: usize, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("No enough data to decode, try again"))]
|
||||
DecodeAgain,
|
||||
|
||||
#[snafu(display("Failed to append entry, source: {}", source))]
|
||||
Append {
|
||||
#[snafu(backtrace)]
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to wait for log file write complete, source: {}", source))]
|
||||
Write { source: tokio::task::JoinError },
|
||||
|
||||
#[snafu(display("Entry corrupted, msg: {}", msg))]
|
||||
Corrupted { msg: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("IO error, source: {}", source))]
|
||||
Io {
|
||||
source: std::io::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to create path {}, source: {}", path, source))]
|
||||
CreateDir {
|
||||
path: String,
|
||||
source: std::io::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to read path {}, source: {}", path, source))]
|
||||
ReadPath {
|
||||
path: String,
|
||||
source: std::io::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to open log file {}, source: {}", file_name, source))]
|
||||
OpenLog {
|
||||
file_name: String,
|
||||
source: std::io::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("File name {} illegal", file_name))]
|
||||
FileNameIllegal {
|
||||
file_name: String,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Internal error, msg: {}", msg))]
|
||||
Internal { msg: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("End of LogFile"))]
|
||||
Eof,
|
||||
|
||||
#[snafu(display("File duplicate on start: {}", msg))]
|
||||
DuplicateFile { msg: String },
|
||||
|
||||
#[snafu(display("Log file suffix is illegal: {}", suffix))]
|
||||
SuffixIllegal { suffix: String },
|
||||
|
||||
#[snafu(display("Failed while waiting for write to finish, source: {}", source))]
|
||||
WaitWrite { source: tokio::task::JoinError },
|
||||
|
||||
#[snafu(display("Invalid logstore status, msg: {}", msg))]
|
||||
InvalidState { msg: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Failed to wait for gc task to stop, source: {}", source))]
|
||||
WaitGcTaskStop {
|
||||
source: JoinError,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to add entry to LogBatch, source: {}", source))]
|
||||
AddEntryLogBatch {
|
||||
source: raft_engine::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to perform raft-engine operation, source: {}", source))]
|
||||
RaftEngine {
|
||||
source: raft_engine::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Log store not started yet"))]
|
||||
IllegalState { backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Namespace is illegal: {}", ns))]
|
||||
IllegalNamespace { ns: u64, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to fetch entries from namespace: {}, start: {}, end: {}, max size: {}, source: {}",
|
||||
start,
|
||||
end,
|
||||
max_size,
|
||||
source,
|
||||
ns
|
||||
))]
|
||||
FetchEntry {
|
||||
ns: u64,
|
||||
start: u64,
|
||||
end: u64,
|
||||
max_size: usize,
|
||||
source: raft_engine::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
}
|
||||
|
||||
impl ErrorExt for Error {
|
||||
|
||||
@@ -1,46 +0,0 @@
|
||||
// Copyright 2022 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 store_api::logstore::entry::{Id, Offset};
|
||||
use store_api::logstore::AppendResponse;
|
||||
|
||||
mod chunk;
|
||||
pub mod config;
|
||||
mod crc;
|
||||
mod entry;
|
||||
mod file;
|
||||
mod file_name;
|
||||
mod index;
|
||||
mod io;
|
||||
pub mod log;
|
||||
mod namespace;
|
||||
pub mod noop;
|
||||
|
||||
#[derive(Debug, PartialEq, Eq)]
|
||||
pub struct AppendResponseImpl {
|
||||
entry_id: Id,
|
||||
offset: Offset,
|
||||
}
|
||||
|
||||
impl AppendResponse for AppendResponseImpl {
|
||||
#[inline]
|
||||
fn entry_id(&self) -> Id {
|
||||
self.entry_id
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn offset(&self) -> Offset {
|
||||
self.offset
|
||||
}
|
||||
}
|
||||
@@ -1,233 +0,0 @@
|
||||
// Copyright 2022 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::collections::LinkedList;
|
||||
|
||||
use common_base::buffer::{Buffer, UnderflowSnafu};
|
||||
use snafu::ensure;
|
||||
|
||||
pub const DEFAULT_CHUNK_SIZE: usize = 4096;
|
||||
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct Chunk {
|
||||
// internal data
|
||||
pub data: Box<[u8]>,
|
||||
// read offset
|
||||
pub read_offset: usize,
|
||||
// write offset
|
||||
pub write_offset: usize,
|
||||
}
|
||||
|
||||
impl Default for Chunk {
|
||||
fn default() -> Self {
|
||||
let data = vec![0u8; DEFAULT_CHUNK_SIZE].into_boxed_slice();
|
||||
Self {
|
||||
write_offset: 0,
|
||||
read_offset: 0,
|
||||
data,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Chunk {
|
||||
#[cfg(test)]
|
||||
pub fn copy_from_slice(s: &[u8]) -> Self {
|
||||
let src_len = s.len();
|
||||
// before [box syntax](https://github.com/rust-lang/rust/issues/49733) becomes stable,
|
||||
// we can only initialize an array on heap like this.
|
||||
let mut data = vec![0u8; src_len].into_boxed_slice();
|
||||
data[0..src_len].copy_from_slice(s);
|
||||
Self {
|
||||
read_offset: 0,
|
||||
write_offset: src_len,
|
||||
data,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn new(data: Box<[u8]>, write: usize) -> Self {
|
||||
Self {
|
||||
write_offset: write,
|
||||
read_offset: 0,
|
||||
data,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn len(&self) -> usize {
|
||||
self.write_offset - self.read_offset
|
||||
}
|
||||
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.len() == 0
|
||||
}
|
||||
|
||||
/// allows short read.
|
||||
/// Calling read **will not** advance read cursor, must call `advance` manually.
|
||||
pub fn read(&self, dst: &mut [u8]) -> usize {
|
||||
let size = self.len().min(dst.len());
|
||||
let range = self.read_offset..(self.read_offset + size);
|
||||
dst[0..size].copy_from_slice(&self.data[range]);
|
||||
size
|
||||
}
|
||||
|
||||
pub fn advance(&mut self, by: usize) -> usize {
|
||||
assert!(
|
||||
self.write_offset >= self.read_offset,
|
||||
"Illegal chunk state, read: {}, write: {}",
|
||||
self.read_offset,
|
||||
self.write_offset
|
||||
);
|
||||
let step = by.min(self.write_offset - self.read_offset);
|
||||
self.read_offset += step;
|
||||
step
|
||||
}
|
||||
}
|
||||
|
||||
pub struct ChunkList {
|
||||
chunks: LinkedList<Chunk>,
|
||||
}
|
||||
|
||||
impl ChunkList {
|
||||
pub fn new() -> Self {
|
||||
Self {
|
||||
chunks: LinkedList::new(),
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn push(&mut self, chunk: Chunk) {
|
||||
self.chunks.push_back(chunk);
|
||||
}
|
||||
}
|
||||
|
||||
impl Buffer for ChunkList {
|
||||
fn remaining_size(&self) -> usize {
|
||||
self.chunks.iter().map(|c| c.len()).sum()
|
||||
}
|
||||
|
||||
fn peek_to_slice(&self, mut dst: &mut [u8]) -> common_base::buffer::Result<()> {
|
||||
ensure!(self.remaining_size() >= dst.len(), UnderflowSnafu);
|
||||
|
||||
for c in &self.chunks {
|
||||
if dst.is_empty() {
|
||||
break;
|
||||
}
|
||||
let read = c.read(dst);
|
||||
dst = &mut dst[read..];
|
||||
}
|
||||
|
||||
ensure!(dst.is_empty(), UnderflowSnafu);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn read_to_slice(&mut self, dst: &mut [u8]) -> common_base::buffer::Result<()> {
|
||||
self.peek_to_slice(dst)?;
|
||||
self.advance_by(dst.len());
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn advance_by(&mut self, by: usize) {
|
||||
let mut left = by;
|
||||
while left > 0 {
|
||||
if let Some(c) = self.chunks.front_mut() {
|
||||
let actual = c.advance(left);
|
||||
if c.is_empty() {
|
||||
self.chunks.pop_front(); // remove first chunk
|
||||
}
|
||||
left -= actual;
|
||||
} else {
|
||||
panic!("Advance step [{by}] exceeds max readable bytes");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
pub fn test_chunk() {
|
||||
let chunk: Chunk = Chunk::copy_from_slice("hello".as_bytes());
|
||||
assert_eq!(5, chunk.write_offset);
|
||||
assert_eq!(0, chunk.read_offset);
|
||||
assert_eq!(5, chunk.len());
|
||||
|
||||
let mut dst = [0u8; 3];
|
||||
assert_eq!(3, chunk.read(&mut dst));
|
||||
assert_eq!(5, chunk.write_offset);
|
||||
assert_eq!(0, chunk.read_offset);
|
||||
assert_eq!(5, chunk.len());
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_chunk_short_read() {
|
||||
let chunk: Chunk = Chunk::copy_from_slice("hello".as_bytes());
|
||||
|
||||
let mut dst = vec![0u8; 8];
|
||||
let read = chunk.read(&mut dst);
|
||||
assert_eq!(5, read);
|
||||
assert_eq!(vec![b'h', b'e', b'l', b'l', b'o', 0x0, 0x0, 0x0], dst);
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_chunk_advance() {
|
||||
let mut chunk: Chunk = Chunk::copy_from_slice("hello".as_bytes());
|
||||
let mut dst = vec![0u8; 8];
|
||||
assert_eq!(5, chunk.read(&mut dst));
|
||||
assert_eq!(0, chunk.read_offset);
|
||||
assert_eq!(5, chunk.write_offset);
|
||||
|
||||
assert_eq!(1, chunk.advance(1));
|
||||
assert_eq!(1, chunk.read_offset);
|
||||
assert_eq!(5, chunk.write_offset);
|
||||
|
||||
assert_eq!(4, chunk.advance(5));
|
||||
assert_eq!(5, chunk.read_offset);
|
||||
assert_eq!(5, chunk.write_offset);
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_composite_chunk_read() {
|
||||
let mut chunks = ChunkList {
|
||||
chunks: LinkedList::new(),
|
||||
};
|
||||
|
||||
chunks.push(Chunk::copy_from_slice("abcd".as_bytes()));
|
||||
chunks.push(Chunk::copy_from_slice("12345".as_bytes()));
|
||||
assert_eq!(9, chunks.remaining_size());
|
||||
|
||||
let mut dst = [0u8; 2];
|
||||
chunks.peek_to_slice(&mut dst).unwrap();
|
||||
chunks.advance_by(2);
|
||||
assert_eq!([b'a', b'b'], dst);
|
||||
assert_eq!(2, chunks.chunks.len());
|
||||
|
||||
let mut dst = [0u8; 3];
|
||||
chunks.peek_to_slice(&mut dst).unwrap();
|
||||
chunks.advance_by(3);
|
||||
assert_eq!([b'c', b'd', b'1'], dst);
|
||||
assert_eq!(4, chunks.remaining_size());
|
||||
assert_eq!(1, chunks.chunks.len());
|
||||
|
||||
let mut dst = [0u8; 4];
|
||||
chunks.peek_to_slice(&mut dst).unwrap();
|
||||
chunks.advance_by(4);
|
||||
assert_eq!([b'2', b'3', b'4', b'5'], dst);
|
||||
assert_eq!(0, chunks.remaining_size());
|
||||
assert_eq!(0, chunks.chunks.len());
|
||||
|
||||
chunks.push(Chunk::copy_from_slice("uvwxyz".as_bytes()));
|
||||
assert_eq!(6, chunks.remaining_size());
|
||||
assert_eq!(1, chunks.chunks.len());
|
||||
}
|
||||
}
|
||||
@@ -1,17 +0,0 @@
|
||||
// Copyright 2022 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 crc::{Crc, CRC_32_ISCSI};
|
||||
|
||||
pub const CRC_ALGO: Crc<u32> = Crc::<u32>::new(&CRC_32_ISCSI);
|
||||
@@ -1,394 +0,0 @@
|
||||
// Copyright 2022 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::pin::Pin;
|
||||
use std::task::{Context, Poll};
|
||||
|
||||
use bytes::{Bytes, BytesMut};
|
||||
use common_base::buffer::{Buffer, BufferMut};
|
||||
use futures::Stream;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use store_api::logstore::entry::{Encode, Entry, Epoch, Id, Offset};
|
||||
use store_api::logstore::entry_stream::{EntryStream, SendableEntryStream};
|
||||
use store_api::logstore::namespace::{Id as NamespaceId, Namespace};
|
||||
|
||||
use crate::error::{CorruptedSnafu, DecodeAgainSnafu, DecodeSnafu, EncodeSnafu, Error};
|
||||
use crate::fs::crc;
|
||||
use crate::fs::namespace::LocalNamespace;
|
||||
|
||||
// length + offset + namespace id + epoch + crc
|
||||
const ENTRY_MIN_LEN: usize = HEADER_LENGTH + 4;
|
||||
// length + offset + namespace id + epoch
|
||||
const HEADER_LENGTH: usize = 4 + 8 + 8 + 8;
|
||||
|
||||
#[derive(Debug, PartialEq, Eq, Clone)]
|
||||
pub struct EntryImpl {
|
||||
pub data: Vec<u8>,
|
||||
pub offset: Offset,
|
||||
pub id: Id,
|
||||
pub namespace_id: NamespaceId,
|
||||
pub epoch: Epoch,
|
||||
}
|
||||
|
||||
impl EntryImpl {
|
||||
#[cfg(test)]
|
||||
fn set_offset(&mut self, offset: Offset) {
|
||||
self.offset = offset;
|
||||
}
|
||||
}
|
||||
|
||||
impl Encode for EntryImpl {
|
||||
type Error = Error;
|
||||
|
||||
/// Entry binary format (Little endian):
|
||||
///
|
||||
// ```text
|
||||
// +--------+--------------+-------+--------+--------+--------+
|
||||
// |entry id| namespace id | epoch | length | data | CRC |
|
||||
// +--------+--------------+-------+--------+--------+--------+
|
||||
// | 8 bytes| 8 bytes |8 bytes| 4 bytes|<length>| 4 bytes|
|
||||
// +--------+--------------+-------+--------+--------+--------+
|
||||
// ```
|
||||
///
|
||||
fn encode_to<T: BufferMut>(&self, buf: &mut T) -> Result<usize, Self::Error> {
|
||||
let data_length = self.data.len();
|
||||
buf.write_u64_le(self.id).context(EncodeSnafu)?;
|
||||
buf.write_u64_le(self.namespace_id).context(EncodeSnafu)?;
|
||||
buf.write_u64_le(self.epoch).context(EncodeSnafu)?;
|
||||
buf.write_u32_le(data_length as u32).context(EncodeSnafu)?;
|
||||
buf.write_from_slice(self.data.as_slice())
|
||||
.context(EncodeSnafu)?;
|
||||
let checksum = crc::CRC_ALGO.checksum(buf.as_slice());
|
||||
buf.write_u32_le(checksum).context(EncodeSnafu)?;
|
||||
Ok(data_length + ENTRY_MIN_LEN)
|
||||
}
|
||||
|
||||
fn decode<T: Buffer>(buf: &mut T) -> Result<Self, Self::Error> {
|
||||
ensure!(buf.remaining_size() >= HEADER_LENGTH, DecodeAgainSnafu);
|
||||
|
||||
macro_rules! map_err {
|
||||
($stmt: expr, $var: ident) => {
|
||||
$stmt.map_err(|_| {
|
||||
DecodeSnafu {
|
||||
size: $var.remaining_size(),
|
||||
}
|
||||
.build()
|
||||
})
|
||||
};
|
||||
}
|
||||
|
||||
let mut digest = crc::CRC_ALGO.digest();
|
||||
let mut header = [0u8; HEADER_LENGTH];
|
||||
buf.peek_to_slice(&mut header).unwrap();
|
||||
|
||||
let mut header = &header[..];
|
||||
let id = header.read_u64_le().unwrap(); // unwrap here is safe because header bytes must be present
|
||||
digest.update(&id.to_le_bytes());
|
||||
|
||||
let namespace_id = header.read_u64_le().unwrap();
|
||||
digest.update(&namespace_id.to_le_bytes());
|
||||
|
||||
let epoch = header.read_u64_le().unwrap();
|
||||
digest.update(&epoch.to_le_bytes());
|
||||
|
||||
let data_len = header.read_u32_le().unwrap();
|
||||
digest.update(&data_len.to_le_bytes());
|
||||
|
||||
ensure!(
|
||||
buf.remaining_size() >= ENTRY_MIN_LEN + data_len as usize,
|
||||
DecodeAgainSnafu
|
||||
);
|
||||
|
||||
buf.advance_by(HEADER_LENGTH);
|
||||
|
||||
let mut data = vec![0u8; data_len as usize];
|
||||
map_err!(buf.peek_to_slice(&mut data), buf)?;
|
||||
digest.update(&data);
|
||||
buf.advance_by(data_len as usize);
|
||||
|
||||
let crc_read = map_err!(buf.peek_u32_le(), buf)?;
|
||||
let crc_calc = digest.finalize();
|
||||
|
||||
ensure!(
|
||||
crc_read == crc_calc,
|
||||
CorruptedSnafu {
|
||||
msg: format!(
|
||||
"CRC mismatch while decoding entry, read: {}, calc: {}",
|
||||
hex::encode_upper(crc_read.to_le_bytes()),
|
||||
hex::encode_upper(crc_calc.to_le_bytes())
|
||||
)
|
||||
}
|
||||
);
|
||||
|
||||
buf.advance_by(4);
|
||||
|
||||
Ok(Self {
|
||||
id,
|
||||
data,
|
||||
epoch,
|
||||
offset: 0,
|
||||
namespace_id,
|
||||
})
|
||||
}
|
||||
|
||||
fn encoded_size(&self) -> usize {
|
||||
self.data.len() + ENTRY_MIN_LEN
|
||||
}
|
||||
}
|
||||
|
||||
impl EntryImpl {
|
||||
pub(crate) fn new(data: impl AsRef<[u8]>, id: Id, namespace: LocalNamespace) -> EntryImpl {
|
||||
EntryImpl {
|
||||
id,
|
||||
data: data.as_ref().to_vec(),
|
||||
offset: 0,
|
||||
epoch: 0,
|
||||
namespace_id: namespace.id(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry for EntryImpl {
|
||||
type Error = Error;
|
||||
type Namespace = LocalNamespace;
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
&self.data
|
||||
}
|
||||
|
||||
fn id(&self) -> Id {
|
||||
self.id
|
||||
}
|
||||
|
||||
fn offset(&self) -> Offset {
|
||||
self.offset
|
||||
}
|
||||
|
||||
fn set_id(&mut self, id: Id) {
|
||||
self.id = id;
|
||||
}
|
||||
|
||||
fn epoch(&self) -> Epoch {
|
||||
self.epoch
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
ENTRY_MIN_LEN + self.data.len()
|
||||
}
|
||||
|
||||
fn is_empty(&self) -> bool {
|
||||
self.data.is_empty()
|
||||
}
|
||||
|
||||
fn namespace(&self) -> Self::Namespace {
|
||||
LocalNamespace::new(self.namespace_id)
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<Bytes> for EntryImpl {
|
||||
type Error = Error;
|
||||
|
||||
fn try_from(mut value: Bytes) -> Result<Self, Self::Error> {
|
||||
EntryImpl::decode(&mut value)
|
||||
}
|
||||
}
|
||||
|
||||
impl From<&EntryImpl> for BytesMut {
|
||||
fn from(e: &EntryImpl) -> Self {
|
||||
let size = e.encoded_size();
|
||||
let mut res = BytesMut::with_capacity(size);
|
||||
e.encode_to(&mut res).unwrap(); // buffer is pre-allocated, so won't fail
|
||||
res
|
||||
}
|
||||
}
|
||||
|
||||
pub struct StreamImpl<'a> {
|
||||
pub inner: SendableEntryStream<'a, EntryImpl, Error>,
|
||||
pub start_entry_id: Id,
|
||||
}
|
||||
|
||||
impl<'a> Stream for StreamImpl<'a> {
|
||||
type Item = Result<Vec<EntryImpl>, Error>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
Pin::new(&mut self.inner).poll_next(cx)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> EntryStream for StreamImpl<'a> {
|
||||
type Error = Error;
|
||||
type Entry = EntryImpl;
|
||||
|
||||
fn start_id(&self) -> u64 {
|
||||
self.start_entry_id
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use async_stream::stream;
|
||||
use byteorder::{ByteOrder, LittleEndian};
|
||||
use futures::pin_mut;
|
||||
use futures_util::StreamExt;
|
||||
use tokio::time::Duration;
|
||||
|
||||
use super::*;
|
||||
use crate::fs::chunk::{Chunk, ChunkList};
|
||||
use crate::fs::crc::CRC_ALGO;
|
||||
|
||||
#[test]
|
||||
pub fn test_entry_deser() {
|
||||
let data = "hello, world";
|
||||
let mut entry = EntryImpl::new(data.as_bytes(), 8, LocalNamespace::new(42));
|
||||
entry.epoch = 9;
|
||||
let mut buf = BytesMut::with_capacity(entry.encoded_size());
|
||||
entry.encode_to(&mut buf).unwrap();
|
||||
assert_eq!(ENTRY_MIN_LEN + data.as_bytes().len(), buf.len());
|
||||
let decoded: EntryImpl = EntryImpl::decode(&mut buf.as_slice()).unwrap();
|
||||
assert_eq!(entry, decoded);
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_rewrite_entry_id() {
|
||||
let data = "hello, world";
|
||||
let entry = EntryImpl::new(data.as_bytes(), 123, LocalNamespace::new(42));
|
||||
let mut buffer = BytesMut::with_capacity(entry.encoded_size());
|
||||
entry.encode_to(&mut buffer).unwrap();
|
||||
assert_eq!(123, entry.id());
|
||||
|
||||
// rewrite entry id.
|
||||
LittleEndian::write_u64(&mut buffer[0..8], 333);
|
||||
let len = buffer.len();
|
||||
let checksum = CRC_ALGO.checksum(&buffer[0..len - 4]);
|
||||
LittleEndian::write_u32(&mut buffer[len - 4..], checksum);
|
||||
|
||||
let entry_impl = EntryImpl::decode(&mut buffer.freeze()).expect("Failed to deserialize");
|
||||
assert_eq!(333, entry_impl.id());
|
||||
}
|
||||
|
||||
fn prepare_entry_bytes(data: &str, id: Id) -> Bytes {
|
||||
let mut entry = EntryImpl::new(data.as_bytes(), id, LocalNamespace::new(42));
|
||||
entry.set_id(123);
|
||||
entry.set_offset(456);
|
||||
let mut buffer = BytesMut::with_capacity(entry.encoded_size());
|
||||
entry.encode_to(&mut buffer).unwrap();
|
||||
let len = buffer.len();
|
||||
let checksum = CRC_ALGO.checksum(&buffer[0..len - 4]);
|
||||
LittleEndian::write_u32(&mut buffer[len - 4..], checksum);
|
||||
buffer.freeze()
|
||||
}
|
||||
|
||||
/// Test decode entry from a composite buffer.
|
||||
#[test]
|
||||
pub fn test_composite_buffer() {
|
||||
let data_1 = "hello, world";
|
||||
let bytes = prepare_entry_bytes(data_1, 0);
|
||||
EntryImpl::decode(&mut bytes.clone()).unwrap();
|
||||
let c1 = Chunk::copy_from_slice(&bytes);
|
||||
|
||||
let data_2 = "LoremIpsumDolor";
|
||||
let bytes = prepare_entry_bytes(data_2, 1);
|
||||
EntryImpl::decode(&mut bytes.clone()).unwrap();
|
||||
let c2 = Chunk::copy_from_slice(&bytes);
|
||||
|
||||
let mut chunks = ChunkList::new();
|
||||
chunks.push(c1);
|
||||
chunks.push(c2);
|
||||
|
||||
assert_eq!(
|
||||
ENTRY_MIN_LEN * 2 + data_2.len() + data_1.len(),
|
||||
chunks.remaining_size()
|
||||
);
|
||||
|
||||
let mut decoded = vec![];
|
||||
while chunks.remaining_size() > 0 {
|
||||
let entry_impl = EntryImpl::decode(&mut chunks).unwrap();
|
||||
decoded.push(entry_impl.data);
|
||||
}
|
||||
|
||||
assert_eq!(
|
||||
vec![data_1.as_bytes().to_vec(), data_2.as_bytes().to_vec()],
|
||||
decoded
|
||||
);
|
||||
}
|
||||
|
||||
// split an encoded entry to two different chunk and try decode from this composite chunk
|
||||
#[test]
|
||||
pub fn test_decode_split_data_from_composite_chunk() {
|
||||
let data = "hello, world";
|
||||
let bytes = prepare_entry_bytes(data, 42);
|
||||
assert_eq!(
|
||||
hex::decode("7B000000000000002A0000000000000000000000000000000C00000068656C6C6F2C20776F726C64E8EE2E57")
|
||||
.unwrap()
|
||||
.as_slice(),
|
||||
&bytes[..]
|
||||
);
|
||||
let original = EntryImpl::decode(&mut bytes.clone()).unwrap();
|
||||
let split_point = bytes.len() / 2;
|
||||
let (left, right) = bytes.split_at(split_point);
|
||||
|
||||
let mut chunks = ChunkList::new();
|
||||
chunks.push(Chunk::copy_from_slice(left));
|
||||
chunks.push(Chunk::copy_from_slice(right));
|
||||
|
||||
assert_eq!(bytes.len(), chunks.remaining_size());
|
||||
let decoded = EntryImpl::decode(&mut chunks).unwrap();
|
||||
assert_eq!(original, decoded);
|
||||
}
|
||||
|
||||
// Tests decode entry from encoded entry data as two chunks
|
||||
#[tokio::test]
|
||||
pub async fn test_decode_from_chunk_stream() {
|
||||
// prepare entry
|
||||
let data = "hello, world";
|
||||
let bytes = prepare_entry_bytes(data, 42);
|
||||
assert_eq!(
|
||||
hex::decode("7b000000000000002a0000000000000000000000000000000c00000068656c6c6f2c20776f726c64e8ee2e57")
|
||||
.unwrap()
|
||||
.as_slice(),
|
||||
&bytes[..]
|
||||
);
|
||||
let original = EntryImpl::decode(&mut bytes.clone()).unwrap();
|
||||
let split_point = bytes.len() / 2;
|
||||
let (left, right) = bytes.split_at(split_point);
|
||||
|
||||
// prepare chunk stream
|
||||
let chunk_stream = stream!({
|
||||
yield Chunk::copy_from_slice(left);
|
||||
tokio::time::sleep(Duration::from_millis(10)).await;
|
||||
yield Chunk::copy_from_slice(right);
|
||||
});
|
||||
|
||||
pin_mut!(chunk_stream);
|
||||
|
||||
let mut chunks = ChunkList::new();
|
||||
let mut decoded = vec![];
|
||||
while let Some(c) = chunk_stream.next().await {
|
||||
chunks.push(c);
|
||||
match EntryImpl::decode(&mut chunks) {
|
||||
Ok(e) => {
|
||||
decoded.push(e);
|
||||
}
|
||||
Err(Error::DecodeAgain { .. }) => {
|
||||
continue;
|
||||
}
|
||||
_ => {
|
||||
panic!()
|
||||
}
|
||||
}
|
||||
}
|
||||
assert_eq!(1, decoded.len());
|
||||
assert_eq!(original, decoded.into_iter().next().unwrap());
|
||||
}
|
||||
}
|
||||
@@ -1,884 +0,0 @@
|
||||
// Copyright 2022 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::fmt::{Debug, Formatter};
|
||||
use std::fs::{File, OpenOptions};
|
||||
use std::pin::Pin;
|
||||
use std::sync::atomic::{AtomicBool, AtomicU64, AtomicUsize, Ordering};
|
||||
use std::sync::{Arc, Mutex};
|
||||
|
||||
use async_stream::stream;
|
||||
use byteorder::{ByteOrder, LittleEndian};
|
||||
use bytes::{Bytes, BytesMut};
|
||||
use common_error::ext::BoxedError;
|
||||
use common_telemetry::logging::{error, info};
|
||||
use common_telemetry::{debug, trace};
|
||||
use futures::Stream;
|
||||
use futures_util::StreamExt;
|
||||
use snafu::ResultExt;
|
||||
use store_api::logstore::entry::{Encode, Entry, Id, Offset};
|
||||
use store_api::logstore::entry_stream::EntryStream;
|
||||
use store_api::logstore::namespace::Namespace;
|
||||
use tokio::sync::mpsc::error::TryRecvError;
|
||||
use tokio::sync::mpsc::{Receiver, Sender as MpscSender};
|
||||
use tokio::sync::oneshot::Sender as OneshotSender;
|
||||
use tokio::sync::{oneshot, Notify};
|
||||
use tokio::task::JoinHandle;
|
||||
use tokio::time;
|
||||
|
||||
use crate::error::Error::Eof;
|
||||
use crate::error::{
|
||||
AppendSnafu, Error, InternalSnafu, IoSnafu, OpenLogSnafu, Result, WaitWriteSnafu, WriteSnafu,
|
||||
};
|
||||
use crate::fs::chunk::{Chunk, ChunkList};
|
||||
use crate::fs::config::LogConfig;
|
||||
use crate::fs::crc::CRC_ALGO;
|
||||
use crate::fs::entry::{EntryImpl, StreamImpl};
|
||||
use crate::fs::file_name::FileName;
|
||||
use crate::fs::namespace::LocalNamespace;
|
||||
use crate::fs::AppendResponseImpl;
|
||||
|
||||
pub const CHUNK_SIZE: usize = 4096;
|
||||
const LOG_WRITER_BATCH_SIZE: usize = 16;
|
||||
|
||||
/// Wraps File operation to get rid of `&mut self` requirements
|
||||
struct FileWriter {
|
||||
inner: Arc<File>,
|
||||
path: String,
|
||||
}
|
||||
|
||||
impl FileWriter {
|
||||
pub fn new(file: Arc<File>, path: String) -> Self {
|
||||
Self { inner: file, path }
|
||||
}
|
||||
|
||||
pub async fn write(&self, data: Bytes, offset: u64) -> Result<()> {
|
||||
let file = self.inner.clone();
|
||||
let handle = common_runtime::spawn_blocking_write(move || {
|
||||
crate::fs::io::pwrite_all(&file, &data, offset)
|
||||
});
|
||||
handle.await.context(WriteSnafu)?
|
||||
}
|
||||
|
||||
/// Writes a batch of `AppendRequest` to file.
|
||||
pub async fn write_batch(self: &Arc<Self>, batch: &Vec<AppendRequest>) -> Result<usize> {
|
||||
let mut futures = Vec::with_capacity(batch.len());
|
||||
|
||||
let mut max_offset = 0;
|
||||
for req in batch {
|
||||
let offset = req.offset;
|
||||
let end = req.data.len() + offset;
|
||||
max_offset = max_offset.max(end);
|
||||
let future = self.write(req.data.clone(), offset as u64);
|
||||
futures.push(future);
|
||||
}
|
||||
debug!(
|
||||
"Write batch, size: {}, max offset: {}",
|
||||
batch.len(),
|
||||
max_offset
|
||||
);
|
||||
futures::future::join_all(futures)
|
||||
.await
|
||||
.into_iter()
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.map(|_| max_offset)
|
||||
}
|
||||
|
||||
pub async fn flush(&self) -> Result<()> {
|
||||
let file = self.inner.clone();
|
||||
common_runtime::spawn_blocking_write(move || file.sync_all().context(IoSnafu))
|
||||
.await
|
||||
.context(WaitWriteSnafu)?
|
||||
}
|
||||
|
||||
pub async fn destroy(&self) -> Result<()> {
|
||||
tokio::fs::remove_file(&self.path).await.context(IoSnafu)?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub type LogFileRef = Arc<LogFile>;
|
||||
|
||||
pub struct LogFile {
|
||||
// name of log file
|
||||
name: FileName,
|
||||
// file writer
|
||||
writer: Arc<FileWriter>,
|
||||
// append request channel
|
||||
pending_request_tx: Option<MpscSender<AppendRequest>>,
|
||||
// flush task notifier
|
||||
notify: Arc<Notify>,
|
||||
// flush task join handle
|
||||
join_handle: Mutex<Option<JoinHandle<Result<()>>>>,
|
||||
// internal state(offset, id counter...)
|
||||
state: Arc<State>,
|
||||
// the start entry id of current log file
|
||||
start_entry_id: u64,
|
||||
// max file size of current log file
|
||||
max_file_size: usize,
|
||||
// buffer size for append request channel. read from config on start.
|
||||
append_buffer_size: usize,
|
||||
}
|
||||
|
||||
impl Drop for LogFile {
|
||||
fn drop(&mut self) {
|
||||
self.state.stopped.store(true, Ordering::Relaxed);
|
||||
info!("Dropping log file {}", self.name);
|
||||
}
|
||||
}
|
||||
|
||||
impl LogFile {
|
||||
/// Opens a file in path with given log config.
|
||||
pub async fn open(path: impl Into<String>, config: &LogConfig) -> Result<Self> {
|
||||
let path = path.into();
|
||||
let file = OpenOptions::new()
|
||||
.write(true)
|
||||
.read(true)
|
||||
.create(true)
|
||||
.open(path.clone())
|
||||
.context(OpenLogSnafu { file_name: &path })?;
|
||||
|
||||
let file_name = FileName::try_from(path.as_str())?;
|
||||
let start_entry_id = file_name.entry_id();
|
||||
|
||||
let mut log = Self {
|
||||
name: file_name,
|
||||
writer: Arc::new(FileWriter::new(Arc::new(file), path.clone())),
|
||||
start_entry_id,
|
||||
pending_request_tx: None,
|
||||
notify: Arc::new(Notify::new()),
|
||||
max_file_size: config.max_log_file_size,
|
||||
join_handle: Mutex::new(None),
|
||||
state: Arc::new(State::default()),
|
||||
append_buffer_size: config.append_buffer_size,
|
||||
};
|
||||
|
||||
let metadata = log.writer.inner.metadata().context(IoSnafu)?;
|
||||
let expect_length = metadata.len() as usize;
|
||||
log.state
|
||||
.write_offset
|
||||
.store(expect_length, Ordering::Relaxed);
|
||||
log.state
|
||||
.flush_offset
|
||||
.store(expect_length, Ordering::Relaxed);
|
||||
|
||||
let replay_start_time = time::Instant::now();
|
||||
let (actual_offset, next_entry_id) = log.replay().await?;
|
||||
|
||||
info!(
|
||||
"Log file {} replay finished, last offset: {}, file start entry id: {}, elapsed time: {}ms",
|
||||
path, actual_offset, start_entry_id,
|
||||
time::Instant::now().duration_since(replay_start_time).as_millis()
|
||||
);
|
||||
|
||||
log.state
|
||||
.write_offset
|
||||
.store(actual_offset, Ordering::Relaxed);
|
||||
log.state
|
||||
.flush_offset
|
||||
.store(actual_offset, Ordering::Relaxed);
|
||||
log.state
|
||||
.last_entry_id
|
||||
.store(next_entry_id, Ordering::Relaxed);
|
||||
Ok(log)
|
||||
}
|
||||
|
||||
/// Returns the persisted size of current log file.
|
||||
#[inline]
|
||||
pub fn persisted_size(&self) -> usize {
|
||||
self.state.flush_offset()
|
||||
}
|
||||
|
||||
/// Starts log file and it's internal components(including flush task, etc.).
|
||||
pub async fn start(&mut self) -> Result<()> {
|
||||
let notify = self.notify.clone();
|
||||
let writer = self.writer.clone();
|
||||
let state = self.state.clone();
|
||||
|
||||
let (tx, mut rx) = tokio::sync::mpsc::channel(self.append_buffer_size);
|
||||
|
||||
let handle = tokio::spawn(async move {
|
||||
while !state.is_stopped() {
|
||||
let batch = Self::recv_batch(&mut rx, &state, ¬ify, true).await;
|
||||
debug!("Receive write request, size: {}", batch.len());
|
||||
if !batch.is_empty() {
|
||||
Self::handle_batch(batch, &state, &writer).await;
|
||||
}
|
||||
}
|
||||
|
||||
// log file stopped
|
||||
let batch = Self::recv_batch(&mut rx, &state, ¬ify, false).await;
|
||||
if !batch.is_empty() {
|
||||
Self::handle_batch(batch, &state, &writer).await;
|
||||
}
|
||||
info!("Writer task finished");
|
||||
Ok(())
|
||||
});
|
||||
|
||||
self.pending_request_tx = Some(tx);
|
||||
*self.join_handle.lock().unwrap() = Some(handle);
|
||||
info!("Flush task started: {}", self.name);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Stops log file.
|
||||
/// # Panics
|
||||
/// Panics when a log file is stopped while not being started ever.
|
||||
pub async fn stop(&self) -> Result<()> {
|
||||
self.state.stopped.store(true, Ordering::Release);
|
||||
let join_handle = self
|
||||
.join_handle
|
||||
.lock()
|
||||
.unwrap()
|
||||
.take()
|
||||
.expect("Join handle should present");
|
||||
self.notify.notify_waiters();
|
||||
let res = join_handle.await.unwrap();
|
||||
info!("LogFile task finished: {:?}", res);
|
||||
res
|
||||
}
|
||||
|
||||
pub async fn destroy(&self) -> Result<()> {
|
||||
self.writer.destroy().await?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn handle_batch(
|
||||
mut batch: Vec<AppendRequest>,
|
||||
state: &Arc<State>,
|
||||
writer: &Arc<FileWriter>,
|
||||
) {
|
||||
// preserve previous write offset
|
||||
let prev_write_offset = state.write_offset();
|
||||
|
||||
let mut last_id = 0;
|
||||
for mut req in &mut batch {
|
||||
req.offset = state
|
||||
.write_offset
|
||||
.fetch_add(req.data.len(), Ordering::AcqRel);
|
||||
last_id = req.id;
|
||||
debug!("Entry id: {}, offset: {}", req.id, req.offset,);
|
||||
}
|
||||
|
||||
match writer.write_batch(&batch).await {
|
||||
Ok(max_offset) => match writer.flush().await {
|
||||
Ok(_) => {
|
||||
let prev_ofs = state.flush_offset.swap(max_offset, Ordering::Acquire);
|
||||
let prev_id = state.last_entry_id.swap(last_id, Ordering::Acquire);
|
||||
debug!(
|
||||
"Flush offset: {} -> {}, max offset in batch: {}, entry id: {}->{}",
|
||||
prev_ofs,
|
||||
state.flush_offset.load(Ordering::Acquire),
|
||||
max_offset,
|
||||
prev_id,
|
||||
state.last_entry_id.load(Ordering::Acquire),
|
||||
);
|
||||
batch.into_iter().for_each(AppendRequest::complete);
|
||||
}
|
||||
Err(e) => {
|
||||
error!(e; "Failed to flush log file");
|
||||
batch.into_iter().for_each(|r| r.fail());
|
||||
state
|
||||
.write_offset
|
||||
.store(prev_write_offset, Ordering::Release);
|
||||
}
|
||||
},
|
||||
Err(e) => {
|
||||
error!(e; "Failed to write append requests");
|
||||
batch.into_iter().for_each(|r| r.fail());
|
||||
state
|
||||
.write_offset
|
||||
.store(prev_write_offset, Ordering::Release);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn recv_batch(
|
||||
rx: &mut Receiver<AppendRequest>,
|
||||
state: &Arc<State>,
|
||||
notify: &Arc<Notify>,
|
||||
wait_on_empty: bool,
|
||||
) -> Vec<AppendRequest> {
|
||||
let mut batch: Vec<AppendRequest> = Vec::with_capacity(LOG_WRITER_BATCH_SIZE);
|
||||
for _ in 0..LOG_WRITER_BATCH_SIZE {
|
||||
match rx.try_recv() {
|
||||
Ok(req) => {
|
||||
batch.push(req);
|
||||
}
|
||||
Err(e) => match e {
|
||||
TryRecvError::Empty => {
|
||||
if batch.is_empty() && wait_on_empty {
|
||||
notify.notified().await;
|
||||
if state.is_stopped() {
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
TryRecvError::Disconnected => {
|
||||
error!("Channel unexpectedly disconnected!");
|
||||
break;
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
batch
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn start_entry_id(&self) -> Id {
|
||||
self.start_entry_id
|
||||
}
|
||||
|
||||
/// Replays current file til last entry read
|
||||
pub async fn replay(&mut self) -> Result<(usize, Id)> {
|
||||
let log_name = self.name.to_string();
|
||||
let previous_offset = self.state.flush_offset();
|
||||
let ns = LocalNamespace::default();
|
||||
let mut stream = self.create_stream(
|
||||
// TODO(hl): LocalNamespace should be filled
|
||||
&ns, 0,
|
||||
);
|
||||
|
||||
let mut last_offset = 0usize;
|
||||
let mut last_entry_id: Option<Id> = None;
|
||||
while let Some(res) = stream.next().await {
|
||||
match res {
|
||||
Ok(entries) => {
|
||||
for e in entries {
|
||||
last_offset += e.len();
|
||||
last_entry_id = Some(e.id());
|
||||
}
|
||||
}
|
||||
Err(e) => {
|
||||
error!(e; "Error while replay log {}", log_name);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
info!(
|
||||
"Replay log {} finished, offset: {} -> {}, last entry id: {:?}",
|
||||
log_name, previous_offset, last_offset, last_entry_id
|
||||
);
|
||||
Ok((last_offset, last_entry_id.unwrap_or(self.start_entry_id)))
|
||||
}
|
||||
|
||||
/// Creates a reader stream that asynchronously generates entries start from given entry id.
|
||||
/// ### Notice
|
||||
/// If the entry with start entry id is not present, the first generated entry will start with
|
||||
/// the first entry with an id greater than `start_entry_id`.
|
||||
pub fn create_stream(
|
||||
&self,
|
||||
_ns: &impl Namespace,
|
||||
start_entry_id: u64,
|
||||
) -> impl EntryStream<Entry = EntryImpl, Error = Error> + '_ {
|
||||
let length = self.state.flush_offset.load(Ordering::Relaxed);
|
||||
|
||||
let mut chunk_stream = file_chunk_stream(self.writer.inner.clone(), 0, length, 0);
|
||||
let entry_stream = stream!({
|
||||
let mut chunks = ChunkList::new();
|
||||
while let Some(chunk) = chunk_stream.next().await {
|
||||
let chunk = chunk.unwrap();
|
||||
chunks.push(chunk);
|
||||
let mut batch = vec![];
|
||||
loop {
|
||||
match EntryImpl::decode(&mut chunks) {
|
||||
Ok(e) => {
|
||||
if e.id() >= start_entry_id {
|
||||
batch.push(e);
|
||||
}
|
||||
}
|
||||
Err(Error::DecodeAgain { .. }) => {
|
||||
// no more data for decoding
|
||||
break;
|
||||
}
|
||||
Err(e) => {
|
||||
yield Err(e);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
trace!("Yield batch size: {}", batch.len());
|
||||
yield Ok(batch);
|
||||
}
|
||||
});
|
||||
|
||||
StreamImpl {
|
||||
inner: Box::pin(entry_stream),
|
||||
start_entry_id,
|
||||
}
|
||||
}
|
||||
|
||||
/// Appends an entry to `LogFile` and return a `Result` containing the id of entry appended.
|
||||
pub async fn append<T: Entry>(&self, e: &mut T) -> Result<AppendResponseImpl>
|
||||
where
|
||||
T: Encode<Error = Error>,
|
||||
{
|
||||
if self.state.is_stopped() {
|
||||
return Err(Error::Eof);
|
||||
}
|
||||
let entry_id = e.id();
|
||||
let mut serialized = BytesMut::with_capacity(e.encoded_size());
|
||||
e.encode_to(&mut serialized)
|
||||
.map_err(BoxedError::new)
|
||||
.context(AppendSnafu)?;
|
||||
let size = serialized.len();
|
||||
|
||||
if size + self.state.write_offset() > self.max_file_size {
|
||||
return Err(Error::Eof);
|
||||
}
|
||||
|
||||
// rewrite encoded data
|
||||
LittleEndian::write_u64(&mut serialized[0..8], entry_id);
|
||||
let checksum = CRC_ALGO.checksum(&serialized[0..size - 4]);
|
||||
LittleEndian::write_u32(&mut serialized[size - 4..], checksum);
|
||||
|
||||
let (tx, rx) = oneshot::channel();
|
||||
self.pending_request_tx
|
||||
.as_ref()
|
||||
.expect("Call start before write to LogFile!")
|
||||
.send(AppendRequest {
|
||||
data: serialized.freeze(),
|
||||
tx,
|
||||
offset: 0,
|
||||
id: entry_id,
|
||||
})
|
||||
.await
|
||||
.map_err(|_| {
|
||||
InternalSnafu {
|
||||
msg: "Send append request",
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
|
||||
self.notify.notify_one(); // notify write thread.
|
||||
|
||||
rx.await
|
||||
.expect("Sender dropped while waiting for append result")
|
||||
.map_err(|_| {
|
||||
InternalSnafu {
|
||||
msg: "Failed to write request".to_string(),
|
||||
}
|
||||
.build()
|
||||
})
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn try_seal(&self) -> bool {
|
||||
self.state
|
||||
.sealed
|
||||
.compare_exchange(false, true, Ordering::AcqRel, Ordering::Acquire)
|
||||
.is_ok()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn is_seal(&self) -> bool {
|
||||
self.state.sealed.load(Ordering::Acquire)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn is_stopped(&self) -> bool {
|
||||
self.state.stopped.load(Ordering::Acquire)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn unseal(&self) {
|
||||
self.state.sealed.store(false, Ordering::Release);
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn file_name(&self) -> String {
|
||||
self.name.to_string()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn last_entry_id(&self) -> Id {
|
||||
self.state.last_entry_id.load(Ordering::Acquire)
|
||||
}
|
||||
}
|
||||
|
||||
impl Debug for LogFile {
|
||||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("LogFile")
|
||||
.field("name", &self.name)
|
||||
.field("start_entry_id", &self.start_entry_id)
|
||||
.field("max_file_size", &self.max_file_size)
|
||||
.field("state", &self.state)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct AppendRequest {
|
||||
tx: OneshotSender<std::result::Result<AppendResponseImpl, ()>>,
|
||||
offset: Offset,
|
||||
id: Id,
|
||||
data: Bytes,
|
||||
}
|
||||
|
||||
impl AppendRequest {
|
||||
#[inline]
|
||||
pub fn complete(self) {
|
||||
let _ = self.tx.send(Ok(AppendResponseImpl {
|
||||
offset: self.offset,
|
||||
entry_id: self.id,
|
||||
}));
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn fail(self) {
|
||||
let _ = self.tx.send(Err(()));
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default, Debug)]
|
||||
struct State {
|
||||
write_offset: AtomicUsize,
|
||||
flush_offset: AtomicUsize,
|
||||
last_entry_id: AtomicU64,
|
||||
sealed: AtomicBool,
|
||||
stopped: AtomicBool,
|
||||
}
|
||||
|
||||
impl State {
|
||||
#[inline]
|
||||
pub fn is_stopped(&self) -> bool {
|
||||
self.stopped.load(Ordering::Acquire)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn write_offset(&self) -> usize {
|
||||
self.write_offset.load(Ordering::Acquire)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn flush_offset(&self) -> usize {
|
||||
self.flush_offset.load(Ordering::Acquire)
|
||||
}
|
||||
}
|
||||
|
||||
type SendableChunkStream = Pin<Box<dyn Stream<Item = Result<Chunk>> + Send>>;
|
||||
|
||||
/// Creates a stream of chunks of data from file. If `buffer_size` is not 0, the returned stream
|
||||
/// will have a bounded buffer and a background thread will do prefetching. When consumer cannot
|
||||
/// catch up with spawned prefetch loop, the prefetch thread will be blocked and wait until buffer
|
||||
/// has enough capacity.
|
||||
///
|
||||
/// If the `buffer_size` is 0, there will not be a prefetching thread. File chunks will not be read
|
||||
/// until stream consumer asks for next chunk.
|
||||
fn file_chunk_stream(
|
||||
file: Arc<File>,
|
||||
mut offset: usize,
|
||||
file_size: usize,
|
||||
buffer_size: usize,
|
||||
) -> SendableChunkStream {
|
||||
if buffer_size == 0 {
|
||||
return file_chunk_stream_sync(file, offset, file_size);
|
||||
}
|
||||
|
||||
let (tx, mut rx) = tokio::sync::mpsc::channel(buffer_size);
|
||||
common_runtime::spawn_blocking_read(move || loop {
|
||||
if offset >= file_size {
|
||||
return;
|
||||
}
|
||||
match read_at(&file, offset, file_size) {
|
||||
Ok(data) => {
|
||||
let data_len = data.len();
|
||||
if tx.blocking_send(Ok(data)).is_err() {
|
||||
break;
|
||||
}
|
||||
offset += data_len;
|
||||
continue;
|
||||
}
|
||||
Err(e) => {
|
||||
error!(e; "Failed to read file chunk");
|
||||
// we're going to break any way so just forget the join result.
|
||||
let _ = tx.blocking_send(Err(e));
|
||||
break;
|
||||
}
|
||||
}
|
||||
});
|
||||
Box::pin(stream!({
|
||||
while let Some(v) = rx.recv().await {
|
||||
yield v;
|
||||
}
|
||||
}))
|
||||
}
|
||||
|
||||
fn file_chunk_stream_sync(
|
||||
file: Arc<File>,
|
||||
mut offset: usize,
|
||||
file_size: usize,
|
||||
) -> SendableChunkStream {
|
||||
let s = stream!({
|
||||
loop {
|
||||
if offset >= file_size {
|
||||
return;
|
||||
}
|
||||
match read_at(&file, offset, file_size) {
|
||||
Ok(data) => {
|
||||
let data_len = data.len();
|
||||
yield Ok(data);
|
||||
offset += data_len;
|
||||
continue;
|
||||
}
|
||||
Err(e) => {
|
||||
error!(e; "Failed to read file chunk");
|
||||
yield Err(e);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
Box::pin(s)
|
||||
}
|
||||
|
||||
/// Reads a chunk of data from file in a blocking manner.
|
||||
/// The file may not contain enough data to fulfill the whole chunk so only data available
|
||||
/// is read into chunk. The `write` field of `Chunk` indicates the end of valid data.
|
||||
fn read_at(file: &Arc<File>, offset: usize, file_length: usize) -> Result<Chunk> {
|
||||
if offset > file_length {
|
||||
return Err(Eof);
|
||||
}
|
||||
let size = CHUNK_SIZE.min(file_length - offset);
|
||||
let mut data = Box::new([0u8; CHUNK_SIZE]);
|
||||
crate::fs::io::pread_exact(file.as_ref(), &mut data[0..size], offset as u64)?;
|
||||
Ok(Chunk::new(data, size))
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::io::Read;
|
||||
|
||||
use common_telemetry::logging;
|
||||
use futures::pin_mut;
|
||||
use futures_util::StreamExt;
|
||||
use tempdir::TempDir;
|
||||
use tokio::io::AsyncWriteExt;
|
||||
|
||||
use super::*;
|
||||
use crate::fs::namespace::LocalNamespace;
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_create_entry_stream() {
|
||||
logging::init_default_ut_logging();
|
||||
let config = LogConfig::default();
|
||||
|
||||
let dir = TempDir::new("greptimedb-store-test").unwrap();
|
||||
let path_buf = dir.path().join("0010.log");
|
||||
let path = path_buf.to_str().unwrap().to_string();
|
||||
File::create(path.as_str()).unwrap();
|
||||
|
||||
let mut file = LogFile::open(path.clone(), &config)
|
||||
.await
|
||||
.unwrap_or_else(|_| panic!("Failed to open file: {path}"));
|
||||
file.start().await.expect("Failed to start log file");
|
||||
|
||||
assert_eq!(
|
||||
10,
|
||||
file.append(&mut EntryImpl::new(
|
||||
"test1".as_bytes(),
|
||||
10,
|
||||
LocalNamespace::new(42)
|
||||
))
|
||||
.await
|
||||
.expect("Failed to append entry 1")
|
||||
.entry_id
|
||||
);
|
||||
|
||||
assert_eq!(
|
||||
11,
|
||||
file.append(&mut EntryImpl::new(
|
||||
"test-2".as_bytes(),
|
||||
11,
|
||||
LocalNamespace::new(42)
|
||||
))
|
||||
.await
|
||||
.expect("Failed to append entry 2")
|
||||
.entry_id
|
||||
);
|
||||
|
||||
let mut log_file = std::fs::File::open(path.clone()).expect("Test log file does not exist");
|
||||
let metadata = log_file.metadata().expect("Failed to read file metadata");
|
||||
info!("Log file metadata: {:?}", metadata);
|
||||
|
||||
assert_eq!(75, metadata.len()); // 32+5+32+6
|
||||
let mut content = vec![0; metadata.len() as usize];
|
||||
log_file
|
||||
.read_exact(&mut content)
|
||||
.expect("Read log file failed");
|
||||
|
||||
info!(
|
||||
"Log file {:?} content: {}, size:{}",
|
||||
dir,
|
||||
hex::encode(content),
|
||||
metadata.len()
|
||||
);
|
||||
|
||||
let ns = LocalNamespace::new(42);
|
||||
let mut stream = file.create_stream(&ns, 0);
|
||||
let mut data = vec![];
|
||||
|
||||
while let Some(v) = stream.next().await {
|
||||
let entries = v.unwrap();
|
||||
for e in entries {
|
||||
let vec = e.data().to_vec();
|
||||
info!("Read entry: {}", String::from_utf8_lossy(&vec));
|
||||
data.push(String::from_utf8(vec).unwrap());
|
||||
}
|
||||
}
|
||||
|
||||
assert_eq!(vec!["test1".to_string(), "test-2".to_string()], data);
|
||||
drop(stream);
|
||||
|
||||
let result = file.stop().await;
|
||||
info!("Stop file res: {:?}", result);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_read_at() {
|
||||
let dir = tempdir::TempDir::new("greptimedb-store-test").unwrap();
|
||||
let file_path = dir.path().join("chunk-stream-file-test");
|
||||
let mut file = tokio::fs::OpenOptions::new()
|
||||
.create(true)
|
||||
.write(true)
|
||||
.read(true)
|
||||
.open(&file_path)
|
||||
.await
|
||||
.unwrap();
|
||||
file.write_all("1234567890ab".as_bytes()).await.unwrap();
|
||||
file.flush().await.unwrap();
|
||||
|
||||
let file = Arc::new(file.into_std().await);
|
||||
let result = read_at(&file, 0, 12).unwrap();
|
||||
|
||||
assert_eq!(12, result.len());
|
||||
assert_eq!("1234567890ab".as_bytes(), &result.data[0..result.len()]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_read_at_center() {
|
||||
let dir = tempdir::TempDir::new("greptimedb-store-test").unwrap();
|
||||
let file_path = dir.path().join("chunk-stream-file-test-center");
|
||||
let mut file = tokio::fs::OpenOptions::new()
|
||||
.create(true)
|
||||
.write(true)
|
||||
.read(true)
|
||||
.open(&file_path)
|
||||
.await
|
||||
.unwrap();
|
||||
file.write_all("1234567890ab".as_bytes()).await.unwrap();
|
||||
file.flush().await.unwrap();
|
||||
|
||||
let file_len = file.metadata().await.unwrap().len();
|
||||
let file = Arc::new(file.into_std().await);
|
||||
let result = read_at(&file, 8, file_len as usize).unwrap();
|
||||
assert_eq!(4, result.len());
|
||||
assert_eq!("90ab".as_bytes(), &result.data[0..result.len()]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_file_chunk_stream() {
|
||||
let dir = tempdir::TempDir::new("greptimedb-store-test").unwrap();
|
||||
let file_path = dir.path().join("chunk-stream-file-test");
|
||||
let mut file = tokio::fs::OpenOptions::new()
|
||||
.create(true)
|
||||
.write(true)
|
||||
.read(true)
|
||||
.open(&file_path)
|
||||
.await
|
||||
.unwrap();
|
||||
file.write_all(&vec![42].repeat(4096 + 1024)).await.unwrap();
|
||||
file.flush().await.unwrap();
|
||||
|
||||
let file_size = file.metadata().await.unwrap().len();
|
||||
let file = Arc::new(file.into_std().await);
|
||||
let stream = file_chunk_stream(file, 0, file_size as usize, 1024);
|
||||
pin_mut!(stream);
|
||||
|
||||
let mut chunks = vec![];
|
||||
while let Some(r) = stream.next().await {
|
||||
chunks.push(r.unwrap());
|
||||
}
|
||||
assert_eq!(
|
||||
vec![4096, 1024],
|
||||
chunks.iter().map(|c| c.write_offset).collect::<Vec<_>>()
|
||||
);
|
||||
assert_eq!(
|
||||
vec![vec![42].repeat(4096), vec![42].repeat(1024)],
|
||||
chunks
|
||||
.iter()
|
||||
.map(|c| &c.data[0..c.write_offset])
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_sync_chunk_stream() {
|
||||
let dir = tempdir::TempDir::new("greptimedb-store-test").unwrap();
|
||||
let file_path = dir.path().join("chunk-stream-file-test");
|
||||
let mut file = tokio::fs::OpenOptions::new()
|
||||
.create(true)
|
||||
.write(true)
|
||||
.read(true)
|
||||
.open(&file_path)
|
||||
.await
|
||||
.unwrap();
|
||||
file.write_all(&vec![42].repeat(4096 + 1024)).await.unwrap();
|
||||
file.flush().await.unwrap();
|
||||
|
||||
let file_size = file.metadata().await.unwrap().len();
|
||||
let file = Arc::new(file.into_std().await);
|
||||
let stream = file_chunk_stream_sync(file, 0, file_size as usize);
|
||||
pin_mut!(stream);
|
||||
|
||||
let mut chunks = vec![];
|
||||
while let Some(r) = stream.next().await {
|
||||
chunks.push(r.unwrap());
|
||||
}
|
||||
assert_eq!(
|
||||
vec![4096, 1024],
|
||||
chunks.iter().map(|c| c.write_offset).collect::<Vec<_>>()
|
||||
);
|
||||
assert_eq!(
|
||||
vec![vec![42].repeat(4096), vec![42].repeat(1024)],
|
||||
chunks
|
||||
.iter()
|
||||
.map(|c| &c.data[0..c.write_offset])
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_shutdown() {
|
||||
logging::init_default_ut_logging();
|
||||
let config = LogConfig::default();
|
||||
let dir = TempDir::new("greptimedb-store-test").unwrap();
|
||||
let path_buf = dir.path().join("0010.log");
|
||||
let path = path_buf.to_str().unwrap().to_string();
|
||||
File::create(path.as_str()).unwrap();
|
||||
|
||||
let mut file = LogFile::open(path.clone(), &config)
|
||||
.await
|
||||
.unwrap_or_else(|_| panic!("Failed to open file: {path}"));
|
||||
|
||||
let state = file.state.clone();
|
||||
file.start().await.unwrap();
|
||||
drop(file);
|
||||
|
||||
assert!(state.stopped.load(Ordering::Relaxed));
|
||||
}
|
||||
}
|
||||
@@ -1,125 +0,0 @@
|
||||
// Copyright 2022 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::fmt::{Display, Formatter};
|
||||
use std::path::Path;
|
||||
|
||||
use snafu::OptionExt;
|
||||
use store_api::logstore::entry::Id;
|
||||
|
||||
use crate::error::{Error, FileNameIllegalSnafu, SuffixIllegalSnafu};
|
||||
use crate::fs::file_name::FileName::Log;
|
||||
|
||||
/// FileName represents the file name with padded leading zeros.
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
pub enum FileName {
|
||||
// File name with .log as suffix.
|
||||
Log(Id),
|
||||
}
|
||||
|
||||
impl TryFrom<&str> for FileName {
|
||||
type Error = Error;
|
||||
|
||||
fn try_from(p: &str) -> Result<Self, Self::Error> {
|
||||
let path = Path::new(p);
|
||||
|
||||
let extension =
|
||||
path.extension()
|
||||
.and_then(|s| s.to_str())
|
||||
.with_context(|| FileNameIllegalSnafu {
|
||||
file_name: path.to_string_lossy(),
|
||||
})?;
|
||||
|
||||
let id: u64 = path
|
||||
.file_stem()
|
||||
.and_then(|s| s.to_str())
|
||||
.and_then(|s| s.parse::<u64>().ok())
|
||||
.with_context(|| FileNameIllegalSnafu {
|
||||
file_name: p.to_string(),
|
||||
})?;
|
||||
|
||||
Self::new_with_suffix(id, extension)
|
||||
}
|
||||
}
|
||||
|
||||
impl From<u64> for FileName {
|
||||
fn from(entry_id: u64) -> Self {
|
||||
Self::log(entry_id)
|
||||
}
|
||||
}
|
||||
|
||||
impl FileName {
|
||||
pub fn log(entry_id: Id) -> Self {
|
||||
Log(entry_id)
|
||||
}
|
||||
|
||||
pub fn new_with_suffix(entry_id: Id, suffix: &str) -> Result<Self, Error> {
|
||||
match suffix {
|
||||
"log" => Ok(Log(entry_id)),
|
||||
_ => SuffixIllegalSnafu { suffix }.fail(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn entry_id(&self) -> Id {
|
||||
match self {
|
||||
Log(id) => *id,
|
||||
}
|
||||
}
|
||||
|
||||
fn suffix(&self) -> &str {
|
||||
match self {
|
||||
Log(_) => ".log",
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for FileName {
|
||||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "{:020}{}", self.entry_id(), self.suffix())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
pub fn test_padding_file_name() {
|
||||
let id = u64::MIN;
|
||||
assert_eq!("00000000000000000000", format!("{id:020}"));
|
||||
let id = 123u64;
|
||||
assert_eq!("00000000000000000123", format!("{id:020}"));
|
||||
let id = 123123123123u64;
|
||||
assert_eq!("00000000123123123123", format!("{id:020}"));
|
||||
let id = u64::MAX;
|
||||
assert_eq!(u64::MAX.to_string(), format!("{id:020}"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_file_name_to_string() {
|
||||
assert_eq!("00000000000000000000.log", FileName::log(0).to_string());
|
||||
assert_eq!(
|
||||
u64::MAX.to_string() + ".log",
|
||||
FileName::log(u64::MAX).to_string()
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_parse_file_name() {
|
||||
let path = "/path/to/any/01010010000.log";
|
||||
let parsed = FileName::try_from(path).expect("Failed to parse file name");
|
||||
assert_eq!(1010010000u64, parsed.entry_id());
|
||||
assert_eq!(".log", parsed.suffix());
|
||||
}
|
||||
}
|
||||
@@ -1,82 +0,0 @@
|
||||
// Copyright 2022 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::collections::BTreeMap;
|
||||
use std::sync::RwLock;
|
||||
|
||||
use store_api::logstore::entry::{Id, Offset};
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::fs::file_name::FileName;
|
||||
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
pub struct Location {
|
||||
pub file_name: FileName,
|
||||
pub offset: Offset,
|
||||
}
|
||||
|
||||
#[allow(dead_code)]
|
||||
impl Location {
|
||||
pub fn new(file_name: FileName, offset: Offset) -> Self {
|
||||
Self { file_name, offset }
|
||||
}
|
||||
}
|
||||
|
||||
/// In-memory entry id to offset index.
|
||||
pub trait EntryIndex {
|
||||
/// Add entry id to offset mapping.
|
||||
fn add_entry_id(&self, id: Id, loc: Location) -> Option<Location>;
|
||||
|
||||
/// Find offset by entry id.
|
||||
fn find_offset_by_id(&self, id: Id) -> Result<Option<Location>>;
|
||||
}
|
||||
|
||||
pub struct MemoryIndex {
|
||||
map: RwLock<BTreeMap<Id, Location>>,
|
||||
}
|
||||
|
||||
#[allow(dead_code)]
|
||||
impl MemoryIndex {
|
||||
pub fn new() -> Self {
|
||||
Self {
|
||||
map: RwLock::new(BTreeMap::new()),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl EntryIndex for MemoryIndex {
|
||||
fn add_entry_id(&self, id: Id, loc: Location) -> Option<Location> {
|
||||
self.map.write().unwrap().insert(id, loc)
|
||||
}
|
||||
|
||||
fn find_offset_by_id(&self, id: Id) -> Result<Option<Location>> {
|
||||
Ok(self.map.read().unwrap().get(&id).cloned())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
pub fn test_entry() {
|
||||
let index = MemoryIndex::new();
|
||||
index.add_entry_id(1, Location::new(FileName::log(0), 1));
|
||||
assert_eq!(
|
||||
Location::new(FileName::log(0), 1),
|
||||
index.find_offset_by_id(1).unwrap().unwrap()
|
||||
);
|
||||
assert_eq!(None, index.find_offset_by_id(2).unwrap());
|
||||
}
|
||||
}
|
||||
@@ -1,29 +0,0 @@
|
||||
// Copyright 2022 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::convert::TryFrom;
|
||||
use std::fs::File;
|
||||
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::error::Error;
|
||||
|
||||
// TODO(hl): Implement pread/pwrite for non-Unix platforms
|
||||
pub fn pread_exact(file: &File, _buf: &mut [u8], _offset: u64) -> Result<(), Error> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
pub fn pwrite_all(file: &File, _buf: &[u8], _offset: u64) -> Result<(), Error> {
|
||||
unimplemented!()
|
||||
}
|
||||
@@ -1,28 +0,0 @@
|
||||
// Copyright 2022 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::fs::File;
|
||||
use std::os::unix::fs::FileExt;
|
||||
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::error::{Error, IoSnafu};
|
||||
|
||||
pub fn pread_exact(file: &File, buf: &mut [u8], offset: u64) -> Result<(), Error> {
|
||||
file.read_exact_at(buf, offset).context(IoSnafu)
|
||||
}
|
||||
|
||||
pub fn pwrite_all(file: &File, buf: &[u8], offset: u64) -> Result<(), Error> {
|
||||
file.write_all_at(buf, offset).context(IoSnafu)
|
||||
}
|
||||
@@ -1,770 +0,0 @@
|
||||
// Copyright 2022 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::collections::{BTreeMap, HashMap};
|
||||
use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
|
||||
use arc_swap::ArcSwap;
|
||||
use async_stream::stream;
|
||||
use common_telemetry::{debug, error, info, warn};
|
||||
use futures::{pin_mut, StreamExt};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::logstore::entry::{Encode, Entry, Id};
|
||||
use store_api::logstore::entry_stream::SendableEntryStream;
|
||||
use store_api::logstore::namespace::{Id as NamespaceId, Namespace};
|
||||
use store_api::logstore::LogStore;
|
||||
use tokio::sync::{Mutex, RwLock};
|
||||
use tokio::task::JoinHandle;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
|
||||
use crate::error::{
|
||||
CreateDirSnafu, DuplicateFileSnafu, Error, FileNameIllegalSnafu, InternalSnafu,
|
||||
InvalidStateSnafu, IoSnafu, ReadPathSnafu, Result, WaitGcTaskStopSnafu,
|
||||
};
|
||||
use crate::fs::config::LogConfig;
|
||||
use crate::fs::entry::EntryImpl;
|
||||
use crate::fs::file::{LogFile, LogFileRef};
|
||||
use crate::fs::file_name::FileName;
|
||||
use crate::fs::namespace::LocalNamespace;
|
||||
use crate::fs::AppendResponseImpl;
|
||||
|
||||
type FileMap = BTreeMap<u64, LogFileRef>;
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct LocalFileLogStore {
|
||||
files: Arc<RwLock<FileMap>>,
|
||||
active: ArcSwap<LogFile>,
|
||||
config: LogConfig,
|
||||
obsolete_ids: Arc<RwLock<HashMap<LocalNamespace, u64>>>,
|
||||
cancel_token: Mutex<Option<CancellationToken>>,
|
||||
gc_task_handle: Mutex<Option<JoinHandle<()>>>,
|
||||
}
|
||||
|
||||
impl LocalFileLogStore {
|
||||
/// Opens a directory as log store directory, initialize directory if it is empty.
|
||||
pub async fn open(config: &LogConfig) -> Result<Self> {
|
||||
// Create the log directory if missing.
|
||||
tokio::fs::create_dir_all(&config.log_file_dir)
|
||||
.await
|
||||
.context(CreateDirSnafu {
|
||||
path: &config.log_file_dir,
|
||||
})?;
|
||||
|
||||
let mut files = Self::load_dir(&config.log_file_dir, config).await?;
|
||||
|
||||
if files.is_empty() {
|
||||
Self::init_on_empty(&mut files, config).await?;
|
||||
info!("Initialized log store directory: {}", config.log_file_dir)
|
||||
}
|
||||
|
||||
let id = *files.keys().max().context(InternalSnafu {
|
||||
msg: format!(
|
||||
"log store directory is empty after initialization: {}",
|
||||
config.log_file_dir
|
||||
),
|
||||
})?;
|
||||
|
||||
info!(
|
||||
"Successfully loaded log store directory, files: {:?}",
|
||||
files
|
||||
);
|
||||
|
||||
let active_file = files
|
||||
.get_mut(&id)
|
||||
.expect("Not expected to fail when initing log store");
|
||||
|
||||
active_file.unseal();
|
||||
let active_file_name = active_file.file_name();
|
||||
info!("Log store active log file: {}", active_file_name);
|
||||
|
||||
// Start active log file
|
||||
Arc::get_mut(active_file)
|
||||
.with_context(|| InternalSnafu {
|
||||
msg: format!(
|
||||
"Concurrent modification on log store {active_file_name} start is not allowed"
|
||||
),
|
||||
})?
|
||||
.start()
|
||||
.await?;
|
||||
info!(
|
||||
"Successfully started current active file: {}",
|
||||
active_file_name
|
||||
);
|
||||
|
||||
let active_file_cloned = active_file.clone();
|
||||
Ok(Self {
|
||||
files: Arc::new(RwLock::new(files)),
|
||||
active: ArcSwap::new(active_file_cloned),
|
||||
config: config.clone(),
|
||||
obsolete_ids: Arc::new(Default::default()),
|
||||
cancel_token: Mutex::new(None),
|
||||
gc_task_handle: Mutex::new(None),
|
||||
})
|
||||
}
|
||||
|
||||
pub async fn init_on_empty(files: &mut FileMap, config: &LogConfig) -> Result<()> {
|
||||
let path = Path::new(&config.log_file_dir).join(FileName::log(0).to_string());
|
||||
let file_path = path.to_str().context(FileNameIllegalSnafu {
|
||||
file_name: config.log_file_dir.clone(),
|
||||
})?;
|
||||
let file = LogFile::open(file_path, config).await?;
|
||||
files.insert(0, Arc::new(file));
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn load_dir(path: impl AsRef<str>, config: &LogConfig) -> Result<FileMap> {
|
||||
let mut map = FileMap::new();
|
||||
let mut dir = tokio::fs::read_dir(Path::new(path.as_ref()))
|
||||
.await
|
||||
.context(ReadPathSnafu {
|
||||
path: path.as_ref(),
|
||||
})?;
|
||||
|
||||
while let Some(f) = dir.next_entry().await.context(IoSnafu)? {
|
||||
let path_buf = f.path();
|
||||
let path = path_buf.to_str().context(FileNameIllegalSnafu {
|
||||
file_name: path.as_ref().to_string(),
|
||||
})?;
|
||||
let file_name = FileName::try_from(path)?;
|
||||
let start_id = file_name.entry_id();
|
||||
let file = LogFile::open(path, config).await?;
|
||||
info!("Load log store file {}: {:?}", start_id, file);
|
||||
if map.contains_key(&start_id) {
|
||||
error!("Log file with start entry id: {start_id} already exists");
|
||||
return DuplicateFileSnafu {
|
||||
msg: format!("File with start id: {start_id} duplicates on start"),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
file.try_seal();
|
||||
map.insert(start_id, Arc::new(file));
|
||||
}
|
||||
Ok(map)
|
||||
}
|
||||
|
||||
/// Mark current active file as closed and create a new log file for writing.
|
||||
async fn roll_next(&self, active: LogFileRef) -> Result<()> {
|
||||
// acquires lock
|
||||
let mut files = self.files.write().await;
|
||||
|
||||
// if active is already sealed, then just return.
|
||||
if active.is_seal() {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
// create and start a new log file
|
||||
let next_entry_id = active.last_entry_id() + 1;
|
||||
let path_buf =
|
||||
Path::new(&self.config.log_file_dir).join(FileName::log(next_entry_id).to_string());
|
||||
let path = path_buf.to_str().context(FileNameIllegalSnafu {
|
||||
file_name: self.config.log_file_dir.clone(),
|
||||
})?;
|
||||
|
||||
let mut new_file = LogFile::open(path, &self.config).await?;
|
||||
new_file.start().await?;
|
||||
|
||||
let new_file = Arc::new(new_file);
|
||||
files.insert(new_file.start_entry_id(), new_file.clone());
|
||||
|
||||
self.active.swap(new_file);
|
||||
active.try_seal();
|
||||
tokio::spawn(async move {
|
||||
active.stop().await.unwrap();
|
||||
info!("Sealed log file {} stopped.", active.file_name());
|
||||
});
|
||||
Ok(()) // release lock
|
||||
}
|
||||
|
||||
pub fn active_file(&self) -> Arc<LogFile> {
|
||||
self.active.load().clone()
|
||||
}
|
||||
}
|
||||
|
||||
async fn gc(
|
||||
files: Arc<RwLock<FileMap>>,
|
||||
obsolete_ids: Arc<RwLock<HashMap<LocalNamespace, u64>>>,
|
||||
) -> Result<()> {
|
||||
if let Some(lowest) = find_lowest_id(obsolete_ids).await {
|
||||
gc_inner(files, lowest).await
|
||||
} else {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
async fn find_lowest_id(obsolete_ids: Arc<RwLock<HashMap<LocalNamespace, u64>>>) -> Option<u64> {
|
||||
let mut lowest_obsolete = None;
|
||||
{
|
||||
let obsolete_ids = obsolete_ids.read().await;
|
||||
for (ns, id) in obsolete_ids.iter() {
|
||||
if *id <= *lowest_obsolete.get_or_insert(*id) {
|
||||
lowest_obsolete = Some(*id);
|
||||
debug!("Current lowest obsolete id: {}, namespace: {:?}", *id, ns);
|
||||
}
|
||||
}
|
||||
}
|
||||
lowest_obsolete
|
||||
}
|
||||
|
||||
async fn gc_inner(files: Arc<RwLock<FileMap>>, obsolete_id: u64) -> Result<()> {
|
||||
let mut files = files.write().await;
|
||||
let files_to_delete = find_files_to_delete(&files, obsolete_id);
|
||||
info!(
|
||||
"Compacting log file up to entry id: {}, files to delete: {:?}",
|
||||
obsolete_id, files_to_delete
|
||||
);
|
||||
for entry_id in files_to_delete {
|
||||
if let Some(f) = files.remove(&entry_id) {
|
||||
if !f.is_stopped() {
|
||||
f.stop().await?;
|
||||
}
|
||||
f.destroy().await?;
|
||||
info!("Destroyed log file: {}", f.file_name());
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn find_files_to_delete<T>(offset_map: &BTreeMap<u64, T>, entry_id: u64) -> Vec<u64> {
|
||||
let mut res = vec![];
|
||||
for (cur, next) in offset_map.keys().zip(offset_map.keys().skip(1)) {
|
||||
if *cur < entry_id && *next <= entry_id {
|
||||
res.push(*cur);
|
||||
}
|
||||
}
|
||||
res
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for LocalFileLogStore {
|
||||
type Error = Error;
|
||||
type Namespace = LocalNamespace;
|
||||
type Entry = EntryImpl;
|
||||
type AppendResponse = AppendResponseImpl;
|
||||
|
||||
async fn start(&self) -> Result<()> {
|
||||
let files = self.files.clone();
|
||||
let obsolete_ids = self.obsolete_ids.clone();
|
||||
let interval = self.config.gc_interval;
|
||||
let token = tokio_util::sync::CancellationToken::new();
|
||||
let child = token.child_token();
|
||||
|
||||
let handle = common_runtime::spawn_bg(async move {
|
||||
loop {
|
||||
if let Err(e) = gc(files.clone(), obsolete_ids.clone()).await {
|
||||
error!(e; "Failed to gc log store");
|
||||
}
|
||||
|
||||
tokio::select! {
|
||||
_ = tokio::time::sleep(interval) => {}
|
||||
_ = child.cancelled() => {
|
||||
info!("LogStore gc task has been cancelled");
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
*self.gc_task_handle.lock().await = Some(handle);
|
||||
*self.cancel_token.lock().await = Some(token);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn stop(&self) -> Result<()> {
|
||||
let handle = self
|
||||
.gc_task_handle
|
||||
.lock()
|
||||
.await
|
||||
.take()
|
||||
.context(InvalidStateSnafu {
|
||||
msg: "Logstore gc task not spawned",
|
||||
})?;
|
||||
let token = self
|
||||
.cancel_token
|
||||
.lock()
|
||||
.await
|
||||
.take()
|
||||
.context(InvalidStateSnafu {
|
||||
msg: "Logstore gc task not spawned",
|
||||
})?;
|
||||
token.cancel();
|
||||
Ok(handle.await.context(WaitGcTaskStopSnafu)?)
|
||||
}
|
||||
|
||||
async fn append(&self, mut entry: Self::Entry) -> Result<Self::AppendResponse> {
|
||||
// TODO(hl): configurable retry times
|
||||
for _ in 0..3 {
|
||||
let current_active_file = self.active_file();
|
||||
match current_active_file.append(&mut entry).await {
|
||||
Ok(r) => return Ok(r),
|
||||
Err(e) => match e {
|
||||
Error::Eof => {
|
||||
self.roll_next(current_active_file.clone()).await?;
|
||||
info!(
|
||||
"Rolled to next file, retry append, entry size: {}",
|
||||
entry.encoded_size()
|
||||
);
|
||||
continue;
|
||||
}
|
||||
Error::Internal { .. } => {
|
||||
warn!("File closed, try new file");
|
||||
continue;
|
||||
}
|
||||
_ => {
|
||||
error!(e; "Failed to roll to next log file");
|
||||
return Err(e);
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
return InternalSnafu {
|
||||
msg: "Failed to append entry with max retry time exceeds",
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
|
||||
async fn append_batch(&self, _ns: &Self::Namespace, _e: Vec<Self::Entry>) -> Result<Id> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn read(
|
||||
&self,
|
||||
ns: &Self::Namespace,
|
||||
id: Id,
|
||||
) -> Result<SendableEntryStream<'_, Self::Entry, Self::Error>> {
|
||||
let files = self.files.read().await;
|
||||
let ns = ns.clone();
|
||||
|
||||
let s = stream!({
|
||||
for (start_id, file) in files.iter() {
|
||||
// TODO(hl): Use index to lookup file
|
||||
if *start_id <= id {
|
||||
let s = file.create_stream(&ns, id);
|
||||
pin_mut!(s);
|
||||
while let Some(entries) = s.next().await {
|
||||
match entries {
|
||||
Ok(entries) => {
|
||||
yield Ok(entries
|
||||
.into_iter()
|
||||
.filter(|e| e.namespace().id() == ns.id())
|
||||
.collect::<Vec<_>>())
|
||||
}
|
||||
Err(e) => yield Err(e),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
Ok(Box::pin(s))
|
||||
}
|
||||
|
||||
async fn create_namespace(&mut self, _ns: &Self::Namespace) -> Result<()> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn delete_namespace(&mut self, _ns: &Self::Namespace) -> Result<()> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn entry<D: AsRef<[u8]>>(&self, data: D, id: Id, namespace: Self::Namespace) -> Self::Entry {
|
||||
EntryImpl::new(data, id, namespace)
|
||||
}
|
||||
|
||||
fn namespace(&self, id: NamespaceId) -> Self::Namespace {
|
||||
LocalNamespace::new(id)
|
||||
}
|
||||
|
||||
async fn obsolete(
|
||||
&self,
|
||||
namespace: Self::Namespace,
|
||||
id: Id,
|
||||
) -> std::result::Result<(), Self::Error> {
|
||||
info!("Mark namespace obsolete entry id, {:?}:{}", namespace, id);
|
||||
let mut map = self.obsolete_ids.write().await;
|
||||
let prev = map.insert(namespace, id);
|
||||
info!("Prev: {:?}", prev);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashSet;
|
||||
use std::time::Duration;
|
||||
|
||||
use futures_util::StreamExt;
|
||||
use rand::distributions::Alphanumeric;
|
||||
use rand::Rng;
|
||||
use store_api::logstore::entry::Entry;
|
||||
use tempdir::TempDir;
|
||||
|
||||
use super::*;
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_roll_file() {
|
||||
common_telemetry::logging::init_default_ut_logging();
|
||||
let dir = TempDir::new("greptimedb1").unwrap();
|
||||
let config = LogConfig {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 128,
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let logstore = LocalFileLogStore::open(&config).await.unwrap();
|
||||
assert_eq!(
|
||||
0,
|
||||
logstore
|
||||
.append(EntryImpl::new(
|
||||
generate_data(96),
|
||||
0,
|
||||
LocalNamespace::new(42)
|
||||
),)
|
||||
.await
|
||||
.unwrap()
|
||||
.entry_id
|
||||
);
|
||||
|
||||
assert_eq!(
|
||||
1,
|
||||
logstore
|
||||
.append(EntryImpl::new(
|
||||
generate_data(96),
|
||||
1,
|
||||
LocalNamespace::new(42)
|
||||
))
|
||||
.await
|
||||
.unwrap()
|
||||
.entry_id
|
||||
);
|
||||
}
|
||||
|
||||
fn generate_data(size: usize) -> Vec<u8> {
|
||||
let s: String = rand::thread_rng()
|
||||
.sample_iter(&Alphanumeric)
|
||||
.take(size)
|
||||
.map(char::from)
|
||||
.collect();
|
||||
s.into_bytes()
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_write_and_read_data() {
|
||||
common_telemetry::logging::init_default_ut_logging();
|
||||
let dir = TempDir::new("greptimedb2").unwrap();
|
||||
|
||||
let dir_str = dir.path().to_string_lossy().to_string();
|
||||
info!("dir: {}", dir_str);
|
||||
|
||||
let config = LogConfig {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 128,
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
};
|
||||
let logstore = LocalFileLogStore::open(&config).await.unwrap();
|
||||
let ns = LocalNamespace::new(42);
|
||||
let id = logstore
|
||||
.append(EntryImpl::new(
|
||||
generate_data(96),
|
||||
0,
|
||||
LocalNamespace::new(42),
|
||||
))
|
||||
.await
|
||||
.unwrap()
|
||||
.entry_id;
|
||||
assert_eq!(0, id);
|
||||
|
||||
let stream = logstore.read(&ns, 0).await.unwrap();
|
||||
tokio::pin!(stream);
|
||||
|
||||
let entries = stream.next().await.unwrap().unwrap();
|
||||
assert_eq!(entries.len(), 1);
|
||||
assert_eq!(entries[0].id(), 0);
|
||||
assert_eq!(42, entries[0].namespace_id);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_namespace() {
|
||||
common_telemetry::logging::init_default_ut_logging();
|
||||
let dir = TempDir::new("greptimedb").unwrap();
|
||||
let config = LogConfig {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 1024 * 1024,
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
};
|
||||
let logstore = LocalFileLogStore::open(&config).await.unwrap();
|
||||
assert_eq!(
|
||||
0,
|
||||
logstore
|
||||
.append(EntryImpl::new(
|
||||
generate_data(96),
|
||||
0,
|
||||
LocalNamespace::new(42),
|
||||
))
|
||||
.await
|
||||
.unwrap()
|
||||
.entry_id
|
||||
);
|
||||
|
||||
assert_eq!(
|
||||
1,
|
||||
logstore
|
||||
.append(EntryImpl::new(
|
||||
generate_data(96),
|
||||
1,
|
||||
LocalNamespace::new(43),
|
||||
))
|
||||
.await
|
||||
.unwrap()
|
||||
.entry_id
|
||||
);
|
||||
|
||||
let stream = logstore.read(&LocalNamespace::new(42), 0).await.unwrap();
|
||||
tokio::pin!(stream);
|
||||
|
||||
let entries = stream.next().await.unwrap().unwrap();
|
||||
assert_eq!(entries.len(), 1);
|
||||
assert_eq!(entries[0].id(), 0);
|
||||
assert_eq!(42, entries[0].namespace_id);
|
||||
|
||||
let stream = logstore.read(&LocalNamespace::new(43), 0).await.unwrap();
|
||||
tokio::pin!(stream);
|
||||
|
||||
let entries = stream.next().await.unwrap().unwrap();
|
||||
assert_eq!(entries.len(), 1);
|
||||
assert_eq!(entries[0].id(), 1);
|
||||
assert_eq!(43, entries[0].namespace_id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_find_files_to_delete() {
|
||||
let file_map = vec![(1u64, ()), (11u64, ()), (21u64, ()), (31u64, ())]
|
||||
.into_iter()
|
||||
.collect::<BTreeMap<u64, ()>>();
|
||||
|
||||
assert!(find_files_to_delete(&file_map, 0).is_empty());
|
||||
assert!(find_files_to_delete(&file_map, 1).is_empty());
|
||||
assert!(find_files_to_delete(&file_map, 2).is_empty());
|
||||
assert!(find_files_to_delete(&file_map, 10).is_empty());
|
||||
|
||||
assert_eq!(vec![1], find_files_to_delete(&file_map, 11));
|
||||
assert_eq!(vec![1], find_files_to_delete(&file_map, 20));
|
||||
assert_eq!(vec![1, 11], find_files_to_delete(&file_map, 21));
|
||||
|
||||
assert_eq!(vec![1, 11, 21], find_files_to_delete(&file_map, 31));
|
||||
assert_eq!(vec![1, 11, 21], find_files_to_delete(&file_map, 100));
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_find_lowest_id() {
|
||||
common_telemetry::logging::init_default_ut_logging();
|
||||
let dir = TempDir::new("greptimedb-log-compact").unwrap();
|
||||
let config = LogConfig {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 4096,
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
};
|
||||
let logstore = LocalFileLogStore::open(&config).await.unwrap();
|
||||
assert!(find_lowest_id(logstore.obsolete_ids.clone())
|
||||
.await
|
||||
.is_none());
|
||||
|
||||
logstore
|
||||
.obsolete(LocalNamespace::new(1), 100)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
Some(100),
|
||||
find_lowest_id(logstore.obsolete_ids.clone()).await
|
||||
);
|
||||
|
||||
logstore
|
||||
.obsolete(LocalNamespace::new(2), 200)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
Some(100),
|
||||
find_lowest_id(logstore.obsolete_ids.clone()).await
|
||||
);
|
||||
|
||||
logstore
|
||||
.obsolete(LocalNamespace::new(1), 101)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
Some(101),
|
||||
find_lowest_id(logstore.obsolete_ids.clone()).await
|
||||
);
|
||||
|
||||
logstore
|
||||
.obsolete(LocalNamespace::new(2), 202)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
Some(101),
|
||||
find_lowest_id(logstore.obsolete_ids.clone()).await
|
||||
);
|
||||
|
||||
logstore
|
||||
.obsolete(LocalNamespace::new(1), 300)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
Some(202),
|
||||
find_lowest_id(logstore.obsolete_ids.clone()).await
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_compact_log_file() {
|
||||
common_telemetry::logging::init_default_ut_logging();
|
||||
let dir = TempDir::new("greptimedb-log-compact").unwrap();
|
||||
let config = LogConfig {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 4096,
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
};
|
||||
let logstore = LocalFileLogStore::open(&config).await.unwrap();
|
||||
|
||||
for id in 0..50 {
|
||||
logstore
|
||||
.append(EntryImpl::new(
|
||||
generate_data(990),
|
||||
id,
|
||||
LocalNamespace::new(42),
|
||||
))
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
assert_eq!(
|
||||
vec![0, 4, 8, 12, 16, 20, 24, 28, 32, 36, 40, 44, 48],
|
||||
logstore
|
||||
.files
|
||||
.read()
|
||||
.await
|
||||
.keys()
|
||||
.copied()
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
|
||||
gc_inner(logstore.files.clone(), 10).await.unwrap();
|
||||
|
||||
assert_eq!(
|
||||
vec![8, 12, 16, 20, 24, 28, 32, 36, 40, 44, 48],
|
||||
logstore
|
||||
.files
|
||||
.read()
|
||||
.await
|
||||
.keys()
|
||||
.copied()
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
|
||||
gc_inner(logstore.files.clone(), 28).await.unwrap();
|
||||
|
||||
assert_eq!(
|
||||
vec![28, 32, 36, 40, 44, 48],
|
||||
logstore
|
||||
.files
|
||||
.read()
|
||||
.await
|
||||
.keys()
|
||||
.copied()
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
|
||||
gc_inner(logstore.files.clone(), 50).await.unwrap();
|
||||
|
||||
assert_eq!(
|
||||
vec![48],
|
||||
logstore
|
||||
.files
|
||||
.read()
|
||||
.await
|
||||
.keys()
|
||||
.copied()
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_gc_task() {
|
||||
common_telemetry::logging::init_default_ut_logging();
|
||||
let dir = TempDir::new("greptimedb-log-compact").unwrap();
|
||||
let config = LogConfig {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 4096,
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
gc_interval: Duration::from_millis(100),
|
||||
};
|
||||
let logstore = LocalFileLogStore::open(&config).await.unwrap();
|
||||
logstore.start().await.unwrap();
|
||||
|
||||
for id in 0..50 {
|
||||
logstore
|
||||
.append(EntryImpl::new(
|
||||
generate_data(990),
|
||||
id,
|
||||
LocalNamespace::new(42),
|
||||
))
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
logstore
|
||||
.obsolete(LocalNamespace::new(42), 30)
|
||||
.await
|
||||
.unwrap();
|
||||
tokio::time::sleep(Duration::from_millis(150)).await;
|
||||
let file_ids = logstore
|
||||
.files
|
||||
.read()
|
||||
.await
|
||||
.keys()
|
||||
.cloned()
|
||||
.collect::<Vec<_>>();
|
||||
assert_eq!(vec![28, 32, 36, 40, 44, 48], file_ids);
|
||||
|
||||
let mut files = vec![];
|
||||
let mut readir = tokio::fs::read_dir(dir.path()).await.unwrap();
|
||||
while let Some(r) = readir.next_entry().await.transpose() {
|
||||
let entry = r.unwrap();
|
||||
files.push(entry.file_name().to_str().unwrap().to_string());
|
||||
}
|
||||
|
||||
assert_eq!(
|
||||
vec![
|
||||
"00000000000000000028.log".to_string(),
|
||||
"00000000000000000048.log".to_string(),
|
||||
"00000000000000000040.log".to_string(),
|
||||
"00000000000000000044.log".to_string(),
|
||||
"00000000000000000036.log".to_string(),
|
||||
"00000000000000000032.log".to_string()
|
||||
]
|
||||
.into_iter()
|
||||
.collect::<HashSet<String>>(),
|
||||
files.into_iter().collect::<HashSet<String>>()
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -1,38 +0,0 @@
|
||||
// Copyright 2022 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 store_api::logstore::namespace::{Id, Namespace};
|
||||
|
||||
#[derive(Clone, Debug, PartialEq, Eq, Hash)]
|
||||
pub struct LocalNamespace {
|
||||
pub(crate) id: Id,
|
||||
}
|
||||
|
||||
impl Default for LocalNamespace {
|
||||
fn default() -> Self {
|
||||
LocalNamespace::new(0)
|
||||
}
|
||||
}
|
||||
|
||||
impl LocalNamespace {
|
||||
pub(crate) fn new(id: Id) -> Self {
|
||||
Self { id }
|
||||
}
|
||||
}
|
||||
|
||||
impl Namespace for LocalNamespace {
|
||||
fn id(&self) -> Id {
|
||||
self.id
|
||||
}
|
||||
}
|
||||
@@ -1,93 +0,0 @@
|
||||
// Copyright 2022 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 store_api::logstore::entry::Id;
|
||||
use store_api::logstore::namespace::Id as NamespaceId;
|
||||
use store_api::logstore::LogStore;
|
||||
|
||||
use crate::error::{Error, Result};
|
||||
use crate::fs::entry::EntryImpl;
|
||||
use crate::fs::namespace::LocalNamespace;
|
||||
use crate::fs::AppendResponseImpl;
|
||||
|
||||
/// A noop log store which only for test
|
||||
// TODO: Add a test feature
|
||||
#[derive(Debug, Default)]
|
||||
pub struct NoopLogStore;
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for NoopLogStore {
|
||||
type Error = Error;
|
||||
type Namespace = LocalNamespace;
|
||||
type Entry = EntryImpl;
|
||||
type AppendResponse = AppendResponseImpl;
|
||||
|
||||
async fn start(&self) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn stop(&self) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn append(&self, mut _e: Self::Entry) -> Result<Self::AppendResponse> {
|
||||
Ok(AppendResponseImpl {
|
||||
entry_id: 0,
|
||||
offset: 0,
|
||||
})
|
||||
}
|
||||
|
||||
async fn append_batch(&self, _ns: &Self::Namespace, _e: Vec<Self::Entry>) -> Result<Id> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn read(
|
||||
&self,
|
||||
_ns: &Self::Namespace,
|
||||
_id: Id,
|
||||
) -> Result<store_api::logstore::entry_stream::SendableEntryStream<'_, Self::Entry, Self::Error>>
|
||||
{
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn create_namespace(&mut self, _ns: &Self::Namespace) -> Result<()> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn delete_namespace(&mut self, _ns: &Self::Namespace) -> Result<()> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn entry<D: AsRef<[u8]>>(&self, data: D, id: Id, ns: Self::Namespace) -> Self::Entry {
|
||||
EntryImpl::new(data, id, ns)
|
||||
}
|
||||
|
||||
fn namespace(&self, id: NamespaceId) -> Self::Namespace {
|
||||
LocalNamespace::new(id)
|
||||
}
|
||||
|
||||
async fn obsolete(
|
||||
&self,
|
||||
namespace: Self::Namespace,
|
||||
id: Id,
|
||||
) -> std::result::Result<(), Self::Error> {
|
||||
let _ = namespace;
|
||||
let _ = id;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -12,7 +12,11 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
mod config;
|
||||
pub mod error;
|
||||
pub mod fs;
|
||||
|
||||
mod noop;
|
||||
pub mod raft_engine;
|
||||
pub mod test_util;
|
||||
|
||||
pub use config::LogConfig;
|
||||
pub use noop::NoopLogStore;
|
||||
|
||||
148
src/log-store/src/noop.rs
Normal file
148
src/log-store/src/noop.rs
Normal file
@@ -0,0 +1,148 @@
|
||||
// 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 store_api::logstore::entry::{Entry, Id};
|
||||
use store_api::logstore::namespace::{Id as NamespaceId, Namespace};
|
||||
use store_api::logstore::{AppendResponse, LogStore};
|
||||
|
||||
use crate::error::{Error, Result};
|
||||
|
||||
/// A noop log store which only for test
|
||||
#[derive(Debug, Default)]
|
||||
pub struct NoopLogStore;
|
||||
|
||||
#[derive(Debug, Default, Clone, PartialEq)]
|
||||
pub struct EntryImpl;
|
||||
|
||||
#[derive(Debug, Clone, Default, Hash, PartialEq)]
|
||||
pub struct NamespaceImpl;
|
||||
|
||||
impl Namespace for NamespaceImpl {
|
||||
fn id(&self) -> NamespaceId {
|
||||
0
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry for EntryImpl {
|
||||
type Error = Error;
|
||||
type Namespace = NamespaceImpl;
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
&[]
|
||||
}
|
||||
|
||||
fn id(&self) -> Id {
|
||||
0
|
||||
}
|
||||
|
||||
fn namespace(&self) -> Self::Namespace {
|
||||
Default::default()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for NoopLogStore {
|
||||
type Error = Error;
|
||||
type Namespace = NamespaceImpl;
|
||||
type Entry = EntryImpl;
|
||||
|
||||
async fn stop(&self) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn append(&self, mut _e: Self::Entry) -> Result<AppendResponse> {
|
||||
Ok(AppendResponse { entry_id: 0 })
|
||||
}
|
||||
|
||||
async fn append_batch(&self, _ns: &Self::Namespace, _e: Vec<Self::Entry>) -> Result<Vec<Id>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
async fn read(
|
||||
&self,
|
||||
_ns: &Self::Namespace,
|
||||
_id: Id,
|
||||
) -> Result<store_api::logstore::entry_stream::SendableEntryStream<'_, Self::Entry, Self::Error>>
|
||||
{
|
||||
todo!()
|
||||
}
|
||||
|
||||
async fn create_namespace(&mut self, _ns: &Self::Namespace) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn delete_namespace(&mut self, _ns: &Self::Namespace) -> Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>> {
|
||||
Ok(vec![])
|
||||
}
|
||||
|
||||
fn entry<D: AsRef<[u8]>>(&self, data: D, id: Id, ns: Self::Namespace) -> Self::Entry {
|
||||
let _ = data;
|
||||
let _ = id;
|
||||
let _ = ns;
|
||||
EntryImpl::default()
|
||||
}
|
||||
|
||||
fn namespace(&self, id: NamespaceId) -> Self::Namespace {
|
||||
let _ = id;
|
||||
NamespaceImpl::default()
|
||||
}
|
||||
|
||||
async fn obsolete(
|
||||
&self,
|
||||
namespace: Self::Namespace,
|
||||
id: Id,
|
||||
) -> std::result::Result<(), Self::Error> {
|
||||
let _ = namespace;
|
||||
let _ = id;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_mock_entry() {
|
||||
let e = EntryImpl::default();
|
||||
assert_eq!(0, e.data().len());
|
||||
assert_eq!(0, e.id());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_noop_logstore() {
|
||||
let mut store = NoopLogStore::default();
|
||||
let e = store.entry("".as_bytes(), 1, NamespaceImpl::default());
|
||||
store.append(e.clone()).await.unwrap();
|
||||
store
|
||||
.append_batch(&NamespaceImpl::default(), vec![e])
|
||||
.await
|
||||
.unwrap();
|
||||
store
|
||||
.create_namespace(&NamespaceImpl::default())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(0, store.list_namespaces().await.unwrap().len());
|
||||
store
|
||||
.delete_namespace(&NamespaceImpl::default())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(NamespaceImpl::default(), store.namespace(0));
|
||||
store.obsolete(NamespaceImpl::default(), 1).await.unwrap();
|
||||
}
|
||||
}
|
||||
80
src/log-store/src/raft_engine.rs
Normal file
80
src/log-store/src/raft_engine.rs
Normal file
@@ -0,0 +1,80 @@
|
||||
// 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::hash::{Hash, Hasher};
|
||||
|
||||
use store_api::logstore::entry::{Entry, Id};
|
||||
use store_api::logstore::namespace::Namespace;
|
||||
|
||||
use crate::error::Error;
|
||||
use crate::raft_engine::protos::logstore::{EntryImpl, NamespaceImpl};
|
||||
|
||||
pub mod log_store;
|
||||
|
||||
pub mod protos {
|
||||
include!(concat!(env!("OUT_DIR"), concat!("/", "protos/", "mod.rs")));
|
||||
}
|
||||
|
||||
impl EntryImpl {
|
||||
pub fn create(id: u64, ns: u64, data: Vec<u8>) -> Self {
|
||||
Self {
|
||||
id,
|
||||
namespace_id: ns,
|
||||
data,
|
||||
..Default::default()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl NamespaceImpl {
|
||||
pub fn with_id(id: Id) -> Self {
|
||||
Self {
|
||||
id,
|
||||
..Default::default()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[allow(clippy::derive_hash_xor_eq)]
|
||||
impl Hash for NamespaceImpl {
|
||||
fn hash<H: Hasher>(&self, state: &mut H) {
|
||||
self.id.hash(state);
|
||||
}
|
||||
}
|
||||
|
||||
impl Namespace for NamespaceImpl {
|
||||
fn id(&self) -> store_api::logstore::namespace::Id {
|
||||
self.id
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry for EntryImpl {
|
||||
type Error = Error;
|
||||
type Namespace = NamespaceImpl;
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
self.data.as_slice()
|
||||
}
|
||||
|
||||
fn id(&self) -> Id {
|
||||
self.id
|
||||
}
|
||||
|
||||
fn namespace(&self) -> Self::Namespace {
|
||||
NamespaceImpl {
|
||||
id: self.namespace_id,
|
||||
..Default::default()
|
||||
}
|
||||
}
|
||||
}
|
||||
553
src/log-store/src/raft_engine/log_store.rs
Normal file
553
src/log-store/src/raft_engine/log_store.rs
Normal file
@@ -0,0 +1,553 @@
|
||||
// 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::fmt::{Debug, Formatter};
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_stream::stream;
|
||||
use common_telemetry::{error, info};
|
||||
use raft_engine::{Config, Engine, LogBatch, MessageExt, ReadableSize, RecoveryMode};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::logstore::entry::Id;
|
||||
use store_api::logstore::entry_stream::SendableEntryStream;
|
||||
use store_api::logstore::namespace::Namespace as NamespaceTrait;
|
||||
use store_api::logstore::{AppendResponse, LogStore};
|
||||
use tokio::sync::Mutex;
|
||||
use tokio::task::JoinHandle;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
|
||||
use crate::config::LogConfig;
|
||||
use crate::error::{
|
||||
AddEntryLogBatchSnafu, Error, FetchEntrySnafu, IllegalNamespaceSnafu, IllegalStateSnafu,
|
||||
RaftEngineSnafu, WaitGcTaskStopSnafu,
|
||||
};
|
||||
use crate::raft_engine::protos::logstore::{EntryImpl as Entry, NamespaceImpl as Namespace};
|
||||
|
||||
const NAMESPACE_PREFIX: &str = "__sys_namespace_";
|
||||
const SYSTEM_NAMESPACE: u64 = 0;
|
||||
|
||||
pub struct RaftEngineLogStore {
|
||||
config: LogConfig,
|
||||
engine: Arc<Engine>,
|
||||
cancel_token: Mutex<Option<CancellationToken>>,
|
||||
gc_task_handle: Mutex<Option<JoinHandle<()>>>,
|
||||
started: AtomicBool,
|
||||
}
|
||||
|
||||
impl RaftEngineLogStore {
|
||||
pub async fn try_new(config: LogConfig) -> Result<Self, Error> {
|
||||
// TODO(hl): set according to available disk space
|
||||
let raft_engine_config = Config {
|
||||
dir: config.log_file_dir.clone(),
|
||||
purge_threshold: ReadableSize(config.purge_threshold as u64),
|
||||
recovery_mode: RecoveryMode::TolerateTailCorruption,
|
||||
batch_compression_threshold: ReadableSize::kb(8),
|
||||
target_file_size: ReadableSize(config.file_size as u64),
|
||||
..Default::default()
|
||||
};
|
||||
let engine = Arc::new(Engine::open(raft_engine_config).context(RaftEngineSnafu)?);
|
||||
let log_store = Self {
|
||||
config,
|
||||
engine,
|
||||
cancel_token: Mutex::new(None),
|
||||
gc_task_handle: Mutex::new(None),
|
||||
started: AtomicBool::new(false),
|
||||
};
|
||||
log_store.start().await?;
|
||||
Ok(log_store)
|
||||
}
|
||||
|
||||
pub fn started(&self) -> bool {
|
||||
self.started.load(Ordering::Relaxed)
|
||||
}
|
||||
|
||||
async fn start(&self) -> Result<(), Error> {
|
||||
let engine_clone = self.engine.clone();
|
||||
let interval = self.config.purge_interval;
|
||||
let token = CancellationToken::new();
|
||||
let child = token.child_token();
|
||||
// TODO(hl): Maybe spawn to a blocking runtime.
|
||||
let handle = common_runtime::spawn_bg(async move {
|
||||
loop {
|
||||
tokio::select! {
|
||||
_ = tokio::time::sleep(interval) => {}
|
||||
_ = child.cancelled() => {
|
||||
info!("LogStore gc task has been cancelled");
|
||||
return;
|
||||
}
|
||||
}
|
||||
match engine_clone.purge_expired_files().context(RaftEngineSnafu) {
|
||||
Ok(res) => {
|
||||
// TODO(hl): the retval of purge_expired_files indicates the namespaces need to be compact,
|
||||
// which is useful when monitoring regions failed to flush it's memtable to SSTs.
|
||||
info!(
|
||||
"Successfully purged logstore files, namespaces need compaction: {:?}",
|
||||
res
|
||||
);
|
||||
}
|
||||
Err(e) => {
|
||||
error!(e; "Failed to purge files in logstore");
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
*self.cancel_token.lock().await = Some(token);
|
||||
*self.gc_task_handle.lock().await = Some(handle);
|
||||
self.started.store(true, Ordering::Relaxed);
|
||||
info!("RaftEngineLogStore started with config: {:?}", self.config);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl Debug for RaftEngineLogStore {
|
||||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("RaftEngineLogsStore")
|
||||
.field("config", &self.config)
|
||||
.field("started", &self.started.load(Ordering::Relaxed))
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl LogStore for RaftEngineLogStore {
|
||||
type Error = Error;
|
||||
type Namespace = Namespace;
|
||||
type Entry = Entry;
|
||||
|
||||
async fn stop(&self) -> Result<(), Self::Error> {
|
||||
ensure!(
|
||||
self.started
|
||||
.compare_exchange(true, false, Ordering::Relaxed, Ordering::Relaxed)
|
||||
.is_ok(),
|
||||
IllegalStateSnafu
|
||||
);
|
||||
let handle = self
|
||||
.gc_task_handle
|
||||
.lock()
|
||||
.await
|
||||
.take()
|
||||
.context(IllegalStateSnafu)?;
|
||||
let token = self
|
||||
.cancel_token
|
||||
.lock()
|
||||
.await
|
||||
.take()
|
||||
.context(IllegalStateSnafu)?;
|
||||
token.cancel();
|
||||
handle.await.context(WaitGcTaskStopSnafu)?;
|
||||
info!("RaftEngineLogStore stopped");
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Append an entry to logstore. Currently of existence of entry's namespace is not checked.
|
||||
async fn append(&self, e: Self::Entry) -> Result<AppendResponse, Self::Error> {
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let entry_id = e.id;
|
||||
let mut batch = LogBatch::with_capacity(1);
|
||||
batch
|
||||
.add_entries::<MessageType>(e.namespace_id, &[e])
|
||||
.context(AddEntryLogBatchSnafu)?;
|
||||
|
||||
self.engine
|
||||
.write(&mut batch, self.config.sync_write)
|
||||
.context(RaftEngineSnafu)?;
|
||||
Ok(AppendResponse { entry_id })
|
||||
}
|
||||
|
||||
/// Append a batch of entries to logstore. `RaftEngineLogStore` assures the atomicity of
|
||||
/// batch append.
|
||||
async fn append_batch(
|
||||
&self,
|
||||
ns: &Self::Namespace,
|
||||
entries: Vec<Self::Entry>,
|
||||
) -> Result<Vec<Id>, Self::Error> {
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let entry_ids = entries.iter().map(Entry::get_id).collect::<Vec<_>>();
|
||||
let mut batch = LogBatch::with_capacity(entries.len());
|
||||
batch
|
||||
.add_entries::<MessageType>(ns.id, &entries)
|
||||
.context(AddEntryLogBatchSnafu)?;
|
||||
self.engine
|
||||
.write(&mut batch, self.config.sync_write)
|
||||
.context(RaftEngineSnafu)?;
|
||||
Ok(entry_ids)
|
||||
}
|
||||
|
||||
/// Create a stream of entries from logstore in the given namespace. The end of stream is
|
||||
/// determined by the current "last index" of the namespace.
|
||||
async fn read(
|
||||
&self,
|
||||
ns: &Self::Namespace,
|
||||
id: Id,
|
||||
) -> Result<SendableEntryStream<'_, Self::Entry, Self::Error>, Self::Error> {
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let engine = self.engine.clone();
|
||||
|
||||
let last_index = engine.last_index(ns.id).unwrap_or(0);
|
||||
let mut start_index = id.max(engine.first_index(ns.id).unwrap_or(last_index + 1));
|
||||
|
||||
let max_batch_size = self.config.read_batch_size;
|
||||
let (tx, mut rx) = tokio::sync::mpsc::channel(max_batch_size);
|
||||
let ns = ns.clone();
|
||||
common_runtime::spawn_read(async move {
|
||||
while start_index <= last_index {
|
||||
let mut vec = Vec::with_capacity(max_batch_size);
|
||||
match engine
|
||||
.fetch_entries_to::<MessageType>(
|
||||
ns.id,
|
||||
start_index,
|
||||
last_index + 1,
|
||||
Some(max_batch_size),
|
||||
&mut vec,
|
||||
)
|
||||
.context(FetchEntrySnafu {
|
||||
ns: ns.id,
|
||||
start: start_index,
|
||||
end: last_index,
|
||||
max_size: max_batch_size,
|
||||
}) {
|
||||
Ok(_) => {
|
||||
if let Some(last_entry) = vec.last() {
|
||||
start_index = last_entry.id + 1;
|
||||
}
|
||||
// reader side closed, cancel following reads
|
||||
if tx.send(Ok(vec)).await.is_err() {
|
||||
break;
|
||||
}
|
||||
}
|
||||
Err(e) => {
|
||||
let _ = tx.send(Err(e)).await;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
let s = stream!({
|
||||
while let Some(res) = rx.recv().await {
|
||||
yield res;
|
||||
}
|
||||
});
|
||||
Ok(Box::pin(s))
|
||||
}
|
||||
|
||||
async fn create_namespace(&mut self, ns: &Self::Namespace) -> Result<(), Self::Error> {
|
||||
ensure!(
|
||||
ns.id != SYSTEM_NAMESPACE,
|
||||
IllegalNamespaceSnafu { ns: ns.id }
|
||||
);
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let key = format!("{}{}", NAMESPACE_PREFIX, ns.id).as_bytes().to_vec();
|
||||
let mut batch = LogBatch::with_capacity(1);
|
||||
batch
|
||||
.put_message::<Namespace>(SYSTEM_NAMESPACE, key, ns)
|
||||
.context(RaftEngineSnafu)?;
|
||||
self.engine
|
||||
.write(&mut batch, true)
|
||||
.context(RaftEngineSnafu)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn delete_namespace(&mut self, ns: &Self::Namespace) -> Result<(), Self::Error> {
|
||||
ensure!(
|
||||
ns.id != SYSTEM_NAMESPACE,
|
||||
IllegalNamespaceSnafu { ns: ns.id }
|
||||
);
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let key = format!("{}{}", NAMESPACE_PREFIX, ns.id).as_bytes().to_vec();
|
||||
let mut batch = LogBatch::with_capacity(1);
|
||||
batch.delete(SYSTEM_NAMESPACE, key);
|
||||
self.engine
|
||||
.write(&mut batch, true)
|
||||
.context(RaftEngineSnafu)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn list_namespaces(&self) -> Result<Vec<Self::Namespace>, Self::Error> {
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let mut namespaces: Vec<Namespace> = vec![];
|
||||
self.engine
|
||||
.scan_messages::<Namespace, _>(
|
||||
SYSTEM_NAMESPACE,
|
||||
Some(NAMESPACE_PREFIX.as_bytes()),
|
||||
None,
|
||||
false,
|
||||
|_, v| {
|
||||
namespaces.push(v);
|
||||
true
|
||||
},
|
||||
)
|
||||
.context(RaftEngineSnafu)?;
|
||||
Ok(namespaces)
|
||||
}
|
||||
|
||||
fn entry<D: AsRef<[u8]>>(&self, data: D, id: Id, ns: Self::Namespace) -> Self::Entry {
|
||||
Entry {
|
||||
id,
|
||||
data: data.as_ref().to_vec(),
|
||||
namespace_id: ns.id(),
|
||||
..Default::default()
|
||||
}
|
||||
}
|
||||
|
||||
fn namespace(&self, id: store_api::logstore::namespace::Id) -> Self::Namespace {
|
||||
Namespace {
|
||||
id,
|
||||
..Default::default()
|
||||
}
|
||||
}
|
||||
|
||||
async fn obsolete(&self, namespace: Self::Namespace, id: Id) -> Result<(), Self::Error> {
|
||||
ensure!(self.started(), IllegalStateSnafu);
|
||||
let obsoleted = self.engine.compact_to(namespace.id(), id + 1);
|
||||
info!(
|
||||
"Namespace {} obsoleted {} entries",
|
||||
namespace.id(),
|
||||
obsoleted
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
struct MessageType;
|
||||
|
||||
impl MessageExt for MessageType {
|
||||
type Entry = Entry;
|
||||
|
||||
fn index(e: &Self::Entry) -> u64 {
|
||||
e.id
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashSet;
|
||||
use std::time::Duration;
|
||||
|
||||
use common_telemetry::debug;
|
||||
use futures_util::StreamExt;
|
||||
use raft_engine::ReadableSize;
|
||||
use store_api::logstore::entry_stream::SendableEntryStream;
|
||||
use store_api::logstore::namespace::Namespace as NamespaceTrait;
|
||||
use store_api::logstore::LogStore;
|
||||
use tempdir::TempDir;
|
||||
|
||||
use crate::config::LogConfig;
|
||||
use crate::error::Error;
|
||||
use crate::raft_engine::log_store::RaftEngineLogStore;
|
||||
use crate::raft_engine::protos::logstore::{EntryImpl as Entry, NamespaceImpl as Namespace};
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_open_logstore() {
|
||||
let dir = TempDir::new("raft-engine-logstore-test").unwrap();
|
||||
let logstore = RaftEngineLogStore::try_new(LogConfig {
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
})
|
||||
.await
|
||||
.unwrap();
|
||||
logstore.start().await.unwrap();
|
||||
let namespaces = logstore.list_namespaces().await.unwrap();
|
||||
assert_eq!(0, namespaces.len());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_manage_namespace() {
|
||||
let dir = TempDir::new("raft-engine-logstore-test").unwrap();
|
||||
let mut logstore = RaftEngineLogStore::try_new(LogConfig {
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
})
|
||||
.await
|
||||
.unwrap();
|
||||
logstore.start().await.unwrap();
|
||||
assert!(logstore.list_namespaces().await.unwrap().is_empty());
|
||||
|
||||
logstore
|
||||
.create_namespace(&Namespace::with_id(42))
|
||||
.await
|
||||
.unwrap();
|
||||
let namespaces = logstore.list_namespaces().await.unwrap();
|
||||
assert_eq!(1, namespaces.len());
|
||||
assert_eq!(Namespace::with_id(42), namespaces[0]);
|
||||
|
||||
logstore
|
||||
.delete_namespace(&Namespace::with_id(42))
|
||||
.await
|
||||
.unwrap();
|
||||
assert!(logstore.list_namespaces().await.unwrap().is_empty());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_append_and_read() {
|
||||
let dir = TempDir::new("raft-engine-logstore-test").unwrap();
|
||||
let logstore = RaftEngineLogStore::try_new(LogConfig {
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
})
|
||||
.await
|
||||
.unwrap();
|
||||
logstore.start().await.unwrap();
|
||||
|
||||
let namespace = Namespace::with_id(1);
|
||||
let cnt = 1024;
|
||||
for i in 0..cnt {
|
||||
let response = logstore
|
||||
.append(Entry::create(
|
||||
i,
|
||||
namespace.id,
|
||||
i.to_string().as_bytes().to_vec(),
|
||||
))
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(i, response.entry_id);
|
||||
}
|
||||
let mut entries = HashSet::with_capacity(1024);
|
||||
let mut s = logstore.read(&Namespace::with_id(1), 0).await.unwrap();
|
||||
while let Some(r) = s.next().await {
|
||||
let vec = r.unwrap();
|
||||
entries.extend(vec.into_iter().map(|e| e.id));
|
||||
}
|
||||
assert_eq!((0..cnt).into_iter().collect::<HashSet<_>>(), entries);
|
||||
}
|
||||
|
||||
async fn collect_entries(mut s: SendableEntryStream<'_, Entry, Error>) -> Vec<Entry> {
|
||||
let mut res = vec![];
|
||||
while let Some(r) = s.next().await {
|
||||
res.extend(r.unwrap());
|
||||
}
|
||||
res
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_reopen() {
|
||||
let dir = TempDir::new("raft-engine-logstore-reopen-test").unwrap();
|
||||
{
|
||||
let logstore = RaftEngineLogStore::try_new(LogConfig {
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
})
|
||||
.await
|
||||
.unwrap();
|
||||
logstore
|
||||
.append(Entry::create(1, 1, "1".as_bytes().to_vec()))
|
||||
.await
|
||||
.unwrap();
|
||||
let entries = logstore
|
||||
.read(&Namespace::with_id(1), 1)
|
||||
.await
|
||||
.unwrap()
|
||||
.collect::<Vec<_>>()
|
||||
.await;
|
||||
assert_eq!(1, entries.len());
|
||||
logstore.stop().await.unwrap();
|
||||
}
|
||||
|
||||
let logstore = RaftEngineLogStore::try_new(LogConfig {
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
})
|
||||
.await
|
||||
.unwrap();
|
||||
logstore.start().await.unwrap();
|
||||
|
||||
let entries =
|
||||
collect_entries(logstore.read(&Namespace::with_id(1), 1).await.unwrap()).await;
|
||||
assert_eq!(1, entries.len());
|
||||
assert_eq!(1, entries[0].id);
|
||||
assert_eq!(1, entries[0].namespace_id);
|
||||
}
|
||||
|
||||
async fn wal_dir_usage(path: impl AsRef<str>) -> usize {
|
||||
let mut size: usize = 0;
|
||||
let mut read_dir = tokio::fs::read_dir(path.as_ref()).await.unwrap();
|
||||
while let Ok(dir_entry) = read_dir.next_entry().await {
|
||||
let Some(entry) = dir_entry else {
|
||||
break;
|
||||
};
|
||||
if entry.file_type().await.unwrap().is_file() {
|
||||
let file_name = entry.file_name();
|
||||
let file_size = entry.metadata().await.unwrap().len() as usize;
|
||||
debug!("File: {file_name:?}, size: {file_size}");
|
||||
size += file_size;
|
||||
}
|
||||
}
|
||||
size
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_compaction() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let dir = TempDir::new("raft-engine-logstore-test").unwrap();
|
||||
|
||||
let config = LogConfig {
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
file_size: ReadableSize::mb(2).0 as usize,
|
||||
purge_threshold: ReadableSize::mb(4).0 as usize,
|
||||
purge_interval: Duration::from_secs(5),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let logstore = RaftEngineLogStore::try_new(config).await.unwrap();
|
||||
logstore.start().await.unwrap();
|
||||
let namespace = Namespace::with_id(42);
|
||||
for id in 0..4096 {
|
||||
let entry = Entry::create(id, namespace.id(), [b'x'; 4096].to_vec());
|
||||
logstore.append(entry).await.unwrap();
|
||||
}
|
||||
|
||||
let before_purge = wal_dir_usage(dir.path().to_str().unwrap()).await;
|
||||
logstore.obsolete(namespace, 4000).await.unwrap();
|
||||
|
||||
tokio::time::sleep(Duration::from_secs(6)).await;
|
||||
let after_purge = wal_dir_usage(dir.path().to_str().unwrap()).await;
|
||||
debug!(
|
||||
"Before purge: {}, after purge: {}",
|
||||
before_purge, after_purge
|
||||
);
|
||||
assert!(before_purge > after_purge);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_obsolete() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let dir = TempDir::new("raft-engine-logstore-test").unwrap();
|
||||
|
||||
let config = LogConfig {
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
file_size: ReadableSize::mb(2).0 as usize,
|
||||
purge_threshold: ReadableSize::mb(4).0 as usize,
|
||||
purge_interval: Duration::from_secs(5),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let logstore = RaftEngineLogStore::try_new(config).await.unwrap();
|
||||
logstore.start().await.unwrap();
|
||||
let namespace = Namespace::with_id(42);
|
||||
for id in 0..1024 {
|
||||
let entry = Entry::create(id, namespace.id(), [b'x'; 4096].to_vec());
|
||||
logstore.append(entry).await.unwrap();
|
||||
}
|
||||
|
||||
logstore.obsolete(namespace.clone(), 100).await.unwrap();
|
||||
assert_eq!(101, logstore.engine.first_index(namespace.id).unwrap());
|
||||
|
||||
let res = logstore.read(&namespace, 100).await.unwrap();
|
||||
let mut vec = collect_entries(res).await;
|
||||
vec.sort_by(|a, b| a.id.partial_cmp(&b.id).unwrap());
|
||||
assert_eq!(101, vec.first().unwrap().id);
|
||||
}
|
||||
}
|
||||
@@ -14,19 +14,19 @@
|
||||
|
||||
use tempdir::TempDir;
|
||||
|
||||
use crate::fs::config::LogConfig;
|
||||
use crate::fs::log::LocalFileLogStore;
|
||||
use crate::raft_engine::log_store::RaftEngineLogStore;
|
||||
use crate::LogConfig;
|
||||
|
||||
/// Create a tmp directory for write log, used for test.
|
||||
// TODO: Add a test feature
|
||||
pub async fn create_tmp_local_file_log_store(dir: &str) -> (LocalFileLogStore, TempDir) {
|
||||
pub async fn create_tmp_local_file_log_store(dir: &str) -> (RaftEngineLogStore, TempDir) {
|
||||
let dir = TempDir::new(dir).unwrap();
|
||||
let cfg = LogConfig {
|
||||
append_buffer_size: 128,
|
||||
max_log_file_size: 128,
|
||||
file_size: 128 * 1024,
|
||||
log_file_dir: dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
(LocalFileLogStore::open(&cfg).await.unwrap(), dir)
|
||||
let logstore = RaftEngineLogStore::try_new(cfg).await.unwrap();
|
||||
(logstore, dir)
|
||||
}
|
||||
|
||||
@@ -527,7 +527,7 @@ mod tests {
|
||||
use datatypes::vectors::{
|
||||
Float64Vector, Int32Vector, StringVector, TimestampMillisecondVector, VectorRef,
|
||||
};
|
||||
use log_store::fs::noop::NoopLogStore;
|
||||
use log_store::NoopLogStore;
|
||||
use storage::config::EngineConfig as StorageEngineConfig;
|
||||
use storage::EngineImpl;
|
||||
use store_api::manifest::Manifest;
|
||||
|
||||
@@ -153,13 +153,6 @@ pub enum Error {
|
||||
table_name: String,
|
||||
},
|
||||
|
||||
#[snafu(display("Columns {} not exist in table {}", column_names.join(","), table_name))]
|
||||
ColumnsNotExist {
|
||||
backtrace: Backtrace,
|
||||
column_names: Vec<String>,
|
||||
table_name: String,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to alter table {}, source: {}", table_name, source))]
|
||||
AlterTable {
|
||||
table_name: String,
|
||||
@@ -176,12 +169,6 @@ pub enum Error {
|
||||
column_qualified_name: String,
|
||||
},
|
||||
|
||||
#[snafu(display("Unsupported column default constraint, source: {}", source))]
|
||||
UnsupportedDefaultConstraint {
|
||||
#[snafu(backtrace)]
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to convert metadata from deserialized data, source: {}",
|
||||
source
|
||||
@@ -219,11 +206,8 @@ impl ErrorExt for Error {
|
||||
| ProjectedColumnNotFound { .. }
|
||||
| InvalidPrimaryKey { .. }
|
||||
| MissingTimestampIndex { .. }
|
||||
| UnsupportedDefaultConstraint { .. }
|
||||
| TableNotFound { .. } => StatusCode::InvalidArguments,
|
||||
|
||||
ColumnsNotExist { .. } => StatusCode::TableColumnNotFound,
|
||||
|
||||
TableInfoNotFound { .. } | ConvertRaw { .. } => StatusCode::Unexpected,
|
||||
|
||||
ScanTableManifest { .. } | UpdateTableManifest { .. } => StatusCode::StorageUnavailable,
|
||||
|
||||
@@ -27,18 +27,16 @@ use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_recordbatch::error::{ExternalSnafu, Result as RecordBatchResult};
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream};
|
||||
use common_telemetry::logging;
|
||||
use datatypes::schema::ColumnSchema;
|
||||
use datatypes::vectors::VectorRef;
|
||||
use futures::task::{Context, Poll};
|
||||
use futures::Stream;
|
||||
use object_store::ObjectStore;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::manifest::{self, Manifest, ManifestVersion, MetaActionIterator};
|
||||
use store_api::storage::{
|
||||
AddColumn, AlterOperation, AlterRequest, ChunkReader, PutOperation, ReadContext, Region,
|
||||
RegionMeta, ScanRequest, SchemaRef, Snapshot, WriteContext, WriteRequest,
|
||||
AddColumn, AlterOperation, AlterRequest, ChunkReader, ReadContext, Region, RegionMeta,
|
||||
ScanRequest, SchemaRef, Snapshot, WriteContext, WriteRequest,
|
||||
};
|
||||
use table::error::{Error as TableError, MissingColumnSnafu, Result as TableResult};
|
||||
use table::error::{Error as TableError, Result as TableResult};
|
||||
use table::metadata::{
|
||||
FilterPushDownType, RawTableInfo, TableInfo, TableInfoRef, TableMeta, TableType,
|
||||
};
|
||||
@@ -48,8 +46,8 @@ use table::table::Table;
|
||||
use tokio::sync::Mutex;
|
||||
|
||||
use crate::error::{
|
||||
self, ColumnsNotExistSnafu, ProjectedColumnNotFoundSnafu, Result, ScanTableManifestSnafu,
|
||||
TableInfoNotFoundSnafu, UnsupportedDefaultConstraintSnafu, UpdateTableManifestSnafu,
|
||||
self, ProjectedColumnNotFoundSnafu, Result, ScanTableManifestSnafu, TableInfoNotFoundSnafu,
|
||||
UpdateTableManifestSnafu,
|
||||
};
|
||||
use crate::manifest::action::*;
|
||||
use crate::manifest::TableManifest;
|
||||
@@ -86,66 +84,17 @@ impl<R: Region> Table for MitoTable<R> {
|
||||
|
||||
let mut write_request = self.region.write_request();
|
||||
|
||||
let mut put_op = write_request.put_op();
|
||||
let mut columns_values = request.columns_values;
|
||||
|
||||
let table_info = self.table_info();
|
||||
let schema = self.schema();
|
||||
let key_columns = table_info.meta.row_key_column_names();
|
||||
let value_columns = table_info.meta.value_column_names();
|
||||
let columns_values = request.columns_values;
|
||||
// columns_values is not empty, it's safe to unwrap
|
||||
let rows_num = columns_values.values().next().unwrap().len();
|
||||
|
||||
// Add row key columns
|
||||
for name in key_columns {
|
||||
let column_schema = schema
|
||||
.column_schema_by_name(name)
|
||||
.expect("column schema not found");
|
||||
|
||||
let vector = match columns_values.remove(name) {
|
||||
Some(v) => v,
|
||||
None => Self::try_get_column_default_constraint_vector(column_schema, rows_num)?,
|
||||
};
|
||||
|
||||
put_op
|
||||
.add_key_column(name, vector)
|
||||
.map_err(TableError::new)?;
|
||||
}
|
||||
|
||||
// Add value columns
|
||||
for name in value_columns {
|
||||
let column_schema = schema
|
||||
.column_schema_by_name(name)
|
||||
.expect("column schema not found");
|
||||
|
||||
let vector = match columns_values.remove(name) {
|
||||
Some(v) => v,
|
||||
None => Self::try_get_column_default_constraint_vector(column_schema, rows_num)?,
|
||||
};
|
||||
put_op
|
||||
.add_value_column(name, vector)
|
||||
.map_err(TableError::new)?;
|
||||
}
|
||||
|
||||
ensure!(
|
||||
columns_values.is_empty(),
|
||||
ColumnsNotExistSnafu {
|
||||
table_name: &table_info.name,
|
||||
column_names: columns_values
|
||||
.keys()
|
||||
.into_iter()
|
||||
.map(|s| s.to_string())
|
||||
.collect::<Vec<_>>(),
|
||||
}
|
||||
);
|
||||
|
||||
logging::trace!(
|
||||
"Insert into table {} with put_op: {:?}",
|
||||
table_info.name,
|
||||
put_op
|
||||
"Insert into table {} with data: {:?}",
|
||||
self.table_info().name,
|
||||
columns_values
|
||||
);
|
||||
|
||||
write_request.put(put_op).map_err(TableError::new)?;
|
||||
write_request.put(columns_values).map_err(TableError::new)?;
|
||||
|
||||
let _resp = self
|
||||
.region
|
||||
@@ -375,21 +324,6 @@ impl<R: Region> MitoTable<R> {
|
||||
Ok(MitoTable::new(table_info, region, manifest))
|
||||
}
|
||||
|
||||
fn try_get_column_default_constraint_vector(
|
||||
column_schema: &ColumnSchema,
|
||||
rows_num: usize,
|
||||
) -> TableResult<VectorRef> {
|
||||
// TODO(dennis): when we support altering schema, we should check the schemas difference between table and region
|
||||
let vector = column_schema
|
||||
.create_default_vector(rows_num)
|
||||
.context(UnsupportedDefaultConstraintSnafu)?
|
||||
.context(MissingColumnSnafu {
|
||||
name: &column_schema.name,
|
||||
})?;
|
||||
|
||||
Ok(vector)
|
||||
}
|
||||
|
||||
pub async fn open(
|
||||
table_name: &str,
|
||||
table_dir: &str,
|
||||
|
||||
@@ -20,7 +20,7 @@ use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnSchema, Schema, SchemaBuilder, SchemaRef};
|
||||
use datatypes::vectors::VectorRef;
|
||||
use log_store::fs::noop::NoopLogStore;
|
||||
use log_store::NoopLogStore;
|
||||
use object_store::services::fs::Builder;
|
||||
use object_store::ObjectStore;
|
||||
use storage::config::EngineConfig as StorageEngineConfig;
|
||||
|
||||
@@ -24,7 +24,7 @@ use common_telemetry::logging;
|
||||
use datatypes::prelude::{DataType, Value, VectorRef};
|
||||
use datatypes::schema::{ColumnSchema, Schema};
|
||||
use storage::metadata::{RegionMetaImpl, RegionMetadata};
|
||||
use storage::write_batch::{Mutation, WriteBatch};
|
||||
use storage::write_batch::WriteBatch;
|
||||
use store_api::storage::{
|
||||
AlterRequest, Chunk, ChunkReader, CreateOptions, EngineContext, GetRequest, GetResponse,
|
||||
OpenOptions, ReadContext, Region, RegionDescriptor, RegionId, RegionMeta, ScanRequest,
|
||||
@@ -219,10 +219,10 @@ impl MockRegionInner {
|
||||
|
||||
let mut memtable = self.memtable.write().unwrap();
|
||||
|
||||
for Mutation::Put(put) in request.iter() {
|
||||
for mutation in &request.payload().mutations {
|
||||
for ColumnSchema { name, .. } in metadata.user_schema().column_schemas() {
|
||||
let column = memtable.get_mut(name).unwrap();
|
||||
if let Some(data) = put.column_by_name(name) {
|
||||
if let Some(data) = mutation.record_batch.column_by_name(name) {
|
||||
(0..data.len()).for_each(|i| column.push(data.get(i)));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -68,6 +68,7 @@ mito = { path = "../mito", features = ["test"] }
|
||||
ron = "0.7"
|
||||
serde = { version = "1.0", features = ["derive"] }
|
||||
storage = { path = "../storage" }
|
||||
store-api = { path = "../store-api" }
|
||||
tempdir = "0.3"
|
||||
tokio = { version = "1.18", features = ["full"] }
|
||||
tokio-test = "0.4"
|
||||
|
||||
@@ -101,9 +101,11 @@ mod tests {
|
||||
use query::QueryEngineFactory;
|
||||
|
||||
use super::*;
|
||||
type DefaultEngine = MitoEngine<EngineImpl<LocalFileLogStore>>;
|
||||
use log_store::fs::config::LogConfig;
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
|
||||
type DefaultEngine = MitoEngine<EngineImpl<RaftEngineLogStore>>;
|
||||
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use log_store::LogConfig;
|
||||
use mito::engine::MitoEngine;
|
||||
use storage::config::EngineConfig as StorageEngineConfig;
|
||||
use storage::EngineImpl;
|
||||
@@ -121,8 +123,7 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let log_store = LocalFileLogStore::open(&log_config).await.unwrap();
|
||||
|
||||
let log_store = RaftEngineLogStore::try_new(log_config).await.unwrap();
|
||||
let mock_engine = Arc::new(DefaultEngine::new(
|
||||
TableEngineConfig::default(),
|
||||
EngineImpl::new(
|
||||
|
||||
@@ -7,10 +7,12 @@ license.workspace = true
|
||||
[dependencies]
|
||||
api = { path = "../api" }
|
||||
catalog = { path = "../catalog" }
|
||||
common-base = { path = "../common/base" }
|
||||
common-catalog = { path = "../common/catalog" }
|
||||
common-error = { path = "../common/error" }
|
||||
common-time = { path = "../common/time" }
|
||||
datatypes = { path = "../datatypes" }
|
||||
hex = "0.4"
|
||||
itertools = "0.10"
|
||||
mito = { path = "../mito" }
|
||||
once_cell = "1.10"
|
||||
|
||||
@@ -24,6 +24,7 @@ pub mod statement;
|
||||
use std::str::FromStr;
|
||||
|
||||
use api::helper::ColumnDataTypeWrapper;
|
||||
use common_base::bytes::Bytes;
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use common_time::Timestamp;
|
||||
use datatypes::data_type::DataType;
|
||||
@@ -127,6 +128,26 @@ fn parse_string_to_value(
|
||||
}
|
||||
}
|
||||
|
||||
fn parse_hex_string(s: &str) -> Result<Value> {
|
||||
match hex::decode(s) {
|
||||
Ok(b) => Ok(Value::Binary(Bytes::from(b))),
|
||||
Err(hex::FromHexError::InvalidHexCharacter { c, index }) => ParseSqlValueSnafu {
|
||||
msg: format!(
|
||||
"Fail to parse hex string to Byte: invalid character {c:?} at position {index}"
|
||||
),
|
||||
}
|
||||
.fail(),
|
||||
Err(hex::FromHexError::OddLength) => ParseSqlValueSnafu {
|
||||
msg: "Fail to parse hex string to Byte: odd number of digits".to_string(),
|
||||
}
|
||||
.fail(),
|
||||
Err(e) => ParseSqlValueSnafu {
|
||||
msg: format!("Fail to parse hex string to Byte {s}, {e:?}"),
|
||||
}
|
||||
.fail(),
|
||||
}
|
||||
}
|
||||
|
||||
macro_rules! parse_number_to_value {
|
||||
($data_type: expr, $n: ident, $(($Type: ident, $PrimitiveType: ident)), +) => {
|
||||
match $data_type {
|
||||
@@ -200,6 +221,7 @@ pub fn sql_value_to_value(
|
||||
SqlValue::DoubleQuotedString(s) | SqlValue::SingleQuotedString(s) => {
|
||||
parse_string_to_value(column_name, s.to_owned(), data_type)?
|
||||
}
|
||||
SqlValue::HexStringLiteral(s) => parse_hex_string(s)?,
|
||||
_ => todo!("Other sql value"),
|
||||
})
|
||||
}
|
||||
@@ -299,6 +321,7 @@ pub fn sql_data_type_to_concrete_data_type(data_type: &SqlDataType) -> Result<Co
|
||||
SqlDataType::Double => Ok(ConcreteDataType::float64_datatype()),
|
||||
SqlDataType::Boolean => Ok(ConcreteDataType::boolean_datatype()),
|
||||
SqlDataType::Date => Ok(ConcreteDataType::date_datatype()),
|
||||
SqlDataType::Varbinary(_) => Ok(ConcreteDataType::binary_datatype()),
|
||||
SqlDataType::Custom(obj_name, _) => match &obj_name.0[..] {
|
||||
[type_name] => {
|
||||
if type_name
|
||||
@@ -379,6 +402,10 @@ mod tests {
|
||||
SqlDataType::Timestamp(None, TimezoneInfo::None),
|
||||
ConcreteDataType::timestamp_millisecond_datatype(),
|
||||
);
|
||||
check_type(
|
||||
SqlDataType::Varbinary(None),
|
||||
ConcreteDataType::binary_datatype(),
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -428,6 +455,23 @@ mod tests {
|
||||
),
|
||||
"v is {v:?}",
|
||||
);
|
||||
|
||||
let sql_val = SqlValue::HexStringLiteral("48656c6c6f20776f726c6421".to_string());
|
||||
let v = sql_value_to_value("a", &ConcreteDataType::binary_datatype(), &sql_val).unwrap();
|
||||
assert_eq!(Value::Binary(Bytes::from(b"Hello world!".as_slice())), v);
|
||||
|
||||
let sql_val = SqlValue::HexStringLiteral("9AF".to_string());
|
||||
let v = sql_value_to_value("a", &ConcreteDataType::binary_datatype(), &sql_val);
|
||||
assert!(v.is_err());
|
||||
assert!(
|
||||
format!("{v:?}").contains("odd number of digits"),
|
||||
"v is {v:?}"
|
||||
);
|
||||
|
||||
let sql_val = SqlValue::HexStringLiteral("AG".to_string());
|
||||
let v = sql_value_to_value("a", &ConcreteDataType::binary_datatype(), &sql_val);
|
||||
assert!(v.is_err());
|
||||
assert!(format!("{v:?}").contains("invalid character"), "v is {v:?}",);
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
||||
@@ -13,6 +13,7 @@ bytes = "1.1"
|
||||
common-base = { path = "../common/base" }
|
||||
common-error = { path = "../common/error" }
|
||||
common-query = { path = "../common/query" }
|
||||
common-recordbatch = { path = "../common/recordbatch" }
|
||||
common-runtime = { path = "../common/runtime" }
|
||||
common-telemetry = { path = "../common/telemetry" }
|
||||
common-time = { path = "../common/time" }
|
||||
|
||||
@@ -33,25 +33,13 @@ rows | protobuf | arrow |
|
||||
*/
|
||||
|
||||
fn encode_arrow(batch: &WriteBatch, dst: &mut Vec<u8>) {
|
||||
let encoder = codec::WriteBatchArrowEncoder::new();
|
||||
let result = encoder.encode(batch, dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
|
||||
fn encode_protobuf(batch: &WriteBatch, dst: &mut Vec<u8>) {
|
||||
let encoder = codec::WriteBatchProtobufEncoder {};
|
||||
let result = encoder.encode(batch, dst);
|
||||
let encoder = codec::PayloadEncoder::new();
|
||||
let result = encoder.encode(batch.payload(), dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
|
||||
fn decode_arrow(dst: &[u8], mutation_types: &[i32]) {
|
||||
let decoder = codec::WriteBatchArrowDecoder::new(mutation_types.to_vec());
|
||||
let result = decoder.decode(dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
|
||||
fn decode_protobuf(dst: &[u8], mutation_types: &[i32]) {
|
||||
let decoder = codec::WriteBatchProtobufDecoder::new(mutation_types.to_vec());
|
||||
let decoder = codec::PayloadDecoder::new(mutation_types);
|
||||
let result = decoder.decode(dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
@@ -60,32 +48,16 @@ fn bench_wal_decode(c: &mut Criterion) {
|
||||
let (batch_10, types_10) = gen_new_batch_and_types(1);
|
||||
let (batch_100, types_100) = gen_new_batch_and_types(10);
|
||||
let (batch_10000, types_10000) = gen_new_batch_and_types(100);
|
||||
let mut dst_protobuf_10 = vec![];
|
||||
let mut dst_protobuf_100 = vec![];
|
||||
let mut dst_protobuf_10000 = vec![];
|
||||
|
||||
let mut dst_arrow_10 = vec![];
|
||||
let mut dst_arrow_100 = vec![];
|
||||
let mut dst_arrow_10000 = vec![];
|
||||
|
||||
encode_protobuf(&batch_10, &mut dst_protobuf_10);
|
||||
encode_protobuf(&batch_100, &mut dst_protobuf_100);
|
||||
encode_protobuf(&batch_10000, &mut dst_protobuf_10000);
|
||||
|
||||
encode_arrow(&batch_10, &mut dst_arrow_10);
|
||||
encode_arrow(&batch_100, &mut dst_arrow_100);
|
||||
encode_arrow(&batch_10000, &mut dst_arrow_10000);
|
||||
|
||||
let mut group = c.benchmark_group("wal_decode");
|
||||
group.bench_function("protobuf_decode_with_10_num_rows", |b| {
|
||||
b.iter(|| decode_protobuf(&dst_protobuf_10, &types_10))
|
||||
});
|
||||
group.bench_function("protobuf_decode_with_100_num_rows", |b| {
|
||||
b.iter(|| decode_protobuf(&dst_protobuf_100, &types_100))
|
||||
});
|
||||
group.bench_function("protobuf_decode_with_10000_num_rows", |b| {
|
||||
b.iter(|| decode_protobuf(&dst_protobuf_10000, &types_10000))
|
||||
});
|
||||
group.bench_function("arrow_decode_with_10_num_rows", |b| {
|
||||
b.iter(|| decode_arrow(&dst_arrow_10, &types_10))
|
||||
});
|
||||
|
||||
@@ -33,16 +33,9 @@ rows | protobuf | arrow |
|
||||
*/
|
||||
|
||||
fn encode_arrow(batch: &WriteBatch) {
|
||||
let encoder = codec::WriteBatchArrowEncoder::new();
|
||||
let encoder = codec::PayloadEncoder::new();
|
||||
let mut dst = vec![];
|
||||
let result = encoder.encode(batch, &mut dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
|
||||
fn encode_protobuf(batch: &WriteBatch) {
|
||||
let encoder = codec::WriteBatchProtobufEncoder {};
|
||||
let mut dst = vec![];
|
||||
let result = encoder.encode(batch, &mut dst);
|
||||
let result = encoder.encode(batch.payload(), &mut dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
|
||||
@@ -52,15 +45,6 @@ fn bench_wal_encode(c: &mut Criterion) {
|
||||
let (batch_10000, _) = gen_new_batch_and_types(100);
|
||||
|
||||
let mut group = c.benchmark_group("wal_encode");
|
||||
group.bench_function("protobuf_encode_with_10_num_rows", |b| {
|
||||
b.iter(|| encode_protobuf(&batch_10))
|
||||
});
|
||||
group.bench_function("protobuf_encode_with_100_num_rows", |b| {
|
||||
b.iter(|| encode_protobuf(&batch_100))
|
||||
});
|
||||
group.bench_function("protobuf_encode_with_10000_num_rows", |b| {
|
||||
b.iter(|| encode_protobuf(&batch_10000))
|
||||
});
|
||||
group.bench_function("arrow_encode_with_10_num_rows", |b| {
|
||||
b.iter(|| encode_arrow(&batch_10))
|
||||
});
|
||||
|
||||
@@ -33,22 +33,12 @@ rows | protobuf | arrow |
|
||||
*/
|
||||
|
||||
fn codec_arrow(batch: &WriteBatch, mutation_types: &[i32]) {
|
||||
let encoder = codec::WriteBatchArrowEncoder::new();
|
||||
let encoder = codec::PayloadEncoder::new();
|
||||
let mut dst = vec![];
|
||||
let result = encoder.encode(batch, &mut dst);
|
||||
let result = encoder.encode(batch.payload(), &mut dst);
|
||||
assert!(result.is_ok());
|
||||
|
||||
let decoder = codec::WriteBatchArrowDecoder::new(mutation_types.to_vec());
|
||||
let result = decoder.decode(&dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
fn codec_protobuf(batch: &WriteBatch, mutation_types: &[i32]) {
|
||||
let encoder = codec::WriteBatchProtobufEncoder {};
|
||||
let mut dst = vec![];
|
||||
let result = encoder.encode(batch, &mut dst);
|
||||
assert!(result.is_ok());
|
||||
|
||||
let decoder = codec::WriteBatchProtobufDecoder::new(mutation_types.to_vec());
|
||||
let decoder = codec::PayloadDecoder::new(mutation_types);
|
||||
let result = decoder.decode(&dst);
|
||||
assert!(result.is_ok());
|
||||
}
|
||||
@@ -59,15 +49,6 @@ fn bench_wal_encode_decode(c: &mut Criterion) {
|
||||
let (batch_10000, types_10000) = gen_new_batch_and_types(100);
|
||||
|
||||
let mut group = c.benchmark_group("wal_encode_decode");
|
||||
group.bench_function("protobuf_encode_decode_with_10_num_rows", |b| {
|
||||
b.iter(|| codec_protobuf(&batch_10, &types_10))
|
||||
});
|
||||
group.bench_function("protobuf_encode_decode_with_100_num_rows", |b| {
|
||||
b.iter(|| codec_protobuf(&batch_100, &types_100))
|
||||
});
|
||||
group.bench_function("protobuf_encode_decode_with_10000_num_rows", |b| {
|
||||
b.iter(|| codec_protobuf(&batch_10000, &types_10000))
|
||||
});
|
||||
group.bench_function("arrow_encode_decode_with_10_num_rows", |b| {
|
||||
b.iter(|| codec_arrow(&batch_10, &types_10))
|
||||
});
|
||||
|
||||
@@ -14,17 +14,18 @@
|
||||
|
||||
pub mod write_batch_util;
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use datatypes::prelude::ScalarVector;
|
||||
use datatypes::type_id::LogicalTypeId;
|
||||
use datatypes::vectors::{
|
||||
BooleanVector, Float64Vector, StringVector, TimestampMillisecondVector, UInt64Vector,
|
||||
BooleanVector, Float64Vector, StringVector, TimestampMillisecondVector, UInt64Vector, VectorRef,
|
||||
};
|
||||
use rand::Rng;
|
||||
use storage::proto;
|
||||
use storage::write_batch::{PutData, WriteBatch};
|
||||
use store_api::storage::{consts, PutOperation, WriteRequest};
|
||||
use storage::write_batch::WriteBatch;
|
||||
use store_api::storage::{consts, WriteRequest};
|
||||
|
||||
pub fn new_test_batch() -> WriteBatch {
|
||||
write_batch_util::new_write_batch(
|
||||
@@ -69,25 +70,25 @@ pub fn gen_new_batch_and_types(putdate_nums: usize) -> (WriteBatch, Vec<i32>) {
|
||||
rng.fill(&mut boolvs[..]);
|
||||
rng.fill(&mut tsvs[..]);
|
||||
rng.fill(&mut fvs[..]);
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&intvs));
|
||||
let boolv = Arc::new(BooleanVector::from(boolvs.to_vec()));
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_values(tsvs));
|
||||
let fvs = Arc::new(Float64Vector::from_slice(&fvs));
|
||||
let svs = Arc::new(StringVector::from_slice(&svs));
|
||||
let mut put_data = PutData::default();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
put_data.add_version_column(intv).unwrap();
|
||||
put_data.add_value_column("v1", boolv).unwrap();
|
||||
put_data.add_key_column("ts", tsv.clone()).unwrap();
|
||||
put_data.add_key_column("4", fvs.clone()).unwrap();
|
||||
put_data.add_key_column("5", fvs.clone()).unwrap();
|
||||
put_data.add_key_column("6", fvs.clone()).unwrap();
|
||||
put_data.add_key_column("7", fvs.clone()).unwrap();
|
||||
put_data.add_key_column("8", fvs.clone()).unwrap();
|
||||
put_data.add_key_column("9", fvs.clone()).unwrap();
|
||||
put_data.add_key_column("10", svs.clone()).unwrap();
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&intvs)) as VectorRef;
|
||||
let boolv = Arc::new(BooleanVector::from(boolvs.to_vec())) as VectorRef;
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_values(tsvs)) as VectorRef;
|
||||
let fvs = Arc::new(Float64Vector::from_slice(&fvs)) as VectorRef;
|
||||
let svs = Arc::new(StringVector::from_slice(&svs)) as VectorRef;
|
||||
let mut put_data = HashMap::with_capacity(11);
|
||||
put_data.insert("k1".to_string(), intv.clone());
|
||||
put_data.insert(consts::VERSION_COLUMN_NAME.to_string(), intv);
|
||||
put_data.insert("v1".to_string(), boolv);
|
||||
put_data.insert("ts".to_string(), tsv.clone());
|
||||
put_data.insert("4".to_string(), fvs.clone());
|
||||
put_data.insert("5".to_string(), fvs.clone());
|
||||
put_data.insert("6".to_string(), fvs.clone());
|
||||
put_data.insert("7".to_string(), fvs.clone());
|
||||
put_data.insert("8".to_string(), fvs.clone());
|
||||
put_data.insert("9".to_string(), fvs.clone());
|
||||
put_data.insert("10".to_string(), svs.clone());
|
||||
batch.put(put_data).unwrap();
|
||||
}
|
||||
let types = proto::wal::gen_mutation_types(&batch);
|
||||
let types = proto::wal::gen_mutation_types(batch.payload());
|
||||
(batch, types)
|
||||
}
|
||||
|
||||
@@ -14,6 +14,6 @@
|
||||
|
||||
fn main() {
|
||||
tonic_build::configure()
|
||||
.compile(&["proto/wal.proto", "proto/write_batch.proto"], &["."])
|
||||
.compile(&["proto/wal.proto"], &["."])
|
||||
.expect("compile proto");
|
||||
}
|
||||
|
||||
@@ -3,23 +3,12 @@ syntax = "proto3";
|
||||
package greptime.storage.wal.v1;
|
||||
|
||||
message WalHeader {
|
||||
PayloadType payload_type = 1;
|
||||
uint64 last_manifest_version = 2;
|
||||
uint64 last_manifest_version = 1;
|
||||
// Type of each mutation in payload, now only arrow payload uses this field.
|
||||
repeated MutationType mutation_types = 3;
|
||||
}
|
||||
|
||||
enum PayloadType {
|
||||
NONE = 0;
|
||||
WRITE_BATCH_ARROW = 1;
|
||||
WRITE_BATCH_PROTO = 2;
|
||||
}
|
||||
|
||||
message MutationExtra {
|
||||
MutationType mutation_type = 1;
|
||||
repeated MutationType mutation_types = 2;
|
||||
}
|
||||
|
||||
enum MutationType {
|
||||
PUT = 0;
|
||||
DELETE = 1;
|
||||
DELETE = 0;
|
||||
PUT = 1;
|
||||
}
|
||||
|
||||
@@ -1,95 +0,0 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package greptime.storage.write_batch.v1;
|
||||
|
||||
message WriteBatch {
|
||||
Schema schema = 1;
|
||||
repeated Mutation mutations = 2;
|
||||
}
|
||||
|
||||
message Schema {
|
||||
repeated ColumnSchema column_schemas = 1;
|
||||
TimestampIndex timestamp_index = 2;
|
||||
}
|
||||
|
||||
message TimestampIndex {
|
||||
uint64 value = 1;
|
||||
}
|
||||
|
||||
message ColumnSchema {
|
||||
string name = 1;
|
||||
DataType data_type = 2;
|
||||
bool is_nullable = 3;
|
||||
bool is_time_index = 4;
|
||||
}
|
||||
|
||||
message Mutation {
|
||||
oneof mutation {
|
||||
Put put = 1;
|
||||
Delete delete = 2;
|
||||
}
|
||||
}
|
||||
|
||||
message Put {
|
||||
repeated Column columns = 1;
|
||||
}
|
||||
|
||||
message Delete {
|
||||
// TODO(zxy)
|
||||
}
|
||||
|
||||
message Column {
|
||||
Values values = 1;
|
||||
bytes value_null_mask = 2;
|
||||
uint64 num_rows = 3;
|
||||
}
|
||||
|
||||
// TODO(jiachun): Enum might be insufficient to represent some composite data type such as list, struct.
|
||||
// In the future, may be we can refer to https://github.com/apache/arrow/blob/master/format/Schema.fbs#L398
|
||||
enum DataType {
|
||||
NULL = 0;
|
||||
BOOLEAN = 1;
|
||||
INT8 = 2;
|
||||
INT16 = 3;
|
||||
INT32 = 4;
|
||||
INT64 = 5;
|
||||
UINT8 = 6;
|
||||
UINT16 = 7;
|
||||
UINT32 = 8;
|
||||
UINT64 = 9;
|
||||
FLOAT32 = 10;
|
||||
FLOAT64 = 11;
|
||||
STRING = 12;
|
||||
BINARY = 13;
|
||||
DATE = 14;
|
||||
DATETIME = 15;
|
||||
TIMESTAMP_SECOND = 16;
|
||||
TIMESTAMP_MILLISECOND = 17;
|
||||
TIMESTAMP_MICROSECOND = 18;
|
||||
TIMESTAMP_NANOSECOND = 19;
|
||||
}
|
||||
|
||||
message Values {
|
||||
repeated int32 i8_values = 1;
|
||||
repeated int32 i16_values = 2;
|
||||
repeated int32 i32_values = 3;
|
||||
repeated int64 i64_values = 4;
|
||||
|
||||
repeated uint32 u8_values = 5;
|
||||
repeated uint32 u16_values = 6;
|
||||
repeated uint32 u32_values = 7;
|
||||
repeated uint64 u64_values = 8;
|
||||
|
||||
repeated float f32_values = 9;
|
||||
repeated double f64_values = 10;
|
||||
|
||||
repeated bool bool_values = 11;
|
||||
repeated bytes binary_values = 12;
|
||||
repeated string string_values = 13;
|
||||
repeated int32 date_values = 14;
|
||||
repeated int64 datetime_values = 15;
|
||||
repeated int64 ts_second_values = 16;
|
||||
repeated int64 ts_millisecond_values = 17;
|
||||
repeated int64 ts_microsecond_values = 18;
|
||||
repeated int64 ts_nanosecond_values = 19;
|
||||
}
|
||||
@@ -17,7 +17,7 @@ use std::io::Error as IoError;
|
||||
use std::str::Utf8Error;
|
||||
|
||||
use common_error::prelude::*;
|
||||
use datatypes::arrow;
|
||||
use datatypes::arrow::error::ArrowError;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use serde_json::error::Error as JsonError;
|
||||
use store_api::manifest::action::ProtocolVersion;
|
||||
@@ -25,6 +25,7 @@ use store_api::manifest::ManifestVersion;
|
||||
use store_api::storage::{RegionId, SequenceNumber};
|
||||
|
||||
use crate::metadata::Error as MetadataError;
|
||||
use crate::write_batch;
|
||||
|
||||
#[derive(Debug, Snafu)]
|
||||
#[snafu(visibility(pub))]
|
||||
@@ -60,7 +61,7 @@ pub enum Error {
|
||||
#[snafu(display("Failed to create RecordBatch from vectors, source: {}", source))]
|
||||
NewRecordBatch {
|
||||
backtrace: Backtrace,
|
||||
source: arrow::error::ArrowError,
|
||||
source: ArrowError,
|
||||
},
|
||||
|
||||
#[snafu(display("Fail to read object from path: {}, source: {}", path, source))]
|
||||
@@ -145,12 +146,6 @@ pub enum Error {
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Invalid timestamp in write batch, source: {}", source))]
|
||||
InvalidTimestamp {
|
||||
#[snafu(backtrace)]
|
||||
source: crate::write_batch::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Task already cancelled"))]
|
||||
Cancelled { backtrace: Backtrace },
|
||||
|
||||
@@ -210,11 +205,11 @@ pub enum Error {
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to mark WAL as stable, region id: {}, source: {}",
|
||||
"Failed to mark WAL as obsolete, region id: {}, source: {}",
|
||||
region_id,
|
||||
source
|
||||
))]
|
||||
MarkWalStable {
|
||||
MarkWalObsolete {
|
||||
region_id: u64,
|
||||
#[snafu(backtrace)]
|
||||
source: BoxedError,
|
||||
@@ -293,13 +288,14 @@ pub enum Error {
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to add default value for column {}, source: {}",
|
||||
column,
|
||||
"Failed to create default value for column {}, source: {}",
|
||||
name,
|
||||
source
|
||||
))]
|
||||
AddDefault {
|
||||
column: String,
|
||||
source: crate::write_batch::Error,
|
||||
CreateDefault {
|
||||
name: String,
|
||||
#[snafu(backtrace)]
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
@@ -366,6 +362,78 @@ pub enum Error {
|
||||
#[snafu(backtrace)]
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Unknown column {}", name))]
|
||||
UnknownColumn { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Failed to create record batch for write batch, source:{}", source))]
|
||||
CreateRecordBatch {
|
||||
#[snafu(backtrace)]
|
||||
source: common_recordbatch::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Request is too large, max is {}, current is {}",
|
||||
write_batch::MAX_BATCH_SIZE,
|
||||
num_rows
|
||||
))]
|
||||
RequestTooLarge {
|
||||
num_rows: usize,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Type of column {} does not match type in schema, expect {:?}, given {:?}",
|
||||
name,
|
||||
expect,
|
||||
given
|
||||
))]
|
||||
TypeMismatch {
|
||||
name: String,
|
||||
expect: ConcreteDataType,
|
||||
given: ConcreteDataType,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Column {} is not null but input has null", name))]
|
||||
HasNull { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display(
|
||||
"Length of column {} not equals to other columns, expect {}, given {}",
|
||||
name,
|
||||
expect,
|
||||
given
|
||||
))]
|
||||
LenNotEquals {
|
||||
name: String,
|
||||
expect: usize,
|
||||
given: usize,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to decode write batch, corrupted data {}", message))]
|
||||
BatchCorrupted {
|
||||
message: String,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to decode arrow data, source: {}", source))]
|
||||
DecodeArrow {
|
||||
backtrace: Backtrace,
|
||||
source: ArrowError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to encode arrow data, source: {}", source))]
|
||||
EncodeArrow {
|
||||
backtrace: Backtrace,
|
||||
source: ArrowError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to parse schema, source: {}", source))]
|
||||
ParseSchema {
|
||||
backtrace: Backtrace,
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -378,12 +446,16 @@ impl ErrorExt for Error {
|
||||
InvalidScanIndex { .. }
|
||||
| BatchMissingColumn { .. }
|
||||
| BatchMissingTimestamp { .. }
|
||||
| InvalidTimestamp { .. }
|
||||
| InvalidProjection { .. }
|
||||
| BuildBatch { .. }
|
||||
| NotInSchemaToCompat { .. }
|
||||
| WriteToOldVersion { .. }
|
||||
| IllegalTimestampColumnType { .. } => StatusCode::InvalidArguments,
|
||||
| IllegalTimestampColumnType { .. }
|
||||
| CreateRecordBatch { .. }
|
||||
| RequestTooLarge { .. }
|
||||
| TypeMismatch { .. }
|
||||
| HasNull { .. }
|
||||
| LenNotEquals { .. } => StatusCode::InvalidArguments,
|
||||
|
||||
Utf8 { .. }
|
||||
| EncodeJson { .. }
|
||||
@@ -402,7 +474,11 @@ impl ErrorExt for Error {
|
||||
| CompatRead { .. }
|
||||
| CreateDefaultToRead { .. }
|
||||
| NoDefaultToRead { .. }
|
||||
| NewRecordBatch { .. } => StatusCode::Unexpected,
|
||||
| NewRecordBatch { .. }
|
||||
| BatchCorrupted { .. }
|
||||
| DecodeArrow { .. }
|
||||
| EncodeArrow { .. }
|
||||
| ParseSchema { .. } => StatusCode::Unexpected,
|
||||
|
||||
FlushIo { .. }
|
||||
| WriteParquet { .. }
|
||||
@@ -420,13 +496,15 @@ impl ErrorExt for Error {
|
||||
| InvalidRegionState { .. }
|
||||
| ReadWal { .. } => StatusCode::StorageUnavailable,
|
||||
|
||||
UnknownColumn { .. } => StatusCode::TableColumnNotFound,
|
||||
|
||||
InvalidAlterRequest { source, .. }
|
||||
| InvalidRegionDesc { source, .. }
|
||||
| ConvertColumnSchema { source, .. } => source.status_code(),
|
||||
PushBatch { source, .. } => source.status_code(),
|
||||
AddDefault { source, .. } => source.status_code(),
|
||||
CreateDefault { source, .. } => source.status_code(),
|
||||
ConvertChunk { source, .. } => source.status_code(),
|
||||
MarkWalStable { source, .. } => source.status_code(),
|
||||
MarkWalObsolete { source, .. } => source.status_code(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -441,9 +519,7 @@ impl ErrorExt for Error {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
|
||||
use common_error::prelude::StatusCode::*;
|
||||
use datatypes::arrow::error::ArrowError;
|
||||
use snafu::GenerateImplicitData;
|
||||
|
||||
use super::*;
|
||||
|
||||
@@ -245,7 +245,7 @@ impl<S: LogStore> Job for FlushJob<S> {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use log_store::fs::noop::NoopLogStore;
|
||||
use log_store::NoopLogStore;
|
||||
use regex::Regex;
|
||||
|
||||
use super::*;
|
||||
|
||||
@@ -12,14 +12,12 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use datatypes::vectors::VectorRef;
|
||||
use snafu::OptionExt;
|
||||
use store_api::storage::{ColumnDescriptor, OpType, SequenceNumber};
|
||||
use store_api::storage::{OpType, SequenceNumber};
|
||||
|
||||
use super::MemtableRef;
|
||||
use crate::error::{self, Result};
|
||||
use crate::error::Result;
|
||||
use crate::memtable::KeyValues;
|
||||
use crate::write_batch::{Mutation, PutData, WriteBatch};
|
||||
use crate::write_batch::{Mutation, Payload};
|
||||
|
||||
/// Wraps logic of inserting key/values in [WriteBatch] to [Memtable].
|
||||
pub struct Inserter {
|
||||
@@ -37,21 +35,20 @@ impl Inserter {
|
||||
}
|
||||
}
|
||||
|
||||
// TODO(yingwen): Can we take the WriteBatch?
|
||||
/// Insert write batch into memtable.
|
||||
/// Insert write batch payload into memtable.
|
||||
///
|
||||
/// Won't do schema validation if not configured. Caller (mostly the [`RegionWriter`]) should ensure the
|
||||
/// schemas of `memtable` are consistent with `batch`'s.
|
||||
pub fn insert_memtable(&mut self, batch: &WriteBatch, memtable: &MemtableRef) -> Result<()> {
|
||||
if batch.is_empty() {
|
||||
/// schemas of `memtable` are consistent with `payload`'s.
|
||||
pub fn insert_memtable(&mut self, payload: &Payload, memtable: &MemtableRef) -> Result<()> {
|
||||
if payload.is_empty() {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
// This function only makes effect in debug mode.
|
||||
validate_input_and_memtable_schemas(batch, memtable);
|
||||
validate_input_and_memtable_schemas(payload, memtable);
|
||||
|
||||
// Enough to hold all key or value columns.
|
||||
let total_column_num = batch.schema().num_columns();
|
||||
let total_column_num = payload.schema.num_columns();
|
||||
// Reusable KeyValues buffer.
|
||||
let mut kvs = KeyValues {
|
||||
sequence: self.sequence,
|
||||
@@ -61,12 +58,8 @@ impl Inserter {
|
||||
values: Vec::with_capacity(total_column_num),
|
||||
};
|
||||
|
||||
for mutation in batch {
|
||||
match mutation {
|
||||
Mutation::Put(put_data) => {
|
||||
self.write_one_mutation(put_data, memtable, &mut kvs)?;
|
||||
}
|
||||
}
|
||||
for mutation in &payload.mutations {
|
||||
self.write_one_mutation(mutation, memtable, &mut kvs)?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
@@ -74,21 +67,22 @@ impl Inserter {
|
||||
|
||||
fn write_one_mutation(
|
||||
&mut self,
|
||||
put_data: &PutData,
|
||||
mutation: &Mutation,
|
||||
memtable: &MemtableRef,
|
||||
kvs: &mut KeyValues,
|
||||
) -> Result<()> {
|
||||
let schema = memtable.schema();
|
||||
let num_rows = put_data.num_rows();
|
||||
let num_rows = mutation.record_batch.num_rows();
|
||||
|
||||
kvs.reset(OpType::Put, self.index_in_batch);
|
||||
kvs.reset(mutation.op_type, self.index_in_batch);
|
||||
|
||||
for key_col in schema.row_key_columns() {
|
||||
clone_put_data_column_to(put_data, &key_col.desc, &mut kvs.keys)?;
|
||||
for key_idx in schema.row_key_indices() {
|
||||
kvs.keys.push(mutation.record_batch.column(key_idx).clone());
|
||||
}
|
||||
|
||||
for value_col in schema.value_columns() {
|
||||
clone_put_data_column_to(put_data, &value_col.desc, &mut kvs.values)?;
|
||||
for value_idx in schema.value_indices() {
|
||||
kvs.values
|
||||
.push(mutation.record_batch.column(value_idx).clone());
|
||||
}
|
||||
|
||||
memtable.write(kvs)?;
|
||||
@@ -99,30 +93,20 @@ impl Inserter {
|
||||
}
|
||||
}
|
||||
|
||||
fn validate_input_and_memtable_schemas(batch: &WriteBatch, memtable: &MemtableRef) {
|
||||
fn validate_input_and_memtable_schemas(payload: &Payload, memtable: &MemtableRef) {
|
||||
if cfg!(debug_assertions) {
|
||||
let batch_schema = batch.schema();
|
||||
let payload_schema = &payload.schema;
|
||||
let memtable_schema = memtable.schema();
|
||||
let user_schema = memtable_schema.user_schema();
|
||||
debug_assert_eq!(batch_schema.version(), user_schema.version());
|
||||
debug_assert_eq!(payload_schema.version(), user_schema.version());
|
||||
// Only validate column schemas.
|
||||
debug_assert_eq!(batch_schema.column_schemas(), user_schema.column_schemas());
|
||||
debug_assert_eq!(
|
||||
payload_schema.column_schemas(),
|
||||
user_schema.column_schemas()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
fn clone_put_data_column_to(
|
||||
put_data: &PutData,
|
||||
desc: &ColumnDescriptor,
|
||||
target: &mut Vec<VectorRef>,
|
||||
) -> Result<()> {
|
||||
let vector = put_data
|
||||
.column_by_name(&desc.name)
|
||||
.context(error::BatchMissingColumnSnafu { column: &desc.name })?;
|
||||
target.push(vector.clone());
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Holds `start` and `end` indexes to get a slice `[start, end)` from the vector whose
|
||||
/// timestamps belong to same time range at `range_index`.
|
||||
#[derive(Debug, PartialEq)]
|
||||
@@ -135,13 +119,14 @@ struct SliceIndex {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_time::timestamp::Timestamp;
|
||||
use datatypes::type_id::LogicalTypeId;
|
||||
use datatypes::value::Value;
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector};
|
||||
use store_api::storage::{PutOperation, WriteRequest};
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector, VectorRef};
|
||||
use store_api::storage::WriteRequest;
|
||||
|
||||
use super::*;
|
||||
use crate::memtable::{DefaultMemtableBuilder, IterContext, MemtableBuilder};
|
||||
@@ -149,6 +134,7 @@ mod tests {
|
||||
use crate::schema::RegionSchemaRef;
|
||||
use crate::test_util::descriptor_util::RegionDescBuilder;
|
||||
use crate::test_util::write_batch_util;
|
||||
use crate::write_batch::WriteBatch;
|
||||
|
||||
fn new_test_write_batch() -> WriteBatch {
|
||||
write_batch_util::new_write_batch(
|
||||
@@ -172,11 +158,11 @@ mod tests {
|
||||
}
|
||||
|
||||
fn put_batch(batch: &mut WriteBatch, data: &[(i64, Option<i64>)]) {
|
||||
let mut put_data = PutData::with_num_columns(2);
|
||||
let mut put_data = HashMap::with_capacity(2);
|
||||
let ts = TimestampMillisecondVector::from_values(data.iter().map(|v| v.0));
|
||||
put_data.add_key_column("ts", Arc::new(ts)).unwrap();
|
||||
put_data.insert("ts".to_string(), Arc::new(ts) as VectorRef);
|
||||
let value = Int64Vector::from(data.iter().map(|v| v.1).collect::<Vec<_>>());
|
||||
put_data.add_value_column("value", Arc::new(value)).unwrap();
|
||||
put_data.insert("value".to_string(), Arc::new(value) as VectorRef);
|
||||
|
||||
batch.put(put_data).unwrap();
|
||||
}
|
||||
@@ -232,7 +218,9 @@ mod tests {
|
||||
],
|
||||
);
|
||||
|
||||
inserter.insert_memtable(&batch, &mutable_memtable).unwrap();
|
||||
inserter
|
||||
.insert_memtable(batch.payload(), &mutable_memtable)
|
||||
.unwrap();
|
||||
check_memtable_content(
|
||||
&mutable_memtable,
|
||||
sequence,
|
||||
|
||||
@@ -13,4 +13,3 @@
|
||||
// limitations under the License.
|
||||
|
||||
pub mod wal;
|
||||
pub mod write_batch;
|
||||
|
||||
@@ -15,13 +15,16 @@
|
||||
#![allow(clippy::all)]
|
||||
tonic::include_proto!("greptime.storage.wal.v1");
|
||||
|
||||
use crate::write_batch::{Mutation, WriteBatch};
|
||||
use store_api::storage::OpType;
|
||||
|
||||
pub fn gen_mutation_types(write_batch: &WriteBatch) -> Vec<i32> {
|
||||
write_batch
|
||||
use crate::write_batch::Payload;
|
||||
|
||||
pub fn gen_mutation_types(payload: &Payload) -> Vec<i32> {
|
||||
payload
|
||||
.mutations
|
||||
.iter()
|
||||
.map(|m| match m {
|
||||
Mutation::Put(_) => MutationType::Put.into(),
|
||||
.map(|m| match m.op_type {
|
||||
OpType::Put => MutationType::Put.into(),
|
||||
})
|
||||
.collect::<Vec<_>>()
|
||||
}
|
||||
|
||||
@@ -1,391 +0,0 @@
|
||||
// Copyright 2022 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.
|
||||
|
||||
#![allow(clippy::all)]
|
||||
tonic::include_proto!("greptime.storage.write_batch.v1");
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_base::BitVec;
|
||||
use common_error::prelude::*;
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::prelude::{ScalarVector, ScalarVectorBuilder};
|
||||
use datatypes::schema;
|
||||
use datatypes::types::TimestampType;
|
||||
use datatypes::vectors::{
|
||||
BinaryVector, BinaryVectorBuilder, BooleanVector, BooleanVectorBuilder, DateTimeVector,
|
||||
DateTimeVectorBuilder, DateVector, DateVectorBuilder, Float32Vector, Float32VectorBuilder,
|
||||
Float64Vector, Float64VectorBuilder, Int16Vector, Int16VectorBuilder, Int32Vector,
|
||||
Int32VectorBuilder, Int64Vector, Int64VectorBuilder, Int8Vector, Int8VectorBuilder,
|
||||
StringVector, StringVectorBuilder, TimestampMicrosecondVector,
|
||||
TimestampMicrosecondVectorBuilder, TimestampMillisecondVector,
|
||||
TimestampMillisecondVectorBuilder, TimestampNanosecondVector, TimestampNanosecondVectorBuilder,
|
||||
TimestampSecondVector, TimestampSecondVectorBuilder, UInt16Vector, UInt16VectorBuilder,
|
||||
UInt32Vector, UInt32VectorBuilder, UInt64Vector, UInt64VectorBuilder, UInt8Vector,
|
||||
UInt8VectorBuilder, Vector, VectorRef,
|
||||
};
|
||||
use paste::paste;
|
||||
use snafu::OptionExt;
|
||||
|
||||
#[derive(Debug, Snafu)]
|
||||
pub enum Error {
|
||||
#[snafu(display("Failed to convert datafusion type: {}", from))]
|
||||
Conversion { from: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Empty column values read"))]
|
||||
EmptyColumnValues { backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Invalid data type: {}", data_type))]
|
||||
InvalidDataType {
|
||||
data_type: i32,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to convert schema, source: {}", source))]
|
||||
ConvertSchema {
|
||||
#[snafu(backtrace)]
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
|
||||
impl TimestampIndex {
|
||||
pub fn new(value: u64) -> Self {
|
||||
Self { value }
|
||||
}
|
||||
}
|
||||
|
||||
impl From<&schema::SchemaRef> for Schema {
|
||||
fn from(schema: &schema::SchemaRef) -> Self {
|
||||
let column_schemas = schema
|
||||
.column_schemas()
|
||||
.iter()
|
||||
.map(|column_schema| column_schema.into())
|
||||
.collect();
|
||||
|
||||
Schema {
|
||||
column_schemas,
|
||||
timestamp_index: schema
|
||||
.timestamp_index()
|
||||
.map(|index| TimestampIndex::new(index as u64)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<Schema> for schema::SchemaRef {
|
||||
type Error = Error;
|
||||
|
||||
fn try_from(schema: Schema) -> Result<Self> {
|
||||
let column_schemas = schema
|
||||
.column_schemas
|
||||
.iter()
|
||||
.map(schema::ColumnSchema::try_from)
|
||||
.collect::<Result<Vec<_>>>()?;
|
||||
|
||||
let schema = Arc::new(
|
||||
schema::SchemaBuilder::try_from(column_schemas)
|
||||
.context(ConvertSchemaSnafu)?
|
||||
.build()
|
||||
.context(ConvertSchemaSnafu)?,
|
||||
);
|
||||
|
||||
Ok(schema)
|
||||
}
|
||||
}
|
||||
|
||||
impl From<&schema::ColumnSchema> for ColumnSchema {
|
||||
fn from(cs: &schema::ColumnSchema) -> Self {
|
||||
Self {
|
||||
name: cs.name.clone(),
|
||||
data_type: DataType::from(&cs.data_type).into(),
|
||||
is_nullable: cs.is_nullable(),
|
||||
is_time_index: cs.is_time_index(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<&ColumnSchema> for schema::ColumnSchema {
|
||||
type Error = Error;
|
||||
|
||||
fn try_from(column_schema: &ColumnSchema) -> Result<Self> {
|
||||
if let Some(data_type) = DataType::from_i32(column_schema.data_type) {
|
||||
Ok(schema::ColumnSchema::new(
|
||||
column_schema.name.clone(),
|
||||
data_type.into(),
|
||||
column_schema.is_nullable,
|
||||
)
|
||||
.with_time_index(column_schema.is_time_index))
|
||||
} else {
|
||||
InvalidDataTypeSnafu {
|
||||
data_type: column_schema.data_type,
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<&ConcreteDataType> for DataType {
|
||||
fn from(data_type: &ConcreteDataType) -> Self {
|
||||
match data_type {
|
||||
ConcreteDataType::Boolean(_) => DataType::Boolean,
|
||||
ConcreteDataType::Int8(_) => DataType::Int8,
|
||||
ConcreteDataType::Int16(_) => DataType::Int16,
|
||||
ConcreteDataType::Int32(_) => DataType::Int32,
|
||||
ConcreteDataType::Int64(_) => DataType::Int64,
|
||||
ConcreteDataType::UInt8(_) => DataType::Uint8,
|
||||
ConcreteDataType::UInt16(_) => DataType::Uint16,
|
||||
ConcreteDataType::UInt32(_) => DataType::Uint32,
|
||||
ConcreteDataType::UInt64(_) => DataType::Uint64,
|
||||
ConcreteDataType::Float32(_) => DataType::Float64,
|
||||
ConcreteDataType::Float64(_) => DataType::Float64,
|
||||
ConcreteDataType::String(_) => DataType::String,
|
||||
ConcreteDataType::Null(_) => DataType::Null,
|
||||
ConcreteDataType::Binary(_) => DataType::Binary,
|
||||
ConcreteDataType::Timestamp(unit) => match unit {
|
||||
TimestampType::Second(_) => DataType::TimestampSecond,
|
||||
TimestampType::Millisecond(_) => DataType::TimestampMillisecond,
|
||||
TimestampType::Microsecond(_) => DataType::TimestampMicrosecond,
|
||||
TimestampType::Nanosecond(_) => DataType::TimestampNanosecond,
|
||||
},
|
||||
ConcreteDataType::Date(_)
|
||||
| ConcreteDataType::DateTime(_)
|
||||
| ConcreteDataType::List(_) => {
|
||||
// TODO(jiachun): Maybe support some composite types in the future , such as list, struct, etc.
|
||||
unimplemented!("data type {:?} is not supported", data_type)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<DataType> for ConcreteDataType {
|
||||
fn from(data_type: DataType) -> Self {
|
||||
match data_type {
|
||||
DataType::Boolean => ConcreteDataType::boolean_datatype(),
|
||||
DataType::Int8 => ConcreteDataType::int8_datatype(),
|
||||
DataType::Int16 => ConcreteDataType::int16_datatype(),
|
||||
DataType::Int32 => ConcreteDataType::int32_datatype(),
|
||||
DataType::Int64 => ConcreteDataType::int64_datatype(),
|
||||
DataType::Uint8 => ConcreteDataType::uint8_datatype(),
|
||||
DataType::Uint16 => ConcreteDataType::uint16_datatype(),
|
||||
DataType::Uint32 => ConcreteDataType::uint32_datatype(),
|
||||
DataType::Uint64 => ConcreteDataType::uint64_datatype(),
|
||||
DataType::Float32 => ConcreteDataType::float32_datatype(),
|
||||
DataType::Float64 => ConcreteDataType::float64_datatype(),
|
||||
DataType::String => ConcreteDataType::string_datatype(),
|
||||
DataType::Binary => ConcreteDataType::binary_datatype(),
|
||||
DataType::Null => ConcreteDataType::null_datatype(),
|
||||
DataType::Date => ConcreteDataType::date_datatype(),
|
||||
DataType::Datetime => ConcreteDataType::datetime_datatype(),
|
||||
DataType::TimestampSecond => ConcreteDataType::timestamp_second_datatype(),
|
||||
DataType::TimestampMillisecond => ConcreteDataType::timestamp_millisecond_datatype(),
|
||||
DataType::TimestampMicrosecond => ConcreteDataType::timestamp_microsecond_datatype(),
|
||||
DataType::TimestampNanosecond => ConcreteDataType::timestamp_nanosecond_datatype(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[macro_export]
|
||||
macro_rules! gen_columns {
|
||||
($key: tt, $vec_ty: ty, $vari: ident, $cast: expr) => {
|
||||
paste! {
|
||||
pub fn [<gen_columns_ $key>](vector: &VectorRef) -> Result<Column> {
|
||||
let mut column = Column::default();
|
||||
let mut values = Values::default();
|
||||
let vector_ref =
|
||||
vector
|
||||
.as_any()
|
||||
.downcast_ref::<$vec_ty>()
|
||||
.with_context(|| ConversionSnafu {
|
||||
from: std::format!("{:?}", vector.as_ref().data_type()),
|
||||
})?;
|
||||
let mut bits: Option<BitVec> = None;
|
||||
|
||||
vector_ref
|
||||
.iter_data()
|
||||
.enumerate()
|
||||
.for_each(|(i, value)| match value {
|
||||
Some($vari) => values.[<$key _values>].push($cast),
|
||||
None => {
|
||||
if (bits.is_none()) {
|
||||
bits = Some(BitVec::repeat(false, vector_ref.len()));
|
||||
}
|
||||
bits.as_mut().map(|x| x.set(i, true));
|
||||
}
|
||||
});
|
||||
|
||||
let null_mask = if let Some(bits) = bits {
|
||||
bits.into_vec()
|
||||
} else {
|
||||
Default::default()
|
||||
};
|
||||
|
||||
column.values = Some(values);
|
||||
column.value_null_mask = null_mask;
|
||||
column.num_rows = vector_ref.len() as u64;
|
||||
|
||||
Ok(column)
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
gen_columns!(i8, Int8Vector, v, v as i32);
|
||||
gen_columns!(i16, Int16Vector, v, v as i32);
|
||||
gen_columns!(i32, Int32Vector, v, v as i32);
|
||||
gen_columns!(i64, Int64Vector, v, v as i64);
|
||||
gen_columns!(u8, UInt8Vector, v, v as u32);
|
||||
gen_columns!(u16, UInt16Vector, v, v as u32);
|
||||
gen_columns!(u32, UInt32Vector, v, v as u32);
|
||||
gen_columns!(u64, UInt64Vector, v, v as u64);
|
||||
gen_columns!(f32, Float32Vector, v, v);
|
||||
gen_columns!(f64, Float64Vector, v, v);
|
||||
gen_columns!(bool, BooleanVector, v, v);
|
||||
gen_columns!(binary, BinaryVector, v, v.to_vec());
|
||||
gen_columns!(string, StringVector, v, v.to_string());
|
||||
gen_columns!(date, DateVector, v, v.val());
|
||||
gen_columns!(datetime, DateTimeVector, v, v.val());
|
||||
gen_columns!(ts_second, TimestampSecondVector, v, v.into());
|
||||
gen_columns!(ts_millisecond, TimestampMillisecondVector, v, v.into());
|
||||
gen_columns!(ts_microsecond, TimestampMicrosecondVector, v, v.into());
|
||||
gen_columns!(ts_nanosecond, TimestampNanosecondVector, v, v.into());
|
||||
|
||||
#[macro_export]
|
||||
macro_rules! gen_put_data {
|
||||
($key: tt, $builder_type: ty, $vari: ident, $cast: expr) => {
|
||||
paste! {
|
||||
pub fn [<gen_put_data_ $key>](column: Column) -> Result<VectorRef> {
|
||||
let values = column.values.context(EmptyColumnValuesSnafu {})?;
|
||||
let mut vector_iter = values.[<$key _values>].iter();
|
||||
let num_rows = column.num_rows as usize;
|
||||
let mut builder = <$builder_type>::with_capacity(num_rows);
|
||||
|
||||
if column.value_null_mask.is_empty() {
|
||||
(0..num_rows)
|
||||
.for_each(|_| builder.push(vector_iter.next().map(|$vari| $cast)));
|
||||
} else {
|
||||
BitVec::from_vec(column.value_null_mask)
|
||||
.into_iter()
|
||||
.take(num_rows)
|
||||
.for_each(|is_null| {
|
||||
if is_null {
|
||||
builder.push(None);
|
||||
} else {
|
||||
builder.push(vector_iter.next().map(|$vari| $cast));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
Ok(Arc::new(builder.finish()))
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
gen_put_data!(i8, Int8VectorBuilder, v, *v as i8);
|
||||
gen_put_data!(i16, Int16VectorBuilder, v, *v as i16);
|
||||
gen_put_data!(i32, Int32VectorBuilder, v, *v);
|
||||
gen_put_data!(i64, Int64VectorBuilder, v, *v);
|
||||
gen_put_data!(u8, UInt8VectorBuilder, v, *v as u8);
|
||||
gen_put_data!(u16, UInt16VectorBuilder, v, *v as u16);
|
||||
gen_put_data!(u32, UInt32VectorBuilder, v, *v as u32);
|
||||
gen_put_data!(u64, UInt64VectorBuilder, v, *v as u64);
|
||||
gen_put_data!(f32, Float32VectorBuilder, v, *v as f32);
|
||||
gen_put_data!(f64, Float64VectorBuilder, v, *v as f64);
|
||||
gen_put_data!(bool, BooleanVectorBuilder, v, *v);
|
||||
gen_put_data!(binary, BinaryVectorBuilder, v, v.as_slice());
|
||||
gen_put_data!(string, StringVectorBuilder, v, v.as_str());
|
||||
gen_put_data!(date, DateVectorBuilder, v, (*v).into());
|
||||
gen_put_data!(datetime, DateTimeVectorBuilder, v, (*v).into());
|
||||
gen_put_data!(ts_second, TimestampSecondVectorBuilder, v, (*v).into());
|
||||
gen_put_data!(
|
||||
ts_millisecond,
|
||||
TimestampMillisecondVectorBuilder,
|
||||
v,
|
||||
(*v).into()
|
||||
);
|
||||
gen_put_data!(
|
||||
ts_microsecond,
|
||||
TimestampMicrosecondVectorBuilder,
|
||||
v,
|
||||
(*v).into()
|
||||
);
|
||||
gen_put_data!(
|
||||
ts_nanosecond,
|
||||
TimestampNanosecondVectorBuilder,
|
||||
v,
|
||||
(*v).into()
|
||||
);
|
||||
|
||||
pub fn gen_columns(vector: &VectorRef) -> Result<Column> {
|
||||
let data_type = vector.data_type();
|
||||
match data_type {
|
||||
ConcreteDataType::Boolean(_) => gen_columns_bool(vector),
|
||||
ConcreteDataType::Int8(_) => gen_columns_i8(vector),
|
||||
ConcreteDataType::Int16(_) => gen_columns_i16(vector),
|
||||
ConcreteDataType::Int32(_) => gen_columns_i32(vector),
|
||||
ConcreteDataType::Int64(_) => gen_columns_i64(vector),
|
||||
ConcreteDataType::UInt8(_) => gen_columns_u8(vector),
|
||||
ConcreteDataType::UInt16(_) => gen_columns_u16(vector),
|
||||
ConcreteDataType::UInt32(_) => gen_columns_u32(vector),
|
||||
ConcreteDataType::UInt64(_) => gen_columns_u64(vector),
|
||||
ConcreteDataType::Float32(_) => gen_columns_f32(vector),
|
||||
ConcreteDataType::Float64(_) => gen_columns_f64(vector),
|
||||
ConcreteDataType::Binary(_) => gen_columns_binary(vector),
|
||||
ConcreteDataType::String(_) => gen_columns_string(vector),
|
||||
ConcreteDataType::Date(_) => gen_columns_date(vector),
|
||||
ConcreteDataType::DateTime(_) => gen_columns_datetime(vector),
|
||||
ConcreteDataType::Timestamp(t) => match t {
|
||||
TimestampType::Second(_) => gen_columns_ts_second(vector),
|
||||
TimestampType::Millisecond(_) => gen_columns_ts_millisecond(vector),
|
||||
TimestampType::Microsecond(_) => gen_columns_ts_microsecond(vector),
|
||||
TimestampType::Nanosecond(_) => gen_columns_ts_nanosecond(vector),
|
||||
},
|
||||
ConcreteDataType::Null(_) | ConcreteDataType::List(_) => {
|
||||
// TODO(jiachun): Maybe support some composite types in the future, such as list, struct, etc.
|
||||
unimplemented!("data type {:?} is not supported", data_type)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn gen_put_data_vector(data_type: ConcreteDataType, column: Column) -> Result<VectorRef> {
|
||||
match data_type {
|
||||
ConcreteDataType::Boolean(_) => gen_put_data_bool(column),
|
||||
ConcreteDataType::Int8(_) => gen_put_data_i8(column),
|
||||
ConcreteDataType::Int16(_) => gen_put_data_i16(column),
|
||||
ConcreteDataType::Int32(_) => gen_put_data_i32(column),
|
||||
ConcreteDataType::Int64(_) => gen_put_data_i64(column),
|
||||
ConcreteDataType::UInt8(_) => gen_put_data_u8(column),
|
||||
ConcreteDataType::UInt16(_) => gen_put_data_u16(column),
|
||||
ConcreteDataType::UInt32(_) => gen_put_data_u32(column),
|
||||
ConcreteDataType::UInt64(_) => gen_put_data_u64(column),
|
||||
ConcreteDataType::Float32(_) => gen_put_data_f32(column),
|
||||
ConcreteDataType::Float64(_) => gen_put_data_f64(column),
|
||||
ConcreteDataType::Binary(_) => gen_put_data_binary(column),
|
||||
ConcreteDataType::String(_) => gen_put_data_string(column),
|
||||
ConcreteDataType::Date(_) => gen_put_data_date(column),
|
||||
ConcreteDataType::DateTime(_) => gen_put_data_datetime(column),
|
||||
ConcreteDataType::Timestamp(t) => match t {
|
||||
TimestampType::Second(_) => gen_put_data_ts_second(column),
|
||||
TimestampType::Millisecond(_) => gen_put_data_ts_millisecond(column),
|
||||
TimestampType::Microsecond(_) => gen_put_data_ts_microsecond(column),
|
||||
TimestampType::Nanosecond(_) => gen_put_data_ts_nanosecond(column),
|
||||
},
|
||||
ConcreteDataType::Null(_) | ConcreteDataType::List(_) => {
|
||||
// TODO(jiachun): Maybe support some composite types in the future, such as list, struct, etc.
|
||||
unimplemented!("data type {:?} is not supported", data_type)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -19,17 +19,19 @@ mod basic;
|
||||
mod flush;
|
||||
mod projection;
|
||||
|
||||
use std::collections::HashMap;
|
||||
|
||||
use common_telemetry::logging;
|
||||
use datatypes::prelude::{ScalarVector, WrapperType};
|
||||
use datatypes::timestamp::TimestampMillisecond;
|
||||
use datatypes::type_id::LogicalTypeId;
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector};
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
use log_store::fs::noop::NoopLogStore;
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector, VectorRef};
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use log_store::NoopLogStore;
|
||||
use object_store::backend::fs;
|
||||
use object_store::ObjectStore;
|
||||
use store_api::storage::{
|
||||
consts, Chunk, ChunkReader, PutOperation, ScanRequest, SequenceNumber, Snapshot, WriteRequest,
|
||||
consts, Chunk, ChunkReader, ScanRequest, SequenceNumber, Snapshot, WriteRequest,
|
||||
};
|
||||
use tempdir::TempDir;
|
||||
|
||||
@@ -39,7 +41,6 @@ use crate::manifest::test_utils::*;
|
||||
use crate::memtable::DefaultMemtableBuilder;
|
||||
use crate::test_util::descriptor_util::RegionDescBuilder;
|
||||
use crate::test_util::{self, config_util, schema_util, write_batch_util};
|
||||
use crate::write_batch::PutData;
|
||||
|
||||
/// Create metadata of a region with schema: (timestamp, v0).
|
||||
pub fn new_metadata(region_name: &str, enable_version_column: bool) -> RegionMetadata {
|
||||
@@ -66,6 +67,10 @@ impl<S: LogStore> TesterBase<S> {
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn close(&self) {
|
||||
self.region.inner.wal.close().await.unwrap();
|
||||
}
|
||||
|
||||
/// Put without version specified.
|
||||
///
|
||||
/// Format of data: (timestamp, v0), timestamp is key, v0 is value.
|
||||
@@ -125,7 +130,7 @@ impl<S: LogStore> TesterBase<S> {
|
||||
}
|
||||
}
|
||||
|
||||
pub type FileTesterBase = TesterBase<LocalFileLogStore>;
|
||||
pub type FileTesterBase = TesterBase<RaftEngineLogStore>;
|
||||
|
||||
fn new_write_batch_for_test(enable_version_column: bool) -> WriteBatch {
|
||||
if enable_version_column {
|
||||
@@ -156,17 +161,18 @@ fn new_write_batch_for_test(enable_version_column: bool) -> WriteBatch {
|
||||
}
|
||||
}
|
||||
|
||||
fn new_put_data(data: &[(TimestampMillisecond, Option<i64>)]) -> PutData {
|
||||
let mut put_data = PutData::with_num_columns(2);
|
||||
fn new_put_data(data: &[(TimestampMillisecond, Option<i64>)]) -> HashMap<String, VectorRef> {
|
||||
let mut put_data = HashMap::with_capacity(2);
|
||||
|
||||
let timestamps =
|
||||
TimestampMillisecondVector::from_vec(data.iter().map(|v| v.0.into()).collect());
|
||||
let values = Int64Vector::from_owned_iterator(data.iter().map(|kv| kv.1));
|
||||
|
||||
put_data
|
||||
.add_key_column(test_util::TIMESTAMP_NAME, Arc::new(timestamps))
|
||||
.unwrap();
|
||||
put_data.add_value_column("v0", Arc::new(values)).unwrap();
|
||||
put_data.insert(
|
||||
test_util::TIMESTAMP_NAME.to_string(),
|
||||
Arc::new(timestamps) as VectorRef,
|
||||
);
|
||||
put_data.insert("v0".to_string(), Arc::new(values) as VectorRef);
|
||||
|
||||
put_data
|
||||
}
|
||||
|
||||
@@ -12,17 +12,17 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::BTreeMap;
|
||||
use std::collections::{BTreeMap, HashMap};
|
||||
use std::sync::Arc;
|
||||
|
||||
use datatypes::prelude::*;
|
||||
use datatypes::timestamp::TimestampMillisecond;
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector};
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector, VectorRef};
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use store_api::storage::{
|
||||
AddColumn, AlterOperation, AlterRequest, Chunk, ChunkReader, ColumnDescriptor,
|
||||
ColumnDescriptorBuilder, ColumnId, PutOperation, Region, RegionMeta, ScanRequest, SchemaRef,
|
||||
Snapshot, WriteRequest, WriteResponse,
|
||||
ColumnDescriptorBuilder, ColumnId, Region, RegionMeta, ScanRequest, SchemaRef, Snapshot,
|
||||
WriteRequest, WriteResponse,
|
||||
};
|
||||
use tempdir::TempDir;
|
||||
|
||||
@@ -31,11 +31,10 @@ use crate::region::{OpenOptions, RawRegionMetadata, RegionImpl, RegionMetadata};
|
||||
use crate::test_util;
|
||||
use crate::test_util::config_util;
|
||||
use crate::test_util::descriptor_util::RegionDescBuilder;
|
||||
use crate::write_batch::PutData;
|
||||
|
||||
const REGION_NAME: &str = "region-alter-0";
|
||||
|
||||
async fn create_region_for_alter(store_dir: &str) -> RegionImpl<LocalFileLogStore> {
|
||||
async fn create_region_for_alter(store_dir: &str) -> RegionImpl<RaftEngineLogStore> {
|
||||
// Always disable version column in this test.
|
||||
let metadata = tests::new_metadata(REGION_NAME, false);
|
||||
|
||||
@@ -69,8 +68,8 @@ impl DataRow {
|
||||
}
|
||||
}
|
||||
|
||||
fn new_put_data(data: &[DataRow]) -> PutData {
|
||||
let mut put_data = PutData::with_num_columns(4);
|
||||
fn new_put_data(data: &[DataRow]) -> HashMap<String, VectorRef> {
|
||||
let mut put_data = HashMap::with_capacity(4);
|
||||
let keys = Int64Vector::from(data.iter().map(|v| v.key).collect::<Vec<_>>());
|
||||
let timestamps = TimestampMillisecondVector::from(
|
||||
data.iter()
|
||||
@@ -80,13 +79,14 @@ fn new_put_data(data: &[DataRow]) -> PutData {
|
||||
let values1 = Int64Vector::from(data.iter().map(|kv| kv.v0).collect::<Vec<_>>());
|
||||
let values2 = Int64Vector::from(data.iter().map(|kv| kv.v1).collect::<Vec<_>>());
|
||||
|
||||
put_data.add_key_column("k0", Arc::new(keys)).unwrap();
|
||||
put_data
|
||||
.add_key_column(test_util::TIMESTAMP_NAME, Arc::new(timestamps))
|
||||
.unwrap();
|
||||
put_data.insert("k0".to_string(), Arc::new(keys) as VectorRef);
|
||||
put_data.insert(
|
||||
test_util::TIMESTAMP_NAME.to_string(),
|
||||
Arc::new(timestamps) as VectorRef,
|
||||
);
|
||||
|
||||
put_data.add_value_column("v0", Arc::new(values1)).unwrap();
|
||||
put_data.add_value_column("v1", Arc::new(values2)).unwrap();
|
||||
put_data.insert("v0".to_string(), Arc::new(values1) as VectorRef);
|
||||
put_data.insert("v1".to_string(), Arc::new(values2) as VectorRef);
|
||||
|
||||
put_data
|
||||
}
|
||||
@@ -103,6 +103,9 @@ impl AlterTester {
|
||||
|
||||
async fn reopen(&mut self) {
|
||||
// Close the old region.
|
||||
if let Some(base) = self.base.as_ref() {
|
||||
base.close().await;
|
||||
}
|
||||
self.base = None;
|
||||
// Reopen the region.
|
||||
let store_config = config_util::new_store_config(REGION_NAME, &self.store_dir).await;
|
||||
|
||||
@@ -1,10 +1,10 @@
|
||||
// Copyright 2022 Greptime Team
|
||||
// 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
|
||||
// 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,
|
||||
@@ -14,7 +14,8 @@
|
||||
|
||||
//! Region read/write tests.
|
||||
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
use common_telemetry::info;
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use store_api::storage::{OpenOptions, SequenceNumber, WriteResponse};
|
||||
use tempdir::TempDir;
|
||||
|
||||
@@ -30,7 +31,7 @@ async fn create_region_for_basic(
|
||||
region_name: &str,
|
||||
store_dir: &str,
|
||||
enable_version_column: bool,
|
||||
) -> RegionImpl<LocalFileLogStore> {
|
||||
) -> RegionImpl<RaftEngineLogStore> {
|
||||
let metadata = tests::new_metadata(region_name, enable_version_column);
|
||||
|
||||
let store_config = config_util::new_store_config(region_name, store_dir).await;
|
||||
@@ -70,6 +71,11 @@ impl Tester {
|
||||
|
||||
async fn try_reopen(&mut self) -> Result<bool> {
|
||||
// Close the old region.
|
||||
if let Some(base) = self.base.as_ref() {
|
||||
info!("Reopen tester base");
|
||||
base.close().await;
|
||||
}
|
||||
|
||||
self.base = None;
|
||||
// Reopen the region.
|
||||
let store_config = config_util::new_store_config(&self.region_name, &self.store_dir).await;
|
||||
|
||||
@@ -17,7 +17,7 @@
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::sync::Arc;
|
||||
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use store_api::storage::{OpenOptions, WriteResponse};
|
||||
use tempdir::TempDir;
|
||||
|
||||
@@ -34,7 +34,7 @@ async fn create_region_for_flush(
|
||||
store_dir: &str,
|
||||
enable_version_column: bool,
|
||||
flush_strategy: FlushStrategyRef,
|
||||
) -> RegionImpl<LocalFileLogStore> {
|
||||
) -> RegionImpl<RaftEngineLogStore> {
|
||||
let metadata = tests::new_metadata(REGION_NAME, enable_version_column);
|
||||
|
||||
let mut store_config = config_util::new_store_config(REGION_NAME, store_dir).await;
|
||||
@@ -63,6 +63,9 @@ impl FlushTester {
|
||||
|
||||
async fn reopen(&mut self) {
|
||||
// Close the old region.
|
||||
if let Some(base) = self.base.as_ref() {
|
||||
base.close().await;
|
||||
}
|
||||
self.base = None;
|
||||
// Reopen the region.
|
||||
let mut store_config = config_util::new_store_config(REGION_NAME, &self.store_dir).await;
|
||||
|
||||
@@ -12,23 +12,23 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::prelude::ScalarVector;
|
||||
use datatypes::type_id::LogicalTypeId;
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector};
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector, VectorRef};
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::storage::{
|
||||
Chunk, ChunkReader, PutOperation, ReadContext, Region, ScanRequest, Snapshot, WriteContext,
|
||||
WriteRequest,
|
||||
Chunk, ChunkReader, ReadContext, Region, ScanRequest, Snapshot, WriteContext, WriteRequest,
|
||||
};
|
||||
use tempdir::TempDir;
|
||||
|
||||
use crate::region::{RegionImpl, RegionMetadata};
|
||||
use crate::test_util::{self, config_util, descriptor_util, write_batch_util};
|
||||
use crate::write_batch::{PutData, WriteBatch};
|
||||
use crate::write_batch::WriteBatch;
|
||||
|
||||
/// Create metadata with schema (k0, timestamp, v0, v1)
|
||||
fn new_metadata(region_name: &str) -> RegionMetadata {
|
||||
@@ -60,20 +60,31 @@ fn new_write_batch_for_test() -> WriteBatch {
|
||||
/// v0: [initial_value, ...., initial_value]
|
||||
/// v1: [initial_value, ..., initial_value + len - 1]
|
||||
/// ```
|
||||
fn new_put_data(len: usize, key_start: i64, ts_start: i64, initial_value: i64) -> PutData {
|
||||
let mut put_data = PutData::with_num_columns(4);
|
||||
fn new_put_data(
|
||||
len: usize,
|
||||
key_start: i64,
|
||||
ts_start: i64,
|
||||
initial_value: i64,
|
||||
) -> HashMap<String, VectorRef> {
|
||||
let mut put_data = HashMap::with_capacity(4);
|
||||
|
||||
let k0 = Int64Vector::from_values((0..len).map(|v| key_start + v as i64));
|
||||
let ts = TimestampMillisecondVector::from_values((0..len).map(|v| ts_start + v as i64));
|
||||
let v0 = Int64Vector::from_values(std::iter::repeat(initial_value).take(len));
|
||||
let v1 = Int64Vector::from_values((0..len).map(|v| initial_value + v as i64));
|
||||
let k0 = Arc::new(Int64Vector::from_values(
|
||||
(0..len).map(|v| key_start + v as i64),
|
||||
)) as VectorRef;
|
||||
let ts = Arc::new(TimestampMillisecondVector::from_values(
|
||||
(0..len).map(|v| ts_start + v as i64),
|
||||
)) as VectorRef;
|
||||
let v0 = Arc::new(Int64Vector::from_values(
|
||||
std::iter::repeat(initial_value).take(len),
|
||||
)) as VectorRef;
|
||||
let v1 = Arc::new(Int64Vector::from_values(
|
||||
(0..len).map(|v| initial_value + v as i64),
|
||||
)) as VectorRef;
|
||||
|
||||
put_data.add_key_column("k0", Arc::new(k0)).unwrap();
|
||||
put_data
|
||||
.add_key_column(test_util::TIMESTAMP_NAME, Arc::new(ts))
|
||||
.unwrap();
|
||||
put_data.add_value_column("v0", Arc::new(v0)).unwrap();
|
||||
put_data.add_value_column("v1", Arc::new(v1)).unwrap();
|
||||
put_data.insert("k0".to_string(), k0);
|
||||
put_data.insert(test_util::TIMESTAMP_NAME.to_string(), ts);
|
||||
put_data.insert("v0".to_string(), v0);
|
||||
put_data.insert("v1".to_string(), v1);
|
||||
|
||||
put_data
|
||||
}
|
||||
@@ -155,7 +166,7 @@ impl<S: LogStore> ProjectionTester<S> {
|
||||
|
||||
const REGION_NAME: &str = "region-projection-0";
|
||||
|
||||
async fn new_tester(store_dir: &str) -> ProjectionTester<LocalFileLogStore> {
|
||||
async fn new_tester(store_dir: &str) -> ProjectionTester<RaftEngineLogStore> {
|
||||
let metadata = new_metadata(REGION_NAME);
|
||||
|
||||
let store_config = config_util::new_store_config(REGION_NAME, store_dir).await;
|
||||
|
||||
@@ -35,7 +35,7 @@ use crate::region::{RecoverdMetadata, RecoveredMetadataMap, RegionManifest, Shar
|
||||
use crate::schema::compat::CompatWrite;
|
||||
use crate::sst::AccessLayerRef;
|
||||
use crate::version::{VersionControl, VersionControlRef, VersionEdit};
|
||||
use crate::wal::{Payload, Wal};
|
||||
use crate::wal::Wal;
|
||||
use crate::write_batch::WriteBatch;
|
||||
|
||||
pub type RegionWriterRef = Arc<RegionWriter>;
|
||||
@@ -216,8 +216,7 @@ impl RegionWriter {
|
||||
version_control.set_committed_sequence(next_sequence);
|
||||
|
||||
let header = WalHeader::with_last_manifest_version(manifest_version);
|
||||
wal.write_to_wal(next_sequence, header, Payload::None)
|
||||
.await?;
|
||||
wal.write_to_wal(next_sequence, header, None).await?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
@@ -311,16 +310,12 @@ impl WriterInner {
|
||||
let wal_header = WalHeader::with_last_manifest_version(version.manifest_version());
|
||||
writer_ctx
|
||||
.wal
|
||||
.write_to_wal(
|
||||
next_sequence,
|
||||
wal_header,
|
||||
Payload::WriteBatchArrow(&request),
|
||||
)
|
||||
.write_to_wal(next_sequence, wal_header, Some(request.payload()))
|
||||
.await?;
|
||||
|
||||
// Insert batch into memtable.
|
||||
let mut inserter = Inserter::new(next_sequence);
|
||||
inserter.insert_memtable(&request, version.mutable_memtable())?;
|
||||
inserter.insert_memtable(request.payload(), version.mutable_memtable())?;
|
||||
|
||||
// Update committed_sequence to make current batch visible. The `&mut self` of WriterInner
|
||||
// guarantees the writer is exclusive.
|
||||
@@ -350,7 +345,7 @@ impl WriterInner {
|
||||
// Read starts from the first entry after last flushed entry, so the start sequence
|
||||
// should be flushed_sequence + 1.
|
||||
let mut stream = writer_ctx.wal.read_from_wal(flushed_sequence + 1).await?;
|
||||
while let Some((req_sequence, _header, request)) = stream.try_next().await? {
|
||||
while let Some((req_sequence, _header, payload)) = stream.try_next().await? {
|
||||
while let Some((sequence_before_alter, _)) = next_apply_metadata {
|
||||
// There might be multiple metadata changes to be applied, so a loop is necessary.
|
||||
if req_sequence > sequence_before_alter {
|
||||
@@ -370,7 +365,7 @@ impl WriterInner {
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(request) = request {
|
||||
if let Some(payload) = payload {
|
||||
num_requests += 1;
|
||||
// Note that memtables of `Version` may be updated during replay.
|
||||
let version = version_control.current();
|
||||
@@ -398,7 +393,7 @@ impl WriterInner {
|
||||
// TODO(yingwen): Trigger flush if the size of memtables reach the flush threshold to avoid
|
||||
// out of memory during replay, but we need to do it carefully to avoid dead lock.
|
||||
let mut inserter = Inserter::new(last_sequence);
|
||||
inserter.insert_memtable(&request, version.mutable_memtable())?;
|
||||
inserter.insert_memtable(&payload, version.mutable_memtable())?;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -121,6 +121,11 @@ impl RegionSchema {
|
||||
self.store_schema.row_key_indices()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn value_indices(&self) -> impl Iterator<Item = usize> {
|
||||
self.store_schema.value_indices()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn column_metadata(&self, idx: usize) -> &ColumnMetadata {
|
||||
self.columns.column_metadata(idx)
|
||||
|
||||
@@ -151,6 +151,11 @@ impl StoreSchema {
|
||||
0..self.row_key_end
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn value_indices(&self) -> impl Iterator<Item = usize> {
|
||||
self.row_key_end..self.user_column_end
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn column_name(&self, idx: usize) -> &str {
|
||||
&self.schema.column_schemas()[idx].name
|
||||
@@ -288,6 +293,8 @@ mod tests {
|
||||
assert_eq!(4, store_schema.op_type_index());
|
||||
let row_key_indices: Vec<_> = store_schema.row_key_indices().collect();
|
||||
assert_eq!([0, 1], &row_key_indices[..]);
|
||||
let value_indices: Vec<_> = store_schema.value_indices().collect();
|
||||
assert_eq!([2], &value_indices[..]);
|
||||
|
||||
// Test batch and chunk conversion.
|
||||
let batch = tests::new_batch();
|
||||
|
||||
@@ -157,30 +157,34 @@ impl<'a> ParquetReader<'a> {
|
||||
|
||||
let adapter = ReadAdapter::new(store_schema.clone(), self.projected_schema.clone())?;
|
||||
|
||||
let pruned_row_groups = self.predicate.prune_row_groups(
|
||||
store_schema.schema().clone(),
|
||||
builder.metadata().row_groups(),
|
||||
);
|
||||
let pruned_row_groups = self
|
||||
.predicate
|
||||
.prune_row_groups(
|
||||
store_schema.schema().clone(),
|
||||
builder.metadata().row_groups(),
|
||||
)
|
||||
.into_iter()
|
||||
.enumerate()
|
||||
.flat_map(|(idx, valid)| if valid { Some(idx) } else { None })
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let projection = ProjectionMask::roots(
|
||||
builder.metadata().file_metadata().schema_descr(),
|
||||
adapter.fields_to_read(),
|
||||
);
|
||||
|
||||
let mut masked_stream = builder
|
||||
let mut stream = builder
|
||||
.with_projection(projection)
|
||||
.with_row_groups(pruned_row_groups)
|
||||
.build()
|
||||
.context(ReadParquetSnafu {
|
||||
file: self.file_path,
|
||||
})?
|
||||
.zip(futures_util::stream::iter(pruned_row_groups.into_iter()));
|
||||
})?;
|
||||
|
||||
let file_name = self.file_path.to_string();
|
||||
let chunk_stream = try_stream!({
|
||||
while let Some((record_batch, valid)) = masked_stream.next().await {
|
||||
if valid {
|
||||
yield record_batch.context(ReadParquetSnafu { file: &file_name })?
|
||||
}
|
||||
while let Some(res) = stream.next().await {
|
||||
yield res.context(ReadParquetSnafu { file: &file_name })?
|
||||
}
|
||||
});
|
||||
|
||||
@@ -330,6 +334,65 @@ mod tests {
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_parquet_read_large_batch() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let schema = memtable_tests::schema_for_test();
|
||||
let memtable = DefaultMemtableBuilder::default().build(schema.clone());
|
||||
|
||||
let rows_total = 4096 * 4;
|
||||
let mut keys_vec = Vec::with_capacity(rows_total);
|
||||
let mut values_vec = Vec::with_capacity(rows_total);
|
||||
|
||||
for i in 0..rows_total {
|
||||
keys_vec.push((i as i64, i as u64));
|
||||
values_vec.push((Some(i as u64), Some(i as u64)));
|
||||
}
|
||||
|
||||
memtable_tests::write_kvs(
|
||||
&*memtable,
|
||||
10, // sequence
|
||||
OpType::Put,
|
||||
&keys_vec, // keys
|
||||
&values_vec, // values
|
||||
);
|
||||
|
||||
let dir = TempDir::new("write_parquet").unwrap();
|
||||
let path = dir.path().to_str().unwrap();
|
||||
let backend = Builder::default().root(path).build().unwrap();
|
||||
let object_store = ObjectStore::new(backend);
|
||||
let sst_file_name = "test-read-large.parquet";
|
||||
let iter = memtable.iter(&IterContext::default()).unwrap();
|
||||
let writer = ParquetWriter::new(sst_file_name, iter, object_store.clone());
|
||||
|
||||
writer
|
||||
.write_sst(&sst::WriteOptions::default())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let operator = ObjectStore::new(
|
||||
object_store::backend::fs::Builder::default()
|
||||
.root(dir.path().to_str().unwrap())
|
||||
.build()
|
||||
.unwrap(),
|
||||
);
|
||||
|
||||
let projected_schema = Arc::new(ProjectedSchema::new(schema, Some(vec![1])).unwrap());
|
||||
let reader = ParquetReader::new(
|
||||
"test-read-large.parquet",
|
||||
operator,
|
||||
projected_schema,
|
||||
Predicate::empty(),
|
||||
);
|
||||
|
||||
let mut rows_fetched = 0;
|
||||
let mut stream = reader.chunk_stream().await.unwrap();
|
||||
while let Some(res) = stream.next_batch().await.unwrap() {
|
||||
rows_fetched += res.num_rows();
|
||||
}
|
||||
assert_eq!(rows_total, rows_fetched);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_parquet_reader() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
@@ -14,8 +14,8 @@
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use log_store::fs::config::LogConfig;
|
||||
use log_store::fs::log::LocalFileLogStore;
|
||||
use log_store::raft_engine::log_store::RaftEngineLogStore;
|
||||
use log_store::LogConfig;
|
||||
use object_store::backend::fs::Builder;
|
||||
use object_store::ObjectStore;
|
||||
|
||||
@@ -35,7 +35,7 @@ fn log_store_dir(store_dir: &str) -> String {
|
||||
pub async fn new_store_config(
|
||||
region_name: &str,
|
||||
store_dir: &str,
|
||||
) -> StoreConfig<LocalFileLogStore> {
|
||||
) -> StoreConfig<RaftEngineLogStore> {
|
||||
let parent_dir = "";
|
||||
let sst_dir = engine::region_sst_dir(parent_dir, region_name);
|
||||
let manifest_dir = engine::region_manifest_dir(parent_dir, region_name);
|
||||
@@ -50,7 +50,7 @@ pub async fn new_store_config(
|
||||
log_file_dir: log_store_dir(store_dir),
|
||||
..Default::default()
|
||||
};
|
||||
let log_store = Arc::new(LocalFileLogStore::open(&log_config).await.unwrap());
|
||||
let log_store = Arc::new(RaftEngineLogStore::try_new(log_config).await.unwrap());
|
||||
|
||||
StoreConfig {
|
||||
log_store,
|
||||
|
||||
@@ -1,10 +1,10 @@
|
||||
// Copyright 2022 Greptime Team
|
||||
// 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
|
||||
// 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,
|
||||
@@ -19,18 +19,18 @@ use common_error::prelude::BoxedError;
|
||||
use futures::{stream, Stream, TryStreamExt};
|
||||
use prost::Message;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use store_api::logstore::entry::Entry;
|
||||
use store_api::logstore::{AppendResponse, LogStore};
|
||||
use store_api::logstore::entry::{Entry, Id};
|
||||
use store_api::logstore::LogStore;
|
||||
use store_api::storage::{RegionId, SequenceNumber};
|
||||
|
||||
use crate::codec::{Decoder, Encoder};
|
||||
use crate::error::{self, Error, MarkWalStableSnafu, Result};
|
||||
use crate::proto::wal::{self, PayloadType, WalHeader};
|
||||
use crate::write_batch::codec::{
|
||||
WriteBatchArrowDecoder, WriteBatchArrowEncoder, WriteBatchProtobufDecoder,
|
||||
WriteBatchProtobufEncoder,
|
||||
use crate::error::{
|
||||
DecodeWalHeaderSnafu, EncodeWalHeaderSnafu, Error, MarkWalObsoleteSnafu, ReadWalSnafu, Result,
|
||||
WalDataCorruptedSnafu, WriteWalSnafu,
|
||||
};
|
||||
use crate::write_batch::WriteBatch;
|
||||
use crate::proto::wal::{self, WalHeader};
|
||||
use crate::write_batch::codec::{PayloadDecoder, PayloadEncoder};
|
||||
use crate::write_batch::Payload;
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct Wal<S: LogStore> {
|
||||
@@ -39,9 +39,8 @@ pub struct Wal<S: LogStore> {
|
||||
store: Arc<S>,
|
||||
}
|
||||
|
||||
pub type WriteBatchStream<'a> = Pin<
|
||||
Box<dyn Stream<Item = Result<(SequenceNumber, WalHeader, Option<WriteBatch>)>> + Send + 'a>,
|
||||
>;
|
||||
pub type PayloadStream<'a> =
|
||||
Pin<Box<dyn Stream<Item = Result<(SequenceNumber, WalHeader, Option<Payload>)>> + Send + 'a>>;
|
||||
|
||||
// Wal should be cheap to clone, so avoid holding things like String, Vec.
|
||||
impl<S: LogStore> Clone for Wal<S> {
|
||||
@@ -69,7 +68,7 @@ impl<S: LogStore> Wal<S> {
|
||||
.obsolete(self.namespace.clone(), seq)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(MarkWalStableSnafu {
|
||||
.context(MarkWalObsoleteSnafu {
|
||||
region_id: self.region_id,
|
||||
})
|
||||
}
|
||||
@@ -78,6 +77,12 @@ impl<S: LogStore> Wal<S> {
|
||||
pub fn region_id(&self) -> RegionId {
|
||||
self.region_id
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub async fn close(&self) -> Result<()> {
|
||||
let _ = self.store.stop().await;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl<S: LogStore> Wal<S> {
|
||||
@@ -85,12 +90,12 @@ impl<S: LogStore> Wal<S> {
|
||||
///
|
||||
/// ```text
|
||||
/// | |
|
||||
/// |--------------------------> Header Len <-----------------------------| Arrow/Protobuf/... encoded
|
||||
/// |--------------------------> Header Len <-----------------------------| Arrow IPC format
|
||||
/// | |
|
||||
/// v v
|
||||
/// +---------------------+----------------------------------------------------+--------------+-------------+--------------+
|
||||
/// | | Header | | | |
|
||||
/// | Header Len(varint) | (last_manifest_version + mutation_types + ...) | Data Chunk0 | Data Chunk1 | ... |
|
||||
/// | Header Len(varint) | (last_manifest_version + mutation_types + ...) | Payload 0 | Payload 1 | ... |
|
||||
/// | | | | | |
|
||||
/// +---------------------+----------------------------------------------------+--------------+-------------+--------------+
|
||||
/// ```
|
||||
@@ -99,37 +104,26 @@ impl<S: LogStore> Wal<S> {
|
||||
&self,
|
||||
seq: SequenceNumber,
|
||||
mut header: WalHeader,
|
||||
payload: Payload<'_>,
|
||||
) -> Result<(u64, usize)> {
|
||||
header.payload_type = payload.payload_type();
|
||||
if let Payload::WriteBatchArrow(batch) = payload {
|
||||
header.mutation_types = wal::gen_mutation_types(batch);
|
||||
payload: Option<&Payload>,
|
||||
) -> Result<Id> {
|
||||
if let Some(p) = payload {
|
||||
header.mutation_types = wal::gen_mutation_types(p);
|
||||
}
|
||||
|
||||
let mut buf = vec![];
|
||||
|
||||
// header
|
||||
// Encode header
|
||||
let wal_header_encoder = WalHeaderEncoder {};
|
||||
wal_header_encoder.encode(&header, &mut buf)?;
|
||||
|
||||
if let Payload::WriteBatchArrow(batch) = payload {
|
||||
// entry
|
||||
let encoder = WriteBatchArrowEncoder::new();
|
||||
// Encode payload
|
||||
if let Some(p) = payload {
|
||||
let encoder = PayloadEncoder::new();
|
||||
// TODO(jiachun): provide some way to compute data size before encode, so we can preallocate an exactly sized buf.
|
||||
encoder
|
||||
.encode(batch, &mut buf)
|
||||
.encode(p, &mut buf)
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::WriteWalSnafu {
|
||||
region_id: self.region_id(),
|
||||
})?;
|
||||
} else if let Payload::WriteBatchProto(batch) = payload {
|
||||
// entry
|
||||
let encoder = WriteBatchProtobufEncoder {};
|
||||
// TODO(jiachun): provide some way to compute data size before encode, so we can preallocate an exactly sized buf.
|
||||
encoder
|
||||
.encode(batch, &mut buf)
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::WriteWalSnafu {
|
||||
.context(WriteWalSnafu {
|
||||
region_id: self.region_id(),
|
||||
})?;
|
||||
}
|
||||
@@ -138,13 +132,13 @@ impl<S: LogStore> Wal<S> {
|
||||
self.write(seq, &buf).await
|
||||
}
|
||||
|
||||
pub async fn read_from_wal(&self, start_seq: SequenceNumber) -> Result<WriteBatchStream<'_>> {
|
||||
pub async fn read_from_wal(&self, start_seq: SequenceNumber) -> Result<PayloadStream<'_>> {
|
||||
let stream = self
|
||||
.store
|
||||
.read(&self.namespace, start_seq)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::ReadWalSnafu {
|
||||
.context(ReadWalSnafu {
|
||||
region_id: self.region_id(),
|
||||
})?
|
||||
// Handle the error when reading from the stream.
|
||||
@@ -162,34 +156,34 @@ impl<S: LogStore> Wal<S> {
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
|
||||
async fn write(&self, seq: SequenceNumber, bytes: &[u8]) -> Result<(u64, usize)> {
|
||||
async fn write(&self, seq: SequenceNumber, bytes: &[u8]) -> Result<u64> {
|
||||
let e = self.store.entry(bytes, seq, self.namespace.clone());
|
||||
|
||||
let res = self
|
||||
let response = self
|
||||
.store
|
||||
.append(e)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::WriteWalSnafu {
|
||||
.context(WriteWalSnafu {
|
||||
region_id: self.region_id(),
|
||||
})?;
|
||||
|
||||
Ok((res.entry_id(), res.offset()))
|
||||
Ok(response.entry_id)
|
||||
}
|
||||
|
||||
fn decode_entry<E: Entry>(
|
||||
&self,
|
||||
entry: E,
|
||||
) -> Result<(SequenceNumber, WalHeader, Option<WriteBatch>)> {
|
||||
) -> Result<(SequenceNumber, WalHeader, Option<Payload>)> {
|
||||
let seq_num = entry.id();
|
||||
let input = entry.data();
|
||||
|
||||
let wal_header_decoder = WalHeaderDecoder {};
|
||||
let (data_pos, mut header) = wal_header_decoder.decode(input)?;
|
||||
let (data_pos, header) = wal_header_decoder.decode(input)?;
|
||||
|
||||
ensure!(
|
||||
data_pos <= input.len(),
|
||||
error::WalDataCorruptedSnafu {
|
||||
WalDataCorruptedSnafu {
|
||||
region_id: self.region_id(),
|
||||
message: format!(
|
||||
"Not enough input buffer, expected data position={}, actual buffer length={}",
|
||||
@@ -199,55 +193,19 @@ impl<S: LogStore> Wal<S> {
|
||||
}
|
||||
);
|
||||
|
||||
match PayloadType::from_i32(header.payload_type) {
|
||||
Some(PayloadType::None) => Ok((seq_num, header, None)),
|
||||
Some(PayloadType::WriteBatchArrow) => {
|
||||
let mutation_types = std::mem::take(&mut header.mutation_types);
|
||||
let decoder = WriteBatchArrowDecoder::new(mutation_types);
|
||||
let write_batch = decoder
|
||||
.decode(&input[data_pos..])
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::ReadWalSnafu {
|
||||
region_id: self.region_id(),
|
||||
})?;
|
||||
if header.mutation_types.is_empty() {
|
||||
return Ok((seq_num, header, None));
|
||||
}
|
||||
|
||||
Ok((seq_num, header, Some(write_batch)))
|
||||
}
|
||||
Some(PayloadType::WriteBatchProto) => {
|
||||
let mutation_types = std::mem::take(&mut header.mutation_types);
|
||||
let decoder = WriteBatchProtobufDecoder::new(mutation_types);
|
||||
let write_batch = decoder
|
||||
.decode(&input[data_pos..])
|
||||
.map_err(BoxedError::new)
|
||||
.context(error::ReadWalSnafu {
|
||||
region_id: self.region_id(),
|
||||
})?;
|
||||
|
||||
Ok((seq_num, header, Some(write_batch)))
|
||||
}
|
||||
_ => error::WalDataCorruptedSnafu {
|
||||
let decoder = PayloadDecoder::new(&header.mutation_types);
|
||||
let payload = decoder
|
||||
.decode(&input[data_pos..])
|
||||
.map_err(BoxedError::new)
|
||||
.context(ReadWalSnafu {
|
||||
region_id: self.region_id(),
|
||||
message: format!("invalid payload type={}", header.payload_type),
|
||||
}
|
||||
.fail(),
|
||||
}
|
||||
}
|
||||
}
|
||||
})?;
|
||||
|
||||
pub enum Payload<'a> {
|
||||
None, // only header
|
||||
WriteBatchArrow(&'a WriteBatch),
|
||||
#[allow(dead_code)]
|
||||
WriteBatchProto(&'a WriteBatch),
|
||||
}
|
||||
|
||||
impl<'a> Payload<'a> {
|
||||
pub fn payload_type(&self) -> i32 {
|
||||
match self {
|
||||
Payload::None => PayloadType::None.into(),
|
||||
Payload::WriteBatchArrow(_) => PayloadType::WriteBatchArrow.into(),
|
||||
Payload::WriteBatchProto(_) => PayloadType::WriteBatchProto.into(),
|
||||
}
|
||||
Ok((seq_num, header, Some(payload)))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -260,7 +218,7 @@ impl Encoder for WalHeaderEncoder {
|
||||
fn encode(&self, item: &WalHeader, dst: &mut Vec<u8>) -> Result<()> {
|
||||
item.encode_length_delimited(dst)
|
||||
.map_err(|err| err.into())
|
||||
.context(error::EncodeWalHeaderSnafu)
|
||||
.context(EncodeWalHeaderSnafu)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -273,12 +231,12 @@ impl Decoder for WalHeaderDecoder {
|
||||
fn decode(&self, src: &[u8]) -> Result<(usize, WalHeader)> {
|
||||
let mut data_pos = prost::decode_length_delimiter(src)
|
||||
.map_err(|err| err.into())
|
||||
.context(error::DecodeWalHeaderSnafu)?;
|
||||
.context(DecodeWalHeaderSnafu)?;
|
||||
data_pos += prost::length_delimiter_len(data_pos);
|
||||
|
||||
let wal_header = WalHeader::decode_length_delimited(src)
|
||||
.map_err(|err| err.into())
|
||||
.context(error::DecodeWalHeaderSnafu)?;
|
||||
.context(DecodeWalHeaderSnafu)?;
|
||||
|
||||
Ok((data_pos, wal_header))
|
||||
}
|
||||
@@ -298,13 +256,9 @@ mod tests {
|
||||
|
||||
let res = wal.write(0, b"test1").await.unwrap();
|
||||
|
||||
assert_eq!(0, res.0);
|
||||
assert_eq!(0, res.1);
|
||||
|
||||
assert_eq!(0, res);
|
||||
let res = wal.write(1, b"test2").await.unwrap();
|
||||
|
||||
assert_eq!(1, res.0);
|
||||
assert_eq!(5 + 32, res.1);
|
||||
assert_eq!(1, res);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -314,18 +268,15 @@ mod tests {
|
||||
test_util::log_store_util::create_tmp_local_file_log_store("wal_test").await;
|
||||
let wal = Wal::new(0, Arc::new(log_store));
|
||||
let header = WalHeader::with_last_manifest_version(111);
|
||||
let (seq_num, _) = wal.write_to_wal(3, header, Payload::None).await?;
|
||||
|
||||
assert_eq!(3, seq_num);
|
||||
let seq_num = 3;
|
||||
wal.write_to_wal(seq_num, header, None).await?;
|
||||
|
||||
let mut stream = wal.read_from_wal(seq_num).await?;
|
||||
let mut data = vec![];
|
||||
while let Some((seq_num, header, write_batch)) = stream.try_next().await? {
|
||||
data.push((seq_num, header, write_batch));
|
||||
}
|
||||
|
||||
assert_eq!(1, data.len());
|
||||
assert_eq!(seq_num, data[0].0);
|
||||
assert_eq!(111, data[0].1.last_manifest_version);
|
||||
assert!(data[0].2.is_none());
|
||||
|
||||
@@ -335,7 +286,6 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_wal_header_codec() {
|
||||
let wal_header = WalHeader {
|
||||
payload_type: 1,
|
||||
last_manifest_version: 99999999,
|
||||
mutation_types: vec![],
|
||||
};
|
||||
|
||||
@@ -15,420 +15,162 @@
|
||||
pub mod codec;
|
||||
mod compat;
|
||||
|
||||
use std::any::Any;
|
||||
use std::collections::{BTreeSet, HashMap};
|
||||
use std::slice;
|
||||
use std::time::Duration;
|
||||
use std::collections::HashMap;
|
||||
|
||||
use common_error::prelude::*;
|
||||
use common_time::timestamp_millis::BucketAligned;
|
||||
use common_time::RangeMillis;
|
||||
use datatypes::arrow::error::ArrowError;
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::prelude::{ScalarVector, Value};
|
||||
use common_recordbatch::RecordBatch;
|
||||
use datatypes::schema::{ColumnSchema, SchemaRef};
|
||||
use datatypes::vectors::{Int64Vector, TimestampMillisecondVector, VectorRef};
|
||||
use prost::{DecodeError, EncodeError};
|
||||
use datatypes::vectors::VectorRef;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::storage::{consts, PutOperation, WriteRequest};
|
||||
use store_api::storage::{OpType, WriteRequest};
|
||||
|
||||
use crate::proto;
|
||||
|
||||
#[derive(Debug, Snafu)]
|
||||
pub enum Error {
|
||||
#[snafu(display("Duplicate column {} in same request", name))]
|
||||
DuplicateColumn { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Missing column {} in request", name))]
|
||||
MissingColumn { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display(
|
||||
"Type of column {} does not match type in schema, expect {:?}, given {:?}",
|
||||
name,
|
||||
expect,
|
||||
given
|
||||
))]
|
||||
TypeMismatch {
|
||||
name: String,
|
||||
expect: ConcreteDataType,
|
||||
given: ConcreteDataType,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Column {} is not null but input has null", name))]
|
||||
HasNull { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Unknown column {}", name))]
|
||||
UnknownColumn { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display(
|
||||
"Length of column {} not equals to other columns, expect {}, given {}",
|
||||
name,
|
||||
expect,
|
||||
given
|
||||
))]
|
||||
LenNotEquals {
|
||||
name: String,
|
||||
expect: usize,
|
||||
given: usize,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Request is too large, max is {}, current is {}",
|
||||
MAX_BATCH_SIZE,
|
||||
num_rows
|
||||
))]
|
||||
RequestTooLarge {
|
||||
num_rows: usize,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Cannot align timestamp: {}", ts))]
|
||||
TimestampOverflow { ts: i64 },
|
||||
|
||||
#[snafu(display("Failed to encode, source: {}", source))]
|
||||
EncodeArrow {
|
||||
backtrace: Backtrace,
|
||||
source: ArrowError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to decode, source: {}", source))]
|
||||
DecodeArrow {
|
||||
backtrace: Backtrace,
|
||||
source: ArrowError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to encode into protobuf, source: {}", source))]
|
||||
EncodeProtobuf {
|
||||
backtrace: Backtrace,
|
||||
source: EncodeError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to decode from protobuf, source: {}", source))]
|
||||
DecodeProtobuf {
|
||||
backtrace: Backtrace,
|
||||
source: DecodeError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to parse schema, source: {}", source))]
|
||||
ParseSchema {
|
||||
backtrace: Backtrace,
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to decode, corrupted data {}", message))]
|
||||
DataCorrupted {
|
||||
message: String,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to decode vector, source {}", source))]
|
||||
DecodeVector {
|
||||
backtrace: Backtrace,
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to convert into protobuf struct, source {}", source))]
|
||||
ToProtobuf {
|
||||
source: proto::write_batch::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to convert from protobuf struct, source {}", source))]
|
||||
FromProtobuf {
|
||||
source: proto::write_batch::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to create default value for column {}, source: {}",
|
||||
name,
|
||||
source
|
||||
))]
|
||||
CreateDefault {
|
||||
name: String,
|
||||
#[snafu(backtrace)]
|
||||
source: datatypes::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
use crate::error::{
|
||||
BatchMissingColumnSnafu, CreateDefaultSnafu, CreateRecordBatchSnafu, Error, HasNullSnafu,
|
||||
LenNotEqualsSnafu, RequestTooLargeSnafu, Result, TypeMismatchSnafu, UnknownColumnSnafu,
|
||||
};
|
||||
|
||||
/// Max number of updates of a write batch.
|
||||
const MAX_BATCH_SIZE: usize = 1_000_000;
|
||||
pub(crate) const MAX_BATCH_SIZE: usize = 1_000_000;
|
||||
|
||||
impl ErrorExt for Error {
|
||||
fn status_code(&self) -> StatusCode {
|
||||
StatusCode::InvalidArguments
|
||||
/// Data of [WriteBatch].
|
||||
///
|
||||
/// We serialize this struct to the WAL instead of the whole `WriteBatch` to avoid
|
||||
/// storing unnecessary information.
|
||||
#[derive(Debug, PartialEq)]
|
||||
pub struct Payload {
|
||||
/// Schema of the payload.
|
||||
///
|
||||
/// This schema doesn't contain internal columns.
|
||||
pub schema: SchemaRef,
|
||||
pub mutations: Vec<Mutation>,
|
||||
}
|
||||
|
||||
impl Payload {
|
||||
/// Creates a new payload with given `schema`.
|
||||
fn new(schema: SchemaRef) -> Payload {
|
||||
Payload {
|
||||
schema,
|
||||
mutations: Vec::new(),
|
||||
}
|
||||
}
|
||||
|
||||
fn backtrace_opt(&self) -> Option<&Backtrace> {
|
||||
ErrorCompat::backtrace(self)
|
||||
/// Returns true if there is no mutation in the payload.
|
||||
#[inline]
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.mutations.is_empty()
|
||||
}
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
/// A write operation to the region.
|
||||
#[derive(Debug, PartialEq)]
|
||||
pub struct Mutation {
|
||||
/// Type of the mutation.
|
||||
pub op_type: OpType,
|
||||
/// Data of the mutation.
|
||||
pub record_batch: RecordBatch,
|
||||
}
|
||||
|
||||
/// Implementation of [WriteRequest].
|
||||
#[derive(Debug)]
|
||||
pub struct WriteBatch {
|
||||
schema: SchemaRef,
|
||||
mutations: Vec<Mutation>,
|
||||
num_rows: usize,
|
||||
payload: Payload,
|
||||
/// Number of rows this batch need to mutate (put, delete, etc).
|
||||
///
|
||||
/// We use it to check whether this batch is too large.
|
||||
num_rows_to_mutate: usize,
|
||||
}
|
||||
|
||||
impl WriteRequest for WriteBatch {
|
||||
type Error = Error;
|
||||
type PutOp = PutData;
|
||||
|
||||
fn put(&mut self, mut data: PutData) -> Result<()> {
|
||||
fn put(&mut self, data: HashMap<String, VectorRef>) -> Result<()> {
|
||||
let data = NameToVector::new(data)?;
|
||||
if data.is_empty() {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
self.preprocess_put_data(&mut data)?;
|
||||
let record_batch = self.process_put_data(data)?;
|
||||
|
||||
self.add_num_rows(data.num_rows())?;
|
||||
self.mutations.push(Mutation::Put(data));
|
||||
self.add_num_rows_to_mutate(record_batch.num_rows())?;
|
||||
self.payload.mutations.push(Mutation {
|
||||
op_type: OpType::Put,
|
||||
record_batch,
|
||||
});
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Aligns timestamps in write batch specified by schema to durations.
|
||||
///
|
||||
/// A negative timestamp means "before Unix epoch".
|
||||
/// Valid timestamp range is `[i64::MIN + duration, i64::MAX-(i64::MAX%duration))`.
|
||||
fn time_ranges(&self, duration: Duration) -> Result<Vec<RangeMillis>> {
|
||||
let ts_col_name = match self.schema.timestamp_column() {
|
||||
None => {
|
||||
// write batch does not have a timestamp column
|
||||
return Ok(Vec::new());
|
||||
}
|
||||
Some(ts_col) => &ts_col.name,
|
||||
};
|
||||
let durations_millis = duration.as_millis() as i64;
|
||||
let mut aligned_timestamps: BTreeSet<i64> = BTreeSet::new();
|
||||
for m in &self.mutations {
|
||||
match m {
|
||||
Mutation::Put(put_data) => {
|
||||
let column = put_data
|
||||
.column_by_name(ts_col_name)
|
||||
.unwrap_or_else(|| panic!("Cannot find column by name: {ts_col_name}"));
|
||||
if column.is_const() {
|
||||
let ts = match column.get(0) {
|
||||
Value::Timestamp(ts) => ts,
|
||||
_ => unreachable!(),
|
||||
};
|
||||
let aligned = align_timestamp(ts.value(), durations_millis)
|
||||
.context(TimestampOverflowSnafu { ts: ts.value() })?;
|
||||
|
||||
aligned_timestamps.insert(aligned);
|
||||
} else {
|
||||
match column.data_type() {
|
||||
ConcreteDataType::Timestamp(_) => {
|
||||
let ts_vector = column
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampMillisecondVector>()
|
||||
.unwrap();
|
||||
for ts in ts_vector.iter_data().flatten() {
|
||||
let aligned = align_timestamp(ts.into(), durations_millis)
|
||||
.context(TimestampOverflowSnafu { ts: i64::from(ts) })?;
|
||||
aligned_timestamps.insert(aligned);
|
||||
}
|
||||
}
|
||||
ConcreteDataType::Int64(_) => {
|
||||
let ts_vector =
|
||||
column.as_any().downcast_ref::<Int64Vector>().unwrap();
|
||||
for ts in ts_vector.iter_data().flatten() {
|
||||
let aligned = align_timestamp(ts, durations_millis)
|
||||
.context(TimestampOverflowSnafu { ts })?;
|
||||
aligned_timestamps.insert(aligned);
|
||||
}
|
||||
}
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let ranges = aligned_timestamps
|
||||
.iter()
|
||||
.map(|t| RangeMillis::new(*t, *t + durations_millis).unwrap())
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
Ok(ranges)
|
||||
}
|
||||
|
||||
fn put_op(&self) -> Self::PutOp {
|
||||
PutData::new()
|
||||
}
|
||||
|
||||
fn put_op_with_columns(num_columns: usize) -> Self::PutOp {
|
||||
PutData::with_num_columns(num_columns)
|
||||
}
|
||||
}
|
||||
|
||||
/// Aligns timestamp to nearest time interval.
|
||||
/// Negative ts means a timestamp before Unix epoch.
|
||||
/// If arithmetic overflows, this function returns None.
|
||||
/// So timestamp within `[i64::MIN, i64::MIN + duration)` or
|
||||
/// `[i64::MAX-(i64::MAX%duration), i64::MAX]` is not a valid input.
|
||||
fn align_timestamp(ts: i64, duration: i64) -> Option<i64> {
|
||||
let aligned = ts.align_by_bucket(duration)?.as_i64();
|
||||
// Also ensure end timestamp won't overflow.
|
||||
aligned.checked_add(duration)?;
|
||||
Some(aligned)
|
||||
}
|
||||
|
||||
// WriteBatch pub methods.
|
||||
impl WriteBatch {
|
||||
pub fn new(schema: SchemaRef) -> Self {
|
||||
Self {
|
||||
schema,
|
||||
mutations: Vec::new(),
|
||||
num_rows: 0,
|
||||
payload: Payload::new(schema),
|
||||
num_rows_to_mutate: 0,
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn schema(&self) -> &SchemaRef {
|
||||
&self.schema
|
||||
&self.payload.schema
|
||||
}
|
||||
|
||||
pub fn iter(&self) -> slice::Iter<'_, Mutation> {
|
||||
self.mutations.iter()
|
||||
}
|
||||
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.mutations.is_empty()
|
||||
#[inline]
|
||||
pub fn payload(&self) -> &Payload {
|
||||
&self.payload
|
||||
}
|
||||
}
|
||||
|
||||
/// Enum to wrap different operations.
|
||||
pub enum Mutation {
|
||||
Put(PutData),
|
||||
}
|
||||
impl WriteBatch {
|
||||
/// Validates `data` and converts it into a [RecordBatch].
|
||||
///
|
||||
/// It fills missing columns by schema's default values.
|
||||
fn process_put_data(&self, data: NameToVector) -> Result<RecordBatch> {
|
||||
let num_rows = data.num_rows();
|
||||
let mut columns = Vec::with_capacity(self.schema().num_columns());
|
||||
|
||||
#[derive(Default, Debug)]
|
||||
pub struct PutData {
|
||||
columns: HashMap<String, VectorRef>,
|
||||
}
|
||||
|
||||
impl PutData {
|
||||
pub(crate) fn new() -> PutData {
|
||||
PutData::default()
|
||||
}
|
||||
|
||||
pub(crate) fn with_num_columns(num_columns: usize) -> PutData {
|
||||
PutData {
|
||||
columns: HashMap::with_capacity(num_columns),
|
||||
}
|
||||
}
|
||||
|
||||
fn add_column_by_name(&mut self, name: &str, vector: VectorRef) -> Result<()> {
|
||||
ensure!(
|
||||
!self.columns.contains_key(name),
|
||||
DuplicateColumnSnafu { name }
|
||||
);
|
||||
|
||||
if let Some(col) = self.columns.values().next() {
|
||||
ensure!(
|
||||
col.len() == vector.len(),
|
||||
LenNotEqualsSnafu {
|
||||
name,
|
||||
expect: col.len(),
|
||||
given: vector.len(),
|
||||
for column_schema in self.schema().column_schemas() {
|
||||
match data.0.get(&column_schema.name) {
|
||||
Some(col) => {
|
||||
validate_column(column_schema, col)?;
|
||||
columns.push(col.clone());
|
||||
}
|
||||
None => {
|
||||
// If column is not provided, fills it by default value.
|
||||
let col = new_column_with_default_value(column_schema, num_rows)?;
|
||||
columns.push(col);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Check all columns in data also exists in schema, which means we
|
||||
// are not inserting unknown columns.
|
||||
for name in data.0.keys() {
|
||||
ensure!(
|
||||
self.schema().contains_column(name),
|
||||
UnknownColumnSnafu { name }
|
||||
);
|
||||
}
|
||||
|
||||
self.columns.insert(name.to_string(), vector);
|
||||
RecordBatch::new(self.schema().clone(), columns).context(CreateRecordBatchSnafu)
|
||||
}
|
||||
|
||||
fn add_num_rows_to_mutate(&mut self, len: usize) -> Result<()> {
|
||||
let num_rows = self.num_rows_to_mutate + len;
|
||||
ensure!(
|
||||
num_rows <= MAX_BATCH_SIZE,
|
||||
RequestTooLargeSnafu { num_rows }
|
||||
);
|
||||
self.num_rows_to_mutate = num_rows;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Add columns by its default value.
|
||||
fn add_default_by_name(&mut self, column_schema: &ColumnSchema) -> Result<()> {
|
||||
let num_rows = self.num_rows();
|
||||
|
||||
// If column is not provided, fills it by default value.
|
||||
let vector = column_schema
|
||||
.create_default_vector(num_rows)
|
||||
.context(CreateDefaultSnafu {
|
||||
name: &column_schema.name,
|
||||
})?
|
||||
.context(MissingColumnSnafu {
|
||||
name: &column_schema.name,
|
||||
})?;
|
||||
|
||||
validate_column(column_schema, &vector)?;
|
||||
|
||||
self.add_column_by_name(&column_schema.name, vector)
|
||||
}
|
||||
}
|
||||
|
||||
impl PutOperation for PutData {
|
||||
type Error = Error;
|
||||
|
||||
fn add_key_column(&mut self, name: &str, vector: VectorRef) -> Result<()> {
|
||||
self.add_column_by_name(name, vector)
|
||||
}
|
||||
|
||||
fn add_version_column(&mut self, vector: VectorRef) -> Result<()> {
|
||||
self.add_column_by_name(consts::VERSION_COLUMN_NAME, vector)
|
||||
}
|
||||
|
||||
fn add_value_column(&mut self, name: &str, vector: VectorRef) -> Result<()> {
|
||||
self.add_column_by_name(name, vector)
|
||||
}
|
||||
}
|
||||
|
||||
// PutData pub methods.
|
||||
impl PutData {
|
||||
pub fn column_by_name(&self, name: &str) -> Option<&VectorRef> {
|
||||
self.columns.get(name)
|
||||
}
|
||||
|
||||
/// Returns number of columns in data.
|
||||
pub fn num_columns(&self) -> usize {
|
||||
self.columns.len()
|
||||
}
|
||||
|
||||
/// Returns number of rows in data.
|
||||
pub fn num_rows(&self) -> usize {
|
||||
self.columns
|
||||
.values()
|
||||
.next()
|
||||
.map(|col| col.len())
|
||||
.unwrap_or(0)
|
||||
}
|
||||
|
||||
/// Returns true if no rows in data.
|
||||
///
|
||||
/// `PutData` with empty column will also be considered as empty.
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.num_rows() == 0
|
||||
}
|
||||
|
||||
/// Returns slice of [PutData] in range `[start, end)`.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if `start > end`.
|
||||
pub fn slice(&self, start: usize, end: usize) -> PutData {
|
||||
assert!(start <= end);
|
||||
|
||||
let columns = self
|
||||
.columns
|
||||
.iter()
|
||||
.map(|(k, v)| (k.clone(), v.slice(start, end - start)))
|
||||
.collect();
|
||||
|
||||
PutData { columns }
|
||||
}
|
||||
/// Returns the length of the first vector in `data`.
|
||||
fn first_vector_len(data: &HashMap<String, VectorRef>) -> usize {
|
||||
data.values().next().map(|col| col.len()).unwrap_or(0)
|
||||
}
|
||||
|
||||
/// Checks whether `col` matches given `column_schema`.
|
||||
fn validate_column(column_schema: &ColumnSchema, col: &VectorRef) -> Result<()> {
|
||||
if !col.data_type().is_null() {
|
||||
// This allow us to use NullVector for columns that only have null value.
|
||||
@@ -454,55 +196,63 @@ fn validate_column(column_schema: &ColumnSchema, col: &VectorRef) -> Result<()>
|
||||
Ok(())
|
||||
}
|
||||
|
||||
impl WriteBatch {
|
||||
/// Validate [PutData] and fill missing columns by default value.
|
||||
fn preprocess_put_data(&self, data: &mut PutData) -> Result<()> {
|
||||
for column_schema in self.schema.column_schemas() {
|
||||
match data.column_by_name(&column_schema.name) {
|
||||
Some(col) => {
|
||||
validate_column(column_schema, col)?;
|
||||
}
|
||||
None => {
|
||||
// If column is not provided, fills it by default value.
|
||||
data.add_default_by_name(column_schema)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
/// Creates a new column and fills it by default value.
|
||||
///
|
||||
/// `num_rows` MUST be greater than 0. This function will also validate the schema.
|
||||
pub(crate) fn new_column_with_default_value(
|
||||
column_schema: &ColumnSchema,
|
||||
num_rows: usize,
|
||||
) -> Result<VectorRef> {
|
||||
// If column is not provided, fills it by default value.
|
||||
let vector = column_schema
|
||||
.create_default_vector(num_rows)
|
||||
.context(CreateDefaultSnafu {
|
||||
name: &column_schema.name,
|
||||
})?
|
||||
.context(BatchMissingColumnSnafu {
|
||||
column: &column_schema.name,
|
||||
})?;
|
||||
|
||||
// Check all columns in data also exists in schema.
|
||||
for name in data.columns.keys() {
|
||||
validate_column(column_schema, &vector)?;
|
||||
|
||||
Ok(vector)
|
||||
}
|
||||
|
||||
/// Vectors in [NameToVector] have same length.
|
||||
///
|
||||
/// MUST construct it via [`NameToVector::new()`] to ensure the vector lengths are validated.
|
||||
struct NameToVector(HashMap<String, VectorRef>);
|
||||
|
||||
impl NameToVector {
|
||||
fn new(data: HashMap<String, VectorRef>) -> Result<NameToVector> {
|
||||
let num_rows = first_vector_len(&data);
|
||||
for (name, vector) in &data {
|
||||
ensure!(
|
||||
self.schema.column_schema_by_name(name).is_some(),
|
||||
UnknownColumnSnafu { name }
|
||||
num_rows == vector.len(),
|
||||
LenNotEqualsSnafu {
|
||||
name,
|
||||
expect: num_rows,
|
||||
given: vector.len(),
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
Ok(NameToVector(data))
|
||||
}
|
||||
|
||||
fn add_num_rows(&mut self, len: usize) -> Result<()> {
|
||||
let num_rows = self.num_rows + len;
|
||||
ensure!(
|
||||
num_rows <= MAX_BATCH_SIZE,
|
||||
RequestTooLargeSnafu { num_rows }
|
||||
);
|
||||
self.num_rows = num_rows;
|
||||
Ok(())
|
||||
fn num_rows(&self) -> usize {
|
||||
first_vector_len(&self.0)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> IntoIterator for &'a WriteBatch {
|
||||
type Item = &'a Mutation;
|
||||
type IntoIter = slice::Iter<'a, Mutation>;
|
||||
|
||||
fn into_iter(self) -> slice::Iter<'a, Mutation> {
|
||||
self.iter()
|
||||
fn is_empty(&self) -> bool {
|
||||
self.num_rows() == 0
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub(crate) fn new_test_batch() -> WriteBatch {
|
||||
use datatypes::type_id::LogicalTypeId;
|
||||
use store_api::storage::consts;
|
||||
|
||||
use crate::test_util::write_batch_util;
|
||||
|
||||
@@ -522,251 +272,165 @@ mod tests {
|
||||
use std::iter;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_error::prelude::*;
|
||||
use datatypes::prelude::ScalarVector;
|
||||
use datatypes::type_id::LogicalTypeId;
|
||||
use datatypes::vectors::{
|
||||
BooleanVector, ConstantVector, Int32Vector, Int64Vector, TimestampMillisecondVector,
|
||||
UInt64Vector,
|
||||
BooleanVector, Int32Vector, Int64Vector, TimestampMillisecondVector, UInt64Vector,
|
||||
};
|
||||
use store_api::storage::consts;
|
||||
|
||||
use super::*;
|
||||
use crate::test_util::write_batch_util;
|
||||
|
||||
#[test]
|
||||
fn test_put_data_basic() {
|
||||
let mut put_data = PutData::new();
|
||||
assert!(put_data.is_empty());
|
||||
fn test_name_to_vector_basic() {
|
||||
let columns = NameToVector::new(HashMap::new()).unwrap();
|
||||
assert!(columns.is_empty());
|
||||
|
||||
let vector1 = Arc::new(Int32Vector::from_slice(&[1, 2, 3, 4, 5]));
|
||||
let vector2 = Arc::new(UInt64Vector::from_slice(&[0, 2, 4, 6, 8]));
|
||||
let vector1 = Arc::new(Int32Vector::from_slice(&[1, 2, 3, 4, 5])) as VectorRef;
|
||||
let vector2 = Arc::new(UInt64Vector::from_slice(&[0, 2, 4, 6, 8])) as VectorRef;
|
||||
|
||||
put_data.add_key_column("k1", vector1.clone()).unwrap();
|
||||
put_data.add_version_column(vector2).unwrap();
|
||||
put_data.add_value_column("v1", vector1).unwrap();
|
||||
let mut put_data = HashMap::with_capacity(3);
|
||||
put_data.insert("k1".to_string(), vector1.clone());
|
||||
put_data.insert(consts::VERSION_COLUMN_NAME.to_string(), vector2);
|
||||
put_data.insert("v1".to_string(), vector1);
|
||||
|
||||
assert_eq!(5, put_data.num_rows());
|
||||
assert!(!put_data.is_empty());
|
||||
|
||||
assert!(put_data.column_by_name("no such column").is_none());
|
||||
assert!(put_data.column_by_name("k1").is_some());
|
||||
assert!(put_data.column_by_name("v1").is_some());
|
||||
assert!(put_data
|
||||
.column_by_name(consts::VERSION_COLUMN_NAME)
|
||||
.is_some());
|
||||
let columns = NameToVector::new(put_data).unwrap();
|
||||
assert_eq!(5, columns.num_rows());
|
||||
assert!(!columns.is_empty());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_data_empty_vector() {
|
||||
let mut put_data = PutData::with_num_columns(1);
|
||||
assert!(put_data.is_empty());
|
||||
fn test_name_to_vector_empty_vector() {
|
||||
let vector1 = Arc::new(Int32Vector::from_slice(&[])) as VectorRef;
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("k1".to_string(), vector1);
|
||||
|
||||
let vector1 = Arc::new(Int32Vector::from_slice(&[]));
|
||||
put_data.add_key_column("k1", vector1).unwrap();
|
||||
|
||||
assert_eq!(0, put_data.num_rows());
|
||||
assert!(put_data.is_empty());
|
||||
let columns = NameToVector::new(put_data).unwrap();
|
||||
assert_eq!(0, columns.num_rows());
|
||||
assert!(columns.is_empty());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_batch_put() {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3]));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true]));
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_vec(vec![0, 0, 0]));
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3])) as VectorRef;
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true])) as VectorRef;
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_slice(&[0, 0, 0])) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
put_data.add_version_column(intv).unwrap();
|
||||
put_data.add_value_column("v1", boolv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
let mut put_data = HashMap::with_capacity(4);
|
||||
put_data.insert("k1".to_string(), intv.clone());
|
||||
put_data.insert(consts::VERSION_COLUMN_NAME.to_string(), intv);
|
||||
put_data.insert("v1".to_string(), boolv);
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
|
||||
let mut batch = new_test_batch();
|
||||
assert!(batch.is_empty());
|
||||
assert!(batch.payload().is_empty());
|
||||
batch.put(put_data).unwrap();
|
||||
assert!(!batch.is_empty());
|
||||
assert!(!batch.payload().is_empty());
|
||||
|
||||
let mut iter = batch.iter();
|
||||
let Mutation::Put(put_data) = iter.next().unwrap();
|
||||
assert_eq!(3, put_data.num_rows());
|
||||
let mutation = &batch.payload().mutations[0];
|
||||
assert_eq!(3, mutation.record_batch.num_rows());
|
||||
}
|
||||
|
||||
fn check_err(err: Error, msg: &str) {
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
assert!(err.backtrace_opt().is_some());
|
||||
assert!(err.to_string().contains(msg));
|
||||
assert!(
|
||||
err.to_string().contains(msg),
|
||||
"<{err}> does not contain {msg}",
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_batch_too_large() {
|
||||
let boolv = Arc::new(BooleanVector::from_iterator(
|
||||
iter::repeat(true).take(MAX_BATCH_SIZE + 1),
|
||||
));
|
||||
)) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", boolv).unwrap();
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("k1".to_string(), boolv);
|
||||
|
||||
let mut batch =
|
||||
write_batch_util::new_write_batch(&[("k1", LogicalTypeId::Boolean, false)], None);
|
||||
let err = batch.put(put_data).err().unwrap();
|
||||
let err = batch.put(put_data).unwrap_err();
|
||||
check_err(err, "Request is too large");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_data_duplicate() {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3]));
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
let err = put_data.add_key_column("k1", intv).err().unwrap();
|
||||
check_err(err, "Duplicate column k1");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_data_different_len() {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3]));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false]));
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3])) as VectorRef;
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_slice(&[0, 0])) as VectorRef;
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true])) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv).unwrap();
|
||||
let err = put_data.add_value_column("v1", boolv).err().unwrap();
|
||||
check_err(err, "Length of column v1 not equals");
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("k1".to_string(), intv.clone());
|
||||
put_data.insert(consts::VERSION_COLUMN_NAME.to_string(), intv);
|
||||
put_data.insert("v1".to_string(), boolv.clone());
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
|
||||
let mut batch = new_test_batch();
|
||||
let err = batch.put(put_data).unwrap_err();
|
||||
check_err(err, "not equals to other columns");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_type_mismatch() {
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true]));
|
||||
let tsv = Arc::new(Int64Vector::from_vec(vec![0, 0, 0]));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true])) as VectorRef;
|
||||
let tsv = Arc::new(Int64Vector::from_slice(&[0, 0, 0])) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", boolv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("k1".to_string(), boolv);
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
|
||||
let mut batch = new_test_batch();
|
||||
let err = batch.put(put_data).err().unwrap();
|
||||
let err = batch.put(put_data).unwrap_err();
|
||||
check_err(err, "Type of column k1 does not match");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_type_has_null() {
|
||||
let intv = Arc::new(UInt64Vector::from(vec![Some(1), None, Some(3)]));
|
||||
let tsv = Arc::new(Int64Vector::from_vec(vec![0, 0, 0]));
|
||||
let intv = Arc::new(UInt64Vector::from(vec![Some(1), None, Some(3)])) as VectorRef;
|
||||
let tsv = Arc::new(Int64Vector::from_slice(&[0, 0, 0])) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("k1".to_string(), intv);
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
|
||||
let mut batch = new_test_batch();
|
||||
let err = batch.put(put_data).err().unwrap();
|
||||
let err = batch.put(put_data).unwrap_err();
|
||||
check_err(err, "Column k1 is not null");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_missing_column() {
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true]));
|
||||
let tsv = Arc::new(Int64Vector::from_vec(vec![0, 0, 0]));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true])) as VectorRef;
|
||||
let tsv = Arc::new(Int64Vector::from_slice(&[0, 0, 0])) as VectorRef;
|
||||
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("v1".to_string(), boolv);
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("v1", boolv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
let mut batch = new_test_batch();
|
||||
let err = batch.put(put_data).err().unwrap();
|
||||
let err = batch.put(put_data).unwrap_err();
|
||||
check_err(err, "Missing column k1");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_unknown_column() {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3]));
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_vec(vec![0, 0, 0]));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true]));
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3])) as VectorRef;
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_slice(&[0, 0, 0])) as VectorRef;
|
||||
let boolv = Arc::new(BooleanVector::from(vec![true, false, true])) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
put_data.add_version_column(intv).unwrap();
|
||||
put_data.add_value_column("v1", boolv.clone()).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
put_data.add_value_column("v2", boolv).unwrap();
|
||||
let mut batch = new_test_batch();
|
||||
let err = batch.put(put_data).err().unwrap();
|
||||
check_err(err, "Unknown column v2");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_align_timestamp() {
|
||||
let duration_millis = 20;
|
||||
let ts = [-21, -20, -19, -1, 0, 5, 15, 19, 20, 21];
|
||||
let res = ts.map(|t| align_timestamp(t, duration_millis));
|
||||
assert_eq!(res, [-40, -20, -20, -20, 0, 0, 0, 0, 20, 20].map(Some));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_align_timestamp_overflow() {
|
||||
assert_eq!(Some(i64::MIN), align_timestamp(i64::MIN, 1));
|
||||
assert_eq!(Some(-9223372036854775808), align_timestamp(i64::MIN, 2));
|
||||
assert_eq!(
|
||||
Some(((i64::MIN + 20) / 20 - 1) * 20),
|
||||
align_timestamp(i64::MIN + 20, 20)
|
||||
);
|
||||
assert_eq!(None, align_timestamp(i64::MAX - (i64::MAX % 23), 23));
|
||||
assert_eq!(
|
||||
Some(9223372036854775780),
|
||||
align_timestamp(i64::MAX / 20 * 20 - 1, 20)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_batch_time_range() {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3, 4, 5, 6]));
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_vec(vec![
|
||||
-21, -20, -1, 0, 1, 20,
|
||||
]));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![
|
||||
true, false, true, false, false, false,
|
||||
]));
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
put_data.add_version_column(intv).unwrap();
|
||||
put_data.add_value_column("v1", boolv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("k1".to_string(), intv.clone());
|
||||
put_data.insert(consts::VERSION_COLUMN_NAME.to_string(), intv);
|
||||
put_data.insert("v1".to_string(), boolv.clone());
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
put_data.insert("v2".to_string(), boolv);
|
||||
|
||||
let mut batch = new_test_batch();
|
||||
batch.put(put_data).unwrap();
|
||||
|
||||
let duration_millis = 20i64;
|
||||
let ranges = batch
|
||||
.time_ranges(Duration::from_millis(duration_millis as u64))
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
[-40, -20, 0, 20].map(|v| RangeMillis::new(v, v + duration_millis).unwrap()),
|
||||
ranges.as_slice()
|
||||
)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_batch_time_range_const_vector() {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3, 4, 5, 6]));
|
||||
let tsv = Arc::new(ConstantVector::new(
|
||||
Arc::new(TimestampMillisecondVector::from_vec(vec![20])),
|
||||
6,
|
||||
));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![
|
||||
true, false, true, false, false, false,
|
||||
]));
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
put_data.add_version_column(intv).unwrap();
|
||||
put_data.add_value_column("v1", boolv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
|
||||
let mut batch = new_test_batch();
|
||||
batch.put(put_data).unwrap();
|
||||
|
||||
let duration_millis = 20i64;
|
||||
let ranges = batch
|
||||
.time_ranges(Duration::from_millis(duration_millis as u64))
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
[20].map(|v| RangeMillis::new(v, v + duration_millis).unwrap()),
|
||||
ranges.as_slice()
|
||||
)
|
||||
let err = batch.put(put_data).unwrap_err();
|
||||
assert_eq!(StatusCode::TableColumnNotFound, err.status_code());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -15,298 +15,150 @@
|
||||
use std::io::Cursor;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_recordbatch::RecordBatch;
|
||||
use datatypes::arrow::ipc::reader::StreamReader;
|
||||
use datatypes::arrow::ipc::writer::{IpcWriteOptions, StreamWriter};
|
||||
use datatypes::arrow::record_batch::RecordBatch;
|
||||
use datatypes::schema::{Schema, SchemaRef};
|
||||
use datatypes::vectors::Helper;
|
||||
use prost::Message;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::storage::WriteRequest;
|
||||
use datatypes::schema::Schema;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use store_api::storage::OpType;
|
||||
|
||||
use crate::codec::{Decoder, Encoder};
|
||||
use crate::proto::wal::MutationType;
|
||||
use crate::proto::write_batch::{self, gen_columns, gen_put_data_vector};
|
||||
use crate::write_batch::{
|
||||
DataCorruptedSnafu, DecodeArrowSnafu, DecodeProtobufSnafu, DecodeVectorSnafu, EncodeArrowSnafu,
|
||||
EncodeProtobufSnafu, Error as WriteBatchError, FromProtobufSnafu, MissingColumnSnafu, Mutation,
|
||||
ParseSchemaSnafu, PutData, Result, ToProtobufSnafu, WriteBatch,
|
||||
use crate::error::{
|
||||
BatchCorruptedSnafu, CreateRecordBatchSnafu, DecodeArrowSnafu, EncodeArrowSnafu, Error,
|
||||
ParseSchemaSnafu, Result,
|
||||
};
|
||||
use crate::proto::wal::MutationType;
|
||||
use crate::write_batch::{Mutation, Payload};
|
||||
|
||||
// TODO(jiachun): We can make a comparison with protobuf, including performance, storage cost,
|
||||
// CPU consumption, etc
|
||||
#[derive(Default)]
|
||||
pub struct WriteBatchArrowEncoder {}
|
||||
pub struct PayloadEncoder {}
|
||||
|
||||
impl WriteBatchArrowEncoder {
|
||||
impl PayloadEncoder {
|
||||
pub fn new() -> Self {
|
||||
Self::default()
|
||||
}
|
||||
}
|
||||
|
||||
impl Encoder for WriteBatchArrowEncoder {
|
||||
type Item = WriteBatch;
|
||||
type Error = WriteBatchError;
|
||||
impl Encoder for PayloadEncoder {
|
||||
type Item = Payload;
|
||||
type Error = Error;
|
||||
|
||||
fn encode(&self, item: &WriteBatch, dst: &mut Vec<u8>) -> Result<()> {
|
||||
let item_schema = item.schema();
|
||||
let arrow_schema = item_schema.arrow_schema();
|
||||
fn encode(&self, item: &Payload, dst: &mut Vec<u8>) -> Result<()> {
|
||||
let arrow_schema = item.schema.arrow_schema();
|
||||
|
||||
let opts = IpcWriteOptions::default();
|
||||
let mut writer = StreamWriter::try_new_with_options(dst, arrow_schema, opts)
|
||||
.context(EncodeArrowSnafu)?;
|
||||
|
||||
for mutation in item.iter() {
|
||||
let rb = match mutation {
|
||||
Mutation::Put(put) => {
|
||||
let arrays = item_schema
|
||||
.column_schemas()
|
||||
.iter()
|
||||
.map(|column_schema| {
|
||||
let vector = put.column_by_name(&column_schema.name).context(
|
||||
MissingColumnSnafu {
|
||||
name: &column_schema.name,
|
||||
},
|
||||
)?;
|
||||
Ok(vector.to_arrow_array())
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()?;
|
||||
|
||||
RecordBatch::try_new(arrow_schema.clone(), arrays).context(EncodeArrowSnafu)?
|
||||
}
|
||||
};
|
||||
writer.write(&rb).context(EncodeArrowSnafu)?;
|
||||
for mutation in &item.mutations {
|
||||
let record_batch = mutation.record_batch.df_record_batch();
|
||||
writer.write(record_batch).context(EncodeArrowSnafu)?;
|
||||
}
|
||||
writer.finish().context(EncodeArrowSnafu)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub struct WriteBatchArrowDecoder {
|
||||
mutation_types: Vec<i32>,
|
||||
pub struct PayloadDecoder<'a> {
|
||||
mutation_types: &'a [i32],
|
||||
}
|
||||
|
||||
impl WriteBatchArrowDecoder {
|
||||
pub fn new(mutation_types: Vec<i32>) -> Self {
|
||||
impl<'a> PayloadDecoder<'a> {
|
||||
pub fn new(mutation_types: &'a [i32]) -> Self {
|
||||
Self { mutation_types }
|
||||
}
|
||||
}
|
||||
|
||||
impl Decoder for WriteBatchArrowDecoder {
|
||||
type Item = WriteBatch;
|
||||
type Error = WriteBatchError;
|
||||
impl<'a> Decoder for PayloadDecoder<'a> {
|
||||
type Item = Payload;
|
||||
type Error = Error;
|
||||
|
||||
fn decode(&self, src: &[u8]) -> Result<WriteBatch> {
|
||||
fn decode(&self, src: &[u8]) -> Result<Payload> {
|
||||
let reader = Cursor::new(src);
|
||||
let mut reader = StreamReader::try_new(reader, None).context(DecodeArrowSnafu)?;
|
||||
let arrow_schema = reader.schema();
|
||||
let mut chunks = Vec::with_capacity(self.mutation_types.len());
|
||||
|
||||
for maybe_record_batch in reader.by_ref() {
|
||||
let record_batch = maybe_record_batch.context(DecodeArrowSnafu)?;
|
||||
chunks.push(record_batch);
|
||||
// We could let the decoder takes a schema as input if possible, then we don't
|
||||
// need to rebuild the schema here.
|
||||
let schema = Arc::new(Schema::try_from(arrow_schema).context(ParseSchemaSnafu)?);
|
||||
let mut mutations = Vec::with_capacity(self.mutation_types.len());
|
||||
|
||||
for (record_batch, mutation_type) in reader.by_ref().zip(self.mutation_types) {
|
||||
let record_batch = record_batch.context(DecodeArrowSnafu)?;
|
||||
let record_batch = RecordBatch::try_from_df_record_batch(schema.clone(), record_batch)
|
||||
.context(CreateRecordBatchSnafu)?;
|
||||
let op_type = match MutationType::from_i32(*mutation_type) {
|
||||
Some(MutationType::Delete) => {
|
||||
unimplemented!("delete mutation is not implemented")
|
||||
}
|
||||
Some(MutationType::Put) => OpType::Put,
|
||||
None => {
|
||||
return BatchCorruptedSnafu {
|
||||
message: format!("Unexpceted mutation type: {mutation_type}"),
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
};
|
||||
mutations.push(Mutation {
|
||||
op_type,
|
||||
record_batch,
|
||||
});
|
||||
}
|
||||
|
||||
// check if exactly finished
|
||||
ensure!(
|
||||
reader.is_finished(),
|
||||
DataCorruptedSnafu {
|
||||
message: "Impossible, the num of data chunks is different than expected."
|
||||
BatchCorruptedSnafu {
|
||||
message: "The num of data chunks is different than expected."
|
||||
}
|
||||
);
|
||||
|
||||
ensure!(
|
||||
chunks.len() == self.mutation_types.len(),
|
||||
DataCorruptedSnafu {
|
||||
mutations.len() == self.mutation_types.len(),
|
||||
BatchCorruptedSnafu {
|
||||
message: format!(
|
||||
"expected {} mutations, but got {}",
|
||||
self.mutation_types.len(),
|
||||
chunks.len()
|
||||
mutations.len()
|
||||
)
|
||||
}
|
||||
);
|
||||
|
||||
let schema = Arc::new(Schema::try_from(arrow_schema).context(ParseSchemaSnafu)?);
|
||||
let mut write_batch = WriteBatch::new(schema.clone());
|
||||
|
||||
for (mutation_type, record_batch) in self.mutation_types.iter().zip(chunks.into_iter()) {
|
||||
match MutationType::from_i32(*mutation_type) {
|
||||
Some(MutationType::Put) => {
|
||||
let mut put_data = PutData::with_num_columns(schema.num_columns());
|
||||
for (column_schema, array) in schema
|
||||
.column_schemas()
|
||||
.iter()
|
||||
.zip(record_batch.columns().iter())
|
||||
{
|
||||
let vector = Helper::try_into_vector(array).context(DecodeVectorSnafu)?;
|
||||
put_data.add_column_by_name(&column_schema.name, vector)?;
|
||||
}
|
||||
|
||||
write_batch.put(put_data)?;
|
||||
}
|
||||
Some(MutationType::Delete) => {
|
||||
unimplemented!("delete mutation is not implemented")
|
||||
}
|
||||
_ => {
|
||||
return DataCorruptedSnafu {
|
||||
message: format!("Unexpected mutation type: {mutation_type}"),
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(write_batch)
|
||||
}
|
||||
}
|
||||
|
||||
pub struct WriteBatchProtobufEncoder {}
|
||||
|
||||
impl Encoder for WriteBatchProtobufEncoder {
|
||||
type Item = WriteBatch;
|
||||
type Error = WriteBatchError;
|
||||
|
||||
fn encode(&self, item: &WriteBatch, dst: &mut Vec<u8>) -> Result<()> {
|
||||
let schema = item.schema().into();
|
||||
|
||||
let mutations = item
|
||||
.iter()
|
||||
.map(|mtn| match mtn {
|
||||
Mutation::Put(put_data) => item
|
||||
.schema()
|
||||
.column_schemas()
|
||||
.iter()
|
||||
.map(|cs| {
|
||||
let vector = put_data
|
||||
.column_by_name(&cs.name)
|
||||
.context(MissingColumnSnafu { name: &cs.name })?;
|
||||
gen_columns(vector).context(ToProtobufSnafu)
|
||||
})
|
||||
.collect::<Result<Vec<_>>>(),
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()?
|
||||
.into_iter()
|
||||
.map(|columns| write_batch::Mutation {
|
||||
mutation: Some(write_batch::mutation::Mutation::Put(write_batch::Put {
|
||||
columns,
|
||||
})),
|
||||
})
|
||||
.collect();
|
||||
|
||||
let write_batch = write_batch::WriteBatch {
|
||||
schema: Some(schema),
|
||||
mutations,
|
||||
};
|
||||
|
||||
write_batch.encode(dst).context(EncodeProtobufSnafu)
|
||||
}
|
||||
}
|
||||
|
||||
pub struct WriteBatchProtobufDecoder {
|
||||
mutation_types: Vec<i32>,
|
||||
}
|
||||
|
||||
impl WriteBatchProtobufDecoder {
|
||||
#[allow(dead_code)]
|
||||
pub fn new(mutation_types: Vec<i32>) -> Self {
|
||||
Self { mutation_types }
|
||||
}
|
||||
}
|
||||
|
||||
impl Decoder for WriteBatchProtobufDecoder {
|
||||
type Item = WriteBatch;
|
||||
type Error = WriteBatchError;
|
||||
|
||||
fn decode(&self, src: &[u8]) -> Result<WriteBatch> {
|
||||
let write_batch = write_batch::WriteBatch::decode(src).context(DecodeProtobufSnafu)?;
|
||||
|
||||
let schema = write_batch.schema.context(DataCorruptedSnafu {
|
||||
message: "schema required",
|
||||
})?;
|
||||
|
||||
let schema = SchemaRef::try_from(schema).context(FromProtobufSnafu {})?;
|
||||
|
||||
ensure!(
|
||||
write_batch.mutations.len() == self.mutation_types.len(),
|
||||
DataCorruptedSnafu {
|
||||
message: &format!(
|
||||
"expected {} mutations, but got {}",
|
||||
self.mutation_types.len(),
|
||||
write_batch.mutations.len()
|
||||
)
|
||||
}
|
||||
);
|
||||
|
||||
let mutations = write_batch
|
||||
.mutations
|
||||
.into_iter()
|
||||
.map(|mtn| match mtn.mutation {
|
||||
Some(write_batch::mutation::Mutation::Put(put)) => {
|
||||
let mut put_data = PutData::with_num_columns(put.columns.len());
|
||||
|
||||
let res = schema
|
||||
.column_schemas()
|
||||
.iter()
|
||||
.map(|column| (column.name.clone(), column.data_type.clone()))
|
||||
.zip(put.columns.into_iter())
|
||||
.map(|((name, data_type), column)| {
|
||||
gen_put_data_vector(data_type, column)
|
||||
.map(|vector| (name, vector))
|
||||
.context(FromProtobufSnafu)
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()?
|
||||
.into_iter()
|
||||
.map(|(name, vector)| put_data.add_column_by_name(&name, vector))
|
||||
.collect::<Result<Vec<_>>>();
|
||||
|
||||
res.map(|_| Mutation::Put(put_data))
|
||||
}
|
||||
Some(write_batch::mutation::Mutation::Delete(_)) => todo!(),
|
||||
_ => DataCorruptedSnafu {
|
||||
message: "invalid mutation type",
|
||||
}
|
||||
.fail(),
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()?;
|
||||
|
||||
let mut write_batch = WriteBatch::new(schema);
|
||||
|
||||
mutations
|
||||
.into_iter()
|
||||
.try_for_each(|mutation| match mutation {
|
||||
Mutation::Put(put_data) => write_batch.put(put_data),
|
||||
})?;
|
||||
|
||||
Ok(write_batch)
|
||||
Ok(Payload { schema, mutations })
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use datatypes::vectors::{BooleanVector, TimestampMillisecondVector, UInt64Vector};
|
||||
use store_api::storage::PutOperation;
|
||||
use datatypes::vectors::{BooleanVector, TimestampMillisecondVector, UInt64Vector, VectorRef};
|
||||
use store_api::storage::{consts, WriteRequest};
|
||||
|
||||
use super::*;
|
||||
use crate::write_batch::WriteBatch;
|
||||
use crate::{proto, write_batch};
|
||||
|
||||
fn gen_new_batch_and_types() -> (WriteBatch, Vec<i32>) {
|
||||
let mut batch = write_batch::new_test_batch();
|
||||
for i in 0..10 {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3]));
|
||||
let boolv = Arc::new(BooleanVector::from(vec![Some(true), Some(false), None]));
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_vec(vec![i, i, i]));
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3])) as VectorRef;
|
||||
let boolv =
|
||||
Arc::new(BooleanVector::from(vec![Some(true), Some(false), None])) as VectorRef;
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_vec(vec![i, i, i])) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
put_data.add_version_column(intv).unwrap();
|
||||
put_data.add_value_column("v1", boolv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
let mut put_data = HashMap::new();
|
||||
put_data.insert("k1".to_string(), intv.clone());
|
||||
put_data.insert(consts::VERSION_COLUMN_NAME.to_string(), intv);
|
||||
put_data.insert("v1".to_string(), boolv);
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
|
||||
batch.put(put_data).unwrap();
|
||||
}
|
||||
|
||||
let types = proto::wal::gen_mutation_types(&batch);
|
||||
let types = proto::wal::gen_mutation_types(batch.payload());
|
||||
|
||||
(batch, types)
|
||||
}
|
||||
@@ -315,32 +167,15 @@ mod tests {
|
||||
fn test_codec_arrow() -> Result<()> {
|
||||
let (batch, mutation_types) = gen_new_batch_and_types();
|
||||
|
||||
let encoder = WriteBatchArrowEncoder::new();
|
||||
let encoder = PayloadEncoder::new();
|
||||
let mut dst = vec![];
|
||||
let result = encoder.encode(&batch, &mut dst);
|
||||
let result = encoder.encode(batch.payload(), &mut dst);
|
||||
assert!(result.is_ok());
|
||||
|
||||
let decoder = WriteBatchArrowDecoder::new(mutation_types);
|
||||
let decoder = PayloadDecoder::new(&mutation_types);
|
||||
let result = decoder.decode(&dst);
|
||||
let batch2 = result?;
|
||||
assert_eq!(batch.num_rows, batch2.num_rows);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_codec_protobuf() -> Result<()> {
|
||||
let (batch, mutation_types) = gen_new_batch_and_types();
|
||||
|
||||
let encoder = WriteBatchProtobufEncoder {};
|
||||
let mut dst = vec![];
|
||||
let result = encoder.encode(&batch, &mut dst);
|
||||
assert!(result.is_ok());
|
||||
|
||||
let decoder = WriteBatchProtobufDecoder::new(mutation_types);
|
||||
let result = decoder.decode(&dst);
|
||||
let batch2 = result?;
|
||||
assert_eq!(batch.num_rows, batch2.num_rows);
|
||||
let payload = result?;
|
||||
assert_eq!(*batch.payload(), payload);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
@@ -348,18 +183,18 @@ mod tests {
|
||||
fn gen_new_batch_and_types_with_none_column() -> (WriteBatch, Vec<i32>) {
|
||||
let mut batch = write_batch::new_test_batch();
|
||||
for _ in 0..10 {
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3]));
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_vec(vec![0, 0, 0]));
|
||||
let intv = Arc::new(UInt64Vector::from_slice(&[1, 2, 3])) as VectorRef;
|
||||
let tsv = Arc::new(TimestampMillisecondVector::from_vec(vec![0, 0, 0])) as VectorRef;
|
||||
|
||||
let mut put_data = PutData::new();
|
||||
put_data.add_key_column("k1", intv.clone()).unwrap();
|
||||
put_data.add_version_column(intv).unwrap();
|
||||
put_data.add_key_column("ts", tsv).unwrap();
|
||||
let mut put_data = HashMap::with_capacity(3);
|
||||
put_data.insert("k1".to_string(), intv.clone());
|
||||
put_data.insert(consts::VERSION_COLUMN_NAME.to_string(), intv);
|
||||
put_data.insert("ts".to_string(), tsv);
|
||||
|
||||
batch.put(put_data).unwrap();
|
||||
}
|
||||
|
||||
let types = proto::wal::gen_mutation_types(&batch);
|
||||
let types = proto::wal::gen_mutation_types(batch.payload());
|
||||
|
||||
(batch, types)
|
||||
}
|
||||
@@ -368,31 +203,15 @@ mod tests {
|
||||
fn test_codec_with_none_column_arrow() -> Result<()> {
|
||||
let (batch, mutation_types) = gen_new_batch_and_types_with_none_column();
|
||||
|
||||
let encoder = WriteBatchArrowEncoder::new();
|
||||
let encoder = PayloadEncoder::new();
|
||||
let mut dst = vec![];
|
||||
let result = encoder.encode(&batch, &mut dst);
|
||||
let result = encoder.encode(batch.payload(), &mut dst);
|
||||
assert!(result.is_ok());
|
||||
|
||||
let decoder = WriteBatchArrowDecoder::new(mutation_types);
|
||||
let decoder = PayloadDecoder::new(&mutation_types);
|
||||
let result = decoder.decode(&dst);
|
||||
let batch2 = result?;
|
||||
assert_eq!(batch.num_rows, batch2.num_rows);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_codec_with_none_column_protobuf() -> Result<()> {
|
||||
let (batch, mutation_types) = gen_new_batch_and_types_with_none_column();
|
||||
|
||||
let encoder = WriteBatchProtobufEncoder {};
|
||||
let mut dst = vec![];
|
||||
encoder.encode(&batch, &mut dst).unwrap();
|
||||
|
||||
let decoder = WriteBatchProtobufDecoder::new(mutation_types);
|
||||
let result = decoder.decode(&dst);
|
||||
let batch2 = result?;
|
||||
assert_eq!(batch.num_rows, batch2.num_rows);
|
||||
let payload = result?;
|
||||
assert_eq!(*batch.payload(), payload);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -12,20 +12,22 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use common_recordbatch::RecordBatch;
|
||||
use datatypes::schema::{ColumnSchema, SchemaRef};
|
||||
use snafu::{ensure, ResultExt};
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::schema::compat::CompatWrite;
|
||||
use crate::write_batch::{Mutation, PutData, WriteBatch};
|
||||
use crate::write_batch::{self, Mutation, WriteBatch};
|
||||
|
||||
impl CompatWrite for WriteBatch {
|
||||
fn compat_write(&mut self, dest_schema: &SchemaRef) -> Result<()> {
|
||||
let (data_version, schema_version) = (dest_schema.version(), self.schema.version());
|
||||
let data_version = dest_schema.version();
|
||||
let schema_version = self.schema().version();
|
||||
// Fast path, nothing to do if schema version of the write batch is equal to version
|
||||
// of destination.
|
||||
if data_version == schema_version {
|
||||
debug_assert_eq!(dest_schema.column_schemas(), self.schema.column_schemas());
|
||||
debug_assert_eq!(dest_schema.column_schemas(), self.schema().column_schemas());
|
||||
|
||||
return Ok(());
|
||||
}
|
||||
@@ -39,7 +41,7 @@ impl CompatWrite for WriteBatch {
|
||||
);
|
||||
|
||||
// For columns not in schema, returns error instead of discarding the column silently.
|
||||
let column_not_in = column_not_in_schema(dest_schema, self.schema.column_schemas());
|
||||
let column_not_in = column_not_in_schema(dest_schema, self.schema().column_schemas());
|
||||
ensure!(
|
||||
column_not_in.is_none(),
|
||||
error::NotInSchemaToCompatSnafu {
|
||||
@@ -48,37 +50,39 @@ impl CompatWrite for WriteBatch {
|
||||
}
|
||||
);
|
||||
|
||||
for m in &mut self.mutations {
|
||||
match m {
|
||||
Mutation::Put(put_data) => {
|
||||
put_data.compat_write(dest_schema)?;
|
||||
}
|
||||
}
|
||||
for mutation in &mut self.payload.mutations {
|
||||
mutation.compat_write(dest_schema)?;
|
||||
}
|
||||
|
||||
// Change schema to `dest_schema`.
|
||||
self.schema = dest_schema.clone();
|
||||
self.payload.schema = dest_schema.clone();
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl CompatWrite for PutData {
|
||||
impl CompatWrite for Mutation {
|
||||
fn compat_write(&mut self, dest_schema: &SchemaRef) -> Result<()> {
|
||||
if self.is_empty() {
|
||||
if self.record_batch.num_rows() == 0 {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
let num_rows = self.record_batch.num_rows();
|
||||
let mut columns = Vec::with_capacity(dest_schema.num_columns());
|
||||
for column_schema in dest_schema.column_schemas() {
|
||||
if self.column_by_name(&column_schema.name).is_none() {
|
||||
if let Some(vector) = self.record_batch.column_by_name(&column_schema.name) {
|
||||
columns.push(vector.clone());
|
||||
} else {
|
||||
// We need to fill the column by null or its default value.
|
||||
self.add_default_by_name(column_schema)
|
||||
.context(error::AddDefaultSnafu {
|
||||
column: &column_schema.name,
|
||||
})?;
|
||||
let vector = write_batch::new_column_with_default_value(column_schema, num_rows)?;
|
||||
columns.push(vector);
|
||||
}
|
||||
}
|
||||
|
||||
// Using dest schema to build RecordBatch.
|
||||
self.record_batch = RecordBatch::new(dest_schema.clone(), columns)
|
||||
.context(error::CreateRecordBatchSnafu)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -95,12 +99,13 @@ fn column_not_in_schema(schema: &SchemaRef, column_schemas: &[ColumnSchema]) ->
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnDefaultConstraint, SchemaBuilder};
|
||||
use datatypes::vectors::{Int32Vector, TimestampMillisecondVector};
|
||||
use store_api::storage::{PutOperation, WriteRequest};
|
||||
use datatypes::vectors::{Int32Vector, TimestampMillisecondVector, VectorRef};
|
||||
use store_api::storage::WriteRequest;
|
||||
|
||||
use super::*;
|
||||
use crate::error::Error;
|
||||
@@ -135,23 +140,31 @@ mod tests {
|
||||
Arc::new(schema)
|
||||
}
|
||||
|
||||
fn new_put_data() -> PutData {
|
||||
let mut put_data = PutData::new();
|
||||
let k0 = Arc::new(Int32Vector::from_slice(&[1, 2, 3]));
|
||||
let ts = Arc::new(TimestampMillisecondVector::from_values([11, 12, 13]));
|
||||
fn new_put_data() -> HashMap<String, VectorRef> {
|
||||
let mut put_data = HashMap::new();
|
||||
let k0 = Arc::new(Int32Vector::from_slice(&[1, 2, 3])) as VectorRef;
|
||||
let ts = Arc::new(TimestampMillisecondVector::from_values([11, 12, 13])) as VectorRef;
|
||||
|
||||
put_data.add_key_column("k0", k0).unwrap();
|
||||
put_data.add_key_column("ts", ts).unwrap();
|
||||
put_data.insert("k0".to_string(), k0);
|
||||
put_data.insert("ts".to_string(), ts);
|
||||
|
||||
put_data
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_put_data_compat_write() {
|
||||
let mut put_data = new_put_data();
|
||||
fn test_mutation_compat_write() {
|
||||
let put_data = new_put_data();
|
||||
let schema_old = new_test_schema(None);
|
||||
// Mutation doesn't check schema version, so we don't have to bump the version here.
|
||||
let schema = new_test_schema(Some(Some(ColumnDefaultConstraint::null_value())));
|
||||
put_data.compat_write(&schema).unwrap();
|
||||
let v0 = put_data.column_by_name("v0").unwrap();
|
||||
// Use WriteBatch to build a payload and its mutation.
|
||||
let mut batch = WriteBatch::new(schema_old);
|
||||
batch.put(put_data).unwrap();
|
||||
|
||||
let mutation = &mut batch.payload.mutations[0];
|
||||
mutation.compat_write(&schema).unwrap();
|
||||
|
||||
let v0 = mutation.record_batch.column_by_name("v0").unwrap();
|
||||
assert!(v0.only_null());
|
||||
}
|
||||
|
||||
@@ -170,8 +183,9 @@ mod tests {
|
||||
);
|
||||
batch.compat_write(&schema_new).unwrap();
|
||||
assert_eq!(schema_new, *batch.schema());
|
||||
let Mutation::Put(put_data) = batch.iter().next().unwrap();
|
||||
put_data.column_by_name("v0").unwrap();
|
||||
|
||||
let mutation = &batch.payload().mutations[0];
|
||||
mutation.record_batch.column_by_name("v0").unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
||||
@@ -16,7 +16,7 @@
|
||||
|
||||
use common_error::prelude::ErrorExt;
|
||||
|
||||
use crate::logstore::entry::{Entry, Id, Offset};
|
||||
use crate::logstore::entry::{Entry, Id};
|
||||
use crate::logstore::entry_stream::SendableEntryStream;
|
||||
use crate::logstore::namespace::Namespace;
|
||||
|
||||
@@ -30,21 +30,20 @@ pub trait LogStore: Send + Sync + 'static + std::fmt::Debug {
|
||||
type Error: ErrorExt + Send + Sync + 'static;
|
||||
type Namespace: Namespace;
|
||||
type Entry: Entry;
|
||||
type AppendResponse: AppendResponse;
|
||||
|
||||
async fn start(&self) -> Result<(), Self::Error>;
|
||||
|
||||
/// Stop components of logstore.
|
||||
async fn stop(&self) -> Result<(), Self::Error>;
|
||||
|
||||
/// Append an `Entry` to WAL with given namespace
|
||||
async fn append(&self, mut e: Self::Entry) -> Result<Self::AppendResponse, Self::Error>;
|
||||
/// Append an `Entry` to WAL with given namespace and return append response containing
|
||||
/// the entry id.
|
||||
async fn append(&self, mut e: Self::Entry) -> Result<AppendResponse, Self::Error>;
|
||||
|
||||
/// Append a batch of entries atomically and return the offset of first entry.
|
||||
async fn append_batch(
|
||||
&self,
|
||||
ns: &Self::Namespace,
|
||||
e: Vec<Self::Entry>,
|
||||
) -> Result<Id, Self::Error>;
|
||||
) -> Result<Vec<Id>, Self::Error>;
|
||||
|
||||
/// Create a new `EntryStream` to asynchronously generates `Entry` with ids
|
||||
/// starting from `id`.
|
||||
@@ -76,8 +75,7 @@ pub trait LogStore: Send + Sync + 'static + std::fmt::Debug {
|
||||
async fn obsolete(&self, namespace: Self::Namespace, id: Id) -> Result<(), Self::Error>;
|
||||
}
|
||||
|
||||
pub trait AppendResponse: Send + Sync {
|
||||
fn entry_id(&self) -> Id;
|
||||
|
||||
fn offset(&self) -> Offset;
|
||||
#[derive(Debug)]
|
||||
pub struct AppendResponse {
|
||||
pub entry_id: Id,
|
||||
}
|
||||
|
||||
@@ -12,7 +12,6 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use common_base::buffer::{Buffer, BufferMut};
|
||||
use common_error::ext::ErrorExt;
|
||||
|
||||
use crate::logstore::namespace::Namespace;
|
||||
@@ -22,7 +21,7 @@ pub type Epoch = u64;
|
||||
pub type Id = u64;
|
||||
|
||||
/// Entry is the minimal data storage unit in `LogStore`.
|
||||
pub trait Entry: Encode + Send + Sync {
|
||||
pub trait Entry: Send + Sync {
|
||||
type Error: ErrorExt + Send + Sync;
|
||||
type Namespace: Namespace;
|
||||
|
||||
@@ -32,32 +31,5 @@ pub trait Entry: Encode + Send + Sync {
|
||||
/// Return entry id that monotonically increments.
|
||||
fn id(&self) -> Id;
|
||||
|
||||
/// Return file offset of entry.
|
||||
fn offset(&self) -> Offset;
|
||||
|
||||
fn set_id(&mut self, id: Id);
|
||||
|
||||
/// Returns epoch of entry.
|
||||
fn epoch(&self) -> Epoch;
|
||||
|
||||
fn len(&self) -> usize;
|
||||
|
||||
fn is_empty(&self) -> bool;
|
||||
|
||||
fn namespace(&self) -> Self::Namespace;
|
||||
}
|
||||
|
||||
pub trait Encode: Sized {
|
||||
type Error: ErrorExt + Send + Sync + 'static;
|
||||
|
||||
/// Encodes item to given byte slice and return encoded size on success;
|
||||
/// # Panics
|
||||
/// If given buffer is not large enough to hold the encoded item.
|
||||
fn encode_to<T: BufferMut>(&self, buf: &mut T) -> Result<usize, Self::Error>;
|
||||
|
||||
/// Decodes item from given buffer.
|
||||
fn decode<T: Buffer>(buf: &mut T) -> Result<Self, Self::Error>;
|
||||
|
||||
/// Return the size in bytes of the encoded item;
|
||||
fn encoded_size(&self) -> usize;
|
||||
}
|
||||
|
||||
@@ -36,18 +36,13 @@ mod tests {
|
||||
use futures::StreamExt;
|
||||
|
||||
use super::*;
|
||||
use crate::logstore::entry::{Encode, Epoch, Id, Offset};
|
||||
pub use crate::logstore::entry::Id;
|
||||
|
||||
pub struct SimpleEntry {
|
||||
/// Offset of current entry
|
||||
offset: Offset,
|
||||
/// Epoch of current entry
|
||||
epoch: Epoch,
|
||||
/// Binary data of current entry
|
||||
data: Vec<u8>,
|
||||
}
|
||||
|
||||
use common_base::buffer::{Buffer, BufferMut};
|
||||
use common_error::prelude::{ErrorExt, Snafu};
|
||||
use snafu::{Backtrace, ErrorCompat};
|
||||
|
||||
@@ -74,23 +69,6 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
impl Encode for SimpleEntry {
|
||||
type Error = Error;
|
||||
|
||||
fn encode_to<T: BufferMut>(&self, buf: &mut T) -> Result<usize, Self::Error> {
|
||||
buf.write_from_slice(self.data.as_slice()).unwrap();
|
||||
Ok(self.data.as_slice().len())
|
||||
}
|
||||
|
||||
fn decode<T: Buffer>(_buf: &mut T) -> Result<Self, Self::Error> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
fn encoded_size(&self) -> usize {
|
||||
self.data.as_slice().len()
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry for SimpleEntry {
|
||||
type Error = Error;
|
||||
type Namespace = Namespace;
|
||||
@@ -103,37 +81,15 @@ mod tests {
|
||||
0u64
|
||||
}
|
||||
|
||||
fn offset(&self) -> Offset {
|
||||
self.offset
|
||||
}
|
||||
|
||||
fn set_id(&mut self, _id: Id) {}
|
||||
|
||||
fn epoch(&self) -> Epoch {
|
||||
self.epoch
|
||||
}
|
||||
|
||||
fn len(&self) -> usize {
|
||||
self.data.len()
|
||||
}
|
||||
|
||||
fn is_empty(&self) -> bool {
|
||||
self.data.is_empty()
|
||||
}
|
||||
|
||||
fn namespace(&self) -> Self::Namespace {
|
||||
Namespace {}
|
||||
}
|
||||
}
|
||||
|
||||
impl SimpleEntry {
|
||||
pub fn new(data: impl AsRef<[u8]>, offset: Offset, epoch: u64) -> Self {
|
||||
pub fn new(data: impl AsRef<[u8]>) -> Self {
|
||||
let data = data.as_ref().to_vec();
|
||||
Self {
|
||||
data,
|
||||
offset,
|
||||
epoch,
|
||||
}
|
||||
Self { data }
|
||||
}
|
||||
}
|
||||
|
||||
@@ -165,9 +121,8 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_entry_stream() {
|
||||
let stream = async_stream::stream!({
|
||||
yield Ok(vec![SimpleEntry::new("test_entry".as_bytes(), 0, 128)])
|
||||
});
|
||||
let stream =
|
||||
async_stream::stream!({ yield Ok(vec![SimpleEntry::new("test_entry".as_bytes())]) });
|
||||
|
||||
let mut stream_impl = EntryStreamImpl {
|
||||
inner: Box::pin(stream),
|
||||
|
||||
@@ -16,6 +16,6 @@ use std::hash::Hash;
|
||||
|
||||
pub type Id = u64;
|
||||
|
||||
pub trait Namespace: Send + Sync + Clone + std::fmt::Debug + Hash + Eq {
|
||||
pub trait Namespace: Send + Sync + Clone + std::fmt::Debug + Hash + PartialEq {
|
||||
fn id(&self) -> Id;
|
||||
}
|
||||
|
||||
@@ -36,7 +36,7 @@ pub use self::engine::{CreateOptions, EngineContext, OpenOptions, StorageEngine}
|
||||
pub use self::metadata::RegionMeta;
|
||||
pub use self::region::{Region, WriteContext};
|
||||
pub use self::requests::{
|
||||
AddColumn, AlterOperation, AlterRequest, GetRequest, PutOperation, ScanRequest, WriteRequest,
|
||||
AddColumn, AlterOperation, AlterRequest, GetRequest, ScanRequest, WriteRequest,
|
||||
};
|
||||
pub use self::responses::{GetResponse, ScanResponse, WriteResponse};
|
||||
pub use self::snapshot::{ReadContext, Snapshot};
|
||||
|
||||
@@ -12,12 +12,10 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::HashSet;
|
||||
use std::time::Duration;
|
||||
use std::collections::{HashMap, HashSet};
|
||||
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_query::logical_plan::Expr;
|
||||
use common_time::RangeMillis;
|
||||
use datatypes::vectors::VectorRef;
|
||||
|
||||
use crate::storage::{ColumnDescriptor, RegionDescriptor, SequenceNumber};
|
||||
@@ -28,35 +26,11 @@ use crate::storage::{ColumnDescriptor, RegionDescriptor, SequenceNumber};
|
||||
/// the request follows the same schema restriction.
|
||||
pub trait WriteRequest: Send {
|
||||
type Error: ErrorExt + Send + Sync;
|
||||
type PutOp: PutOperation;
|
||||
|
||||
/// Add put operation to the request.
|
||||
fn put(&mut self, put: Self::PutOp) -> Result<(), Self::Error>;
|
||||
|
||||
/// Returns all possible time ranges that contain the timestamp in this batch.
|
||||
///
|
||||
/// Each time range is aligned to given `duration`.
|
||||
fn time_ranges(&self, duration: Duration) -> Result<Vec<RangeMillis>, Self::Error>;
|
||||
|
||||
/// Create a new put operation.
|
||||
fn put_op(&self) -> Self::PutOp;
|
||||
|
||||
/// Create a new put operation with capacity reserved for `num_columns`.
|
||||
fn put_op_with_columns(num_columns: usize) -> Self::PutOp;
|
||||
}
|
||||
|
||||
/// Put multiple rows.
|
||||
pub trait PutOperation: Send + std::fmt::Debug {
|
||||
type Error: ErrorExt + Send + Sync;
|
||||
|
||||
/// Put data to the key column.
|
||||
fn add_key_column(&mut self, name: &str, vector: VectorRef) -> Result<(), Self::Error>;
|
||||
|
||||
/// Put data to the version column.
|
||||
fn add_version_column(&mut self, vector: VectorRef) -> Result<(), Self::Error>;
|
||||
|
||||
/// Put data to the value column.
|
||||
fn add_value_column(&mut self, name: &str, vector: VectorRef) -> Result<(), Self::Error>;
|
||||
/// `data` is the columnar format of the data to put.
|
||||
fn put(&mut self, data: HashMap<String, VectorRef>) -> Result<(), Self::Error>;
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
|
||||
@@ -39,9 +39,6 @@ pub enum InnerError {
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Missing column when insert, column: {}", name))]
|
||||
MissingColumn { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Poll stream failed, source: {}", source))]
|
||||
PollStream {
|
||||
source: ArrowError,
|
||||
@@ -119,9 +116,9 @@ impl ErrorExt for InnerError {
|
||||
| InnerError::PollStream { .. }
|
||||
| InnerError::SchemaConversion { .. }
|
||||
| InnerError::TableProjection { .. } => StatusCode::EngineExecuteQuery,
|
||||
InnerError::MissingColumn { .. }
|
||||
| InnerError::RemoveColumnInIndex { .. }
|
||||
| InnerError::BuildColumnDescriptor { .. } => StatusCode::InvalidArguments,
|
||||
InnerError::RemoveColumnInIndex { .. } | InnerError::BuildColumnDescriptor { .. } => {
|
||||
StatusCode::InvalidArguments
|
||||
}
|
||||
InnerError::TablesRecordBatch { .. } => StatusCode::Unexpected,
|
||||
InnerError::ColumnExists { .. } => StatusCode::TableColumnExists,
|
||||
InnerError::SchemaBuild { source, .. } => source.status_code(),
|
||||
@@ -166,12 +163,16 @@ mod tests {
|
||||
Err(DataFusionError::NotImplemented("table test".to_string())).context(DatafusionSnafu)?
|
||||
}
|
||||
|
||||
fn throw_missing_column_inner() -> std::result::Result<(), InnerError> {
|
||||
MissingColumnSnafu { name: "test" }.fail()
|
||||
fn throw_column_exists_inner() -> std::result::Result<(), InnerError> {
|
||||
ColumnExistsSnafu {
|
||||
column_name: "col",
|
||||
table_name: "test",
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
|
||||
fn throw_missing_column() -> Result<()> {
|
||||
Ok(throw_missing_column_inner()?)
|
||||
Ok(throw_column_exists_inner()?)
|
||||
}
|
||||
|
||||
fn throw_arrow() -> Result<()> {
|
||||
@@ -186,7 +187,7 @@ mod tests {
|
||||
|
||||
let err = throw_missing_column().err().unwrap();
|
||||
assert!(err.backtrace_opt().is_some());
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
assert_eq!(StatusCode::TableColumnExists, err.status_code());
|
||||
|
||||
let err = throw_arrow().err().unwrap();
|
||||
assert!(err.backtrace_opt().is_some());
|
||||
@@ -195,15 +196,15 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_into_record_batch_error() {
|
||||
let err = throw_missing_column_inner().err().unwrap();
|
||||
let err = throw_column_exists_inner().err().unwrap();
|
||||
let err: RecordBatchError = err.into();
|
||||
assert!(err.backtrace_opt().is_some());
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
assert_eq!(StatusCode::TableColumnExists, err.status_code());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_into_df_error() {
|
||||
let err = throw_missing_column_inner().err().unwrap();
|
||||
let err = throw_column_exists_inner().err().unwrap();
|
||||
let err: DataFusionError = err.into();
|
||||
assert!(matches!(err, DataFusionError::External(_)));
|
||||
}
|
||||
|
||||
@@ -23,7 +23,7 @@ use axum::Router;
|
||||
use catalog::CatalogManagerRef;
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, MIN_USER_TABLE_ID};
|
||||
use common_runtime::Builder as RuntimeBuilder;
|
||||
use datanode::datanode::{DatanodeOptions, ObjectStoreConfig};
|
||||
use datanode::datanode::{DatanodeOptions, ObjectStoreConfig, WalConfig};
|
||||
use datanode::error::{CreateTableSnafu, Result};
|
||||
use datanode::instance::{Instance, InstanceRef};
|
||||
use datanode::sql::SqlHandler;
|
||||
@@ -147,7 +147,10 @@ pub fn create_tmp_dir_and_datanode_opts(
|
||||
let (storage, data_tmp_dir) = get_test_store_config(&store_type, name);
|
||||
|
||||
let opts = DatanodeOptions {
|
||||
wal_dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
wal: WalConfig {
|
||||
dir: wal_tmp_dir.path().to_str().unwrap().to_string(),
|
||||
..Default::default()
|
||||
},
|
||||
storage,
|
||||
mode: Mode::Standalone,
|
||||
..Default::default()
|
||||
|
||||
Reference in New Issue
Block a user