diff --git a/rfcs/0068-error-record-table.md b/rfcs/0068-error-record-table.md new file mode 100644 index 00000000..ca89a1d9 --- /dev/null +++ b/rfcs/0068-error-record-table.md @@ -0,0 +1,96 @@ +--- +feature: error_record_table +authors: + - "TennyZhuang" +start_date: "2023/07/31" +--- + +# Error Record Table + +## Summary + +Our current streaming engine does not help users to discover, debug, and handle errors well. When user met an data record error, they can only find a log record like ``ExprError: Parse error: expected `,` or `]` at line 1 column 10 (ProjectExecutor: fragment_id=19007)``. + +User can't view the eror record, and can't replay with the error record. + +We want to introduce the Error Record Table (ERT) to resolve the problem. + +## Motivation + +There are several benefits to maintain the error records ourselves: + +1. We can ensure that our storage engine can handle the volume of erroneous data, as it is of the same magnitude as the source. +2. We can ensure the error records are durable. +3. Users can view the error records directly over psql. +4. Users can reproduce the error easily by the similar SQL. + +Some common errors include: + +1. Expression errors (division by zero, cast fail, json extraction fail) +2. UDF errors (timeout, invalid records, user errors) +3. Source error (paring failed) + +## Design + +### Creating + +The ERTs are automatically created as internal tables when an operator is created. In most cases, an operator will have n ERTs, where n corresponds to the number of inputs it has. + +Connectors should also ouptut the error records to the source ERT. + +### Naming + +Same as other internal tables while suffixed by `error_{seq}`. + +### Schema + +The schema of ERT should have the same fields as their input, with several extra columns: + +1. `id bigint`: The ID can be generated by the similar method like `row_id` (vnode + local monotical ID). +2. `error_reason varchar`: A human-readable error message. + +### Modification + +To keep things simple, we do not permit any DML operations over the ERT. Only the `TRUNCATE TABLE` operation is permitted. + +### The relationship between ERT and the log system + +We should keep the warning entry in our log, and we can give the error record ID in the log entry. + +We can even give a SQL to query the error record in the log entry if it's helpful to user. + +## Unresolved questions + +Should we allow creating sink over ERT? (Similar to side_output in Flink) + +## Alternatives + +One alternative solution is to output the complete error record directly to the log system. There are some concerns: + +1. The data record may be too large to record, e.g. several tens of KB. +2. Errors may occur continuously, causing the log system to fill up quickly. + +From the viewpoint of users, many of them do not consider logs as being reliable. In our default error handling approach, we do not halt the stream when encountering stream errors, which may lead users to anticipate greater reliability in those logs. + +## Future possibilities + +### Data correction + +ERT could potentially be used to correct data, for example, users could clean up the data within ERT and then reimport it into the source. + +```sql +SELECT v1, v2, error_reason FROM __rw_internal_1023_source_1134_error_1; +# 10000, 0, "division by zero" +CREATE TEMP TABLE fixing_1234 (v1 int, v2 int); +INSERT INTO fixing_1234 ( + SELECT v1, v2 FROM __rw_internal_1023_source_1134_error_1); +UPDATE fixing_1234 SET (v2 = 1) WHERE v2 = 0; +INSERT INTO source_table ( + SELECT * FROM fixing_1234 +); +TRUNCATE TABLE __rw_internal_1023_source_1134_error_1; +``` + +### Sink + +For advanced users, we can still allow them sink the error records to their own system.