Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
import org.apache.flink.runtime.client.JobInitializationException;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.rest.HttpHeader;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkUserCodeClassLoaders;
import org.apache.flink.util.SerializedThrowable;
Expand All @@ -43,6 +44,8 @@

import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.Executors;
Expand Down Expand Up @@ -185,4 +188,24 @@ public static ScheduledExecutorService reportHeartbeatPeriodically(
TimeUnit.MILLISECONDS);
return scheduledExecutor;
}

public static Collection<HttpHeader> readHeadersFromEnvironmentVariable(String envVarName) {
List<HttpHeader> headers = new ArrayList<>();
String rawHeaders = System.getenv(envVarName);

if (rawHeaders != null) {
String[] lines = rawHeaders.split("\n");
for (String line : lines) {
String[] keyValue = line.split(":", 2);
if (keyValue.length == 2) {
headers.add(new HttpHeader(keyValue[0], keyValue[1]));
} else {
LOG.info(
"Skipped a malformed header {} from FLINK_REST_CLIENT_HEADERS env variable. Expecting newline-separated headers in format header_name:header_value.",
line);
}
}
}
return headers;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,18 +16,20 @@
* limitations under the License.
*/

package org.apache.flink.table.gateway.rest.header.util;
package org.apache.flink.client.program.rest;

import org.apache.flink.runtime.rest.HttpMethodWrapper;
import org.apache.flink.runtime.rest.RestClient;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.runtime.rest.messages.MessageParameters;
import org.apache.flink.runtime.rest.messages.RequestBody;
import org.apache.flink.runtime.rest.messages.ResponseBody;
import org.apache.flink.table.gateway.rest.header.SqlGatewayMessageHeaders;
import org.apache.flink.runtime.rest.versioning.RestAPIVersion;

import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;

import java.util.Collection;

import static org.apache.flink.shaded.guava31.com.google.common.base.Preconditions.checkNotNull;

/**
Expand All @@ -41,7 +43,7 @@
*/
public class UrlPrefixDecorator<
R extends RequestBody, P extends ResponseBody, M extends MessageParameters>
implements SqlGatewayMessageHeaders<R, P, M> {
implements MessageHeaders<R, P, M> {

private final String prefixedUrl;
private final MessageHeaders<R, P, M> decorated;
Expand Down Expand Up @@ -100,4 +102,9 @@ public Class<R> getRequestClass() {
public M getUnresolvedMessageParameters() {
return decorated.getUnresolvedMessageParameters();
}

@Override
public Collection<? extends RestAPIVersion<?>> getSupportedAPIVersions() {
return decorated.getSupportedAPIVersions();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
package org.apache.flink.table.client.gateway;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.client.ClientUtils;
import org.apache.flink.client.program.rest.UrlPrefixDecorator;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
Expand Down Expand Up @@ -52,7 +54,6 @@
import org.apache.flink.table.gateway.rest.header.statement.ExecuteStatementHeaders;
import org.apache.flink.table.gateway.rest.header.statement.FetchResultsHeaders;
import org.apache.flink.table.gateway.rest.header.util.GetApiVersionHeaders;
import org.apache.flink.table.gateway.rest.header.util.UrlPrefixDecorator;
import org.apache.flink.table.gateway.rest.message.operation.OperationMessageParameters;
import org.apache.flink.table.gateway.rest.message.operation.OperationStatusResponseBody;
import org.apache.flink.table.gateway.rest.message.session.CloseSessionResponseBody;
Expand Down Expand Up @@ -84,7 +85,6 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
Expand Down Expand Up @@ -178,7 +178,8 @@ public ExecutorImpl(DefaultContext defaultContext, URL gatewayUrl, String sessio
this.gatewayUrl = gatewayUrl;
this.rowFormat = rowFormat;
this.customHttpHeaders =
readHeadersFromEnvironmentVariable(ConfigConstants.FLINK_REST_CLIENT_HEADERS);
ClientUtils.readHeadersFromEnvironmentVariable(
ConfigConstants.FLINK_REST_CLIENT_HEADERS);
try {
// register required resource
this.executorService = Executors.newCachedThreadPool();
Expand Down Expand Up @@ -595,26 +596,6 @@ private void closeSession() throws SqlExecutionException {
}
}

private static Collection<HttpHeader> readHeadersFromEnvironmentVariable(String envVarName) {
List<HttpHeader> headers = new ArrayList<>();
String rawHeaders = System.getenv(envVarName);

if (rawHeaders != null) {
String[] lines = rawHeaders.split("\n");
for (String line : lines) {
String[] keyValue = line.split(":", 2);
if (keyValue.length == 2) {
headers.add(new HttpHeader(keyValue[0], keyValue[1]));
} else {
LOG.info(
"Skipped a malformed header {} from FLINK_REST_CLIENT_HEADERS env variable. Expecting newline-separated headers in format header_name:header_value.",
line);
}
}
}
return headers;
}

@VisibleForTesting
Collection<HttpHeader> getCustomHttpHeaders() {
return customHttpHeaders;
Expand Down