This is an automated email from the ASF dual-hosted git repository. chesnay pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git
commit 66f996aecaf501f24a4d883a8d7bca9544d0db3b Author: Chesnay Schepler <[email protected]> AuthorDate: Wed Sep 14 10:21:06 2022 +0200 [FLINK-29287][tests] Rewrite sql-client packaging tests --- flink-connectors/flink-sql-connector-kafka/pom.xml | 6 +++ .../flink/connectors/kafka/PackagingITCase.java | 40 +++++++++++++++++++ .../flink-sql-connector-kinesis/pom.xml | 6 +++ .../flink/connectors/kinesis/PackagingITCase.java | 46 ++++++++++++++++++++++ .../test-scripts/test_sql_client.sh | 41 ------------------- flink-formats/flink-sql-avro/pom.xml | 6 +++ .../apache/flink/formats/avro/PackagingITCase.java | 40 +++++++++++++++++++ 7 files changed, 144 insertions(+), 41 deletions(-) diff --git a/flink-connectors/flink-sql-connector-kafka/pom.xml b/flink-connectors/flink-sql-connector-kafka/pom.xml index caf662ba375..da576b11566 100644 --- a/flink-connectors/flink-sql-connector-kafka/pom.xml +++ b/flink-connectors/flink-sql-connector-kafka/pom.xml @@ -44,6 +44,12 @@ under the License. <artifactId>flink-connector-kafka</artifactId> <version>${project.version}</version> </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-test-utils</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> </dependencies> <build> diff --git a/flink-connectors/flink-sql-connector-kafka/src/test/java/org/apache/flink/connectors/kafka/PackagingITCase.java b/flink-connectors/flink-sql-connector-kafka/src/test/java/org/apache/flink/connectors/kafka/PackagingITCase.java new file mode 100644 index 00000000000..cb3a761f99f --- /dev/null +++ b/flink-connectors/flink-sql-connector-kafka/src/test/java/org/apache/flink/connectors/kafka/PackagingITCase.java @@ -0,0 +1,40 @@ +/* + * 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.flink.connectors.kafka; + +import org.apache.flink.packaging.PackagingTestUtils; +import org.apache.flink.table.factories.Factory; +import org.apache.flink.test.resources.ResourceTestUtils; + +import org.junit.jupiter.api.Test; + +import java.nio.file.Path; +import java.util.Arrays; + +class PackagingITCase { + + @Test + void testPackaging() throws Exception { + final Path jar = ResourceTestUtils.getResource(".*/flink-sql-connector-kafka[^/]*\\.jar"); + + PackagingTestUtils.assertJarContainsOnlyFilesMatching( + jar, Arrays.asList("org/apache/flink/", "META-INF/")); + PackagingTestUtils.assertJarContainsServiceEntry(jar, Factory.class); + } +} diff --git a/flink-connectors/flink-sql-connector-kinesis/pom.xml b/flink-connectors/flink-sql-connector-kinesis/pom.xml index 9a4a485cd22..d59800dc75f 100644 --- a/flink-connectors/flink-sql-connector-kinesis/pom.xml +++ b/flink-connectors/flink-sql-connector-kinesis/pom.xml @@ -44,6 +44,12 @@ under the License. <artifactId>flink-connector-kinesis</artifactId> <version>${project.version}</version> </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-test-utils</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> </dependencies> <build> diff --git a/flink-connectors/flink-sql-connector-kinesis/src/test/java/org/apache/flink/connectors/kinesis/PackagingITCase.java b/flink-connectors/flink-sql-connector-kinesis/src/test/java/org/apache/flink/connectors/kinesis/PackagingITCase.java new file mode 100644 index 00000000000..fb4404cab82 --- /dev/null +++ b/flink-connectors/flink-sql-connector-kinesis/src/test/java/org/apache/flink/connectors/kinesis/PackagingITCase.java @@ -0,0 +1,46 @@ +/* + * 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.flink.connectors.kinesis; + +import org.apache.flink.packaging.PackagingTestUtils; +import org.apache.flink.table.factories.Factory; +import org.apache.flink.test.resources.ResourceTestUtils; + +import org.junit.jupiter.api.Test; + +import java.nio.file.Path; +import java.util.Arrays; + +class PackagingITCase { + + @Test + void testPackaging() throws Exception { + final Path jar = ResourceTestUtils.getResource(".*/flink-sql-connector-kinesis[^/]*\\.jar"); + + PackagingTestUtils.assertJarContainsOnlyFilesMatching( + jar, + Arrays.asList( + "org/apache/flink/", + "META-INF/", + "amazon-kinesis-producer-native-binaries/", + "cacerts/", + "google/")); + PackagingTestUtils.assertJarContainsServiceEntry(jar, Factory.class); + } +} diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index ea6a343589b..3cb3a22f8cd 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -40,47 +40,6 @@ source "$(dirname "$0")"/elasticsearch-common.sh SQL_TOOLBOX_JAR=$END_TO_END_DIR/flink-sql-client-test/target/SqlToolbox.jar SQL_JARS_DIR=$END_TO_END_DIR/flink-sql-client-test/target/sql-jars -################################################################################ -# Verify existing SQL jars -################################################################################ - -EXTRACTED_JAR=$TEST_DATA_DIR/extracted - -mkdir -p $EXTRACTED_JAR - -for SQL_JAR in $SQL_JARS_DIR/*.jar; do - echo "Checking SQL JAR: $SQL_JAR" - (cd $EXTRACTED_JAR && jar xf $SQL_JAR) - - # check for proper shading - for EXTRACTED_FILE in $(find $EXTRACTED_JAR -type f); do - - if ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/org/apache/flink"* ]] && \ - ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/META-INF"* ]] && \ - ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/LICENSE"* ]] && \ - ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/NOTICE"* ]] && \ - ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/org/apache/avro"* ]] && \ - # Following required by amazon-kinesis-producer in flink-connector-kinesis - ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/amazon-kinesis-producer-native-binaries"* ]] && \ - ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/cacerts"* ]] && \ - ! [[ $EXTRACTED_FILE = "$EXTRACTED_JAR/google"* ]] ; then - echo "Bad file in JAR: $EXTRACTED_FILE" - exit 1 - fi - done - - # check for table factory - if [ ! -f $EXTRACTED_JAR/META-INF/services/org.apache.flink.table.factories.Factory ]; then - echo "No table factory found in JAR: $SQL_JAR" - exit 1 - fi - - # clean up - rm -r $EXTRACTED_JAR/* -done - -rm -r $EXTRACTED_JAR - ################################################################################ # Prepare connectors ################################################################################ diff --git a/flink-formats/flink-sql-avro/pom.xml b/flink-formats/flink-sql-avro/pom.xml index 49c82e3b5c4..bdab96c2c06 100644 --- a/flink-formats/flink-sql-avro/pom.xml +++ b/flink-formats/flink-sql-avro/pom.xml @@ -43,6 +43,12 @@ under the License. <artifactId>flink-avro</artifactId> <version>${project.version}</version> </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-test-utils</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> </dependencies> <build> diff --git a/flink-formats/flink-sql-avro/src/test/java/org/apache/flink/formats/avro/PackagingITCase.java b/flink-formats/flink-sql-avro/src/test/java/org/apache/flink/formats/avro/PackagingITCase.java new file mode 100644 index 00000000000..3aed948864f --- /dev/null +++ b/flink-formats/flink-sql-avro/src/test/java/org/apache/flink/formats/avro/PackagingITCase.java @@ -0,0 +1,40 @@ +/* + * 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.flink.formats.avro; + +import org.apache.flink.packaging.PackagingTestUtils; +import org.apache.flink.table.factories.Factory; +import org.apache.flink.test.resources.ResourceTestUtils; + +import org.junit.jupiter.api.Test; + +import java.nio.file.Path; +import java.util.Arrays; + +class PackagingITCase { + + @Test + void testPackaging() throws Exception { + final Path jar = ResourceTestUtils.getResource(".*/flink-sql-avro[^/]*\\.jar"); + + PackagingTestUtils.assertJarContainsOnlyFilesMatching( + jar, Arrays.asList("org/apache/flink/", "META-INF/")); + PackagingTestUtils.assertJarContainsServiceEntry(jar, Factory.class); + } +}
