This is an automated email from the ASF dual-hosted git repository.
fcsaky pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/flink-connector-http.git
The following commit(s) were added to refs/heads/main by this push:
new 89c3ff5 [FLINK-38743] Update connector identifiers
89c3ff5 is described below
commit 89c3ff57379e94e3eb4fadf69f8f1f0f203c5226
Author: David Radley <[email protected]>
AuthorDate: Tue Dec 2 11:35:46 2025 +0000
[FLINK-38743] Update connector identifiers
---
docs/content.zh/docs/connectors/table/http.md | 29 ++++++++++++----------
docs/content/docs/connectors/table/http.md | 28 +++++++++++----------
.../http/HttpPostRequestCallbackFactory.java | 2 +-
.../connector/http/LookupQueryCreatorFactory.java | 2 +-
.../table/lookup/HttpLookupTableSourceFactory.java | 4 ++-
.../Slf4jHttpLookupPostRequestCallbackFactory.java | 2 +-
.../ElasticSearchLiteQueryCreatorFactory.java | 2 +-
.../GenericGetQueryCreatorFactory.java | 2 +-
.../GenericJsonAndUrlQueryCreatorFactory.java | 2 +-
.../GenericJsonQueryCreatorFactory.java | 2 +-
.../table/sink/HttpDynamicTableSinkFactory.java | 2 +-
.../sink/Slf4jHttpPostRequestCallbackFactory.java | 2 +-
.../http/HttpPostRequestCallbackFactoryTest.java | 2 +-
.../flink/connector/http/StreamTableJob.java | 2 +-
.../lookup/HttpLookupTableSourceFactoryTest.java | 4 +--
.../lookup/HttpLookupTableSourceITCaseTest.java | 26 +++++++++----------
.../table/lookup/HttpLookupTableSourceTest.java | 2 +-
17 files changed, 61 insertions(+), 54 deletions(-)
diff --git a/docs/content.zh/docs/connectors/table/http.md
b/docs/content.zh/docs/connectors/table/http.md
index e73ada3..d1ec6d8 100644
--- a/docs/content.zh/docs/connectors/table/http.md
+++ b/docs/content.zh/docs/connectors/table/http.md
@@ -45,8 +45,8 @@ The HTTP source connector supports [Lookup
Joins](https://nightlies.apache.org/f
* [Using a HTTP Lookup Source in a lookup
join](#using-a-http-lookup-source-in-a-lookup-join)
* [Lookup Source Connector Options](#lookup-source-connector-options)
* [Query Creators](#query-creators)
- * [generic-json-url Query Creator](#generic-json-url-query-creator)
- * [generic-json-url Query Creator](#generic-json-url-query-creator-1)
+ * [http-generic-json-url Query
Creator](#http-generic-json-url-query-creator)
+ * [http-generic-json-url Query
Creator](#http-generic-json-url-query-creator-1)
* [Http headers](#http-headers)
* [Timeouts](#timeouts)
* [Source table HTTP status code](#source-table-http-status-code)
@@ -81,8 +81,11 @@ The GetInData HTTP connector was donated to Flink in
[FLIP-532](https://cwiki.ap
The Flink connector does have some changes that you need to be aware of if you
are migrating from using the original connector:
* Existing java applications will need to be recompiled to pick up the new
flink package names.
-* Existing application and SQL need to be amended to use the new connector
option names. The new option names do not have
+* Existing application and SQL need to be amended to use the new connector
option names. The new option names do not have
the _com.getindata.http_ prefix, the prefix is now _http_.
+* The name of the connector and the identifiers of components that are
discovered have been changed, so that the GetInData jar file can co-exist
+* with this connector's jar file. Be aware that if you have created custom
pluggable components; you will need to recompile against
+* this connector.
## Working with HTTP lookup source tables
@@ -102,7 +105,7 @@ CREATE TABLE Customers (
>
>
) WITH (
-'connector' = 'rest-lookup',
+'connector' = 'http',
'format' = 'json',
'url' = 'http://localhost:8080/client',
'asyncPolling' = 'true'
@@ -157,7 +160,7 @@ Note the options with the prefix _http_ are the HTTP
connector specific options,
| Option |
Required | Description/Value
[...]
|:-----------------------------------------------------------------------|----------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
[...]
-| connector |
required | The Value should be set to _rest-lookup_
[...]
+| connector |
required | The Value should be set to _http_
[...]
| format |
required | Flink's format name that should be used to decode REST response, Use
`json` for a typical REST endpoint.
[...]
| url |
required | The base URL that should be use for GET requests. For example
_http://localhost:8080/client_
[...]
| asyncPolling |
optional | true/false - determines whether Async Polling should be used.
Mechanism is based on Flink's Async I/O.
[...]
@@ -218,13 +221,13 @@ The HTTP connector supplies a number of Query Creators
that you can use define t
</thead>
<tbody>
<tr>
- <td><h5>generic-json-url</h5></td>
+ <td><h5>http-generic-json-url</h5></td>
<td>✓</td>
<td>✓</td>
<td>✓</td>
</tr>
<tr>
- <td><h5>generic-get-query</h5></td>
+ <td><h5>http-generic-get-query</h5></td>
<td><✓ for GETs/td>
<td></td>
<td>✓ for PUTs and POSTs</td>
@@ -233,20 +236,20 @@ The HTTP connector supplies a number of Query Creators
that you can use define t
</tbody>
</table>
-### generic-json-url Query Creator
+### http-generic-json-url Query Creator
-The recommended Query creator for json is called _generic-json-url_, which
allows column content to be mapped as URL, path, body and query parameter
request values; it supports
+The recommended Query creator for json is called _http-generic-json-url_,
which allows column content to be mapped as URL, path, body and query parameter
request values; it supports
POST, PUT and GET operations. This query creator allows you to issue json
requests without needing to code
your own custom http connector. The mappings from columns to the json request
are supplied in the query creator configuration
parameters `http.request.query-param-fields`, `http.request.body-fields` and
`http.request.url-map`.
-### generic-json-url Query Creator
+### http-generic-json-url Query Creator
-The default Query Creator is called _generic-json-url_. For body based
queries such as POST/PUT requests, the
+The default Query Creator is called _http-generic-json-url_. For body based
queries such as POST/PUT requests, the
([GenericGetQueryCreator](flink-connector-http/src/main/java/org/apache/flink/connectors/http/internal/table/lookup/querycreators/GenericGetQueryCreator.java))is
provided as a default query creator. This implementation uses Flink's
[json-format](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/table/formats/json/)
to convert RowData object into Json String.
For GET requests can be used for query parameter based queries.
-The _generic-json-url_ allows for using custom formats that will perform
serialization to Json. Thanks to this, users can create their own logic for
converting RowData to Json Strings suitable for their HTTP endpoints and use
this logic as custom format
+The _http-generic-json-url_ allows for using custom formats that will perform
serialization to Json. Thanks to this, users can create their own logic for
converting RowData to Json Strings suitable for their HTTP endpoints and use
this logic as custom format
with HTTP Lookup connector and SQL queries.
To create a custom format user has to implement Flink's `SerializationSchema`
and `SerializationFormatFactory` interfaces and register custom format factory
along other factories in
`resources/META-INF.services/org.apache.flink.table.factories.Factory` file.
This is common Flink mechanism for providing custom implementations for various
factories.
@@ -266,7 +269,7 @@ CREATE TABLE http-lookup (
id bigint,
some_field string
) WITH (
- 'connector' = 'rest-lookup',
+ 'connector' = 'http',
'format' = 'json',
'url' = 'http://localhost:8080/client',
'asyncPolling' = 'true',
diff --git a/docs/content/docs/connectors/table/http.md
b/docs/content/docs/connectors/table/http.md
index a2cc233..ada2636 100644
--- a/docs/content/docs/connectors/table/http.md
+++ b/docs/content/docs/connectors/table/http.md
@@ -45,8 +45,8 @@ The HTTP source connector supports [Lookup
Joins](https://nightlies.apache.org/f
* [Using a HTTP Lookup Source in a lookup
join](#using-a-http-lookup-source-in-a-lookup-join)
* [Lookup Source Connector Options](#lookup-source-connector-options)
* [Query Creators](#query-creators)
- * [generic-json-url Query Creator](#generic-json-url-query-creator)
- * [generic-json-url Query Creator](#generic-json-url-query-creator-1)
+ * [http-generic-json-url Query
Creator](#http-generic-json-url-query-creator)
+ * [http-generic-json-url Query
Creator](#http-generic-json-url-query-creator-1)
* [Http headers](#http-headers)
* [Timeouts](#timeouts)
* [Source table HTTP status code](#source-table-http-status-code)
@@ -81,8 +81,10 @@ The GetInData HTTP connector was donated to Flink in
[FLIP-532](https://cwiki.ap
The Flink connector does have some changes that you need to be aware of if you
are migrating from using the original connector:
* Existing java applications will need to be recompiled to pick up the new
flink package names.
-* Existing application and SQL need to be amended to use the new connector
option names. The new option names do not have
+* Existing application and SQL need to be amended to use the new connector
option names. The new option names do not have
the _com.getindata.http_ prefix, the prefix is now _http_.
+* The name of the connector and the identifiers of components that are
discovered have been changed, so that the GetInData jar file can co-exist
+* with this connector's jar file. Be aware that if you have created custom
pluggable components; you will need to recompile against
## Working with HTTP lookup source tables
@@ -102,7 +104,7 @@ CREATE TABLE Customers (
>
>
) WITH (
-'connector' = 'rest-lookup',
+'connector' = 'http',
'format' = 'json',
'url' = 'http://localhost:8080/client',
'asyncPolling' = 'true'
@@ -157,7 +159,7 @@ Note the options with the prefix _http_ are the HTTP
connector specific options,
| Option |
Required | Description/Value
[...]
|:-----------------------------------------------------------------------|----------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
[...]
-| connector |
required | The Value should be set to _rest-lookup_
[...]
+| connector |
required | The Value should be set to _http_
[...]
| format |
required | Flink's format name that should be used to decode REST response, Use
`json` for a typical REST endpoint.
[...]
| url |
required | The base URL that should be use for GET requests. For example
_http://localhost:8080/client_
[...]
| asyncPolling |
optional | true/false - determines whether Async Polling should be used.
Mechanism is based on Flink's Async I/O.
[...]
@@ -218,13 +220,13 @@ The HTTP connector supplies a number of Query Creators
that you can use define t
</thead>
<tbody>
<tr>
- <td><h5>generic-json-url</h5></td>
+ <td><h5>http-generic-json-url</h5></td>
<td>✓</td>
<td>✓</td>
<td>✓</td>
</tr>
<tr>
- <td><h5>generic-get-query</h5></td>
+ <td><h5>http-generic-get-query</h5></td>
<td><✓ for GETs/td>
<td></td>
<td>✓ for PUTs and POSTs</td>
@@ -233,20 +235,20 @@ The HTTP connector supplies a number of Query Creators
that you can use define t
</tbody>
</table>
-### generic-json-url Query Creator
+### http-generic-json-url Query Creator
-The recommended Query creator for json is called _generic-json-url_, which
allows column content to be mapped as URL, path, body and query parameter
request values; it supports
+The recommended Query creator for json is called _http-generic-json-url_,
which allows column content to be mapped as URL, path, body and query parameter
request values; it supports
POST, PUT and GET operations. This query creator allows you to issue json
requests without needing to code
your own custom http connector. The mappings from columns to the json request
are supplied in the query creator configuration
parameters `http.request.query-param-fields`, `http.request.body-fields` and
`http.request.url-map`.
-### generic-json-url Query Creator
+### http-generic-json-url Query Creator
-The default Query Creator is called _generic-json-url_. For body based
queries such as POST/PUT requests, the
+The default Query Creator is called _http-generic-json-url_. For body based
queries such as POST/PUT requests, the
([GenericGetQueryCreator](flink-connector-http/src/main/java/org/apache/flink/connectors/http/internal/table/lookup/querycreators/GenericGetQueryCreator.java))is
provided as a default query creator. This implementation uses Flink's
[json-format](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/table/formats/json/)
to convert RowData object into Json String.
For GET requests can be used for query parameter based queries.
-The _generic-json-url_ allows for using custom formats that will perform
serialization to Json. Thanks to this, users can create their own logic for
converting RowData to Json Strings suitable for their HTTP endpoints and use
this logic as custom format
+The _http-generic-json-url_ allows for using custom formats that will perform
serialization to Json. Thanks to this, users can create their own logic for
converting RowData to Json Strings suitable for their HTTP endpoints and use
this logic as custom format
with HTTP Lookup connector and SQL queries.
To create a custom format user has to implement Flink's `SerializationSchema`
and `SerializationFormatFactory` interfaces and register custom format factory
along other factories in
`resources/META-INF.services/org.apache.flink.table.factories.Factory` file.
This is common Flink mechanism for providing custom implementations for various
factories.
@@ -266,7 +268,7 @@ CREATE TABLE http-lookup (
id bigint,
some_field string
) WITH (
- 'connector' = 'rest-lookup',
+ 'connector' = 'http',
'format' = 'json',
'url' = 'http://localhost:8080/client',
'asyncPolling' = 'true',
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactory.java
index 155053c..f48ff0d 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactory.java
@@ -64,7 +64,7 @@ import org.apache.flink.table.factories.Factory;
* CREATE TABLE httplookup (
* id bigint
* ) with (
- * 'connector' = 'rest-lookup',
+ * 'connector' = 'http',
* 'url' = 'http://example.com/myendpoint',
* 'format' = 'json',
* 'http.source.lookup.request-callback' = 'my-callback'
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/LookupQueryCreatorFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/LookupQueryCreatorFactory.java
index d3a6322..04092f8 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/LookupQueryCreatorFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/LookupQueryCreatorFactory.java
@@ -46,7 +46,7 @@ import java.io.Serializable;
* id bigint,
* some_field string
* ) WITH (
- * 'connector' = 'rest-lookup',
+ * 'connector' = 'http',
* 'format' = 'json',
* 'url' = 'http://example.com/myendpoint',
* 'http.source.lookup.query-creator' = 'my-query-creator'
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactory.java
index f9b07ab..c5d0d3c 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactory.java
@@ -74,6 +74,8 @@ import static
org.apache.flink.table.types.utils.DataTypeUtils.removeTimeAttribu
/** Http Lookup Table Source Factory. */
public class HttpLookupTableSourceFactory implements DynamicTableSourceFactory
{
+ private static final String IDENTIFIER = "http";
+
private static DataTypes.Field columnToField(Column column) {
return FIELD(
column.getName(),
@@ -141,7 +143,7 @@ public class HttpLookupTableSourceFactory implements
DynamicTableSourceFactory {
@Override
public String factoryIdentifier() {
- return "rest-lookup";
+ return IDENTIFIER;
}
@Override
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/Slf4jHttpLookupPostRequestCallbackFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/Slf4jHttpLookupPostRequestCallbackFactory.java
index 39e839e..5a8cae7 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/Slf4jHttpLookupPostRequestCallbackFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/Slf4jHttpLookupPostRequestCallbackFactory.java
@@ -29,7 +29,7 @@ import java.util.Set;
public class Slf4jHttpLookupPostRequestCallbackFactory
implements
HttpPostRequestCallbackFactory<HttpLookupSourceRequestEntry> {
- public static final String IDENTIFIER = "slf4j-lookup-logger";
+ public static final String IDENTIFIER = "http-slf4j-lookup-logger";
@Override
public HttpPostRequestCallback<HttpLookupSourceRequestEntry>
createHttpPostRequestCallback() {
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/ElasticSearchLiteQueryCreatorFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/ElasticSearchLiteQueryCreatorFactory.java
index 6b0e9c2..734525a 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/ElasticSearchLiteQueryCreatorFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/ElasticSearchLiteQueryCreatorFactory.java
@@ -29,7 +29,7 @@ import java.util.Set;
/** Factory for creating {@link ElasticSearchLiteQueryCreator}. */
public class ElasticSearchLiteQueryCreatorFactory implements
LookupQueryCreatorFactory {
- public static final String IDENTIFIER = "elasticsearch-lite";
+ public static final String IDENTIFIER = "http-elasticsearch-lite";
@Override
public LookupQueryCreator createLookupQueryCreator(
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericGetQueryCreatorFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericGetQueryCreatorFactory.java
index 2894ff1..63351de 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericGetQueryCreatorFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericGetQueryCreatorFactory.java
@@ -29,7 +29,7 @@ import java.util.Set;
/** Factory for creating {@link GenericGetQueryCreator}. */
public class GenericGetQueryCreatorFactory implements
LookupQueryCreatorFactory {
- public static final String IDENTIFIER = "generic-get-query";
+ public static final String IDENTIFIER = "http-generic-get-query";
@Override
public LookupQueryCreator createLookupQueryCreator(
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonAndUrlQueryCreatorFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonAndUrlQueryCreatorFactory.java
index a718371..c03ce7e 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonAndUrlQueryCreatorFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonAndUrlQueryCreatorFactory.java
@@ -54,7 +54,7 @@ import static
org.apache.flink.connector.http.table.lookup.HttpLookupConnectorOp
public class GenericJsonAndUrlQueryCreatorFactory implements
LookupQueryCreatorFactory {
private static final long serialVersionUID = 1L;
- public static final String ID = "generic-json-url";
+ public static final String ID = "http-generic-json-url";
public static final ConfigOption<List<String>> REQUEST_QUERY_PARAM_FIELDS =
key("http.request.query-param-fields")
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonQueryCreatorFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonQueryCreatorFactory.java
index 75cb22e..bf7c97d 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonQueryCreatorFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/querycreators/GenericJsonQueryCreatorFactory.java
@@ -39,7 +39,7 @@ import static
org.apache.flink.connector.http.table.lookup.HttpLookupConnectorOp
/** Factory for creating {@link GenericJsonQueryCreatorFactory}. */
public class GenericJsonQueryCreatorFactory implements
LookupQueryCreatorFactory {
- public static final String IDENTIFIER = "generic-json-query";
+ public static final String IDENTIFIER = "http-generic-json-query";
@Override
public LookupQueryCreator createLookupQueryCreator(
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/HttpDynamicTableSinkFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/HttpDynamicTableSinkFactory.java
index 6d4b2d1..d3ef903 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/HttpDynamicTableSinkFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/HttpDynamicTableSinkFactory.java
@@ -38,7 +38,7 @@ import static
org.apache.flink.connector.http.table.sink.HttpDynamicSinkConnecto
/** Factory for creating {@link HttpDynamicSink}. */
public class HttpDynamicTableSinkFactory extends AsyncDynamicTableSinkFactory {
- public static final String IDENTIFIER = "http-sink";
+ public static final String IDENTIFIER = "http-async-sink";
@Override
public DynamicTableSink createDynamicTableSink(Context context) {
diff --git
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/Slf4jHttpPostRequestCallbackFactory.java
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/Slf4jHttpPostRequestCallbackFactory.java
index 78d00bf..2dc9e97 100644
---
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/Slf4jHttpPostRequestCallbackFactory.java
+++
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/sink/Slf4jHttpPostRequestCallbackFactory.java
@@ -29,7 +29,7 @@ import java.util.Set;
public class Slf4jHttpPostRequestCallbackFactory
implements HttpPostRequestCallbackFactory<HttpRequest> {
- public static final String IDENTIFIER = "slf4j-logger";
+ public static final String IDENTIFIER = "http-slf4j-logger";
@Override
public HttpPostRequestCallback<HttpRequest>
createHttpPostRequestCallback() {
diff --git
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactoryTest.java
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactoryTest.java
index ef20a3f..f5bf9ac 100644
---
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactoryTest.java
+++
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/HttpPostRequestCallbackFactoryTest.java
@@ -147,7 +147,7 @@ public class HttpPostRequestCallbackFactoryTest {
+ " 'format' = 'json',\n"
+ " 'http.source.lookup.request-callback' =
'%s'\n"
+ ")",
- "rest-lookup",
+ "http",
"http://localhost:" + SERVER_PORT + "/myendpoint",
TEST_LOOKUP_POST_REQUEST_CALLBACK_IDENT);
tEnv.executeSql(createTable2);
diff --git
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/StreamTableJob.java
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/StreamTableJob.java
index 24153bc..32be150 100644
---
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/StreamTableJob.java
+++
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/StreamTableJob.java
@@ -49,7 +49,7 @@ public class StreamTableJob {
+ " 'fields.id2.kind' = 'sequence', 'fields.id2.start'
= '2',"
+ " 'fields.id2.end' = '120')");
tableEnv.executeSql(
- "CREATE TABLE Customers (id STRING, id2 STRING, msg STRING,
uuid STRING, isActive STRING, balance STRING) WITH ('connector' =
'rest-lookup',"
+ "CREATE TABLE Customers (id STRING, id2 STRING, msg STRING,
uuid STRING, isActive STRING, balance STRING) WITH ('connector' = 'http',"
+ " 'url' = 'http://localhost:"
+ WireMockServerPortAllocator.getServerPort()
+ "/client', "
diff --git
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactoryTest.java
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactoryTest.java
index 08e031a..6ac8dce 100644
---
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactoryTest.java
+++
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceFactoryTest.java
@@ -97,7 +97,7 @@ public class HttpLookupTableSourceFactoryTest {
@Test
void shouldThrowIfMissingUrl() {
- Map<String, String> options = Collections.singletonMap("connector",
"rest-lookup");
+ Map<String, String> options = Collections.singletonMap("connector",
"http");
assertThatExceptionOfType(ValidationException.class)
.isThrownBy(() -> createTableSource(SCHEMA, options));
}
@@ -120,7 +120,7 @@ public class HttpLookupTableSourceFactoryTest {
private Map<String, String> getMandatoryOptions() {
return Map.of(
- "connector", "rest-lookup",
+ "connector", "http",
"url", "http://localhost:" +
WireMockServerPortAllocator.PORT_BASE + "/service",
"format", "json");
}
diff --git
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceITCaseTest.java
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceITCaseTest.java
index ade60a0..1ffbf63 100644
---
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceITCaseTest.java
+++
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceITCaseTest.java
@@ -182,7 +182,7 @@ class HttpLookupTableSourceITCaseTest {
+ ">"
+ ") WITH ("
+ "'format' = 'json',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ ((StringUtils.isNullOrWhitespaceOnly(methodName))
? ""
: "'lookup-method' = '" + methodName + "',")
@@ -222,7 +222,7 @@ class HttpLookupTableSourceITCaseTest {
+ ">"
+ ") WITH ("
+ "'format' = 'json',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'url' = 'http://localhost:"
+ serverPort
+ "/client',"
@@ -269,7 +269,7 @@ class HttpLookupTableSourceITCaseTest {
+ ">"
+ ") WITH ("
+ "'format' = 'json',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'url' = 'http://localhost:"
+ serverPort
+ "/client',"
@@ -322,7 +322,7 @@ class HttpLookupTableSourceITCaseTest {
+ ">"
+ ") WITH ("
+ "'format' = 'json',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'url' = 'http://localhost:"
+ serverPort
+ "/client',"
@@ -362,7 +362,7 @@ class HttpLookupTableSourceITCaseTest {
+ ">"
+ ") WITH ("
+ "'format' = 'json',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'url' = 'https://localhost:"
+ secServerPort
+ "/client',"
@@ -421,7 +421,7 @@ class HttpLookupTableSourceITCaseTest {
+ "'format' = 'json',"
+ "'lookup-request.format' = 'json',"
+ "'lookup-request.format.json.fail-on-missing-field'
= 'true',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'lookup-method' = 'POST',"
+ "'url' = 'http://localhost:"
+ serverPort
@@ -497,7 +497,7 @@ class HttpLookupTableSourceITCaseTest {
+ "'format' = 'json',"
+ "'lookup-request.format' = 'json',"
+ "'lookup-request.format.json.fail-on-missing-field'
= 'true',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'lookup-method' = 'POST',"
+ "'url' = 'http://localhost:"
+ serverPort
@@ -573,7 +573,7 @@ class HttpLookupTableSourceITCaseTest {
+ "'format' = 'json',"
+ "'lookup-request.format' = 'json',"
+ "'lookup-request.format.json.fail-on-missing-field'
= 'true',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'lookup-method' = 'POST',"
+ "'url' = 'http://localhost:"
+ serverPort
@@ -643,7 +643,7 @@ class HttpLookupTableSourceITCaseTest {
+ "'format' = 'json',"
+ "'lookup-request.format' = 'json',"
+ "'lookup-request.format.json.fail-on-missing-field'
= 'true',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'lookup-method' = 'POST',"
+ "'url' = 'http://localhost:"
+ serverPort
@@ -753,7 +753,7 @@ class HttpLookupTableSourceITCaseTest {
+ fields
+ ") WITH ("
+ "'format' = 'json',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'lookup-method' = 'POST',"
+ "'url' = 'http://localhost:"
+ serverPort
@@ -863,7 +863,7 @@ class HttpLookupTableSourceITCaseTest {
+ "'lookup-request.format' = 'json',"
+ "'lookup-request.format.json.fail-on-missing-field'
= 'true',"
+ "'lookup-method' = 'POST',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'url' = 'http://localhost:"
+ serverPort
+ "/client',"
@@ -933,7 +933,7 @@ class HttpLookupTableSourceITCaseTest {
+ ">"
+ ") WITH ("
+ "'format' = 'json',"
- + "'connector' = 'rest-lookup',"
+ + "'connector' = 'http',"
+ "'lookup-method' = 'GET',"
+ "'url' = 'http://localhost:"
+ serverPort
@@ -1528,7 +1528,7 @@ class HttpLookupTableSourceITCaseTest {
.append("completionState STRING METADATA from
'http-completion-state'");
}
- sql.append(") WITH (").append("'format' =
'json',").append("'connector' = 'rest-lookup',");
+ sql.append(") WITH (").append("'format' =
'json',").append("'connector' = 'http',");
if (!StringUtils.isNullOrWhitespaceOnly(spec.methodName)) {
sql.append("'lookup-method' =
'").append(spec.methodName).append("',");
diff --git
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceTest.java
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceTest.java
index 7bae0f2..429f30b 100644
---
a/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceTest.java
+++
b/flink-connector-http/src/test/java/org/apache/flink/connector/http/table/lookup/HttpLookupTableSourceTest.java
@@ -345,7 +345,7 @@ class HttpLookupTableSourceTest {
private Map<String, String> getOptions() {
return Map.of(
- "connector", "rest-lookup",
+ "connector", "http",
"url", "http://localhost:8080/service",
"format", "json");
}