zhuzhurk commented on code in PR #24764:
URL: https://github.com/apache/flink/pull/24764#discussion_r1596210598


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOptions.java:
##########
@@ -58,11 +60,32 @@ public class HiveOptions {
                             "If is false, parallelism of source are set by 
config.\n"
                                     + "If is true, source parallelism is 
inferred according to splits number.\n");
 
+    public static final ConfigOption<InferMode> 
TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MODE =

Review Comment:
   Let's annotate it with `@PublicEvolving`



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceDynamicFileEnumerator.java:
##########
@@ -178,7 +178,7 @@ public Collection<FileSourceSplit> enumerateSplits(Path[] 
paths, int minDesiredS
     }
 
     @VisibleForTesting

Review Comment:
   `@VisibleForTesting` should be removed because it is not only visible for 
testing with these changes.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOptions.java:
##########
@@ -58,11 +60,32 @@ public class HiveOptions {
                             "If is false, parallelism of source are set by 
config.\n"
                                     + "If is true, source parallelism is 
inferred according to splits number.\n");
 
+    public static final ConfigOption<InferMode> 
TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MODE =
+            key("table.exec.hive.infer-source-parallelism.mode")
+                    .enumType(InferMode.class)
+                    .defaultValue(InferMode.DYNAMIC)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "An option for selecting the hive 
source parallelism inference mode to infer parallelism according to splits 
number.")
+                                    .list(
+                                            text(
+                                                    "'static' represents 
static inference, which will infer source parallelism at job create stage."),

Review Comment:
   create -> creation



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveStaticParallelismInferenceFactory.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.flink.connectors.hive;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * The factory class for {@link HiveParallelismInference} to support Hive 
source static parallelism
+ * inference.
+ */
+class HiveStaticParallelismInferenceFactory implements 
HiveParallelismInference.Provider {
+
+    private final ObjectPath tablePath;
+    private final ReadableConfig flinkConf;
+
+    HiveStaticParallelismInferenceFactory(ObjectPath tablePath, ReadableConfig 
flinkConf) {
+        this.tablePath = tablePath;
+        this.flinkConf = flinkConf;
+    }
+
+    @Override
+    public HiveParallelismInference create() {
+        boolean inferEnabled = 
flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM);
+        HiveOptions.InferMode inferMode =
+                
flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MODE);
+        // This logic should be fixed if config option 
`table.exec.hive.infer-source-parallelism`
+        // is deprecated.

Review Comment:
   deprecated -> removed



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicParallelismInferenceFactory.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.flink.connectors.hive;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.table.catalog.ObjectPath;
+
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * The factory class for {@link HiveParallelismInference} to support Hive 
source dynamic parallelism
+ * inference.
+ */
+class HiveDynamicParallelismInferenceFactory implements 
HiveParallelismInference.Provider {
+
+    private final ObjectPath tablePath;
+    private final JobConf jobConf;
+    private final int globalMaxParallelism;
+
+    HiveDynamicParallelismInferenceFactory(
+            ObjectPath tablePath, JobConf jobConf, int globalMaxParallelism) {
+        this.tablePath = tablePath;
+        this.jobConf = jobConf;
+        this.globalMaxParallelism = globalMaxParallelism;
+    }
+
+    @Override
+    public HiveParallelismInference create() {
+        boolean inferEnabled =
+                jobConf.getBoolean(
+                        
HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM.key(),
+                        
HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM.defaultValue());
+        HiveOptions.InferMode inferMode =
+                jobConf.getEnum(
+                        
HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MODE.key(),
+                        
HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MODE.defaultValue());

Review Comment:
   Not sure if the hive jobConf contains this config?
   Seems it is not set in `HiveSourceBuilder#setFlinkConfigurationToJobConf()`



-- 
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.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to