Skip to content

Commit

Permalink
#1494 [feat-1494][http] http support array data (#1495)
Browse files Browse the repository at this point in the history
  • Loading branch information
yanghuaiGit authored Feb 6, 2023
1 parent de5b214 commit 58fbb3b
Show file tree
Hide file tree
Showing 17 changed files with 834 additions and 76 deletions.
15 changes: 15 additions & 0 deletions chunjun-connectors/chunjun-connector-http/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,21 @@
<artifactId>chunjun-connector-http</artifactId>
<name>ChunJun : Connectors : Http</name>


<dependencies>
<dependency>
<groupId>org.dom4j</groupId>
<artifactId>dom4j</artifactId>
<version>2.1.3</version>
</dependency>

<dependency>
<groupId>net.sourceforge.javacsv</groupId>
<artifactId>javacsv</artifactId>
<version>2.0</version>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* 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 com.dtstack.chunjun.connector.http.client;

import com.dtstack.chunjun.connector.http.common.HttpRestConfig;
import com.dtstack.chunjun.converter.AbstractRowConverter;

import com.csvreader.CsvReader;
import org.apache.commons.collections.CollectionUtils;

import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
import java.util.HashMap;
import java.util.Map;

public class CsvResponseParse extends ResponseParse {
private CsvReader csvReader;
private Reader reader;
private String responseValue;
private HttpRequestParam requestParam;

public CsvResponseParse(HttpRestConfig config, AbstractRowConverter converter) {
super(config, converter);
if (CollectionUtils.isEmpty(columns)) {
throw new RuntimeException("please configure column when decode is csv");
}
}

@Override
public boolean hasNext() throws IOException {
return csvReader.readRecord();
}

@Override
public ResponseValue next() throws Exception {
String[] data = csvReader.getValues();
HashMap<String, Object> stringObjectHashMap = new HashMap<>(columns.size());
for (int i = 0; i < columns.size(); i++) {
if (columns.get(i).getValue() != null) {
stringObjectHashMap.put(columns.get(i).getName(), columns.get(i).getValue());
} else {
stringObjectHashMap.put(columns.get(i).getName(), data[i]);
}
}

return new ResponseValue(
converter.toInternal(stringObjectHashMap), requestParam, responseValue);
}

@Override
public void parse(String responseValue, int responseStatus, HttpRequestParam requestParam) {
this.responseValue = responseValue;
this.requestParam = requestParam;
this.reader = new StringReader(responseValue);
this.csvReader = new CsvReader(reader);
csvReader.setDelimiter(config.getCsvDelimiter().charAt(0));

Map<String, Object> csvConfig = config.getCsvConfig();
// 是否跳过空行
csvReader.setSkipEmptyRecords((Boolean) csvConfig.getOrDefault("skipEmptyRecords", true));
// 是否使用csv转义字符
csvReader.setUseTextQualifier((Boolean) csvConfig.getOrDefault("useTextQualifier", true));
csvReader.setTrimWhitespace((Boolean) csvConfig.getOrDefault("trimWhitespace", false));
// 单列长度是否限制100000字符
csvReader.setSafetySwitch((Boolean) csvConfig.getOrDefault("safetySwitch", false));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -163,12 +163,6 @@ public HttpRequestParam buildRequestParam(
return requestParam;
}

@Override
public ResponseValue buildResponseValue(
String decode, String responseValue, String fields, HttpRequestParam requestParam) {
return new ResponseValue(responseValue, requestParam, responseValue);
}

/**
* 根据指定的key 构建一个新的response
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.dtstack.chunjun.connector.http.common.HttpRestConfig;
import com.dtstack.chunjun.connector.http.common.HttpUtil;
import com.dtstack.chunjun.connector.http.common.MetaParam;
import com.dtstack.chunjun.converter.AbstractRowConverter;
import com.dtstack.chunjun.util.ExceptionUtil;
import com.dtstack.chunjun.util.GsonUtil;

Expand All @@ -41,6 +42,10 @@
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;

import static com.dtstack.chunjun.connector.http.common.ConstantValue.CSV_DECODE;
import static com.dtstack.chunjun.connector.http.common.ConstantValue.TEXT_DECODE;
import static com.dtstack.chunjun.connector.http.common.ConstantValue.XML_DECODE;

/**
* httpClient
*
Expand All @@ -60,6 +65,8 @@ public class HttpClient {

private final RestHandler restHandler;

protected final ResponseParse responseParse;

private int requestRetryTime;

/** origin body */
Expand All @@ -86,11 +93,14 @@ public class HttpClient {

private boolean running;

protected long requestNumber;

public HttpClient(
HttpRestConfig httpRestConfig,
List<MetaParam> originalBodyList,
List<MetaParam> originalParamList,
List<MetaParam> originalHeaderList) {
List<MetaParam> originalHeaderList,
AbstractRowConverter converter) {
this.restConfig = httpRestConfig;
this.originalHeaderList = originalHeaderList;
this.originalBodyList = originalBodyList;
Expand All @@ -102,14 +112,16 @@ public HttpClient(
this.queue = new LinkedBlockingQueue<>();
this.scheduledExecutorService =
new ScheduledThreadPoolExecutor(1, r -> new Thread(r, THREAD_NAME));
this.httpClient = HttpUtil.getHttpsClient();
this.httpClient = HttpUtil.getHttpsClient((int) restConfig.getTimeOut());
this.restHandler = new DefaultRestHandler();
this.responseParse = getResponseParse(converter);

this.prevResponse = "";
this.first = true;
this.currentParam = new HttpRequestParam();
this.reachEnd = false;
this.requestRetryTime = 2;
this.requestNumber = 1;
}

public void start() {
Expand Down Expand Up @@ -172,6 +184,7 @@ public void execute() {
doExecute(ConstantValue.REQUEST_RETRY_TIME);
first = false;
requestRetryTime = 3;
requestNumber++;
}

public void doExecute(int retryTime) {
Expand All @@ -192,6 +205,7 @@ public void doExecute(int retryTime) {

// 执行请求
String responseValue = null;
int responseStatus;
try {

HttpUriRequest request =
Expand All @@ -211,6 +225,7 @@ public void doExecute(int retryTime) {
}

responseValue = EntityUtils.toString(httpResponse.getEntity());
responseStatus = httpResponse.getStatusLine().getStatusCode();
} catch (Throwable e) {
// 只要本次请求中出现了异常 都会进行重试,如果重试次数达到了就真正结束任务
LOG.warn(
Expand Down Expand Up @@ -244,27 +259,28 @@ public void doExecute(int retryTime) {
case ConstantValue.STRATEGY_STOP:
reachEnd = true;
running = false;
// stop 此次请求数据有问题 任务直接异常结束
processData(new ResponseValue(0, null, strategy.toString(), null, null));
break;
default:
break;
}
}

ResponseValue value =
restHandler.buildResponseValue(
restConfig.getDecode(),
responseValue,
restConfig.getFields(),
HttpRequestParam.copy(currentParam));
responseParse.parse(responseValue, responseStatus, HttpRequestParam.copy(currentParam));
while (responseParse.hasNext()) {
processData(responseParse.next());
}

if (-1 != restConfig.getCycles() && requestNumber >= restConfig.getCycles()) {
reachEnd = true;
running = false;
}

if (reachEnd) {
// 如果结束了 需要告诉format 结束了
if (value.isNormal()) {
value.setStatus(0);
// 触发的策略信息返回上游
value.setErrorMsg(strategy.toString());
}
processData(new ResponseValue(2, null, null, null, null));
}
processData(value);

prevParam = currentParam;
prevResponse = responseValue;
Expand Down Expand Up @@ -321,6 +337,19 @@ public void close() {
}
}

protected ResponseParse getResponseParse(AbstractRowConverter converter) {
switch (restConfig.getDecode()) {
case CSV_DECODE:
return new CsvResponseParse(restConfig, converter);
case XML_DECODE:
return new XmlResponseParse(restConfig, converter);
case TEXT_DECODE:
return new TextResponseParse(restConfig, converter);
default:
return new JsonResponseParse(restConfig, converter);
}
}

@Override
public String toString() {
return "HttpClient{"
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* 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 com.dtstack.chunjun.connector.http.client;

import com.dtstack.chunjun.conf.FieldConf;
import com.dtstack.chunjun.connector.http.common.HttpRestConfig;
import com.dtstack.chunjun.connector.http.util.JsonPathUtil;
import com.dtstack.chunjun.constants.ConstantValue;
import com.dtstack.chunjun.converter.AbstractRowConverter;
import com.dtstack.chunjun.util.GsonUtil;
import com.dtstack.chunjun.util.MapUtil;

import com.google.common.collect.Lists;
import com.google.gson.Gson;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.StringUtils;

import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

public class JsonResponseParse extends ResponseParse {
private String responseValue;
private HttpRequestParam requestParam;
private final Gson gson;
private final List<FieldConf> fields;
private Iterator<Map<String, Object>> iterator;

public JsonResponseParse(HttpRestConfig config, AbstractRowConverter converter) {
super(config, converter);
this.gson = GsonUtil.setTypeAdapter(new Gson());
if (StringUtils.isNotBlank(config.getFields())) {
fields =
Arrays.stream(config.getFields().split(","))
.map(
i -> {
FieldConf fieldConf = new FieldConf();
fieldConf.setName(i);
return fieldConf;
})
.collect(Collectors.toList());
} else {
fields = null;
}
}

@Override
public boolean hasNext() throws IOException {
return iterator.hasNext();
}

@Override
public ResponseValue next() throws Exception {
Map next = iterator.next();

if (CollectionUtils.isEmpty(columns)) {
if (CollectionUtils.isNotEmpty(fields)) {
LinkedHashMap<String, Object> map =
buildResponseByKey(next, fields, ConstantValue.POINT_SYMBOL);
HashMap<String, Object> data = new HashMap<>();
// 需要拆分key
((Map<String, Object>) map)
.forEach(
(k, v) -> {
MapUtil.buildMap(k, ConstantValue.POINT_SYMBOL, v, data);
});
return new ResponseValue(converter.toInternal(data), requestParam, responseValue);
} else {
return new ResponseValue(converter.toInternal(next), requestParam, responseValue);
}

} else {
LinkedHashMap<String, Object> data =
buildResponseByKey(next, columns, ConstantValue.POINT_SYMBOL);
return new ResponseValue(converter.toInternal(data), requestParam, responseValue);
}
}

@Override
public void parse(String responseValue, int responseStatus, HttpRequestParam requestParam) {
this.responseValue = responseValue;
this.requestParam = requestParam;

Map<String, Object> map = gson.fromJson(responseValue, GsonUtil.gsonMapTypeToken);
if (StringUtils.isNotBlank(config.getDataSubject())) {
Object valueByKey =
MapUtil.getValueByKey(
map,
JsonPathUtil.parseJsonPath(config.getDataSubject()),
ConstantValue.POINT_SYMBOL);
if (valueByKey instanceof List) {
this.iterator = ((List) valueByKey).iterator();
} else {
throw new RuntimeException(config.getDataSubject() + " in response is not array");
}
} else {
this.iterator = Lists.newArrayList(map).iterator();
}
}
}
Loading

0 comments on commit 58fbb3b

Please sign in to comment.