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 ab9580d  [FLINK-38803] Add HTTP response deserialize failed completion 
state
ab9580d is described below

commit ab9580de7bea5487b01920d9b18072f6d5829f47
Author: David Radley <[email protected]>
AuthorDate: Sat Dec 27 21:43:49 2025 +0000

    [FLINK-38803] Add HTTP response deserialize failed completion state
---
 docs/content.zh/docs/connectors/table/http.md      | 34 +++++++++++++---------
 docs/content/docs/connectors/table/http.md         | 34 +++++++++++++---------
 .../http/table/lookup/HttpCompletionState.java     |  3 +-
 .../table/lookup/JavaNetHttpPollingClient.java     |  4 +--
 .../lookup/HttpLookupTableSourceITCaseTest.java    | 11 +++----
 5 files changed, 52 insertions(+), 34 deletions(-)

diff --git a/docs/content.zh/docs/connectors/table/http.md 
b/docs/content.zh/docs/connectors/table/http.md
index 60a9fef..b49c915 100644
--- a/docs/content.zh/docs/connectors/table/http.md
+++ b/docs/content.zh/docs/connectors/table/http.md
@@ -32,7 +32,6 @@ under the License.
 {{< label "Lookup Source: Async Mode" >}}
 {{< label "Sink: Batch" >}}
 
-
 The HTTP connector allows for pulling data from external system via HTTP 
