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][elasticsearch-connector] support elasticsearch sql source #8895

Open
wants to merge 5 commits into
base: dev
Choose a base branch
from
Open
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
19 changes: 19 additions & 0 deletions docs/en/connector-v2/source/Elasticsearch.md
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@ support version >= 2.x and <= 8.x.
| index_list | array | no | used to define a multiple table task |
| source | array | no | - |
| query | json | no | {"match_all": {}} |
| search_type | json | no | Search method,sql or dsl,default dsl |
| sql_query | json | no | sql query |
| scroll_time | string | no | 1m |
| scroll_size | int | no | 100 |
| tls_verify_certificate | boolean | no | true |
Expand Down Expand Up @@ -245,6 +247,23 @@ source {
}
```

Demo 6 : sql query
notes: sql does not support map and array types
```hocon
source {
Elasticsearch {
hosts = ["https://elasticsearch:9200"]
username = "elastic"
password = "elasticsearch"
tls_verify_certificate = false
tls_verify_hostname = false
index = "st_index_sql"
sql_query = "select * from st_index_sql where c_int>=10 and c_int<=20"
search_type = "sql"
}
}
```

## Changelog

### next version
Expand Down
55 changes: 37 additions & 18 deletions docs/zh/connector-v2/source/Elasticsearch.md
Original file line number Diff line number Diff line change
Expand Up @@ -17,25 +17,27 @@

## 配置参数选项

| 参数名称 | 类型 | 是否必须 | 默认值或者描述 |
| ----------------------- | ------- | -------- | ------------------------------------------------------- |
| hosts | 数组 | | - |
| username | string | no | - |
| password | string | no | - |
| 参数名称 | 类型 | 是否必须 | 默认值或者描述 |
| ----------------------- | ------- | -------- |-------------------------------------|
| hosts | 数组 | | - |
| username | string | no | - |
| password | string | no | - |
| index | string | No | 单索引同步配置,如果index_list没有配置,则必须配置index |
| index_list | array | no | 用来定义多索引同步任务 |
| source | array | no | - |
| query | json | no | {"match_all": {}} |
| scroll_time | string | no | 1m |
| scroll_size | int | no | 100 |
| tls_verify_certificate | boolean | no | true |
| tls_verify_hostnames | boolean | no | true |
| array_column | map | no | |
| tls_keystore_path | string | no | - |
| tls_keystore_password | string | no | - |
| tls_truststore_path | string | no | - |
| tls_truststore_password | string | no | - |
| common-options | | no | - |
| index_list | array | no | 用来定义多索引同步任务 |
| source | array | no | - |
| query | json | no | {"match_all": {}} |
| search_type | json | no | 查询方式,sql或者dsl,默认 dsl |
| sql_query | json | no | sql 查询语句 |
| scroll_time | string | no | 1m |
| scroll_size | int | no | 100 |
| tls_verify_certificate | boolean | no | true |
| tls_verify_hostnames | boolean | no | true |
| array_column | map | no | |
| tls_keystore_path | string | no | - |
| tls_keystore_password | string | no | - |
| tls_truststore_path | string | no | - |
| tls_truststore_password | string | no | - |
| common-options | | no | - |

### hosts [array]

Expand Down Expand Up @@ -244,4 +246,21 @@ source {
tls_keystore_password = "${your password}"
}
}
```

案例六 : sql 方式查询
注意: sql查询不支持map和数组类型
```hocon
source {
Elasticsearch {
hosts = ["https://elasticsearch:9200"]
username = "elastic"
password = "elasticsearch"
tls_verify_certificate = false
tls_verify_hostname = false
index = "st_index_sql"
sql_query = "select * from st_index_sql where c_int>=10 and c_int<=20"
search_type = "sql"
}
}
```
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.BYTE;
import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.COMPLETION;
import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.DATE;
import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.DATETIME;
import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.DATE_NANOS;
import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.DATE_RANGE;
import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.DENSE_VECTOR;
Expand Down Expand Up @@ -125,6 +126,7 @@ public Column convert(BasicTypeDefine<EsType> typeDefine) {
builder.dataType(BasicType.BOOLEAN_TYPE);
break;
case DATE:
case DATETIME:
builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE);
builder.scale(3);
break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,10 +63,13 @@
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;

