This is an automated email from the ASF dual-hosted git repository.

pvary pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/main by this push:
     new 26d62c06bc Flink: Added error handling and default logic for Flink 
version detection (#9452)
26d62c06bc is described below

commit 26d62c06bce6f50d46f51860f0014bba2f9538d2
Author: Geoffrey Jacoby <[email protected]>
AuthorDate: Wed Jan 31 08:11:20 2024 -0500

    Flink: Added error handling and default logic for Flink version detection 
(#9452)
---
 .../apache/iceberg/flink/util/FlinkPackage.java    | 34 ++++++++++++++++++++--
 .../iceberg/flink/util/TestFlinkPackage.java       | 23 +++++++++++++++
 .../apache/iceberg/flink/util/FlinkPackage.java    | 34 ++++++++++++++++++++--
 .../iceberg/flink/util/TestFlinkPackage.java       | 23 +++++++++++++++
 .../apache/iceberg/flink/util/FlinkPackage.java    | 34 ++++++++++++++++++++--
 .../iceberg/flink/util/TestFlinkPackage.java       | 23 +++++++++++++++
 6 files changed, 162 insertions(+), 9 deletions(-)

diff --git 
a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
 
b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
index 00d74d8d34..353cee56be 100644
--- 
a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
+++ 
b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
@@ -18,16 +18,44 @@
  */
 package org.apache.iceberg.flink.util;
 
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
 
 public class FlinkPackage {
-  /** Choose {@link DataStream} class because it is one of the core Flink API. 
*/
-  private static final String VERSION = 
DataStream.class.getPackage().getImplementationVersion();
+
+  private static final AtomicReference<String> VERSION = new 
AtomicReference<>();
+  public static final String FLINK_UNKNOWN_VERSION = "FLINK-UNKNOWN-VERSION";
 
   private FlinkPackage() {}
 
   /** Returns Flink version string like x.y.z */
   public static String version() {
-    return VERSION;
+    if (null == VERSION.get()) {
+      String detectedVersion = null;
+      try {
+        detectedVersion = versionFromJar();
+        // use unknown version in case exact implementation version can't be 
found from the jar
+        // (this can happen if the DataStream class appears multiple times in 
the same classpath
+        // such as with shading)
+        detectedVersion = detectedVersion != null ? detectedVersion : 
FLINK_UNKNOWN_VERSION;
+      } catch (Exception e) {
+        detectedVersion = FLINK_UNKNOWN_VERSION;
+      }
+      VERSION.set(detectedVersion);
+    }
+
+    return VERSION.get();
+  }
+
+  @VisibleForTesting
+  static String versionFromJar() {
+    // Choose {@link DataStream} class because it is one of the core Flink API
+    return DataStream.class.getPackage().getImplementationVersion();
+  }
+
+  @VisibleForTesting
+  static void setVersion(String version) {
+    VERSION.set(version);
   }
 }
diff --git 
a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
 
b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
index 4c9b8aea82..23cefa8347 100644
--- 
a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
+++ 
b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
@@ -20,6 +20,8 @@ package org.apache.iceberg.flink.util;
 
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.MockedStatic;
+import org.mockito.Mockito;
 
 public class TestFlinkPackage {
 
@@ -28,4 +30,25 @@ public class TestFlinkPackage {
   public void testVersion() {
     Assert.assertEquals("1.16.2", FlinkPackage.version());
   }
+
+  @Test
+  public void testDefaultVersion() {
+    // It's difficult to reproduce a reflection error in a unit test, so we 
just inject a mocked
+    // fault to test the default logic
+
+    // First make sure we're not caching a version result from a previous test
+    FlinkPackage.setVersion(null);
+    try (MockedStatic<FlinkPackage> mockedStatic = 
Mockito.mockStatic(FlinkPackage.class)) {
+      
mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class);
+      mockedStatic.when(FlinkPackage::version).thenCallRealMethod();
+      Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, 
FlinkPackage.version());
+    }
+    FlinkPackage.setVersion(null);
+    try (MockedStatic<FlinkPackage> mockedStatic = 
Mockito.mockStatic(FlinkPackage.class)) {
+      mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null);
+      mockedStatic.when(FlinkPackage::version).thenCallRealMethod();
+      FlinkPackage.setVersion(null);
+      Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, 
FlinkPackage.version());
+    }
+  }
 }
diff --git 
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
 
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
index 00d74d8d34..353cee56be 100644
--- 
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
+++ 
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
@@ -18,16 +18,44 @@
  */
 package org.apache.iceberg.flink.util;
 
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
 
 public class FlinkPackage {
-  /** Choose {@link DataStream} class because it is one of the core Flink API. 
*/
-  private static final String VERSION = 
DataStream.class.getPackage().getImplementationVersion();
+
+  private static final AtomicReference<String> VERSION = new 
AtomicReference<>();
+  public static final String FLINK_UNKNOWN_VERSION = "FLINK-UNKNOWN-VERSION";
 
   private FlinkPackage() {}
 
   /** Returns Flink version string like x.y.z */
   public static String version() {
-    return VERSION;
+    if (null == VERSION.get()) {
+      String detectedVersion = null;
+      try {
+        detectedVersion = versionFromJar();
+        // use unknown version in case exact implementation version can't be 
found from the jar
+        // (this can happen if the DataStream class appears multiple times in 
the same classpath
+        // such as with shading)
+        detectedVersion = detectedVersion != null ? detectedVersion : 
FLINK_UNKNOWN_VERSION;
+      } catch (Exception e) {
+        detectedVersion = FLINK_UNKNOWN_VERSION;
+      }
+      VERSION.set(detectedVersion);
+    }
+
+    return VERSION.get();
+  }
+
+  @VisibleForTesting
+  static String versionFromJar() {
+    // Choose {@link DataStream} class because it is one of the core Flink API
+    return DataStream.class.getPackage().getImplementationVersion();
+  }
+
+  @VisibleForTesting
+  static void setVersion(String version) {
+    VERSION.set(version);
   }
 }
