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

[Fix][Connector-V2] Fix cdc use default value when value is null #7950

Merged
merged 2 commits into from
Oct 31, 2024
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 @@ -85,7 +85,7 @@ public SeaTunnelRow convert(SourceRecord record, Struct struct, Schema schema)
if (field == null) {
row.setField(i, null);
} else {
Object fieldValue = struct.get(fieldName);
Object fieldValue = struct.getWithoutDefault(fieldName);
Schema fieldSchema = field.schema();
Object convertedField =
SeaTunnelRowDebeziumDeserializationConverters.convertField(
Expand Down Expand Up @@ -494,11 +494,11 @@ public Object convert(Object dbzObj, Schema schema) throws Exception {
SeaTunnelRow row = new SeaTunnelRow(arity);
for (int i = 0; i < arity; i++) {
String fieldName = fieldNames[i];
Object fieldValue = struct.get(fieldName);
Field field = schema.field(fieldName);
if (field == null) {
row.setField(i, null);
} else {
Object fieldValue = struct.getWithoutDefault(fieldName);
Schema fieldSchema = field.schema();
Object convertedField =
SeaTunnelRowDebeziumDeserializationConverters.convertField(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.seatunnel.connectors.cdc.debezium.row;

import org.apache.seatunnel.api.table.type.BasicType;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationConverterFactory;
import org.apache.seatunnel.connectors.cdc.debezium.MetadataConverter;

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.source.SourceRecord;

import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;

import java.time.ZoneId;
import java.util.ArrayList;
import java.util.HashMap;

public class SeaTunnelRowDebeziumDeserializationConvertersTest {

@Test
void testDefaultValueNotUsed() throws Exception {
SeaTunnelRowDebeziumDeserializationConverters converters =
new SeaTunnelRowDebeziumDeserializationConverters(
new SeaTunnelRowType(
new String[] {"id", "name"},
new SeaTunnelDataType[] {
BasicType.INT_TYPE, BasicType.STRING_TYPE
}),
new MetadataConverter[] {},
ZoneId.systemDefault(),
DebeziumDeserializationConverterFactory.DEFAULT);
Schema schema =
SchemaBuilder.struct()
.field("id", SchemaBuilder.int32().build())
.field("name", SchemaBuilder.string().defaultValue("UL"))
.build();
Struct value = new Struct(schema);
// the value of `name` is null, so do not put value for it
value.put("id", 1);
SourceRecord record =
new SourceRecord(
new HashMap<>(),
new HashMap<>(),
"topicName",
null,
SchemaBuilder.int32().build(),
1,
schema,
value,
null,
new ArrayList<>());

SeaTunnelRow row = converters.convert(record, value, schema);
Assertions.assertEquals(row.getField(0), 1);
Assertions.assertNull(row.getField(1));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,24 +19,25 @@
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
import org.apache.seatunnel.api.sink.SinkWriter;
import org.apache.seatunnel.api.sink.SupportMultiTableSink;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink;
import org.apache.seatunnel.connectors.seatunnel.http.config.HttpConfig;
import org.apache.seatunnel.connectors.seatunnel.http.config.HttpParameter;

import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;

public class PrometheusSink extends AbstractSimpleSink<SeaTunnelRow, Void>
implements SupportMultiTableSink {

protected final HttpParameter httpParameter = new HttpParameter();
protected SeaTunnelRowType seaTunnelRowType;
protected CatalogTable catalogTable;
protected ReadonlyConfig pluginConfig;

public PrometheusSink(ReadonlyConfig pluginConfig, SeaTunnelRowType rowType) {
public PrometheusSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) {
this.pluginConfig = pluginConfig;
httpParameter.setUrl(pluginConfig.get(HttpConfig.URL));
if (pluginConfig.getOptional(HttpConfig.HEADERS).isPresent()) {
Expand All @@ -45,7 +46,7 @@ public PrometheusSink(ReadonlyConfig pluginConfig, SeaTunnelRowType rowType) {
if (pluginConfig.getOptional(HttpConfig.PARAMS).isPresent()) {
httpParameter.setHeaders(pluginConfig.get(HttpConfig.PARAMS));
}
this.seaTunnelRowType = rowType;
this.catalogTable = catalogTable;

if (Objects.isNull(httpParameter.getHeaders())) {
Map<String, String> headers = new HashMap<>();
Expand All @@ -67,6 +68,12 @@ public String getPluginName() {

@Override
public PrometheusWriter createWriter(SinkWriter.Context context) {
return new PrometheusWriter(seaTunnelRowType, httpParameter, pluginConfig);
return new PrometheusWriter(
catalogTable.getSeaTunnelRowType(), httpParameter, pluginConfig);
}

@Override
public Optional<CatalogTable> getWriteCatalogTable() {
return Optional.ofNullable(catalogTable);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ public TableSink createSink(TableSinkFactoryContext context) {

ReadonlyConfig readonlyConfig = context.getOptions();
CatalogTable catalogTable = context.getCatalogTable();
return () -> new PrometheusSink(readonlyConfig, catalogTable.getSeaTunnelRowType());
return () -> new PrometheusSink(readonlyConfig, catalogTable);
}

@Override
Expand Down
Loading