Skip to content

Commit

Permalink
[Fix] Fix Debezium format cannot parse date/time/timestamp
Browse files Browse the repository at this point in the history
  • Loading branch information
Hisoka-X committed Nov 21, 2023
1 parent c6b03e9 commit 66a8eaa
Show file tree
Hide file tree
Showing 12 changed files with 577 additions and 78 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,10 @@ public TableIdentifier getTableId() {
return tableId;
}

public TablePath getTablePath() {
return tableId.toTablePath();
}

public TableSchema getTableSchema() {
return tableSchema;
}
Expand Down

Large diffs are not rendered by default.

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -38,10 +38,48 @@ source {
format = debezium_json
schema = {
fields {
id = "int"
name = "string"
description = "string"
weight = "float"
id = "int"
f_binary = "bytes"
f_blob = "bytes"
f_long_varbinary = "bytes"
f_longblob = "bytes"
f_tinyblob = "bytes"
f_varbinary = "string"
f_smallint = "smallint"
f_smallint_unsigned = "int"
f_mediumint = "int"
f_mediumint_unsigned = "int"
f_int = "int"
f_int_unsigned = "bigint"
f_integer = "int"
f_integer_unsigned = "bigint"
f_bigint = "bigint"
f_bigint_unsigned = "decimal(10, 0)"
f_numeric = "decimal(10, 0)"
f_decimal = "decimal(10, 0)"
f_float = "float"
f_double = "double"
f_double_precision = "double"
f_longtext = "string"
f_mediumtext = "string"
f_text = "string"
f_tinytext = "string"
f_varchar = "string"
f_date = "date"
f_datetime = "timestamp"
f_timestamp = "timestamp"
f_bit1 = "boolean"
f_bit64 = "tinyint"
f_char = "string"
f_enum = "string"
f_mediumblob = "bytes"
f_long_varchar = "string"
f_real = "double"
f_time = "time"
f_tinyint = "tinyint"
f_tinyint_unsigned = "int"
f_json = "string"
f_year = "int"
}
}
}
Expand All @@ -55,7 +93,7 @@ sink {
password = test
generate_sink_sql = true
database = test
table = public.sink
table = public.sink2
primary_keys = ["id"]
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,9 +39,47 @@ source {
schema = {
fields {
id = "int"
name = "string"
description = "string"
weight = "float"
f_binary = "bytes"
f_blob = "bytes"
f_long_varbinary = "bytes"
f_longblob = "bytes"
f_tinyblob = "bytes"
f_varbinary = "string"
f_smallint = "smallint"
f_smallint_unsigned = "int"
f_mediumint = "int"
f_mediumint_unsigned = "int"
f_int = "int"
f_int_unsigned = "bigint"
f_integer = "int"
f_integer_unsigned = "bigint"
f_bigint = "bigint"
f_bigint_unsigned = "decimal(10, 0)"
f_numeric = "decimal(10, 0)"
f_decimal = "decimal(10, 0)"
f_float = "float"
f_double = "double"
f_double_precision = "double"
f_longtext = "string"
f_mediumtext = "string"
f_text = "string"
f_tinytext = "string"
f_varchar = "string"
f_date = "date"
f_datetime = "timestamp"
f_timestamp = "timestamp"
f_bit1 = "boolean"
f_bit64 = "tinyint"
f_char = "string"
f_enum = "string"
f_mediumblob = "bytes"
f_long_varchar = "string"
f_real = "double"
f_time = "time"
f_tinyint = "tinyint"
f_tinyint_unsigned = "int"
f_json = "string"
f_year = "int"
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,8 +60,7 @@ public SeaTunnelRow deserialize(SourceRecord record)
String key = debeziumJsonConverter.serializeKey(record);
String value = debeziumJsonConverter.serializeValue(record);
Object[] fields = new Object[] {record.topic(), key, value};
SeaTunnelRow row = new SeaTunnelRow(fields);
return row;
return new SeaTunnelRow(fields);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@

@RequiredArgsConstructor
public class CompatibleDebeziumJsonSerializationSchema implements SerializationSchema {
public static final String IDENTIFIER = CompatibleDebeziumJsonDeserializationSchema.IDENTIFIER;

private final int index;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@ public class DebeziumJsonConverter implements Serializable {

private final boolean keySchemaEnable;
private final boolean valueSchemaEnable;
private transient JsonConverter keyConverter;
private transient JsonConverter valueConverter;
private transient volatile JsonConverter keyConverter;
private transient volatile JsonConverter valueConverter;
private transient Method keyConverterMethod;
private transient Method valueConverterMethod;

Expand Down
2 changes: 0 additions & 2 deletions seatunnel-formats/seatunnel-format-json/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,6 @@
<artifactId>seatunnel-format-json</artifactId>
<name>SeaTunnel : Formats : Json</name>

<properties />

<dependencies>

<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,10 @@
import org.apache.seatunnel.format.json.JsonDeserializationSchema;
import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException;

import com.fasterxml.jackson.core.JsonProcessingException;

import java.io.IOException;
import java.lang.reflect.InvocationTargetException;

public class DebeziumJsonDeserializationSchema implements DeserializationSchema<SeaTunnelRow> {
private static final long serialVersionUID = 1L;
Expand All @@ -48,6 +51,8 @@ public class DebeziumJsonDeserializationSchema implements DeserializationSchema<

private final JsonDeserializationSchema jsonDeserializer;

private final DebeziumRowConverter debeziumRowConverter;

private final boolean ignoreParseErrors;

private final boolean debeziumEnabledSchema;
Expand All @@ -57,6 +62,7 @@ public DebeziumJsonDeserializationSchema(SeaTunnelRowType rowType, boolean ignor
this.ignoreParseErrors = ignoreParseErrors;
this.jsonDeserializer =
new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType));
this.debeziumRowConverter = new DebeziumRowConverter(rowType);
this.debeziumEnabledSchema = false;
}

Expand All @@ -66,6 +72,7 @@ public DebeziumJsonDeserializationSchema(
this.ignoreParseErrors = ignoreParseErrors;
this.jsonDeserializer =
new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType));
this.debeziumRowConverter = new DebeziumRowConverter(rowType);
this.debeziumEnabledSchema = debeziumEnabledSchema;
}

Expand Down Expand Up @@ -153,8 +160,9 @@ private JsonNode convertBytes(byte[] message) {
}
}

private SeaTunnelRow convertJsonNode(JsonNode root) {
return jsonDeserializer.convertToRowData(root);
private SeaTunnelRow convertJsonNode(JsonNode root)
throws InvocationTargetException, IllegalAccessException, JsonProcessingException {
return debeziumRowConverter.serializeValue(root.toString());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.seatunnel.format.json.debezium;

import org.apache.seatunnel.api.configuration.util.OptionRule;
import org.apache.seatunnel.api.serialization.DeserializationSchema;
import org.apache.seatunnel.api.serialization.SerializationSchema;
import org.apache.seatunnel.api.table.connector.DeserializationFormat;
import org.apache.seatunnel.api.table.connector.SerializationFormat;
Expand Down Expand Up @@ -58,13 +57,6 @@ public DeserializationFormat createDeserializationFormat(TableFactoryContext con
Map<String, String> options = context.getOptions().toMap();
boolean ignoreParseErrors = DebeziumJsonFormatOptions.getIgnoreParseErrors(options);
boolean schemaInclude = DebeziumJsonFormatOptions.getSchemaInclude(options);

// TODO config SeaTunnelRowType
return new DeserializationFormat() {
@Override
public DeserializationSchema createDeserializationSchema() {
return new DebeziumJsonDeserializationSchema(null, ignoreParseErrors);
}
};
return () -> new DebeziumJsonDeserializationSchema(null, ignoreParseErrors);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
/*
* 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.format.json.debezium;

import org.apache.seatunnel.api.table.type.ArrayType;
import org.apache.seatunnel.api.table.type.DecimalType;
import org.apache.seatunnel.api.table.type.MapType;
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.api.table.type.SqlType;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;

import java.io.IOException;
import java.io.Serializable;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.ZoneOffset;
import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;

public class DebeziumRowConverter implements Serializable {

private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private final SeaTunnelRowType rowType;
public DebeziumRowConverter(SeaTunnelRowType rowType) {
this.rowType = rowType;
}

public SeaTunnelRow serializeValue(String json) throws JsonProcessingException {
JsonNode node = OBJECT_MAPPER.readTree(json);
return (SeaTunnelRow) getValue(rowType, node);
}

private Object getValue(SeaTunnelDataType<?> dataType, JsonNode value) {
SqlType sqlType = dataType.getSqlType();
if (value == null) {
return null;
}
switch (sqlType) {
case BOOLEAN:
return value.booleanValue();
case TINYINT:
return (byte) value.intValue();
case SMALLINT:
return (short) value.intValue();
case INT:
return value.intValue();
case BIGINT:
return value.longValue();
case FLOAT:
return value.floatValue();
case DOUBLE:
return value.doubleValue();
case DECIMAL:
if (value.isNumber()) {
return value.decimalValue();
}
if (value.isBinary() || value.isTextual()) {
try {
return new BigDecimal(
new BigInteger(value.binaryValue()),
((DecimalType) dataType).getScale());
} catch (Exception e) {
throw new RuntimeException("Invalid bytes for Decimal field", e);
}
}
case STRING:
return value.textValue();
case BYTES:
try {
return value.binaryValue();
} catch (IOException e) {
throw new RuntimeException("Invalid bytes field", e);
}
case DATE:
return LocalDate.ofEpochDay(value.intValue());
case TIME:
return LocalTime.ofNanoOfDay(value.longValue() * 1000L);
case TIMESTAMP:
try {
long timestamp = Long.parseLong(value.toString());
return LocalDateTime.ofInstant(Instant.ofEpochMilli(timestamp), ZoneOffset.UTC);
} catch (NumberFormatException e) {
return LocalDateTime.parse(
value.textValue(),
DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'"));
}
case ARRAY:
List<Object> arrayValue = new ArrayList<>();
for (JsonNode o : value) {
arrayValue.add(getValue(((ArrayType) dataType).getElementType(), o));
}
return arrayValue;
case MAP:
Map<Object, Object> mapValue = new LinkedHashMap<>();
for (Iterator<Map.Entry<String, JsonNode>> it = value.fields(); it.hasNext(); ) {
Map.Entry<String, JsonNode> entry = it.next();
mapValue.put(
entry.getKey(),
getValue(((MapType) dataType).getValueType(), entry.getValue()));
}
return mapValue;
case ROW:
SeaTunnelRowType rowType = (SeaTunnelRowType) dataType;
SeaTunnelRow row = new SeaTunnelRow(rowType.getTotalFields());
for (int i = 0; i < rowType.getTotalFields(); i++) {
row.setField(
i,
getValue(rowType.getFieldType(i), value.get(rowType.getFieldName(i))));
}
return row;
default:
throw new UnsupportedOperationException("Unsupported type: " + sqlType);
}
}
}

0 comments on commit 66a8eaa

Please sign in to comment.