Skip to content

Commit

Permalink
Implemented record simplified new state extractor
Browse files Browse the repository at this point in the history
This extractor transforms record in a similar way to
io.debezium.transforms.ExtractNewRecordState, but does not
differentiate between NULL and no value. Thanks to this, values
don't need to be packed in a 1-field struct, and can be represented
directly instead. This transformer makes the connector compatibile
with Elasticsearch Sink - quick setup is documented in
`QUICKSTART-ELASTICSEARCH-INTEGRATION.md` file.
  • Loading branch information
Lorak-mmk authored Aug 17, 2021
1 parent 7fd425f commit ecbeb1d
Show file tree
Hide file tree
Showing 6 changed files with 226 additions and 0 deletions.
66 changes: 66 additions & 0 deletions QUICKSTART-ELASTICSEARCH-INTEGRATION.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
# Connecting Scylla to Elasticsearch

Scylla CDC Source Connector can be paired with Elasticsearch Sink, allowing you to easily search trough your database. This quickstart will show you how to achieve that.

## Prerequisites

This guide assumes that you have already set up Scylla and CDC Source Connector - instructions for that are available in `README-QUICKSTART.md`.

## Source connector configuration

In order for the connector to be compatible with Elasticsearch Sink (and other sinks) you need to use a `ScyllaExtractNewState` transformer. This transformer is described in more detail in [README.md](README.md) file in "`ScyllaExtractNewState` transformer" section.

1. Open the Confluent Control Center. By default, it is started at port `9021`:
![Confluent Control Center main page](images/scylla-cdc-source-connector-control-center1.png)

2. Click on the cluster you want to start the connector in and open the "Connect" tab:
![Confluent Control Center "Connect" tab](images/scylla-cdc-source-connector-control-center2.png)

3. Click on the Kafka Connect cluster:
![Confluent Control Center "connect-default" cluster](images/scylla-cdc-source-connector-control-center3.png)

4. Select CDC source connector that you want to use:
![Confluent Control Center select connector](images/scylla-cdc-source-connector-control-center8.png)

5. Go to "Settings" tab:
![Confluent Control Center "settings" tab](images/scylla-cdc-source-connector-control-center9.png)

6. Add the transformation. In the "Transforms" field add a transformation, with any name. After doing that, a new section, named `Transforms: YourNameHere` will appear. In that section, from drop-down list, select class name `com.scylladb.cdc.debezium.connector.transforms.ScyllaExtractNewRecordState`.

7. Click "Next" and then "Launch".

## Running Elasticsearch