methods and HTTP Sink that allows for sending data to external system via HTTP 
requests.
 
 The HTTP source connector supports [Lookup 
Joins](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sourcessinks/#lookup-table-source)
 in [Table API and 
SQL](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/overview/).
@@ -40,18 +39,22 @@ The HTTP source connector supports [Lookup 
Joins](https://nightlies.apache.org/f
 <!-- TOC -->
 * [HTTP Connector](#http-connector)
   * [Dependencies](#dependencies)
+  * [Migration from GetInData HTTP 
connector](#migration-from-getindata-http-connector)
   * [Working with HTTP lookup source 
tables](#working-with-http-lookup-source-tables)
     * [HTTP Lookup Table API and SQL Source 
example](#http-lookup-table-api-and-sql-source-example)
     * [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)
     * [http-generic-json-url Query 
Creator](#http-generic-json-url-query-creator)
+    * [Format considerations](#format-considerations)
+      * [For HTTP requests](#for-http-requests)
+      * [For HTTP responses](#for-http-responses)
     * [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)
-    * [Retries (Lookup source)](#retries-lookup-source)
-      * [Retry strategy](#retry-strategy)
+    * [Retries and handling errors (Lookup 
source)](#retries-and-handling-errors-lookup-source)
+        * [Retry strategy](#retry-strategy)
       * [Lookup multiple results](#lookup-multiple-results)
   * [Working with HTTP sink tables](#working-with-http-sink-tables)
     * [HTTP Sink](#http-sink)
@@ -61,11 +64,13 @@ The HTTP source connector supports [Lookup 
Joins](https://nightlies.apache.org/f
     * [Batch submission mode](#batch-submission-mode)
     * [Single submission mode](#single-submission-mode)
   * [Available Metadata](#available-metadata)
+    * [http-completion-state possible 
values](#http-completion-state-possible-values)
   * [HTTP status code handler](#http-status-code-handler)
   * [Security considerations](#security-considerations)
     * [TLS (more secure replacement for SSL) and mTLS 
support](#tls-more-secure-replacement-for-ssl-and-mtls-support)
     * [Basic Authentication](#basic-authentication)
     * [OIDC Bearer Authentication](#oidc-bearer-authentication)
+  * [Logging the HTTP content](#logging-the-http-content)
       * [Restrictions at this time](#restrictions-at-this-time)
 <!-- TOC -->
 ## Dependencies
@@ -502,22 +507,25 @@ Metadata columns can be specified and hold http 
information. They are optional r
 
 | Key                   | Data Type                        | Description       
                     |
 
|-----------------------|----------------------------------|----------------------------------------|
-| error-string          | STRING NULL                      | A message 
associated with the error    |
+| error-string          | STRING NULL                      | A string 
associated with the error     |
 | http-status-code      | INT NULL                         | The HTTP status 
code                   |
 | http-headers-map      | MAP <STRING, ARRAY<STRING>> NULL | The headers 
returned with the response |
 | http-completion-state | STRING NULL                      | The completion 
state of the http call. |
 
 ### http-completion-state possible values
 
-| Value             | Description            |
-|:------------------|------------------------|
-| SUCCESS           | Success                |
-| HTTP_ERROR_STATUS | HTTP error status code |
-| EXCEPTION         | An Exception occurred  |
+| Value                          | Description                         |
+|:-------------------------------|-------------------------------------|
+| SUCCESS                        | Success                             |
+| HTTP_ERROR_STATUS              | HTTP error status code              |
+| EXCEPTION                      | An Exception occurred               |
+| UNABLE_TO_DESERIALIZE_RESPONSE | Unable to deserialize HTTP response |
 
 If the `error-string` metadata column is defined on the table and the call 
succeeds then it will have a null value.
+When the HTTP response cannot be deserialized, then the 
`http-completion-state` will be `UNABLE_TO_DESERIALIZE_RESPONSE`
+and the `error-string` will be the response body.
 
-When a http lookup call fails and populates the metadata columns with the 
error information, the expected enrichment columns from the http call
+When a HTTP lookup call fails and populates the metadata columns with the 
error information, the expected enrichment columns from the HTTP call
 are not populated, this means that they will be null for nullable columns and 
hold a default value for the type for non-nullable columns.
 
 If you are using the Table API `TableResult` and have an `await` with a 
timeout, this Timeout exception will cause the job to terminate,
@@ -562,8 +570,8 @@ To disable retries, set `'lookup.max-retries' = '0'`.
 
 ### TLS (more secure replacement for SSL) and mTLS support
 
-Both Http Sink and Lookup Source connectors support HTTPS communication using 
TLS 1.2 and mTLS.
-To enable Https communication simply use `https` protocol in endpoint's URL.
+Both HTTP Sink and Lookup Source connectors support HTTPS communication using 
TLS 1.2 and mTLS.
+To enable HTTPS communication simply use `https` protocol in endpoint's URL.
 
 To specify certificate(s) to be used by the server, use 
`http.security.cert.server` connector property;
 the value is a comma separated list of paths to certificate(s), for example 
you can use your organization's CA
@@ -577,7 +585,7 @@ and `http.security.key.client` connector properties to 
specify paths to the cert
 private key. The key MUST be in `PCKS8` format. Both PEM and DER keys are
 allowed.
 
-For non production environments it is sometimes necessary to use Https 
connection and accept all certificates.
+For non production environments it is sometimes necessary to use HTTPS 
connection and accept all certificates.
 In this special case, you can configure connector to trust all certificates 
without adding them to keystore.
 To enable this option use `http.security.cert.server.allowSelfSigned` property 
setting its value to `true`.
 
diff --git a/docs/content/docs/connectors/table/http.md 
b/docs/content/docs/connectors/table/http.md
index 60a9fef..0738dde 100644
--- a/docs/content/docs/connectors/table/http.md
+++ b/docs/content/docs/connectors/table/http.md
@@ -32,7 +32,6 @@ under the License.
 {{< label "Lookup Source: Async Mode" >}}
 {{< label "Sink: Batch" >}}
 
-
 The HTTP connector allows for pulling data from external system via HTTP 
methods and HTTP Sink that allows for sending data to external system via HTTP 
requests.
 
 The HTTP source connector supports [Lookup 
Joins](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sourcessinks/#lookup-table-source)
 in [Table API and 
SQL](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/overview/).
@@ -40,17 +39,21 @@ The HTTP source connector supports [Lookup 
Joins](https://nightlies.apache.org/f
 <!-- TOC -->
 * [HTTP Connector](#http-connector)
   * [Dependencies](#dependencies)
+  * [Migration from GetInData HTTP 
connector](#migration-from-getindata-http-connector)
   * [Working with HTTP lookup source 
tables](#working-with-http-lookup-source-tables)
     * [HTTP Lookup Table API and SQL Source 
example](#http-lookup-table-api-and-sql-source-example)
     * [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)
     * [http-generic-json-url Query 
Creator](#http-generic-json-url-query-creator)
+    * [Format considerations](#format-considerations)
+      * [For HTTP requests](#for-http-requests)
+      * [For HTTP responses](#for-http-responses)
     * [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)
-    * [Retries (Lookup source)](#retries-lookup-source)
+    * [Retries and handling errors (Lookup 
source)](#retries-and-handling-errors-lookup-source)
       * [Retry strategy](#retry-strategy)
       * [Lookup multiple results](#lookup-multiple-results)
   * [Working with HTTP sink tables](#working-with-http-sink-tables)
@@ -61,12 +64,14 @@ The HTTP source connector supports [Lookup 
Joins](https://nightlies.apache.org/f
     * [Batch submission mode](#batch-submission-mode)
     * [Single submission mode](#single-submission-mode)
   * [Available Metadata](#available-metadata)
+    * [http-completion-state possible 
values](#http-completion-state-possible-values)
   * [HTTP status code handler](#http-status-code-handler)
   * [Security considerations](#security-considerations)
     * [TLS (more secure replacement for SSL) and mTLS 
support](#tls-more-secure-replacement-for-ssl-and-mtls-support)
     * [Basic Authentication](#basic-authentication)
     * [OIDC Bearer Authentication](#oidc-bearer-authentication)
-      * [Restrictions at this time](#restrictions-at-this-time)
+  * [Logging the HTTP content](#logging-the-http-content)
+    * [Restrictions at this time](#restrictions-at-this-time)
 <!-- TOC -->
 ## Dependencies
 
@@ -502,22 +507,25 @@ Metadata columns can be specified and hold http 
information. They are optional r
 
 | Key                   | Data Type                        | Description       
                     |
 
|-----------------------|----------------------------------|----------------------------------------|
-| error-string          | STRING NULL                      | A message 
associated with the error    |
+| error-string          | STRING NULL                      | A string 
associated with the error     |
 | http-status-code      | INT NULL                         | The HTTP status 
code                   |
 | http-headers-map      | MAP <STRING, ARRAY<STRING>> NULL | The headers 
returned with the response |
 | http-completion-state | STRING NULL                      | The completion 
state of the http call. |
 
 ### http-completion-state possible values
 
-| Value             | Description            |
-|:------------------|------------------------|
-| SUCCESS           | Success                |
-| HTTP_ERROR_STATUS | HTTP error status code |
-| EXCEPTION         | An Exception occurred  |
+| Value                          | Description                         |
+|:-------------------------------|-------------------------------------|
+| SUCCESS                        | Success                             |
+| HTTP_ERROR_STATUS              | HTTP error status code              |
+| EXCEPTION                      | An Exception occurred               |
+| UNABLE_TO_DESERIALIZE_RESPONSE | Unable to deserialize HTTP response |
 
 If the `error-string` metadata column is defined on the table and the call 
succeeds then it will have a null value.
+When the HTTP response cannot be deserialized, then the 
`http-completion-state` will be `UNABLE_TO_DESERIALIZE_RESPONSE`
+and the `error-string` will be the response body.
 
-When a http lookup call fails and populates the metadata columns with the 
error information, the expected enrichment columns from the http call
+When a HTTP lookup call fails and populates the metadata columns with the 
error information, the expected enrichment columns from the HTTP call
 are not populated, this means that they will be null for nullable columns and 
hold a default value for the type for non-nullable columns.
 
 If you are using the Table API `TableResult` and have an `await` with a 
timeout, this Timeout exception will cause the job to terminate,
@@ -562,8 +570,8 @@ To disable retries, set `'lookup.max-retries' = '0'`.
 
 ### TLS (more secure replacement for SSL) and mTLS support
 
-Both Http Sink and Lookup Source connectors support HTTPS communication using 
TLS 1.2 and mTLS.
-To enable Https communication simply use `https` protocol in endpoint's URL.
+Both HTTP Sink and Lookup Source connectors support HTTPS communication using 
TLS 1.2 and mTLS.
+To enable HTTPS communication simply use `https` protocol in endpoint's URL.
 
 To specify certificate(s) to be used by the server, use 
`http.security.cert.server` connector property;
 the value is a comma separated list of paths to certificate(s), for example 
you can use your organization's CA
@@ -577,7 +585,7 @@ and `http.security.key.client` connector properties to 
specify paths to the cert
 private key. The key MUST be in `PCKS8` format. Both PEM and DER keys are
 allowed.
 
-For non production environments it is sometimes necessary to use Https 
connection and accept all certificates.
+For non production environments it is sometimes necessary to use HTTPS 
connection and accept all certificates.
 In this special case, you can configure connector to trust all certificates 
without adding them to keystore.
 To enable this option use `http.security.cert.server.allowSelfSigned` property 
setting its value to `true`.
 
diff --git 
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpCompletionState.java
 
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpCompletionState.java
index 91cfff7..6d3a802 100644
--- 
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpCompletionState.java
+++ 
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/HttpCompletionState.java
@@ -21,5 +21,6 @@ package org.apache.flink.connector.http.table.lookup;
 public enum HttpCompletionState {
     HTTP_ERROR_STATUS,
     EXCEPTION,
-    SUCCESS
+    SUCCESS,
+    UNABLE_TO_DESERIALIZE_RESPONSE
 }
diff --git 
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/JavaNetHttpPollingClient.java
 
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/JavaNetHttpPollingClient.java
index ef9b1ed..dd9f60e 100644
--- 
a/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/JavaNetHttpPollingClient.java
+++ 
b/flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/JavaNetHttpPollingClient.java
@@ -278,8 +278,8 @@ public class JavaNetHttpPollingClient implements 
PollingClient {
                 if (!this.continueOnError) {
                     throw e;
                 }
-                httpCompletionState = HttpCompletionState.EXCEPTION;
-                errMessage = e.getMessage();
+                httpCompletionState = 
HttpCompletionState.UNABLE_TO_DESERIALIZE_RESPONSE;
+                errMessage = responseBody;
             }
             return HttpRowDataWrapper.builder()
                     .data(rowData)
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 b126968..ff94415 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
@@ -102,6 +102,8 @@ class HttpLookupTableSourceITCaseTest {
 
     private static final String ENDPOINT = "/client";
 
+    public static final String A_TEST_STRING_THAT_IS_NOT_JSON = "A test string 
that is not json";
+
     /** Comparator for Flink SQL result. */
     private static final Comparator<Row> ROW_COMPARATOR =
             (row1, row2) -> {
@@ -1176,13 +1178,12 @@ class HttpLookupTableSourceITCaseTest {
                         assertThat(row.getField("balance")).isNull();
                         // metadata
                         assertThat(row.getField("errStr"))
-                                .isEqualTo(
-                                        "Failed to deserialize JSON 'A test 
string that is not json'.");
+                                .isEqualTo(A_TEST_STRING_THAT_IS_NOT_JSON);
                         assertThat(row.getField("headers")).isNotNull();
                         assertThat(row.getField("statusCode")).isEqualTo(200);
                         assertEquals(
                                 row.getField("completionState"),
-                                HttpCompletionState.EXCEPTION.name());
+                                
HttpCompletionState.UNABLE_TO_DESERIALIZE_RESPONSE.name());
                     }
                 });
     }
@@ -1309,7 +1310,7 @@ class HttpLookupTableSourceITCaseTest {
             if (isDeserErr) {
                 methodStub.willReturn(
                         aResponse()
-                                .withBody("A test string that is not json")
+                                .withBody(A_TEST_STRING_THAT_IS_NOT_JSON)
                                 .withStatus(200)
                                 .withHeader("Content-Type", "text/plain"));
             } else {
@@ -1517,7 +1518,7 @@ class HttpLookupTableSourceITCaseTest {
                                 .withHeader("Content-Type", 
equalTo("application/json"))
                                 .willReturn(
                                         aResponse()
-                                                .withBody("A test string that 
is not json")
+                                                
.withBody(A_TEST_STRING_THAT_IS_NOT_JSON)
                                                 .withStatus(200)));
             } else {
                 setUpServerBodyStub(

Reply via email to