import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.AGGREGATE_METRIC_DOUBLE;
Expand Down Expand Up @@ -294,6 +297,43 @@ public ScrollResult searchByScroll(
return getDocsFromScrollRequest(endpoint, JsonUtils.toJsonString(param));
}

/**
* first time to request search documents by scroll call /_sql?format=json
*
* @param scrollSize fetch documents count in one request
*/
public ScrollResult searchBySql(String query, int scrollSize) {
Map<String, Object> param = new HashMap<>();
param.put("query", query);
param.put("fetch_size", scrollSize);
String endpoint = "/_sql?format=json";
return getDocsFromCursorResult(endpoint, JsonUtils.toJsonString(param), null);
}

/** first time to request search documents by scroll call /_sql?format=json */
public Map<String, BasicTypeDefine<EsType>> getSqlMapping(String query, List<String> source) {
Map<String, Object> param = new HashMap<>();
String limitRegex = "(?i)\\s+LIMIT\\s+\\d+";
Pattern pattern = Pattern.compile(limitRegex);
Matcher matcher = pattern.matcher(query);
if (matcher.find()) {
query = matcher.replaceAll(" LIMIT 0");
} else {
query = query.trim() + " LIMIT 0";
}
param.put("query", query);
String endpoint = "/_sql?format=json";
ScrollResult cursorResult =
getDocsFromCursorResult(endpoint, JsonUtils.toJsonString(param), null);
JsonNode columnNodes = cursorResult.getColumnNodes();
Map<String, Object> columnMap = new LinkedHashMap<>();
for (JsonNode columnNode : columnNodes) {
String fieldName = columnNode.get("name").asText();
columnMap.put(fieldName, columnNode);
}
return getFieldTypeMappingFromProperties(JsonUtils.toJsonNode(columnMap), source);
}

