zhangjun0x01 commented on a change in pull request #1936:
URL: https://github.com/apache/iceberg/pull/1936#discussion_r555467464
##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -195,7 +205,29 @@ public FlinkInputFormat buildFormat() {
Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not
be null");
FlinkInputFormat format = buildFormat();
if (isBounded(context)) {
- return env.createInput(format, rowTypeInfo);
+ int parallelism =
flinkConf.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM);
+ if
(flinkConf.get(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) {
+ int max =
flinkConf.get(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX);
+ if (max < 1) {
+ throw new IllegalConfigurationException(
+
FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() + "
cannot be less than 1");
+ }
+
+ int splitNum = 0;
+ try {
+ FlinkInputSplit[] splits = format.createInputSplits(0);
+ splitNum = splits.length;
+ } catch (IOException e) {
+ throw new RuntimeException("get input split error.", e);
+ }
+
+ parallelism = Math.min(splitNum, max);
+ }
+
+ parallelism = limit > 0 ? Math.min(parallelism, (int) limit) :
parallelism;
Review comment:
the `parallelism` is int type and the `limit` is long type,
`Math.min(parallelism, limit)` will throws an exception,I add a judgment to
prevent overflow.
```
int limitInt = limit > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int)
limit;
parallelism = limitInt > 0 ? Math.min(parallelism, limitInt) :
parallelism;
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]