-
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 3 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 |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/* | ||
* 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.util; | ||
|
||
import org.apache.flink.streaming.api.datastream.DataStream; | ||
|
||
public class FlinkVersionDetector { | ||
public String version() { | ||
String version = null; | ||
try { | ||
version = getVersionFromJar(); | ||
} catch (Exception e) { | ||
/* 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), so the best we can do is say it's | ||
1.16.x below | ||
*/ | ||
} | ||
if (version == null) { | ||
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. Why not set this directly in the catch block? 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. Setting the default in a 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. For me this is more readable:
The reasons:
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 have made the changes you suggested. |
||
version = "1.16.x"; | ||
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. This beats the reason for the automatic version detection. The original goal was to avoid versions in java code. See: #6206 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. Maybe go for: "Flink-UNKNOWN" in this case? 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 switched the patch to use a single "Flink-UNKNOWN" constant as you suggested. |
||
} | ||
return version; | ||
} | ||
|
||
String getVersionFromJar() { | ||
/* Choose {@link DataStream} class because it is one of the core Flink API. */ | ||
return DataStream.class.getPackage().getImplementationVersion(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/* | ||
* 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.util; | ||
|
||
import org.apache.flink.streaming.api.datastream.DataStream; | ||
|
||
public class FlinkVersionDetector { | ||
public String version() { | ||
String version = null; | ||
try { | ||
version = getVersionFromJar(); | ||
} catch (Exception e) { | ||
/* 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), so the best we can do is say it's | ||
1.17.x below | ||
*/ | ||
} | ||
if (version == null) { | ||
version = "1.17.x"; | ||
} | ||
return version; | ||
} | ||
|
||
String getVersionFromJar() { | ||
/* Choose {@link DataStream} class because it is one of the core Flink API. */ | ||
return DataStream.class.getPackage().getImplementationVersion(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/* | ||
* 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.util; | ||
|
||
import org.apache.flink.streaming.api.datastream.DataStream; | ||
|
||
public class FlinkVersionDetector { | ||
public String version() { | ||
String version = null; | ||
try { | ||
version = getVersionFromJar(); | ||
} catch (Exception e) { | ||
/* we can't detect the exact implementation version from the jar (this can happen if the DataStream class | ||
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. Isn't it a packaging issue, if we have multiple In this case the load order of these classes might depend on timining, and could cause issues. 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. @pvary - Yes, it's a packaging issue. In my use case that reproduced the problem, the two instances of DataStream on the classpath are actually identical, due to unrelocated shading. That normally would cause no issues aside from build warnings. Of course we're fixing that too. Whether the packaging problem causes other bugs or not though, Iceberg's reaction shouldn't be "throw an NPE and crash the Flink pipeline", which is what currently happens and this PR fixes. 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. Oh, and just fyi, I'm not affiliated with the engineers who posted the original issue this patch fixes; I just came across it while I was investigating. So this has happened at least twice in real-world use. 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. Why did we get 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. I don't know for sure, but the shading messing up the metadata is my best guess too. In addition to the unit test in the patch, I did confirm that the patch resolves the issue in a real Flink cluster. |
||
appears multiple times in the same classpath such as with shading), so the best we can do is say it's | ||
1.18.x below | ||
*/ | ||
} | ||
if (version == null) { | ||
version = "1.18.x"; | ||
} | ||
return version; | ||
} | ||
|
||
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.
No need to create a new class for this
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.
I originally made the separate class to make mocking the logic in the unit test easier. Just moved all the logic to FlinkPackage and used static mocking to do the test.