Skip to content

Commit

Permalink
Merge branch 'master' into database-sync-single-sink
Browse files Browse the repository at this point in the history
# Conflicts:
#	flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/JsonDebeziumSchemaSerializer.java
  • Loading branch information
wudi committed Nov 29, 2023
2 parents 03542e0 + ba280cc commit 3868071
Show file tree
Hide file tree
Showing 13 changed files with 139 additions and 56 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,11 @@
import org.apache.doris.flink.cfg.DorisExecutionOptions;
import org.apache.doris.flink.cfg.DorisOptions;
import org.apache.doris.flink.cfg.DorisReadOptions;
import org.apache.doris.flink.sink.writer.serializer.DorisRecordSerializer;
import org.apache.doris.flink.sink.writer.LabelGenerator;
import org.apache.doris.flink.sink.writer.serializer.DorisRecord;
import org.apache.doris.flink.sink.writer.serializer.DorisRecordSerializer;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.api.connector.sink2.SinkWriter;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.StringUtils;
import org.apache.flink.util.concurrent.ExecutorThreadFactory;
Expand Down Expand Up @@ -93,18 +93,17 @@ public void write(IN in, Context context) throws IOException, InterruptedExcepti
checkFlushException();
String db = this.database;
String tbl = this.table;
Tuple2<String, byte[]> rowTuple = serializer.serialize(in);
if(rowTuple == null || rowTuple.f1 == null){
DorisRecord record = serializer.serialize(in);
if(record == null || record.getRow() == null){
//ddl or value is null
return;
}
//multi table load
if(rowTuple.f0 != null){
String[] tableInfo = rowTuple.f0.split("\\.");
db = tableInfo[0];
tbl = tableInfo[1];
if(record.getTableIdentifier() != null){
db = record.getDatabase();
tbl = record.getTable();
}
batchStreamLoad.writeRecord(db, tbl, rowTuple.f1);
batchStreamLoad.writeRecord(db, tbl, record.getRow());
}
@Override
public void flush(boolean flush) throws IOException, InterruptedException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,12 @@
import org.apache.doris.flink.sink.BackendUtil;
import org.apache.doris.flink.sink.DorisCommittable;
import org.apache.doris.flink.sink.HttpUtil;
import org.apache.doris.flink.sink.writer.serializer.DorisRecord;
import org.apache.doris.flink.sink.writer.serializer.DorisRecordSerializer;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.api.connector.sink2.StatefulSink;
import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
import org.apache.flink.util.concurrent.ExecutorThreadFactory;
import org.slf4j.Logger;
Expand Down Expand Up @@ -152,14 +152,14 @@ public void write(IN in, Context context) throws IOException {
checkLoadException();
String tableKey = dorisOptions.getTableIdentifier();

Tuple2<String, byte[]> rowTuple = serializer.serialize(in);
if(rowTuple == null || rowTuple.f1 == null){
DorisRecord record = serializer.serialize(in);
if(record == null || record.getRow() == null){
//ddl or value is null
return;
}
//multi table load
if(rowTuple.f0 != null){
tableKey = rowTuple.f0;
if(record.getTableIdentifier() != null){
tableKey = record.getTableIdentifier();
}

DorisStreamLoad streamLoader = getStreamLoader(tableKey);
Expand All @@ -171,7 +171,7 @@ public void write(IN in, Context context) throws IOException {
loadingMap.put(tableKey, true);
globalLoading = true;
}
streamLoader.writeRecord(rowTuple.f1);
streamLoader.writeRecord(record.getRow());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
// 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.doris.flink.sink.writer.serializer;

import java.io.Serializable;

public class DorisRecord implements Serializable {
private String database;
private String table;
private byte[] row;

public DorisRecord() {
}

public DorisRecord(String database, String table, byte[] row) {
this.database = database;
this.table = table;
this.row = row;
}

public String getTableIdentifier(){
if(database == null || table == null){
return null;
}
return database + "." + table;
}

public String getDatabase() {
return database;
}

public void setDatabase(String database) {
this.database = database;
}

public String getTable() {
return table;
}

public void setTable(String table) {
this.table = table;
}

public byte[] getRow() {
return row;
}

public void setRow(byte[] row) {
this.row = row;
}

public static DorisRecord of(String database, String table, byte[] row) {
return new DorisRecord(database, table, row);
}

public static DorisRecord of(String tableIdentifier, byte[] row) {
if(tableIdentifier != null) {
String[] dbTbl = tableIdentifier.split("\\.");
if(dbTbl.length == 2){
String database = dbTbl[0];
String table = dbTbl[1];
return new DorisRecord(database, table, row);
}
}
return null;
}

public static DorisRecord of(byte[] row) {
return new DorisRecord(null, null, row);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.doris.flink.sink.writer.serializer;

import org.apache.flink.api.java.tuple.Tuple2;

import java.io.IOException;
import java.io.Serializable;

Expand All @@ -34,6 +32,6 @@ public interface DorisRecordSerializer<T> extends Serializable {
* @return [tableIdentifer,byte array]
* @throws IOException
*/
Tuple2<String, byte[]> serialize(T record) throws IOException;
DorisRecord serialize(T record) throws IOException;

}
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ public JsonDebeziumSchemaSerializer(DorisOptions dorisOptions,
}

@Override
public Tuple2<String, byte[]> serialize(String record) throws IOException {
public DorisRecord serialize(String record) throws IOException {
LOG.debug("received debezium json data {} :", record);
JsonNode recordRoot = objectMapper.readValue(record, JsonNode.class);

Expand Down Expand Up @@ -171,7 +171,7 @@ public Tuple2<String, byte[]> serialize(String record) throws IOException {
addDeleteSign(valueMap, false);
break;
case OP_UPDATE:
return Tuple2.of(null, extractUpdate(recordRoot));
return DorisRecord.of(extractUpdate(recordRoot));
case OP_DELETE:
valueMap = extractBeforeRow(recordRoot);
addDeleteSign(valueMap, true);
Expand All @@ -181,7 +181,7 @@ public Tuple2<String, byte[]> serialize(String record) throws IOException {
return null;
}

return Tuple2.of(dorisTableIdentifier, objectMapper.writeValueAsString(valueMap).getBytes(StandardCharsets.UTF_8));
return DorisRecord.of(dorisTableIdentifier, objectMapper.writeValueAsString(valueMap).getBytes(StandardCharsets.UTF_8));
}

/**
Expand Down Expand Up @@ -471,6 +471,8 @@ public void fillOriginSchema(JsonNode columns) {
}
}
} else {
LOG.error("Current schema change failed! You need to ensure that "
+ "there is data in the table." + dorisOptions.getTableIdentifier());
originFieldSchemaMap = new LinkedHashMap<>();
columns.forEach(column -> buildFieldSchema(originFieldSchemaMap, column));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

import org.apache.commons.lang3.StringUtils;
import org.apache.doris.flink.sink.batch.RecordWithMeta;
import org.apache.flink.api.java.tuple.Tuple2;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -30,15 +29,14 @@ public class RecordWithMetaSerializer implements DorisRecordSerializer<RecordWit
private static final Logger LOG = LoggerFactory.getLogger(RecordWithMetaSerializer.class);

@Override
public Tuple2<String, byte[]> serialize(RecordWithMeta record) throws IOException {
public DorisRecord serialize(RecordWithMeta record) throws IOException {
if(StringUtils.isBlank(record.getTable())
|| StringUtils.isBlank(record.getDatabase())
|| record.getRecord() == null){
LOG.warn("Record or meta format is incorrect, ignore record db:{}, table:{}, row:{}",
record.getDatabase(), record.getTable(), record.getRecord());
return null;
}
String tableKey = record.getDatabase() + "." + record.getTable();
return Tuple2.of(tableKey, record.getRecord().getBytes(StandardCharsets.UTF_8));
return DorisRecord.of(record.getDatabase(), record.getTable(), record.getRecord().getBytes(StandardCharsets.UTF_8));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.doris.flink.deserialization.converter.DorisRowConverter;
import org.apache.doris.flink.sink.EscapeHandler;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.types.RowKind;
Expand Down Expand Up @@ -60,7 +59,7 @@ private RowDataSerializer(String[] fieldNames, DataType[] dataTypes, String type
}

@Override
public Tuple2<String, byte[]> serialize(RowData record) throws IOException{
public DorisRecord serialize(RowData record) throws IOException{
int maxIndex = Math.min(record.getArity(), fieldNames.length);
String valString;
if (JSON.equals(type)) {
Expand All @@ -70,7 +69,7 @@ public Tuple2<String, byte[]> serialize(RowData record) throws IOException{
} else {
throw new IllegalArgumentException("The type " + type + " is not supported!");
}
return Tuple2.of(null, valString.getBytes(StandardCharsets.UTF_8));
return DorisRecord.of(valString.getBytes(StandardCharsets.UTF_8));
}

public String buildJsonString(RowData record, int maxIndex) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.doris.flink.sink.writer.serializer;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.conversion.RowRowConverter;
Expand Down Expand Up @@ -55,7 +54,7 @@ private RowSerializer(String[] fieldNames, DataType[] dataTypes, String type, St
}

@Override
public Tuple2<String, byte[]> serialize(Row record) throws IOException{
public DorisRecord serialize(Row record) throws IOException{
RowData rowDataRecord = this.rowRowConverter.toInternal(record);
return this.rowDataSerializer.serialize(rowDataRecord);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
public class SimpleStringSerializer implements DorisRecordSerializer<String> {

@Override
public Tuple2<String, byte[]> serialize(String record) throws IOException {
return Tuple2.of(null, record.getBytes(StandardCharsets.UTF_8));
public DorisRecord serialize(String record) throws IOException {
return DorisRecord.of(record.getBytes(StandardCharsets.UTF_8));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public void testConvert() throws IOException {
.setFieldDelimiter("|")
.setFieldNames(new String[]{"f1","f2","f3","f4","f5","f6","f7","f8","f9","f10","f11","f12","f13","f14","f15","f16"})
.build();
String s = new String(serializer.serialize(rowData).f1);
String s = new String(serializer.serialize(rowData).getRow());
Assert.assertEquals("\\N|true|1.2|1.2345|24|10|1|32|64|128|10.12|2021-01-01 08:00:00.0|2021-01-01 08:00:00.0|2021-01-01|a|doris", s);
}

Expand Down
Loading

0 comments on commit 3868071

Please sign in to comment.