You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by st...@apache.org on 2022/10/18 21:10:41 UTC
[iceberg] branch master updated: Flink: Fix NoClassDefFound with Flink runtime jar / Add integration test (#6001)
This is an automated email from the ASF dual-hosted git repository.
stevenwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git
The following commit(s) were added to refs/heads/master by this push:
new ef3bbe79f8 Flink: Fix NoClassDefFound with Flink runtime jar / Add integration test (#6001)
ef3bbe79f8 is described below
commit ef3bbe79f8a220799166987e84f19748e451b4fc
Author: Eduard Tudenhöfner <et...@gmail.com>
AuthorDate: Tue Oct 18 23:10:33 2022 +0200
Flink: Fix NoClassDefFound with Flink runtime jar / Add integration test (#6001)
* Flink: Fix NoClassDefFound with Flink runtime jar
* Flink: Add Smoke test
---
flink/v1.14/build.gradle | 74 +++++++++++++++++++++-
.../iceberg/flink/IcebergConnectorSmokeTest.java | 29 +++++++++
flink/v1.15/build.gradle | 74 +++++++++++++++++++++-
.../iceberg/flink/IcebergConnectorSmokeTest.java | 29 +++++++++
4 files changed, 200 insertions(+), 6 deletions(-)
diff --git a/flink/v1.14/build.gradle b/flink/v1.14/build.gradle
index 55ce9c40da..04764a7d98 100644
--- a/flink/v1.14/build.gradle
+++ b/flink/v1.14/build.gradle
@@ -117,9 +117,15 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
tasks.jar.dependsOn tasks.shadowJar
+ sourceSets {
+ integration {
+ java.srcDir "$projectDir/src/integration/java"
+ resources.srcDir "$projectDir/src/integration/resources"
+ }
+ }
+
configurations {
implementation {
- exclude group: 'org.apache.flink'
// included in Flink
exclude group: 'org.slf4j'
exclude group: 'org.apache.commons'
@@ -132,7 +138,9 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
}
dependencies {
- implementation project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")
+ implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) {
+ exclude group: 'org.apache.flink'
+ }
implementation project(':iceberg-aws')
implementation(project(':iceberg-aliyun')) {
exclude group: 'edu.umd.cs.findbugs', module: 'findbugs'
@@ -144,7 +152,57 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
}
// for dropwizard histogram metrics implementation
- implementation "org.apache.flink:flink-metrics-dropwizard:${flinkMajorVersion}"
+ implementation "org.apache.flink:flink-metrics-dropwizard:${flinkVersion}"
+
+ // for integration testing with the flink-runtime-jar
+ // all of those dependencies are required because the integration test extends FlinkTestBase
+ integrationCompileOnly project(':iceberg-api')
+ integrationImplementation 'org.junit.vintage:junit-vintage-engine'
+ integrationImplementation 'org.assertj:assertj-core'
+ integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts")
+ integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')
+ integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
+ integrationImplementation("org.apache.flink:flink-test-utils_${scalaVersion}:${flinkVersion}") {
+ exclude group: "org.apache.curator", module: 'curator-test'
+ exclude group: 'junit'
+ }
+
+ integrationImplementation "org.apache.flink:flink-table-api-java-bridge_${scalaVersion}:${flinkVersion}"
+ integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${flinkVersion}"
+
+ integrationImplementation "org.apache.hadoop:hadoop-common"
+ integrationImplementation "org.apache.hadoop:hadoop-hdfs"
+ integrationImplementation("org.apache.hadoop:hadoop-minicluster") {
+ exclude group: 'org.apache.avro', module: 'avro'
+ }
+
+ integrationImplementation("org.apache.hive:hive-metastore") {
+ exclude group: 'org.apache.avro', module: 'avro'
+ exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+ exclude group: 'org.pentaho' // missing dependency
+ exclude group: 'org.apache.hbase'
+ exclude group: 'org.apache.logging.log4j'
+ exclude group: 'co.cask.tephra'
+ exclude group: 'com.google.code.findbugs', module: 'jsr305'
+ exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all'
+ exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet'
+ exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle'
+ exclude group: 'com.tdunning', module: 'json'
+ exclude group: 'javax.transaction', module: 'transaction-api'
+ exclude group: 'com.zaxxer', module: 'HikariCP'
+ }
+
+ integrationImplementation("org.apache.hive:hive-exec::core") {
+ exclude group: 'org.apache.avro', module: 'avro'
+ exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+ exclude group: 'org.pentaho' // missing dependency
+ exclude group: 'org.apache.hive', module: 'hive-llap-tez'
+ exclude group: 'org.apache.logging.log4j'
+ exclude group: 'com.google.protobuf', module: 'protobuf-java'
+ exclude group: 'org.apache.calcite'
+ exclude group: 'org.apache.calcite.avatica'
+ exclude group: 'com.google.code.findbugs', module: 'jsr305'
+ }
}
shadowJar {
@@ -174,6 +232,16 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
classifier null
}
+ task integrationTest(type: Test) {
+ description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}"
+ group = "verification"
+ testClassesDirs = sourceSets.integration.output.classesDirs
+ classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path)
+ inputs.file(shadowJar.archiveFile.get().asFile.path)
+ }
+ integrationTest.dependsOn shadowJar
+ check.dependsOn integrationTest
+
jar {
enabled = false
}
diff --git a/flink/v1.14/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.14/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java
new file mode 100644
index 0000000000..65cfba1ec8
--- /dev/null
+++ b/flink/v1.14/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.iceberg.flink;
+
+import java.util.Map;
+
+public class IcebergConnectorSmokeTest extends TestIcebergConnector {
+
+ public IcebergConnectorSmokeTest(
+ String catalogName, Map<String, String> properties, boolean isStreaming) {
+ super(catalogName, properties, isStreaming);
+ }
+}
diff --git a/flink/v1.15/build.gradle b/flink/v1.15/build.gradle
index fd042c7a8c..36ef7b1665 100644
--- a/flink/v1.15/build.gradle
+++ b/flink/v1.15/build.gradle
@@ -119,9 +119,15 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
tasks.jar.dependsOn tasks.shadowJar
+ sourceSets {
+ integration {
+ java.srcDir "$projectDir/src/integration/java"
+ resources.srcDir "$projectDir/src/integration/resources"
+ }
+ }
+
configurations {
implementation {
- exclude group: 'org.apache.flink'
// included in Flink
exclude group: 'org.slf4j'
exclude group: 'org.apache.commons'
@@ -134,7 +140,9 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
}
dependencies {
- implementation project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")
+ implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) {
+ exclude group: 'org.apache.flink'
+ }
implementation project(':iceberg-aws')
implementation(project(':iceberg-aliyun')) {
exclude group: 'edu.umd.cs.findbugs', module: 'findbugs'
@@ -146,7 +154,57 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
}
// for dropwizard histogram metrics implementation
- implementation "org.apache.flink:flink-metrics-dropwizard:${flinkMajorVersion}"
+ implementation "org.apache.flink:flink-metrics-dropwizard:${flinkVersion}"
+
+ // for integration testing with the flink-runtime-jar
+ // all of those dependencies are required because the integration test extends FlinkTestBase
+ integrationCompileOnly project(':iceberg-api')
+ integrationImplementation 'org.junit.vintage:junit-vintage-engine'
+ integrationImplementation 'org.assertj:assertj-core'
+ integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts")
+ integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')
+ integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
+ integrationImplementation("org.apache.flink:flink-test-utils:${flinkVersion}") {
+ exclude group: "org.apache.curator", module: 'curator-test'
+ exclude group: 'junit'
+ }
+
+ integrationImplementation "org.apache.flink:flink-table-api-java-bridge:${flinkVersion}"
+ integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${flinkVersion}"
+
+ integrationImplementation "org.apache.hadoop:hadoop-common"
+ integrationImplementation "org.apache.hadoop:hadoop-hdfs"
+ integrationImplementation("org.apache.hadoop:hadoop-minicluster") {
+ exclude group: 'org.apache.avro', module: 'avro'
+ }
+
+ integrationImplementation("org.apache.hive:hive-metastore") {
+ exclude group: 'org.apache.avro', module: 'avro'
+ exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+ exclude group: 'org.pentaho' // missing dependency
+ exclude group: 'org.apache.hbase'
+ exclude group: 'org.apache.logging.log4j'
+ exclude group: 'co.cask.tephra'
+ exclude group: 'com.google.code.findbugs', module: 'jsr305'
+ exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all'
+ exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet'
+ exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle'
+ exclude group: 'com.tdunning', module: 'json'
+ exclude group: 'javax.transaction', module: 'transaction-api'
+ exclude group: 'com.zaxxer', module: 'HikariCP'
+ }
+
+ integrationImplementation("org.apache.hive:hive-exec::core") {
+ exclude group: 'org.apache.avro', module: 'avro'
+ exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+ exclude group: 'org.pentaho' // missing dependency
+ exclude group: 'org.apache.hive', module: 'hive-llap-tez'
+ exclude group: 'org.apache.logging.log4j'
+ exclude group: 'com.google.protobuf', module: 'protobuf-java'
+ exclude group: 'org.apache.calcite'
+ exclude group: 'org.apache.calcite.avatica'
+ exclude group: 'com.google.code.findbugs', module: 'jsr305'
+ }
}
shadowJar {
@@ -176,6 +234,16 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") {
classifier null
}
+ task integrationTest(type: Test) {
+ description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}"
+ group = "verification"
+ testClassesDirs = sourceSets.integration.output.classesDirs
+ classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path)
+ inputs.file(shadowJar.archiveFile.get().asFile.path)
+ }
+ integrationTest.dependsOn shadowJar
+ check.dependsOn integrationTest
+
jar {
enabled = false
}
diff --git a/flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java
new file mode 100644
index 0000000000..65cfba1ec8
--- /dev/null
+++ b/flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.iceberg.flink;
+
+import java.util.Map;
+
+public class IcebergConnectorSmokeTest extends TestIcebergConnector {
+
+ public IcebergConnectorSmokeTest(
+ String catalogName, Map<String, String> properties, boolean isStreaming) {
+ super(catalogName, properties, isStreaming);
+ }
+}