Sort Plugin
This article describes how to extend a new source (abstracted as extract node in inlong) or a new sink (abstracted as load node in inlong) in InLong Sort. The architecture of inlong sort can be represented by UML object relation diagram as:
The concepts of each component are:
To extend the extract node or load node, you need to do the following:
- Inherit the node class (such as MyExtractNode) and build specific extract or load usage logic;
- In a specific node class (such as MyExtractNode), specify the corresponding Flink connector;
- Use specific node classes in specific ETL implementation logic (such as MyExtractNode)
In the second step, you can use the existing flick connector or extend it yourself. How to extend the flink connector, please refer to the official flink documentationDataStream Connectors.
Step 1:Inherit the ExtractNode class,the location of the class is:
Specify the connector in the implemented ExtractNode.
@EqualsAndHashCode(callSuper = true)
@JsonTypeName("MongoExtract")
@Data
public class MongoExtractNode extends ExtractNode implements Serializable {
@JsonInclude(Include.NON_NULL)
@JsonProperty("primaryKey")
private String primaryKey;
...
@JsonCreator
public MongoExtractNode(@JsonProperty("id") String id, ...) { ... }
@Override
public Map<String, String> tableOptions() {
Map<String, String> options = super.tableOptions();
// configure the specified connector, here is mongodb-cdc
options.put("connector", "mongodb-cdc");
...
}
}
Step 2:add the Extract to JsonSubTypes in ExtractNode and Node
Step 3:Expand the Sort connector and check whether the corresponding connector already exists in the (InLong Agentinlong-sort/sort-connectors/mongodb-cdc
) directory. If you haven’t already, you need to refer to the official flink documentation to extend, directly call the existing flink-connector (such asinlong-sort/sort-connectors/mongodb-cdc
) or implement the related connector by yourself.
There are three steps to extend an LoadNode:
inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/LoadNode.java
specify the connector in the implemented LoadNode.
Step 2:add the Load to JsonSubTypes in ExtractNode and Node
// add field in JsonSubTypes of LoadNode and Node
@JsonSubTypes({
@JsonSubTypes.Type(value = KafkaLoadNode.class, name = "kafkaLoad")
})
...
public abstract class LoadNode implements Node{...}
...
@JsonSubTypes({
@JsonSubTypes.Type(value = KafkaLoadNode.class, name = "kafkaLoad")
})
public interface Node {...}
Step 3:Extend the Sort connector, Kafka’s sort connector is in inlong-sort/sort-connectors/kafka
.
To integrate extract and load into the InLong Sort mainstream, you need to implement the semantics mentioned in the overview section: group, stream, node, etc. The entry class of InLong Sort is in :
How to integrate extract and load into InLong Sort can refer to the following ut. First, build the corresponding extractnode and loadnode, then build noderelation, streaminfo and groupinfo, and finally use FlinkSqlParser to execute.
public class MongoExtractToKafkaLoad extends AbstractTestBase {
// create MongoExtractNode
private MongoExtractNode buildMongoNode() {
List<FieldInfo> fields = Arrays.asList(new FieldInfo("name", new StringFormatInfo()), ...);
return new MongoExtractNode(..., fields, ...);
}
// create KafkaLoadNode
private KafkaLoadNode buildAllMigrateKafkaNode() {
List<FieldRelation> relations = Arrays.asList(new FieldRelation(new FieldInfo("name", new StringFormatInfo()), ...), ...);
CsvFormat csvFormat = new CsvFormat();
}
// create NodeRelation
private NodeRelation buildNodeRelation(List<Node> inputs, List<Node> outputs) {
List<String> inputIds = inputs.stream().map(Node::getId).collect(Collectors.toList());
List<String> outputIds = outputs.stream().map(Node::getId).collect(Collectors.toList());
return new NodeRelation(inputIds, outputIds);
}
// test the main flow: mongodb to kafka
@Test
public void testMongoDbToKafka() throws Exception {
EnvironmentSettings settings = EnvironmentSettings. ... .build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
...
StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings);
Node inputNode = buildMongoNode();
Node outputNode = buildAllMigrateKafkaNode();
StreamInfo streamInfo = new StreamInfo("1", Arrays.asList(inputNode, outputNode), ...);
GroupInfo groupInfo = new GroupInfo("1", Collections.singletonList(streamInfo));
FlinkSqlParser parser = FlinkSqlParser.getInstance(tableEnv, groupInfo);
ParseResult result = parser.parse();
Assert.assertTrue(result.tryExecute());
}