[
https://issues.apache.org/jira/browse/FLINK-37186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17927899#comment-17927899
]
Feng Jin commented on FLINK-37186:
----------------------------------
*FLIP-480 Test Report*
*Conclusion:*
The overall behavior meets expectations. However, in the *Kubernetes
Application* mode, there is an issue when submitting scripts using the *SQL
Client Embedded* mode.
*Test Procedures:*
*YARN-Application Test:*
*1. Start the Hadoop Cluster*
{{}}
{code:java}
{code}
{{export END_TO_END_DIR=flink/flink-end-to-end-tests/ docker compose -f
"${END_TO_END_DIR}/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml"
up -d}}
*2. Prepare the Flink Environment*
*2.1 Build Flink Package*
{{}}
{code:java}
{code}
{{mvn clean install -DskipTests -Dfast}}
*2.2 Package and Copy Flink to the Hadoop Docker Container*
{{}}
{code:java}
{code}
{{tar czf flink.tar.gz -C
flink/flink-dist/target/flink-2.0-SNAPSHOT-bin/flink-2.0-SNAPSHOT/ . docker cp
flink.tar.gz master:/home/hadoop-user/ docker exec master bash -c "tar xzf
/home/hadoop-user/flink.tar.gz --directory /home/hadoop-user/" # Copy OpenJDK
17 image docker cp openjdk-17.0.2_linux-x64_bin.tar.gz
master:/home/hadoop-user/}}
*3. Prepare Hadoop and Flink Packages and Configurations*
{{}}
{code:java}
{code}
{{docker exec -it master bash cd /home/hadoop-user/ export
JAVA_HOME=./jdk-17.0.2 export FLINK_HOME=./ export FLINK_CONF_DIR=./conf export
HADOOP_CLASSPATH=$($HADOOP_HOME/bin/hadoop classpath)}}
*4. Deploy SQL Script*
{{}}
{code:java}
{code}
{{./bin/sql-client.sh -f ./script.sql -j \ -Dexecution.target=yarn-application
\ -Dsecurity.kerberos.login.keytab=/home/hadoop-user/hadoop-user.keytab \
-Dsecurity.kerberos.login.principal=hadoop-user \
-Dyarn.ship-archives=openjdk-17.0.2_linux-x64_bin.tar.gz \
-Denv.java.home=./openjdk-17.0.2_linux-x64_bin.tar.gz/jdk-17.0.2}}
*5. Verify Results*
•The -f argument is correctly converted to the yarn.ship-files parameter:
• *JobManager Log:*
{{2025-02-17 08:54:11,316 INFO
org.apache.flink.configuration.GlobalConfiguration
Loading configuration property: yarn.ship-files,
[./flink-table-filesystem-test-utils-2.0-SNAPSHOT.jar,
/home/hadoop-user/opt/flink-sql-client-2.0-SNAPSHOT.jar]}}
•The job is successfully started and writing to the expected directory:
{code:java}
root@master:/home/hadoop-user# hdfs dfs -ls /user/hadoop-user/test-sink/ Found
1 items -rw-r--r-- 1 hadoop-user hadoop-user 0 2025-02-17 08:54
/user/hadoop-user/test-sink/.part-90dfaa0a-e4ca-48ea-b34e-af2f02c0579c-0-0.inprogress.17a27e53-1efc-4e3b-a1ff-9d51ce9d16e0
{code}
{{}}
*Kubernetes-Application Test:*
*1. Minikube Setup*
{{minikube start}}
*2. Build the Flink Image*
Since this is a test version, the official repository does not provide this
version, requiring a manual build.
*2.1 Compile Flink Source Code*
{{}}
{code:java}
mvn clean install -DskipTests -Dfast -fn{code}
{{}}
*2.2 Build the Docker Image*
{{}}
{code:java}
FROM flink:2.0-scala_2.12-java17# Define build arguments (default: /opt/flink)
ARG FLINK_PATH=/opt/flink
ARG LOCAL_FLINK_PATH=flink# Remove original FLINK_PATH directories
RUN rm -rf $FLINK_PATH/lib $FLINK_PATH/opt $FLINK_PATH/bin# Copy compiled Flink
files into the image
COPY $LOCAL_FLINK_PATH/lib $FLINK_PATH/lib
COPY $LOCAL_FLINK_PATH/opt $FLINK_PATH/opt
COPY $LOCAL_FLINK_PATH/bin $FLINK_PATH/bin
COPY $LOCAL_FLINK_PATH/conf/config.yaml $FLINK_PATH/conf/config.yaml
COPY tmp /tmp# Change ownership to flink:flink
RUN chown -R flink:flink $FLINK_PATH
{code}
{{}}
{{}}
*2.3 Build and Load Image*
{{}}
{code:java}
docker build -t test_kubernetes_application-1:latest . minikube image load
test_kubernetes_application-1:latest{code}
{{}}
*3. Prepare S3 Environment*
{code:java}
docker run -d -P --mount type=bind,source="./s3_test_data",target=/data \
-e "MINIO_ACCESS_KEY=access_key" -e "MINIO_SECRET_KEY=secret_key" -e
"MINIO_DOMAIN=localhost" \
minio/minio server /data# Get S3 endpoint
export S3_ENDPOINT="http://$(docker port "$CONTAINER_ID" 9000 | grep -F
'0.0.0.0' | sed 's/0\.0\.0\.0/localhost/')"
S3_ENDPOINT=$(echo $S3_ENDPOINT | sed 's/localhost/xx.xx.xx.xx/') # Replace
with local IP
{code}
*4. Deploy SQL Script in Embedded Mode*
{code:java}
//代码占位符
CREATE TABLE test_source(a varchar) WITH ('connector' = 'datagen');
CREATE TABLE test_sink(a varchar) WITH (
'connector' = 'test-filesystem',
'path' = 's3://test_bucket',
'format' = 'json'
);
INSERT INTO test_sink SELECT * FROM test_source; {code}
{code:java}
./bin/sql-client.sh -f ./script.sql -j
s3://jar-bucket/flink-table-filesystem-test-utils-2.0-SNAPSHOT.jar \
-Dexecution.target=kubernetes-application \
-Dkubernetes.container.image.ref=test_kubernetes_application-1:latest \
-Djobmanager.memory.process.size=2088m -Dtaskmanager.memory.process.size=2000m \
-Ds3.access-key=access_key -Ds3.secret-key=secret_key
-Ds3.endpoint=http://xx.xx.xx.xx:32768 {code}
{{}}
*Issue in Embedded Mode*
•In *Embedded Mode*, sql-client gets included in the --jar parameter, leading
to an error:
{{}}
{code:java}
Caused by: java.lang.RuntimeException: java.io.FileNotFoundException:
/home/feng/flink-cluster/image-build/flink/opt/flink-sql-client-2.0-SNAPSHOT.jar
(No such file or directory){code}
{{}}
*Gateway Mode Deployment of SQL Script*
{code:java}
ADD JAR 's3://jar-bucket/flink-table-filesystem-test-utils-2.0-SNAPSHOT.jar';
CREATE TABLE test_source(a varchar) WITH ('connector' = 'datagen');
CREATE TABLE test_sink(a varchar) WITH (
'connector' = 'test-filesystem',
'path' = 's3://test_bucket',
'format' = 'json'
);
INSERT INTO test_sink SELECT * FROM test_source;
{code}
{{}}
{code:java}
./bin/sql-client.sh gateway --endpoint 127.0.0.1:8083 -f ./script_s3.sql \
-Dexecution.target=kubernetes-application \
-Dkubernetes.container.image.ref=test_kubernetes_application-1:latest \
-Djobmanager.memory.process.size=2088m \
-Dtaskmanager.memory.process.size=2000m \ -Ds3.access-key=access_key \
-Ds3.secret-key=secret_key \ -Ds3.endpoint=http://xx.xx.xx.xx:32768{code}
{{}}
*Results in Gateway Mode*
•The job is successfully submitted.
*Final Verification Results*
✅ *YARN-Application Mode*: Working as expected.
❌ *Kubernetes-Application (Embedded Mode)*: Issue due to incorrect --jar
argument handling.
✅ {*}Kubernetes-Application (Gateway Mode){*}: Working as expected.
> Cross-team verification for "FLIP-480: Support to deploy SQL script in
> application mode"
> ----------------------------------------------------------------------------------------
>
> Key: FLINK-37186
> URL: https://issues.apache.org/jira/browse/FLINK-37186
> Project: Flink
> Issue Type: Sub-task
> Components: Table SQL / Gateway
> Affects Versions: 2.0.0
> Reporter: Shengkai Fang
> Assignee: Feng Jin
> Priority: Major
>
> Please take a look at the FLIP-480:
> [https://cwiki.apache.org/confluence/display/FLINK/FLIP-480%3A+Support+to+deploy+SQL+script+in+application+mode]
>
> For testers, you can validate the correctness and functionality with large
> datasets using the following steps:
> Preparation Steps:
> # Prepare a Kubernetes environment
> # Compile the branch and build the image according to
> [https://github.com/apache/flink-docker/blob/dev-master/README.md]
> # Push the docker images to the docker hub that your K8S environment can
> access
>
> Validation
> Steps:[|https://issues.apache.org/jira/secure/AddComment!default.jspa?id=13604232]
> # Start the sql-gateway
> # Use the REST API to submit script to the environment
> # Use ADD JAR command to add jar from the remote oss
--
This message was sent by Atlassian Jira
(v8.20.10#820010)