Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

docs: klip-52: bytes data type support #7764

Merged
merged 6 commits into from
Jul 12, 2021
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion design-proposals/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -92,4 +92,4 @@ Next KLIP number: **53**
| KLIP-49: Add source stream/table semantic | Proposal | | | |
| KLIP-50: Partition and offset in ksqlDB | Proposal | 0.23.0 | | [Discussion](https://github.com/confluentinc/ksql/pull/7505) |
| [KLIP-51: ksqlDB .NET LINQ provider](klip-51-ksqldb .NET LINQ provider.md) | Proposal | | | [Discussion](https://github.com/confluentinc/ksql/pull/6883) |
| KLIP-52: BYTES data type support | Proposal | 0.21.0 | | |
| [KLIP-52: BYTES data type support](klip-52-bytes-data-type-support.md) | Proposal | 0.21.0 | | [Discussion](https://github.com/confluentinc/ksql/pull/7764) |
156 changes: 156 additions & 0 deletions design-proposals/klip-52-bytes-data-type-support.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,156 @@
# KLIP 46 - BYTES Data Type Support
jzaralim marked this conversation as resolved.
Show resolved Hide resolved

**Author**: Zara Lim (@jzaralim) |
**Release Target**: 0.21 |
**Status**: _In Discussion_ |
**Discussion**: https://github.com/confluentinc/ksql/pull/7764

**tl;dr:** _Add support for the BYTES data type. This will allow users to work with BLOBs of data that don't fit into any other data type._

## Motivation and background

Currently, ksqlDB can only handle a set of primitive types and combinations of them.
A BYTES data type would allow users to work with data that does not fit into any of
the primitive types such as images, as well as BLOB/binary data from other databases.

## What is in scope
* Add BYTES type to KSQL
* Support BYTE comparisons
* Support BYTES usage in STRUCT, MAP and ARRAY
* Serialization and de-serialization of BYTES to Avro, JSON, Protobuf and Delimited formats
* Adding/updating UDFs to support the BYTES type
* Casting between BYTES and STRING

## What is not in scope
* Fixed sized BYTES (`BYTES(3)` representing 3 bytes, for example) - This is supported by Kafka Connect by adding the `connect.fixed.size`
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why will not be supported? Doesn't fit on any use case?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Two reasons:

  1. The purpose of this KLIP is to support raw data that doesn't fit into other types, and to make it easier to handle data from other sources. BYTES can do both of these on its own.
  2. Fixed size byte arrays are not supported in Protobuf.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok. I don't know a use case for fixed bytes yet. But it's ok to not implement it for now. We don't have fixed strings either.

key in a bytes schema, but this will not be included in this KLIP.

## Public APIS

### BYTES

The BYTES data type will store an array of raw bytes of an unspecified length. The maximum size of
the array is limited by the maximum size of a Kafka message, as well as possibly by the value format being used.
jzaralim marked this conversation as resolved.
Show resolved Hide resolved
The syntax is as follows:

```roomsql
CREATE STREAM stream_name (b BYTES, COL2 STRING) AS ...
CREATE TABLE table_name (col1 STRUCT<field BYTES>) AS ...
```

By default, BYTES will be displayed in console as HEX strings, where each byte is represented by two characters.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

By default, BYTES will be displayed in console as HEX strings

  • Does "console" refer strictly to the ksqlDB CLI? If so, the example above should use the ksql> prompt.
  • Are we choosing hex here for consistency with other databases?
  • What about the REST API? With JSON, it's typical to send byte arrays as base64 strings.
  • Should the Java client automatically convert their wire representation into ByteBuffer objects?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍 just saw later in the KLIP that JSON/delimited will use base64. That covers the REST question above.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Yeah it's the CLI. The two common CLI representations in other databases are hex and UTF-8. IMO hex is easier to use with because sometimes UTF-8 representations show weird characters.
  • The Java client should keep base64 strings as is until a user wants it converted to ByteBuffer (if they call getBytes on the response object.)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added some clarifications to the KLIP on how BYTES will be handled by the REST API/Java client.

For example, the byte array `[91, 67]` will be displayed as:

```roomsql
> SELECT b from STREAM;
'0x5B43'
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it going to be displayed with single quotes?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't have a strong opinion on this. Maybe not, since it's not a string.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Vote for no wrapping in single quotes.

```

Users can also represent BYTES as HEX strings, for example

```roomsql
> INSERT INTO STREAM VALUES ('0x5b43', 'string value');
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure about this. '0x5b43' may be seen as a String. Mysql, for instance:

insert into t1(a) values ('0x61');

mysql> select * from t1;
+------------+
| a          |
+------------+
| 0x30783631 |
+------------+

The 0x61 is converted from string to bytes. Could users also expect this conversion? Could we provide conversion functions instead of doing implicit conversions here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that if users are seeing HEX outputs, then its reasonable to expect HEX strings to work as inputs (Postgres does this). That being said, not doing implicit conversions is also reasonable. Let's go forward with no implicit conversions + no casting.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed implicit conversions from KLIP

```

The input and output formats can be configured using a new property, `ksql.bytes.format`.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about letting users use functions instead of setting configs? It will be easier to identify how the data is displayed by seeing the schema with describe ... when using functions.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That works. The only downside is, the output format will always be a HEX string, though the user can use a conversion function if they want to see another output format.

The accepted encodings are `hex`, `utf8`, `ascii`, and `base64`.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should support functions for different encodings instead of configs. IMHO this is going to let users use different formats in different queries.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you mean something like to_bytes and from_bytes? Or did you have something else in mind?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed property from KLIP


### UDF

The following UDFs will be added:

* `to_bytes(string, inputEncoding, outputEncoding)` - this will convert a STRING value in the specified encoding format to a BYTES in the specified encoding format.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need to specify the output encoding if the result is only bytes? to_bytes('my name', 'utf8') should convert UTF-8 String to bytes, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The existing encode function accepts these two parameters, so it can do things like convert a HEX string to UTF-8. Right now to_bytes basically does encode and then converts the result to bytes. Yeah just having one encoding parameter makes more sense.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated KLIP to remove outputEncoding

The allowed encoders are the same as the ones allowed in the existing `encode` function.
* `decode(bytes, inputEncoding, outputEncoding)` - this will convert a BYTES value in the specified encoding format to a STRING in the specified encoding format.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same question about the output encoding. Btw, I think we should have similar function names, either to_bytes/from_bytes or encode/decode. I prefer to_bytes/from_bytes as it gives better understanding of the conversion.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed, from_bytes makes more sense.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated KLIP with new function name and removed outputEncoding


We will also update some of the existing STRING functions to accept BYTES as a parameter. In general, if a function works on ASCII characters for a STRING parameter,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will these functions assume a particular encoding for the bytes, or will the functions all need to take a parameter to specify the encoding? If the latter, I wonder if it's (conceptually) simpler to just let users nest the from_bytes() function instead.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All DBs that support bytes/binary types also support functions to use strings or bytes as parameters. Using from_bytes() everytime they want to use a string function will have a different behavior, for instance:

Say we have raw = [91, 67], then:
len(raw) = 2, but len(from_bytes(raw, 'hex')) = 6 because from_bytes(raw, 'hex') = '0x5B43'

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yup, so basically they just operate on raw bytes, so no encoding is needed.

then it will work on bytes for a BYTES parameter.

* `len(bytes)` - This will return the length of the stored ByteArray.
* `concat(bytes...)` - Concatenate an arbitrary number of byte fields
* `r/lpad(bytes, target_length, padding_bytes)` - pads input BYTES beginning from the left/right with the specified padding BYTES until the target length is reached.
* `replace(bytes, old_bytes, new_bytes)` - returns the given BYTES value with all occurrences of `old_bytes` with `new_bytes`
* `split(bytes, delimiter)` - splits a BYTES value into an array of BYTES based on a delimiter
* `splittomap(bytes, entryDelimiter, kvDelimiter)` - splits a BYTES value into key-value pairs based on a delimiter and creates a MAP from them
Comment on lines +69 to +71
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are there particular use-cases for using bytes with these functions?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not that I can think of, but supporting binary for string functions is common across almost all databases and there are people on Stack Overflow who use them.

* `substring(bytes, to, from)` - returns the section of the BYTES from the byte at position `to` to `from`
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Substring with bytes feels awkward. I would expect the function to be called slice. Can we make the existing slice function that takes a collection argument work on bytes also?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A lot of databases support substring for binary/blob, so that might actually be expected.

In general, it seems like binary array types are seen as closer to text types than actual arrays.


Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about functions for converting bytes to other data types such as integers or other types with standard bytes encodings? I see below that CAST() will not be supported for this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Most other systems don't support this. If needed, the user could cast the int to a string and then call to_bytes.

## Design
### Serialization/Deserialization

BYTES will be handled by [`java.nio.ByteBuffer`](https://docs.oracle.com/javase/7/docs/api/java/nio/ByteBuffer.html) within ksqlDB.
The underlying Kafka Connect type is the primitive `bytes` type.

#### Avro

`bytes` is a primitive Avro type. When converting to/from Connect data, the Avro converter ksqlDB
uses converts byte arrays to ByteBuffer.

#### Protobuf

`bytes` is a primitive Protobuf type. The maximum number of bytes in a byte array is 2<sup>32</sup>.
When converting to/from Connect data, the Avro converter ksqlDB uses converts byte arrays to ByteBuffer.
Comment on lines +87 to +88
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we have a maximum length for BYTES now that we have a max. in protobuf? I assume this limit will be handled by Protobuf, but what would happen when I try to insert 2^32+1 bytes into a protobuf topic? should it fail or truncated?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The size limit for a string in Protobuf is also 2^32. That's more than 4GB. I suspect that at that size, the ksqlDB code itself would run into problems before Protobuf serialization is a concern.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right. Seems kafka maximum message size is 1GB? Is there a maximum size allowed in Connect?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure if what the maximum message size is, but it's configurable with the default being 1 MB. Same with Connect.


#### JSON/Delimited

Byte arrays will be stored in JSON and CSV files as [Base64 MIME](https://docs.oracle.com/javase/8/docs/api/java/util/Base64.html#mime) encoded binary values.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is base64 how connect also serializes bytes types?

I wonder if we should have (not on this PR) an encoding method as part of the column type, i.e. create stream s1 (b bytes encoding 'hex'). It might even work with strings when we want them utf-16 or utf-8. Just throwing ideas here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For JSON yes. For Delimited, it's configurable, but the default is UTF_8. I'd still rather have them be the same though.

Yeah I think that would be nice to have if the all the SR serializer encodings become configurable.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could make our own serializers for this. Anyway, it is a nice suggestion for the future.

This is because ksqlDB and Schema Registry both use Jackson to serialize and deserialize JSON,
and Jackson serializes binaries to Base64 strings.

The ksqlDB JSON and delimited deserializers will be updated to convert Base64 strings to ByteBuffer.

### Casting

Casting BYTES to STRING will convert the BYTES value to a STRING value encoded by the format specified
by `ksql.bytes.format`. Casting STRING to BYTES will convert the STRING to a BYTES value decoded by the
format specified by `ksql.bytes.format`.

Some other alternatives are:
* Use UTF-8 for all casts, and throw if it fails (BigQuery does this)
* Not support casting. This would make BYTES the only data type that cannot be cast to STRING.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I vote for not supporting casting. Instead, users can use conversion functions with the specified format. Easier than handling configs.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That sounds good to me.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated KLIP to remove support for casting


### Comparisons

Comparisons will only be allowed between two BYTES. They will be compared lexicographically by
unsigned 8-bit values. For example, the following comparisons evaluate to `TRUE`:

```
[10, 11] > [10]
[12] > [10, 11]
```

## Test plan

There will need to be tests for the following:
* Integration with Kafka Connect and Schema Registry
* All serialization formats
* Different types of byte data
* QTTs with all of the new and updated UDFs

## LOEs and Delivery Milestones

The implementation can both be broken up as follows:
* Adding the BYTES type to ksqlDB - 2 days
* Serialization/deserialization - 4 days
* Documentation - 2 days
* Add to Connect integration test - 1 day
* Casting - 2 days
* Comparisons - 2 days
* Adding UDFs + documentation - 1 week
* Buffer time and manual testing - 3 days

## Documentation Updates

* Add and update UDFs to `docs/developer-guide/ksqldb-reference/scalar-functions.md`
* Serialization/deserialization information in `docs/reference/serialization.md`
* Section on casting in `docs/developer-guide/ksqldb-reference/type-coercion.md`
* Detailed description of `BYTES` in `docs/reference/sql/data-types.md`
* New section in `docs/developer-guide/ksqldb-reference/operations.md` for comparisons

## Compatibility Implications

If a user issues a command that includes the BYTES type, then previous versions of KSQL will not
recognize the BYTES type, and the server will enter a DEGRADED state.

## Security Implications

None