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

[Bug][Connectors] Text And Json WriteStrategy lost the sinkColumnsIndexInRow #3863

Merged
merged 2 commits into from
Jan 18, 2023
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 @@ -22,6 +22,7 @@
import static org.apache.parquet.avro.AvroWriteSupport.WRITE_FIXED_AS_INT96;
import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE;

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.common.Constants;
Expand Down Expand Up @@ -113,6 +114,20 @@ public void write(SeaTunnelRow seaTunnelRow) throws FileConnectorException {
currentBatchSize++;
}

protected SeaTunnelRowType buildSchemaWithRowType(SeaTunnelRowType seaTunnelRowType, List<Integer> sinkColumnsIndex) {
SeaTunnelDataType<?>[] fieldTypes = seaTunnelRowType.getFieldTypes();
String[] fieldNames = seaTunnelRowType.getFieldNames();
List<String> newFieldNames = new ArrayList<>();
List<SeaTunnelDataType<?>> newFieldTypes = new ArrayList<>();
sinkColumnsIndex.forEach(index -> {
newFieldNames.add(fieldNames[index]);
newFieldTypes.add(fieldTypes[index]);
});
return new SeaTunnelRowType(
newFieldNames.toArray(new String[0]),
newFieldTypes.toArray(new SeaTunnelDataType[0]));
}

/**
* use hadoop conf generate hadoop configuration
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,8 @@ public JsonWriteStrategy(FileSinkConfig textFileSinkConfig) {
@Override
public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) {
super.setSeaTunnelRowTypeInfo(seaTunnelRowType);
this.serializationSchema = new JsonSerializationSchema(seaTunnelRowType);
this.serializationSchema = new JsonSerializationSchema(
buildSchemaWithRowType(seaTunnelRowType, sinkColumnsIndexInRow));
}

@Override
Expand All @@ -57,7 +58,9 @@ public void write(@NonNull SeaTunnelRow seaTunnelRow) {
String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow);
FSDataOutputStream fsDataOutputStream = getOrCreateOutputStream(filePath);
try {
byte[] rowBytes = serializationSchema.serialize(seaTunnelRow);
byte[] rowBytes = serializationSchema.serialize(seaTunnelRow.copy(sinkColumnsIndexInRow.stream()
.mapToInt(Integer::intValue)
.toArray()));
if (isFirstWrite.get(filePath)) {
isFirstWrite.put(filePath, false);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ public TextWriteStrategy(FileSinkConfig textFileSinkConfig) {
public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) {
super.setSeaTunnelRowTypeInfo(seaTunnelRowType);
this.serializationSchema = TextSerializationSchema.builder()
.seaTunnelRowType(seaTunnelRowType)
.seaTunnelRowType(buildSchemaWithRowType(seaTunnelRowType, sinkColumnsIndexInRow))
.delimiter(fieldDelimiter)
.dateFormatter(dateFormat)
.dateTimeFormatter(dateTimeFormat)
Expand All @@ -85,7 +85,9 @@ public void write(@NonNull SeaTunnelRow seaTunnelRow) {
} else {
fsDataOutputStream.write(rowDelimiter.getBytes());
}
fsDataOutputStream.write(serializationSchema.serialize(seaTunnelRow));
fsDataOutputStream.write(serializationSchema.serialize(seaTunnelRow.copy(sinkColumnsIndexInRow.stream()
.mapToInt(Integer::intValue)
.toArray())));
} catch (IOException e) {
throw new FileConnectorException(CommonErrorCode.FILE_OPERATION_FAILED,
String.format("Write data to file [%s] failed", filePath), e);
Expand Down