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

DBZ-8782 Allow key and value format of type Connect #162

Merged
merged 1 commit into from
Mar 13, 2025
Merged
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@

import io.debezium.DebeziumException;
import io.debezium.embedded.ClientProvided;
import io.debezium.embedded.Connect;
import io.debezium.embedded.async.ConvertingAsyncEngineBuilderFactory;
import io.debezium.engine.ChangeEvent;
import io.debezium.engine.DebeziumEngine;
Expand Down Expand Up @@ -95,6 +96,7 @@ public class DebeziumServer {
private static final String FORMAT_PROTOBUF = Protobuf.class.getSimpleName().toLowerCase();
private static final String FORMAT_BINARY = Binary.class.getSimpleName().toLowerCase();
private static final String FORMAT_STRING = SimpleString.class.getSimpleName().toLowerCase();
private static final String FORMAT_CONNECT = Connect.class.getSimpleName().toLowerCase();
private static final String FORMAT_CLIENT_PROVIDED = ClientProvided.class.getSimpleName().toLowerCase();

private static final Pattern SHELL_PROPERTY_NAME_PATTERN = Pattern.compile("^[a-zA-Z0-9_]+_+[a-zA-Z0-9_]+$");
Expand Down Expand Up @@ -230,6 +232,9 @@ else if (FORMAT_BINARY.equals(formatName)) {
else if (FORMAT_STRING.equals(formatName)) {
return SimpleString.class;
}
else if (FORMAT_CONNECT.equalsIgnoreCase(formatName)) {
return Connect.class;
}
else if (FORMAT_CLIENT_PROVIDED.equals(formatName)) {
return ClientProvided.class;
}
Expand All @@ -244,6 +249,9 @@ private Class<?> getHeaderFormat(Config config) {
else if (FORMAT_JSON_BYTE_ARRAY.equals(formatName)) {
return JsonByteArray.class;
}
else if (FORMAT_CONNECT.equals(formatName)) {
return Connect.class;
}
else if (FORMAT_CLIENT_PROVIDED.equals(formatName)) {
return ClientProvided.class;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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.server;

import static org.assertj.core.api.Assertions.assertThat;

import java.time.Duration;

import jakarta.enterprise.event.Observes;
import jakarta.inject.Inject;

import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.source.SourceRecord;
import org.awaitility.Awaitility;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.condition.DisabledIfSystemProperty;
import org.junit.jupiter.api.condition.EnabledIfSystemProperty;

import io.debezium.server.events.ConnectorCompletedEvent;
import io.debezium.server.events.ConnectorStartedEvent;
import io.debezium.testing.testcontainers.PostgresTestResourceLifecycleManager;
import io.debezium.util.Testing;
import io.quarkus.test.common.QuarkusTestResource;
import io.quarkus.test.junit.QuarkusTest;
import io.quarkus.test.junit.TestProfile;

/**
* Integration test that verifies Debezium server is able to deliver records in Kafka Connect format.
*
* @author vjuranek
*/
@QuarkusTest
@TestProfile(DebeziumServerConnectFormatProfile.class)
@QuarkusTestResource(PostgresTestResourceLifecycleManager.class)
@EnabledIfSystemProperty(named = "test.apicurio", matches = "false", disabledReason = "DebeziumServerConfigProvidersIT doesn't run with apicurio profile.")
@DisabledIfSystemProperty(named = "debezium.format.key", matches = "protobuf")
@DisabledIfSystemProperty(named = "debezium.format.value", matches = "protobuf")
public class DebeziumServerConnectFormatIT {

private static final int MESSAGE_COUNT = 4;
@Inject
DebeziumServer server;

{
Testing.Files.delete(TestConfigSource.OFFSET_STORE_PATH);
}

void setupDependencies(@Observes ConnectorStartedEvent event) {
if (!TestConfigSource.isItTest()) {
return;
}

}

void connectorCompleted(@Observes ConnectorCompletedEvent event) throws Exception {
if (!event.isSuccess()) {
throw (Exception) event.getError().get();
}
}

@Test
public void testPostgresWithSourceRecord() throws Exception {
Testing.Print.enable();
final TestConsumer testConsumer = (TestConsumer) server.getConsumer();
Awaitility.await().atMost(Duration.ofSeconds(TestConfigSource.waitForSeconds()))
.until(() -> (testConsumer.getValues().size() >= MESSAGE_COUNT));
assertThat(testConsumer.getValues().size()).isEqualTo(MESSAGE_COUNT);

SourceRecord record = (SourceRecord) testConsumer.getValues().get(MESSAGE_COUNT - 1);
Struct key = (Struct) record.key();
assertThat(key.getInt32("id")).isEqualTo(1004);

Struct after = ((Struct) record.value()).getStruct("after");
assertThat(after.getInt32("id")).isEqualTo(1004);
assertThat(after.getString("first_name")).isEqualTo("Anne");
assertThat(after.getString("last_name")).isEqualTo("Kretchmar");
assertThat(after.getString("email")).isEqualTo("[email protected]");

ConnectHeaders headers = (ConnectHeaders) record.headers();
assertThat(headers.size()).isEqualTo(0);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.server;

import java.util.HashMap;
import java.util.Map;

import io.debezium.embedded.Connect;
import io.quarkus.test.junit.QuarkusTestProfile;

public class DebeziumServerConnectFormatProfile implements QuarkusTestProfile {

@Override
public Map<String, String> getConfigOverrides() {
Map<String, String> config = new HashMap<String, String>();
config.put("debezium.format.key", Connect.class.getSimpleName().toLowerCase());
config.put("debezium.format.value", Connect.class.getSimpleName().toLowerCase());
config.put("debezium.format.header", Connect.class.getSimpleName().toLowerCase());
return config;
}

}