diff --git 
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
 
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
index cf244f4102..ceb3ed5586 100644
--- 
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
+++ 
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
@@ -20,6 +20,8 @@ package org.apache.iceberg.flink.util;
 
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.MockedStatic;
+import org.mockito.Mockito;
 
 public class TestFlinkPackage {
 
@@ -28,4 +30,25 @@ public class TestFlinkPackage {
   public void testVersion() {
     Assert.assertEquals("1.17.1", FlinkPackage.version());
   }
+
+  @Test
+  public void testDefaultVersion() {
+    // It's difficult to reproduce a reflection error in a unit test, so we 
just inject a mocked
+    // fault to test the default logic
+
+    // First make sure we're not caching a version result from a previous test
+    FlinkPackage.setVersion(null);
+    try (MockedStatic<FlinkPackage> mockedStatic = 
Mockito.mockStatic(FlinkPackage.class)) {
+      
mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class);
+      mockedStatic.when(FlinkPackage::version).thenCallRealMethod();
+      Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, 
FlinkPackage.version());
+    }
+    FlinkPackage.setVersion(null);
+    try (MockedStatic<FlinkPackage> mockedStatic = 
Mockito.mockStatic(FlinkPackage.class)) {
+      mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null);
+      mockedStatic.when(FlinkPackage::version).thenCallRealMethod();
+      FlinkPackage.setVersion(null);
+      Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, 
FlinkPackage.version());
+    }
+  }
 }
diff --git 
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
 
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
index 00d74d8d34..353cee56be 100644
--- 
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
+++ 
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java
@@ -18,16 +18,44 @@
  */
 package org.apache.iceberg.flink.util;
 
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
 
 public class FlinkPackage {
-  /** Choose {@link DataStream} class because it is one of the core Flink API. 
*/
-  private static final String VERSION = 
DataStream.class.getPackage().getImplementationVersion();
+
+  private static final AtomicReference<String> VERSION = new 
AtomicReference<>();
+  public static final String FLINK_UNKNOWN_VERSION = "FLINK-UNKNOWN-VERSION";
 
   private FlinkPackage() {}
 
   /** Returns Flink version string like x.y.z */
   public static String version() {
-    return VERSION;
+    if (null == VERSION.get()) {
+      String detectedVersion = null;
+      try {
+        detectedVersion = versionFromJar();
+        // use unknown version in case exact implementation version can't be 
found from the jar
+        // (this can happen if the DataStream class appears multiple times in 
the same classpath
+        // such as with shading)
+        detectedVersion = detectedVersion != null ? detectedVersion : 
FLINK_UNKNOWN_VERSION;
+      } catch (Exception e) {
+        detectedVersion = FLINK_UNKNOWN_VERSION;
+      }
+      VERSION.set(detectedVersion);
+    }
+
+    return VERSION.get();
+  }
+
+  @VisibleForTesting
+  static String versionFromJar() {
+    // Choose {@link DataStream} class because it is one of the core Flink API
+    return DataStream.class.getPackage().getImplementationVersion();
+  }
+
+  @VisibleForTesting
+  static void setVersion(String version) {
+    VERSION.set(version);
   }
 }
diff --git 
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
 
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
index 487e0a1870..6cef0d1228 100644
--- 
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
+++ 
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java
@@ -20,6 +20,8 @@ package org.apache.iceberg.flink.util;
 
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.MockedStatic;
+import org.mockito.Mockito;
 
 public class TestFlinkPackage {
 
@@ -28,4 +30,25 @@ public class TestFlinkPackage {
   public void testVersion() {
     Assert.assertEquals("1.18.1", FlinkPackage.version());
   }
+
+  @Test
+  public void testDefaultVersion() {
+    // It's difficult to reproduce a reflection error in a unit test, so we 
just inject a mocked
+    // fault to test the default logic
+
+    // First make sure we're not caching a version result from a previous test
+    FlinkPackage.setVersion(null);
+    try (MockedStatic<FlinkPackage> mockedStatic = 
Mockito.mockStatic(FlinkPackage.class)) {
+      
mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class);
+      mockedStatic.when(FlinkPackage::version).thenCallRealMethod();
+      Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, 
FlinkPackage.version());
+    }
+    FlinkPackage.setVersion(null);
+    try (MockedStatic<FlinkPackage> mockedStatic = 
Mockito.mockStatic(FlinkPackage.class)) {
+      mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null);
+      mockedStatic.when(FlinkPackage::version).thenCallRealMethod();
+      FlinkPackage.setVersion(null);
+      Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, 
FlinkPackage.version());
+    }
+  }
 }

Reply via email to