[ 
https://issues.apache.org/jira/browse/PHOENIX-6698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17537404#comment-17537404
 ] 

ASF GitHub Bot commented on PHOENIX-6698:
-----------------------------------------

jichen20210919 commented on code in PR #79:
URL: https://github.com/apache/phoenix-connectors/pull/79#discussion_r873477213


##########
phoenix-hive-base/src/main/java/org/apache/phoenix/hive/mapreduce/PhoenixInputFormat.java:
##########
@@ -121,73 +126,192 @@ public InputSplit[] getSplits(JobConf jobConf, int 
numSplits) throws IOException
     private List<InputSplit> generateSplits(final JobConf jobConf, final 
QueryPlan qplan,
                                             final List<KeyRange> splits, 
String query) throws
             IOException {
-        if (qplan == null){
+        if (qplan == null) {
             throw new NullPointerException();
-        }if (splits == null){
+        }
+        if (splits == null) {
             throw new NullPointerException();
         }
         final List<InputSplit> psplits = new ArrayList<>(splits.size());
 
-        Path[] tablePaths = 
FileInputFormat.getInputPaths(ShimLoader.getHadoopShims()
-                .newJobContext(new Job(jobConf)));
-        boolean splitByStats = 
jobConf.getBoolean(PhoenixStorageHandlerConstants.SPLIT_BY_STATS,
+        final Path[] tablePaths = FileInputFormat.getInputPaths(
+                ShimLoader.getHadoopShims().newJobContext(new Job(jobConf)));
+        final boolean splitByStats = jobConf.getBoolean(
+                PhoenixStorageHandlerConstants.SPLIT_BY_STATS,
                 false);
-
+        final int parallelThreshould = jobConf.getInt(
+                "hive.phoenix.split.parallel.threshold",
+                32);
         setScanCacheSize(jobConf);
+        if (
+                (parallelThreshould <= 0)
+                ||
+                (qplan.getScans().size() < parallelThreshould)
+        ) {
+            LOG.info("generate splits in serial");
+            for (final List<Scan> scans : qplan.getScans()) {
+                psplits.addAll(
+                        generateSplitsInternal(
+                                jobConf,
+                                qplan,
+                                splits,
+                                query,
+                                scans,
+                                splitByStats,
+                                tablePaths)
+                );
+            }
+        } else {
+            final int parallism = jobConf.getInt(

Review Comment:
   parallelism level config is used to control the worker threads count for 
parallel split method, parallel threshold is used to control which  
split-generation method is used, serial or parallel.





> hive-connector will take long time to generate splits for large phoenix 
> tables.
> -------------------------------------------------------------------------------
>
>                 Key: PHOENIX-6698
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-6698
>             Project: Phoenix
>          Issue Type: Improvement
>          Components: hive-connector
>    Affects Versions: 5.1.0
>            Reporter: jichen
>            Assignee: jichen
>            Priority: Minor
>             Fix For: connectors-6.0.0
>
>         Attachments: PHOENIX-6698.master.v1.patch
>
>
> {{{color:#1d1c1d}In our production environment, hive-phoenix connector  will 
> take nearly 30-40 minutes to generate splits for large phoenix table, which 
> has more than 2048 regions.it is because in class PhoenixInputFormat, 
> function  'generateSplits' only uses one thread to generate splits for each 
> scan. My proposal is to use multi-thread to generate splits in parallel. the 
> proposal has been validated in our production environment.by  changing code 
> {color}}}{color:#1d1c1d}to generate splits  in parallel with 24 threads, the 
> time cost is reduced to 2 minutes.  {color}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to