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

[Improve][seatunnel-format-text] Support read & write SeaTunnelRow type #2969

Merged
merged 3 commits into from
Oct 5, 2022
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 @@ -56,5 +56,16 @@ public enum Formatter {
public String getValue() {
return value;
}

public static Formatter parse(String format) {
Formatter[] formatters = Formatter.values();
for (Formatter formatter : formatters) {
if (formatter.getValue().equals(format)) {
return formatter;
}
}
String errorMsg = String.format("Illegal format [%s]", format);
throw new IllegalArgumentException(errorMsg);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -52,5 +52,16 @@ public enum Formatter {
public String getValue() {
return value;
}

public static Formatter parse(String format) {
Formatter[] formatters = Formatter.values();
for (Formatter formatter : formatters) {
if (formatter.getValue().equals(format)) {
return formatter;
}
}
String errorMsg = String.format("Illegal format [%s]", format);
throw new IllegalArgumentException(errorMsg);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,5 +50,16 @@ public enum Formatter {
public String getValue() {
return value;
}

public static Formatter parse(String format) {
Formatter[] formatters = Formatter.values();
for (Formatter formatter : formatters) {
if (formatter.getValue().equals(format)) {
return formatter;
}
}
String errorMsg = String.format("Illegal format [%s]", format);
throw new IllegalArgumentException(errorMsg);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
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 org.apache.seatunnel.common.utils.DateTimeUtils;
import org.apache.seatunnel.common.utils.DateUtils;
import org.apache.seatunnel.common.utils.JsonUtils;
Expand All @@ -32,10 +33,12 @@
import com.fasterxml.jackson.databind.node.ArrayNode;
import lombok.Builder;
import lombok.NonNull;
import org.apache.commons.lang3.StringUtils;

import java.io.IOException;
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedHashMap;
import java.util.Map;

Expand All @@ -55,13 +58,10 @@ public class TextDeserializationSchema implements DeserializationSchema<SeaTunne
@Override
public SeaTunnelRow deserialize(byte[] message) throws IOException {
String content = new String(message);
String[] splits = content.split(delimiter);
if (seaTunnelRowType.getTotalFields() != splits.length) {
throw new IndexOutOfBoundsException("The data does not match the configured schema information, please check");
}
Object[] objects = new Object[splits.length];
for (int i = 0; i < splits.length; i++) {
objects[i] = convert(splits[i], seaTunnelRowType.getFieldType(i));
Map<Integer, String> splitsMap = splitLineBySeaTunnelRowType(content, seaTunnelRowType);
Object[] objects = new Object[splitsMap.size()];
TyrantLucifer marked this conversation as resolved.
Show resolved Hide resolved
for (int i = 0; i < objects.length; i++) {
objects[i] = convert(splitsMap.get(i), seaTunnelRowType.getFieldType(i));
}
return new SeaTunnelRow(objects);
}
Expand All @@ -71,7 +71,30 @@ public SeaTunnelDataType<SeaTunnelRow> getProducedType() {
return seaTunnelRowType;
}

private Map<Integer, String> splitLineBySeaTunnelRowType(String line, SeaTunnelRowType seaTunnelRowType) {
String[] splits = line.split(delimiter, -1);
LinkedHashMap<Integer, String> splitsMap = new LinkedHashMap<>();
SeaTunnelDataType<?>[] fieldTypes = seaTunnelRowType.getFieldTypes();
int cursor = 0;
for (int i = 0; i < fieldTypes.length; i++) {
if (fieldTypes[i].getSqlType() == SqlType.ROW) {
// row type
int totalFields = ((SeaTunnelRowType) fieldTypes[i]).getTotalFields();
ArrayList<String> rowSplits = new ArrayList<>(Arrays.asList(splits).subList(cursor, cursor + totalFields));
splitsMap.put(i, String.join(delimiter, rowSplits));
cursor += totalFields;
} else {
// not row type
splitsMap.put(i, splits[cursor++]);
}
}
return splitsMap;
}

private Object convert(String field, SeaTunnelDataType<?> fieldType) {
if (StringUtils.isBlank(field)) {
return null;
}
switch (fieldType.getSqlType()) {
case ARRAY:
BasicType<?> elementType = ((ArrayType<?, ?>) fieldType).getElementType();
Expand Down Expand Up @@ -134,8 +157,15 @@ private Object convert(String field, SeaTunnelDataType<?> fieldType) {
return TimeUtils.parse(field, timeFormatter);
case TIMESTAMP:
return DateTimeUtils.parse(field, dateTimeFormatter);
case ROW:
Map<Integer, String> splitsMap = splitLineBySeaTunnelRowType(field, (SeaTunnelRowType) fieldType);
Object[] objects = new Object[splitsMap.size()];
for (int i = 0; i < objects.length; i++) {
objects[i] = convert(splitsMap.get(i), ((SeaTunnelRowType) fieldType).getFieldType(i));
}
return new SeaTunnelRow(objects);
default:
throw new UnsupportedOperationException("SeaTunnel format text not supported for parsing [SeaTunnelRow] type");
throw new UnsupportedOperationException("SeaTunnel format text not supported for parsing this type");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@

import lombok.Builder;
import lombok.NonNull;
import org.apache.commons.lang3.StringUtils;

import java.time.LocalDate;
import java.time.LocalDateTime;
Expand Down Expand Up @@ -57,10 +56,13 @@ public byte[] serialize(SeaTunnelRow element) {
for (int i = 0; i < fields.length; i++) {
strings[i] = convert(fields[i], seaTunnelRowType.getFieldType(i));
}
return StringUtils.join(strings, delimiter).getBytes();
return String.join(delimiter, strings).getBytes();
}

private String convert(Object field, SeaTunnelDataType<?> fieldType) {
if (field == null) {
return "";
}
switch (fieldType.getSqlType()) {
case ARRAY:
case MAP:
Expand All @@ -85,8 +87,15 @@ private String convert(Object field, SeaTunnelDataType<?> fieldType) {
return "";
case BYTES:
return new String((byte[]) field);
case ROW:
Object[] fields = ((SeaTunnelRow) field).getFields();
String[] strings = new String[fields.length];
for (int i = 0; i < fields.length; i++) {
strings[i] = convert(fields[i], ((SeaTunnelRowType) fieldType).getFieldType(i));
}
return String.join(delimiter, strings);
default:
throw new UnsupportedOperationException("SeaTunnel format text not supported for parsing [SeaTunnelRow] type");
throw new UnsupportedOperationException("SeaTunnel format text not supported for parsing this type");
}
}
}