tmljob commented on code in PR #1798: URL: https://github.com/apache/incubator-seatunnel/pull/1798#discussion_r865924790
########## seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-http/src/main/java/org/apache/seatunnel/flink/http/source/Http.java: ########## @@ -0,0 +1,136 @@ +/* + * 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.flink.http.source; + +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.config.TypesafeConfigUtils; +import org.apache.seatunnel.flink.FlinkEnvironment; +import org.apache.seatunnel.flink.batch.FlinkBatchSource; +import org.apache.seatunnel.flink.http.source.constant.Settings; +import org.apache.seatunnel.flink.http.source.util.HttpClientResult; +import org.apache.seatunnel.flink.http.source.util.HttpClientUtils; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.DataSource; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class Http implements FlinkBatchSource { + + private static final String GET = "GET"; + private static final String POST = "POST"; + private static final int INITIAL_CAPACITY = 16; + + private static Logger LOG = LoggerFactory.getLogger(Http.class); + + private Config config; + + @Override + public void setConfig(Config config) { + this.config = config; + } + + @Override + public Config getConfig() { + return config; + } + + @Override + public CheckResult checkConfig() { + return CheckConfigUtil.checkAllExists(config, Settings.SOURCE_HTTP_URL); + } + + @Override + public DataSet<Row> getData(FlinkEnvironment env) { + String url = config.getString(Settings.SOURCE_HTTP_URL); + String method = TypesafeConfigUtils.getConfig(config, Settings.SOURCE_HTTP_METHOD, GET); + String header = TypesafeConfigUtils.getConfig(config, Settings.SOURCE_HTTP_HEADER, ""); + String requestParams = TypesafeConfigUtils.getConfig(config, Settings.SOURCE_HTTP_REQUEST_PARAMS, ""); + String syncPath = TypesafeConfigUtils.getConfig(config, Settings.SOURCE_HTTP_SYNC_PATH, ""); + + Map requestMap = jsonToMap(requestParams); + String syncValues = getSyncValues(env.getBatchEnvironment(), syncPath); + LOG.info("sync values->{}", syncValues); + Map syncMap = jsonToMap(syncValues); + if (!syncMap.isEmpty()) { + requestMap.putAll(syncMap); + } + + HttpClientResult response = new HttpClientResult(); + try { + Map headerMap = jsonToMap(header); + if (POST.equals(method)) { + response = HttpClientUtils.doPost(url, headerMap, requestMap); + } else { + response = HttpClientUtils.doGet(url, headerMap, requestMap); + } + } catch (Exception e) { + LOG.error("http call error!", e); + } + + LOG.info("http respond code->{}", response.getCode()); + LOG.info("http respond body->{}", response.getContent()); + + Row row = Row.of(response.getContent()); Review Comment: > I suggest to use `env.getBatchTableEnvironment().toDataSet() ` to define the filed name. Because the responsebody returned by the http call is undefined, it feels like there is no way to predefine the field name. After I change it to the following way, even if there is no transform process, an error will be reported. **code** ``` env.getBatchTableEnvironment().toDataSet( env.getBatchTableEnvironment().from(response.getContent()), Row.class ); ``` **error log** ``` 21:20:47.073 [main] ERROR org.apache.seatunnel.core.base.Seatunnel - =============================================================================== 21:20:47.078 [main] ERROR org.apache.seatunnel.core.base.Seatunnel - Fatal Error, 21:20:47.078 [main] ERROR org.apache.seatunnel.core.base.Seatunnel - Please submit bug report in https://github.com/apache/incubator-seatunnel/issues 21:20:47.080 [main] ERROR org.apache.seatunnel.core.base.Seatunnel - Reason:Execute Flink task error 21:20:47.082 [main] ERROR org.apache.seatunnel.core.base.Seatunnel - Exception StackTrace:java.lang.RuntimeException: Execute Flink task error at org.apache.seatunnel.core.flink.command.FlinkTaskExecuteCommand.execute(FlinkTaskExecuteCommand.java:71) at org.apache.seatunnel.core.base.Seatunnel.run(Seatunnel.java:39) at org.apache.seatunnel.core.flink.SeatunnelFlink.main(SeatunnelFlink.java:33) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:349) at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:219) at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:812) at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:246) at org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:1054) at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1132) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1875) at org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41) at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1132) Caused by: org.apache.flink.table.api.SqlParserException: Invalid SQL identifier { "gzipped": true, "headers": { "Accept-Encoding": "gzip,deflate", "Host": "httpbin.org", "User-Agent": "Apache-HttpClient/4.5.3 (Java/1.8.0_181)", "X-Amzn-Trace-Id": "Root=1-6273cf2e-75e23ad14971e7ad7ec2e4e1" }, "method": "GET", "origin": "103.25.65.102" } . at org.apache.flink.table.calcite.CalciteParser.parseIdentifier(CalciteParser.java:70) at org.apache.flink.table.api.internal.TableEnvImpl.from(TableEnvImpl.scala:437) at org.apache.seatunnel.flink.http.source.Http.getData(Http.java:102) at org.apache.seatunnel.flink.batch.FlinkBatchExecution.start(FlinkBatchExecution.java:55) at org.apache.seatunnel.core.flink.command.FlinkTaskExecuteCommand.execute(FlinkTaskExecuteCommand.java:68) ... 18 more 21:20:47.082 [main] ERROR org.apache.seatunnel.core.base.Seatunnel - =============================================================================== ``` Can you suggest a solution here? -- 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]
