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

[Feature][Connector-V2]Support Doris Fe Node HA #29

Merged
merged 9 commits into from
Dec 18, 2024
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 @@ -240,30 +240,15 @@ static String[] parseIdentifier(String tableIdentifier, Logger logger)
}

@VisibleForTesting
public static String randomEndpoint(String feNodes, Logger logger)
throws DorisConnectorException {
logger.trace("Parse fenodes '{}'.", feNodes);
if (StringUtils.isEmpty(feNodes)) {
String errMsg =
String.format(ErrorMessages.ILLEGAL_ARGUMENT_MESSAGE, "fenodes", feNodes);
throw new DorisConnectorException(DorisConnectorErrorCode.REST_SERVICE_FAILED, errMsg);
}
List<String> nodes = Arrays.asList(feNodes.split(","));
Collections.shuffle(nodes);
return nodes.get(0).trim();
}

@VisibleForTesting
static String getUriStr(
DorisSourceConfig dorisSourceConfig, DorisSourceTable dorisSourceTable, Logger logger)
static String getUriStr(String node, DorisSourceTable dorisSourceTable, Logger logger)
throws DorisConnectorException {
String tableIdentifier =
dorisSourceTable.getTablePath().getDatabaseName()
+ "."
+ dorisSourceTable.getTablePath().getTableName();
String[] identifier = parseIdentifier(tableIdentifier, logger);
return "http://"
+ randomEndpoint(dorisSourceConfig.getFrontends(), logger)
+ node.trim()
+ API_PREFIX
+ "/"
+ identifier[0]
Expand Down Expand Up @@ -298,16 +283,37 @@ public static List<PartitionDefinition> findPartitions(
}
logger.debug("Query SQL Sending to Doris FE is: '{}'.", sql);

HttpPost httpPost =
new HttpPost(getUriStr(dorisSourceConfig, dorisSourceTable, logger) + QUERY_PLAN);
String entity = "{\"sql\": \"" + sql + "\"}";
logger.debug("Post body Sending to Doris FE is: '{}'.", entity);
StringEntity stringEntity = new StringEntity(entity, StandardCharsets.UTF_8);
stringEntity.setContentEncoding("UTF-8");
stringEntity.setContentType("application/json");
httpPost.setEntity(stringEntity);

String resStr = send(dorisSourceConfig, httpPost, logger);
List<String> feNodes = Arrays.asList(dorisSourceConfig.getFrontends().split(","));
Collections.shuffle(feNodes);
int feNodesNum = feNodes.size();
String resStr = null;

for (int i = 0; i < feNodesNum; i++) {
try {
HttpPost httpPost =
new HttpPost(
getUriStr(feNodes.get(i), dorisSourceTable, logger) + QUERY_PLAN);
httpPost.setEntity(stringEntity);
resStr = send(dorisSourceConfig, httpPost, logger);
break;
} catch (Exception e) {
if (i == feNodesNum - 1) {
throw new DorisConnectorException(
DorisConnectorErrorCode.REST_SERVICE_FAILED, e);
}
log.error(
"Find partition error for feNode: {} with exception: {}",
feNodes.get(i),
e.getMessage());
}
}

logger.debug("Find partition response is '{}'.", resStr);
QueryPlan queryPlan = getQueryPlan(resStr, logger);
Map<String, List<Long>> be2Tablets = selectBeForTablet(queryPlan, logger);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.seatunnel.connectors.doris.config.DorisSinkConfig;
import org.apache.seatunnel.connectors.doris.exception.DorisConnectorErrorCode;
import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException;
import org.apache.seatunnel.connectors.doris.rest.RestService;
import org.apache.seatunnel.connectors.doris.rest.models.RespContent;
import org.apache.seatunnel.connectors.doris.serialize.DorisSerializer;
import org.apache.seatunnel.connectors.doris.serialize.SeaTunnelRowSerializer;
Expand Down Expand Up @@ -98,21 +97,36 @@ public DorisSinkWriter(
}

private void initializeLoad() {
String backend = RestService.randomEndpoint(dorisSinkConfig.getFrontends(), log);
try {
this.dorisStreamLoad =
new DorisStreamLoad(
backend,
catalogTable.getTablePath(),
dorisSinkConfig,
labelGenerator,
new HttpUtil().getHttpClient());
if (dorisSinkConfig.getEnable2PC()) {
dorisStreamLoad.abortPreCommit(labelPrefix, lastCheckpointId + 1);

List<String> feNodes = Arrays.asList(dorisSinkConfig.getFrontends().split(","));
Collections.shuffle(feNodes);
int feNodesNum = feNodes.size();

for (int i = 0; i < feNodesNum; i++) {
try {
this.dorisStreamLoad =
new DorisStreamLoad(
feNodes.get(i),
catalogTable.getTablePath(),
dorisSinkConfig,
labelGenerator,
new HttpUtil().getHttpClient());
if (dorisSinkConfig.getEnable2PC()) {
dorisStreamLoad.abortPreCommit(labelPrefix, lastCheckpointId + 1);
}
break;
} catch (Exception e) {
if (i == feNodesNum - 1) {
throw new DorisConnectorException(
DorisConnectorErrorCode.STREAM_LOAD_FAILED, e);
}
log.error(
"stream load error for feNode: {} with exception: {}",
feNodes.get(i),
e.getMessage());
}
} catch (Exception e) {
throw new DorisConnectorException(DorisConnectorErrorCode.STREAM_LOAD_FAILED, e);
}

startLoad(labelGenerator.generateLabel(lastCheckpointId + 1));
// when uploading data in streaming mode, we need to regularly detect whether there are
// exceptions.
Expand Down

This file was deleted.

Loading