Now you need to run Elasticsearch instance. The quickest way to do this is to use docker, as described in [this guide](https://www.elastic.co/guide/en/elasticsearch/reference/current/docker.html). There is one small change required: when run with provided command, Elasticsearch runs out of memory after a short while and crashes. You should instead use command below:

```docker run -p 9200:9200 -p 9300:9300 -e "discovery.type=single-node" -e "bootstrap_memory_lock=true" -e "ES_JAVA_OPTS=-Xms512m -Xmx512m" --ulimit memlock=-1:-1 docker.elastic.co/elasticsearch/elasticsearch:7.14.0```

## Configure Elasticsearch Sink connector

Now comes the final part of setup: connecting CDC Source to Elasticsearch.

1. Install sink connector using Confluent hub: `confluent-hub install confluentinc/kafka-connect-elasticsearch:latest`

2. Open the Confluent Control Center. By default, it is started at port `9021`:
![Confluent Control Center main page](images/scylla-cdc-source-connector-control-center1.png)

3. Click on the cluster you want to start the connector in and open the "Connect" tab:
![Confluent Control Center "Connect" tab](images/scylla-cdc-source-connector-control-center2.png)

4. Click on the Kafka Connect cluster:
![Confluent Control Center "connect-default" cluster](images/scylla-cdc-source-connector-control-center3.png)

5. Click "Add connector":
![Confluent Control Center "Add connector"](images/scylla-cdc-source-connector-control-center10.png)

6. Click "Elasticsearch Sink Connector".

7. Configure the connector:

1. Input the topic name. In our example it would be `QuickstartConnectorNamespace.ks.t`.
2. Input any name you want in the "Name" field.
3. "Key converter class" must be set to `org.apache.kafka.connect.storage.StringConverter`, because Elasticsearch requires document ids to be numbers or strings. This will unfortunately produce pretty ugly looking ids - you can fix that with custom transformer.
4. "Value converter class" must be set to the same value as in the source connector.
5. Set "Connection URLs" in "Connector" section to proper Elasticsearch URL - in our example it would be `http://localhost:9200`

8. Test that everything works. Insert some data to the table, you should see it in Elasticsearch. Delete some rows, they should disappear from Elasticsearch.
80 changes: 80 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,86 @@ If the operation did not modify the `v` column, the data event will contain the

See `UPDATE` example for full data change event's value.

#### ScyllaExtractNewState transformer
Connector provides one single message transformation (SMT), `ScyllaExtractNewState` (class: `com.scylladb.cdc.debezium.connector.transforms.ScyllaExtractNewState`).
This SMT works like exactly like `io.debezium.transforms.ExtractNewRecordState` (in fact it is called underneath), but also flattens structure by extracting values from aforementioned single-field structures.
Such transformation makes message structure simpler (and easier to use with e.g. Elasticsearch), but it makes it impossible to differentiate between NULL value and non-modification.
If the message is as following:
```json
{
"schema": {
"type": "struct",
"fields": [
{
"type": "int32",
"optional": true,
"field": "ck"
},
{
"type": "int32",
"optional": true,
"field": "pk"
},
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": true,
"field": "value"
}
],
"optional": true,
"name": "NS2.ks.t.v.Cell",
"field": "v"
}
],
"optional": false,
"name": "NS2.ks.t.After"
},
"payload": {
"ck": 2,
"pk": 20,
"v": {
"value": 3
}
}
}
```
then the same message transformed by `ScyllaExtractNewState` would be:
```json
{
"schema": {
"type": "struct",
"fields": [
{
"type": "int32",
"optional": true,
"field": "ck"
},
{
"type": "int32",
"optional": true,
"field": "pk"
},
{
"type": "int32",
"optional": true,
"field": "v"
}
],
"optional": false,
"name": "NS2.ks.t.After"
},
"payload": {
"ck": 2,
"pk": 20,
"v": 3
}
}
```
Notice how `v` field is no longer packed in `value`.

### `INSERT` example
Given this Scylla table and `INSERT` operation:
```
Expand Down
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
package com.scylladb.cdc.debezium.connector.transforms;

import io.debezium.transforms.ExtractNewRecordState;

import org.apache.kafka.common.cache.Cache;
import org.apache.kafka.common.cache.LRUCache;
import org.apache.kafka.common.cache.SynchronizedCache;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.transforms.util.SchemaUtil;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Schema.Type;

public class ScyllaExtractNewRecordState<R extends ConnectRecord<R>> extends ExtractNewRecordState<R> {
private Cache<Schema, Schema> schemaUpdateCache = new SynchronizedCache<>(new LRUCache<>(16));

@Override
public R apply(final R record) {
final R ret = super.apply(record);
if (ret == null || !(ret.value() instanceof Struct)) {
return ret;
}

final Struct value = (Struct)ret.value();

Schema updatedSchema = schemaUpdateCache.get(value.schema());
if (updatedSchema == null) {
updatedSchema = makeUpdatedSchema(value.schema());
schemaUpdateCache.put(value.schema(), updatedSchema);
}

final Struct updatedValue = new Struct(updatedSchema);

for (Field field : value.schema().fields()) {
if (isSimplifiableField(field)) {
Struct fieldValue = (Struct) value.get(field);
updatedValue.put(field.name(), fieldValue == null ? null : fieldValue.get("value"));
} else {
updatedValue.put(field.name(), value.get(field));
}
}

return ret.newRecord(ret.topic(), ret.kafkaPartition(), ret.keySchema(), ret.key(), updatedSchema, updatedValue, ret.timestamp());
}

@Override
public void close() {
super.close();
schemaUpdateCache = null;
}

private boolean isSimplifiableField(Field field) {
if (field.schema().type() != Type.STRUCT) {
return false;
}

if (field.schema().fields().size() != 1
|| field.schema().fields().get(0).name() != "value") {
return false;
}

return true;
}

private Schema makeUpdatedSchema(Schema schema) {
final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct());

for (Field field : schema.fields()) {
if (isSimplifiableField(field)) {
builder.field(field.name(), field.schema().field("value").schema());
} else {
builder.field(field.name(), field.schema());
}
}

return builder.build();
}
}

0 comments on commit ecbeb1d

Please sign in to comment.