-
Notifications
You must be signed in to change notification settings - Fork 2.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Flink: Added error handling and default logic for Flink version detection #9452
Changes from 5 commits
c153f38
7428ec2
4384e94
adadc19
bb8c50d
428f367
be5d3aa
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,15 +19,31 @@ | |
package org.apache.iceberg.flink.util; | ||
|
||
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(); | ||
|
||
public static final String FLINK_UNKNOWN_VERSION = "Flink-UNKNOWN"; | ||
|
||
private FlinkPackage() {} | ||
|
||
/** Returns Flink version string like x.y.z */ | ||
public static String version() { | ||
return VERSION; | ||
try { | ||
String version = getVersionFromJar(); | ||
/* If we can't detect the exact implementation version from the jar (this can happen if the DataStream class | ||
appears multiple times in the same classpath such as with shading), then the best we can do is say it's | ||
unknown | ||
*/ | ||
return version != null ? version : FLINK_UNKNOWN_VERSION; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we should store the result in a variable rather than re-fetching the version on every call There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1. this method can be used to initialize the static There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @nastra @stevenzwu - Iceberg's checkstyle config forbids naming a non-final static variable an all-caps name; it wants camel-case. Should I:
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we might want to slightly update the implementation to
Notice that I also removed the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we do this, we might not want to create an extra method for
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. would this simple way work, where initFlinkVersion implements the try-catch?
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @stevenzwu - making VERSION @nastra @pvary - Using an AtomicReference would work, and if that's what you want I'll do it, but isn't that more complexity than this logic requires? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In general, this is a pretty simple patch and it's been going back and forth for a week. Can we come to a consensus on what exactly the patch needs, so that if those things are done it's ready for commit? Thanks! There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
} catch (Exception e) { | ||
return FLINK_UNKNOWN_VERSION; | ||
} | ||
} | ||
|
||
@VisibleForTesting | ||
static String getVersionFromJar() { | ||
/* Choose {@link DataStream} class because it is one of the core Flink API. */ | ||
return DataStream.class.getPackage().getImplementationVersion(); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can we please use // instead of /* */ for inline comments?