Hisoka-X commented on code in PR #4752: URL: https://github.com/apache/incubator-seatunnel/pull/4752#discussion_r1193618395
########## seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/client/sink/StarRocksSinkManagerV2.java: ########## @@ -0,0 +1,274 @@ +/* + * 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.starrocks.client.sink; + +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.connectors.seatunnel.starrocks.client.sink.entity.StreamLoadResponse; +import org.apache.seatunnel.connectors.seatunnel.starrocks.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.starrocks.sink.committer.StarRocksCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.starrocks.sink.state.StarRocksSinkState; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.LockSupport; +import java.util.concurrent.locks.ReentrantLock; + +@Slf4j +public class StarRocksSinkManagerV2 implements StreamLoadManager { + + private Thread manager; + private Thread current; + private volatile Throwable flushException; + private TableRegion tableRegion; + private final long flushFrequencyMs; + private SinkConfig sinkConfig; + private TransactionStreamLoader streamLoader; + protected long checkpointId = 0L; + protected String transactionId; + + private final Lock lock = new ReentrantLock(); + + private Condition writable = lock.newCondition(); + private Condition flushable = lock.newCondition(); + + private volatile boolean savepoint; + + private final long maxCacheBytes; + private long rowCountForCurrentTransaction; + protected int subTaskIndex; + private LabelGenerator labelGenerator; + + public StarRocksSinkManagerV2( + LabelGenerator labelGenerator, SinkConfig sinkConfig, SinkWriter.Context context) { + this.sinkConfig = sinkConfig; + this.flushFrequencyMs = sinkConfig.getFlushFrequencyMs(); + this.maxCacheBytes = sinkConfig.getBatchMaxBytes(); + this.streamLoader = new TransactionStreamLoader(sinkConfig, this); + this.tableRegion = new TransactionTableRegion(this, sinkConfig, streamLoader); + this.subTaskIndex = context.getIndexOfSubtask(); + this.labelGenerator = labelGenerator; + init(); + } + + @Override + public void init() { + this.manager = Review Comment: Are there have a reason for why have to create an new thread? ########## seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/client/HttpHelper.java: ########## @@ -77,100 +77,58 @@ public String doHttpPost(String postUrl, Map<String, String> header, String post } } - public String doHttpGet(String getUrl) throws IOException { - log.info("Executing GET from {}.", getUrl); - try (CloseableHttpClient httpclient = buildHttpClient()) { - HttpGet httpGet = new HttpGet(getUrl); - try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { - HttpEntity respEntity = resp.getEntity(); - if (null == respEntity) { - log.warn("Request failed with empty response."); - return null; - } - return EntityUtils.toString(respEntity); + public String doHttpExecute(HttpClientBuilder clientBuilder, HttpRequestBase httpRequestBase) + throws IOException { + if (Objects.isNull(clientBuilder)) clientBuilder = getDefaultClientBuilder(); + try (CloseableHttpClient client = clientBuilder.build()) { + try (CloseableHttpResponse response = client.execute(httpRequestBase)) { + return parseHttpResponse(response, httpRequestBase.getMethod()); } } } - public Map<String, Object> doHttpGet(String getUrl, Map<String, String> header) - throws IOException { - log.info("Executing GET from {}.", getUrl); - try (CloseableHttpClient httpclient = HttpClients.createDefault()) { - HttpGet httpGet = new HttpGet(getUrl); - if (null != header) { - for (Map.Entry<String, String> entry : header.entrySet()) { - httpGet.setHeader(entry.getKey(), String.valueOf(entry.getValue())); - } - } - try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { - HttpEntity respEntity = getHttpEntity(resp); - if (null == respEntity) { - log.warn("Request failed with empty response."); - return null; - } - return JsonUtils.parseObject(EntityUtils.toString(respEntity), Map.class); - } + public String parseHttpResponse(CloseableHttpResponse response, String requestType) + throws StarRocksConnectorException { + int code = response.getStatusLine().getStatusCode(); + if (307 == code) { Review Comment: Can you change the response code number to same meaning enum? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
