Skip to content

Commit

Permalink
DBZ-5926 Add support for Connect Headers to Debezium Server
Browse files Browse the repository at this point in the history
Add headers support to ChangeEvent
Add SerializationFormat support headers
  • Loading branch information
jeremy-l-ford authored and jpechane committed Jan 18, 2023
1 parent 107f09a commit d775104
Show file tree
Hide file tree
Showing 18 changed files with 542 additions and 73 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -5,10 +5,13 @@
*/
package io.debezium.engine;

import java.util.Collections;
import java.util.List;

import io.debezium.common.annotation.Incubating;

/**
* A data change event with key and value.
* A data change event with key, value, and headers.
*
* @param <K>
* @param <V>
Expand All @@ -20,6 +23,10 @@ public interface ChangeEvent<K, V> {

V value();

default <H> List<Header<H>> headers() {
return Collections.emptyList();
}

/**
* @return A name of the logical destination for which the event is intended
*/
Expand Down
47 changes: 37 additions & 10 deletions debezium-api/src/main/java/io/debezium/engine/DebeziumEngine.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.debezium.common.annotation.Incubating;
import io.debezium.engine.format.ChangeEventFormat;
import io.debezium.engine.format.KeyValueChangeEventFormat;
import io.debezium.engine.format.KeyValueHeaderChangeEventFormat;
import io.debezium.engine.format.SerializationFormat;
import io.debezium.engine.spi.OffsetCommitPolicy;

Expand Down Expand Up @@ -287,16 +288,27 @@ static <K, V> Builder<ChangeEvent<K, V>> create(Class<? extends SerializationFor
return create(KeyValueChangeEventFormat.of(keyFormat, valueFormat));
}

/**
* Obtain a new {@link Builder} instance that can be used to construct runnable {@link DebeziumEngine} instances.
* Different formats are used for key, value, and headers of emitted change events.
* <p>
* Convenience method, equivalent to calling {@code create(KeyValueChangeEventFormat.of(MyKeyFormat.class, MyValueFormat.class, MyHeaderFormat.class)}.
*
* @return the new builder; never null
*/
static <K, V, H> Builder<ChangeEvent<K, V>> create(Class<? extends SerializationFormat<K>> keyFormat,
Class<? extends SerializationFormat<V>> valueFormat,
Class<? extends SerializationFormat<H>> headerFormat) {
return create(KeyValueHeaderChangeEventFormat.of(keyFormat, valueFormat, headerFormat));
}

static <S, T, K extends SerializationFormat<S>, V extends SerializationFormat<T>> Builder<ChangeEvent<S, T>> create(KeyValueChangeEventFormat<K, V> format) {
final ServiceLoader<BuilderFactory> loader = ServiceLoader.load(BuilderFactory.class);
final Iterator<BuilderFactory> iterator = loader.iterator();
if (!iterator.hasNext()) {
throw new DebeziumException("No implementation of Debezium engine builder was found");
}
final BuilderFactory builder = iterator.next();
if (iterator.hasNext()) {
LoggerFactory.getLogger(Builder.class).warn("More than one Debezium engine builder implementation was found, using {}", builder.getClass());
}
final BuilderFactory builder = determineBuilderFactory();
return builder.builder(format);
}

static <S, T, U, K extends SerializationFormat<S>, V extends SerializationFormat<T>, H extends SerializationFormat<U>> Builder<ChangeEvent<S, T>> create(KeyValueHeaderChangeEventFormat<K, V, H> format) {
final BuilderFactory builder = determineBuilderFactory();
return builder.builder(format);
}

Expand All @@ -307,6 +319,11 @@ static <S, T, K extends SerializationFormat<S>, V extends SerializationFormat<T>
* @return the new builder; never null
*/
static <T, V extends SerializationFormat<T>> Builder<RecordChangeEvent<T>> create(ChangeEventFormat<V> format) {
final BuilderFactory builder = determineBuilderFactory();
return builder.builder(format);
}

private static BuilderFactory determineBuilderFactory() {
final ServiceLoader<BuilderFactory> loader = ServiceLoader.load(BuilderFactory.class);
final Iterator<BuilderFactory> iterator = loader.iterator();
if (!iterator.hasNext()) {
Expand All @@ -316,7 +333,7 @@ static <T, V extends SerializationFormat<T>> Builder<RecordChangeEvent<T>> creat
if (iterator.hasNext()) {
LoggerFactory.getLogger(Builder.class).warn("More than one Debezium engine builder implementation was found, using {}", builder.getClass());
}
return builder.builder(format);
return builder;
}

/**
Expand All @@ -340,5 +357,15 @@ interface BuilderFactory {
* @return this builder object so methods can be chained together; never null
*/
<S, T, K extends SerializationFormat<S>, V extends SerializationFormat<T>> Builder<ChangeEvent<S, T>> builder(KeyValueChangeEventFormat<K, V> format);

/**
* Prescribe the output and header formats to be used by the {@link DebeziumEngine}.
* Usually called by {@link DebeziumEngine#create}.
* @param format
* @return this builder object so methods can be chained together; never null
*/
default <S, T, U, K extends SerializationFormat<S>, V extends SerializationFormat<T>, H extends SerializationFormat<U>> Builder<ChangeEvent<S, T>> builder(KeyValueHeaderChangeEventFormat<K, V, H> format) {
throw new UnsupportedOperationException("Method must be implemented in order to support headers");
}
}
}
23 changes: 23 additions & 0 deletions debezium-api/src/main/java/io/debezium/engine/Header.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.engine;

/**
* Represents a header that contains a key and a value.
*/
public interface Header<T> {

/**
* Key of a header.
*/
String getKey();

/**
* Value of a header.
*/
T getValue();

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.engine.format;

/**
* A {@link SerializationFormat} defining the JSON format serialized as byte[].
*/
public class JsonByteArray implements SerializationFormat<byte[]> {
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.engine.format;

import io.debezium.common.annotation.Incubating;

/**
* Describes a change event output format comprising a key, value, and a header.
*/
@Incubating
public interface KeyValueHeaderChangeEventFormat<K extends SerializationFormat<?>, V extends SerializationFormat<?>, H extends SerializationFormat<?>>
extends KeyValueChangeEventFormat<K, V> {
static <K extends SerializationFormat<?>, V extends SerializationFormat<?>, H extends SerializationFormat<?>> KeyValueHeaderChangeEventFormat<K, V, H> of(
Class<K> keyFormat,
Class<V> valueFormat,
Class<H> headerFormat) {
return new KeyValueHeaderChangeEventFormat<>() {

@Override
public Class<K> getKeyFormat() {
return keyFormat;
}

@Override
public Class<V> getValueFormat() {
return valueFormat;
}

@Override
public Class<H> getHeaderFormat() {
return headerFormat;
}
};
}

Class<H> getHeaderFormat();
}
Loading

0 comments on commit d775104

Please sign in to comment.