/**
* scroll to get result call _search/scroll
*
Expand All @@ -307,6 +347,43 @@ public ScrollResult searchWithScrollId(String scrollId, String scrollTime) {
return getDocsFromScrollRequest("/_search/scroll", JsonUtils.toJsonString(param));
}

public ScrollResult searchWithCursor(String cursor, JsonNode columnNodes) {
Map<String, String> param = new HashMap<>();
param.put("cursor", cursor);
String endpoint = "/_sql?format=json";
return getDocsFromCursorResult(endpoint, JsonUtils.toJsonString(param), columnNodes);
}

private ScrollResult getDocsFromCursorResult(
String endpoint, String requestBody, JsonNode columnNodes) {
Request request = new Request("POST", endpoint);
request.setJsonEntity(requestBody);
try {
Response response = restClient.performRequest(request);
if (response == null) {
throw new ElasticsearchConnectorException(
ElasticsearchConnectorErrorCode.SCROLL_REQUEST_ERROR,
"POST " + endpoint + " response null");
}
if (response.getStatusLine().getStatusCode() == HttpStatus.SC_OK) {
String entity = EntityUtils.toString(response.getEntity());
ObjectNode responseJson = JsonUtils.parseObject(entity);
return getDocsFromCursorResponse(responseJson, columnNodes);
} else {
throw new ElasticsearchConnectorException(
ElasticsearchConnectorErrorCode.SCROLL_REQUEST_ERROR,
String.format(
"POST %s response status code=%d,request body=%s",
endpoint, response.getStatusLine().getStatusCode(), requestBody));
}
} catch (IOException e) {
throw new ElasticsearchConnectorException(
ElasticsearchConnectorErrorCode.SCROLL_REQUEST_ERROR,
String.format("POST %s error,request body=%s", endpoint, requestBody),
e);
}
}

private ScrollResult getDocsFromScrollRequest(String endpoint, String requestBody) {
Request request = new Request("POST", endpoint);
request.setJsonEntity(requestBody);
Expand Down Expand Up @@ -346,6 +423,38 @@ private ScrollResult getDocsFromScrollRequest(String endpoint, String requestBod
}
}

private ScrollResult getDocsFromCursorResponse(ObjectNode responseJson, JsonNode columnNodes) {
ScrollResult cursorResult = new ScrollResult();
if (responseJson.get("cursor") != null) {
cursorResult.setScrollId(responseJson.get("cursor").asText());
}
if (columnNodes == null) {
columnNodes = responseJson.get("columns");
}
JsonNode valueNodes = responseJson.get("rows");
List<Map<String, Object>> docs = new ArrayList<>();
if (valueNodes != null) {

for (int i = 0; i < valueNodes.size(); i++) {
JsonNode valueNode = valueNodes.get(i);
Map<String, Object> doc = new HashMap<>();
for (int j = 0; j < columnNodes.size(); j++) {
String fieldName = columnNodes.get(j).get("name").asText();
if (valueNode.get(j) instanceof TextNode) {
doc.put(fieldName, valueNode.get(j).textValue());
} else {
doc.put(fieldName, valueNode.get(j));
}
}
docs.add(doc);
}
}
cursorResult.setDocs(docs);
cursorResult.setColumnNodes(columnNodes);

return cursorResult;
}

private ScrollResult getDocsFromScrollResponse(ObjectNode responseJson) {
ScrollResult scrollResult = new ScrollResult();
String scrollId = responseJson.get("_scroll_id").asText();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ public class EsType {
public static final String BOOLEAN = "boolean";
public static final String COMPLETION = "completion";
public static final String DATE = "date";
public static final String DATETIME = "datetime";
public static final String DATE_NANOS = "date_nanos";
public static final String DENSE_VECTOR = "dense_vector";
public static final String DOUBLE = "double";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,8 @@ public class ElasticsearchConfig implements Serializable {
private Map<String, Object> query;
private String scrollTime;
private int scrollSize;
private SearchTypeEnum searchType;
private String sqlQuery;

private CatalogTable catalogTable;

Expand All @@ -48,6 +50,8 @@ public ElasticsearchConfig clone() {
elasticsearchConfig.setScrollTime(scrollTime);
elasticsearchConfig.setScrollSize(scrollSize);
elasticsearchConfig.setCatalogTable(catalogTable);
elasticsearchConfig.setSearchType(searchType);
elasticsearchConfig.setSqlQuery(sqlQuery);
return elasticsearchConfig;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,18 @@ public class ElasticsearchSourceOptions extends ElasticsearchBaseOptions {
.withDescription(
"Amount of time Elasticsearch will keep the search context alive for scroll requests");

public static final Option<SearchTypeEnum> SEARCH_TYPE =
Options.key("search_type")
.enumType(SearchTypeEnum.class)
.defaultValue(SearchTypeEnum.SQL)
.withDescription("Choose dsl syntax or x-pack sql.");

public static final Option<String> SQL_QUERY =
Options.key("sql_query")
.stringType()
.noDefaultValue()
.withDescription("X-pack sql,if search_type is sql, this value is required.");

public static final Option<Integer> SCROLL_SIZE =
Options.key("scroll_size")
.intType()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* 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.seatunnel.connectors.seatunnel.elasticsearch.config;

public enum SearchTypeEnum {
DSL,
SQL
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source;

import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode;

import lombok.Data;

import java.util.List;
Expand All @@ -27,4 +29,5 @@ public class ScrollResult {

private String scrollId;
private List<Map<String, Object>> docs;
private JsonNode columnNodes;
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ public enum ElasticsearchConnectorErrorCode implements SeaTunnelErrorCode {
SOURCE_CONFIG_ERROR_01(
"ELASTICSEARCH-11",
"'index' or 'index_list' must be configured, with at least one being required."),
SOURCE_CONFIG_ERROR_02("ELASTICSEARCH-12", "'query' must be configured."),
ADD_FIELD_FAILED("ELASTICSEARCH-12", "Field add failed"),
SCHEMA_CHANGE_FAILED("ELASTICSEARCH-13", "Schema change failed"),
;
Expand Down
Loading
Loading