deniskuzZ commented on code in PR #6526:
URL: https://github.com/apache/hive/pull/6526#discussion_r3357354488
##########
ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat.java:
##########
@@ -72,6 +73,71 @@ protected MapredParquetInputFormat(final
ParquetInputFormat<ArrayWritable> input
vectorizedSelf = new VectorizedParquetInputFormat();
}
+ /**
+ * On blob storage with multiple recursive input directories, list them in
parallel instead of the
+ * default serial per-directory listing that dominates split generation.
Listed files flow through
+ * the inherited {@link FileInputFormat#getSplits} unchanged; all other
cases defer to the default.
+ */
+ @Override
+ protected FileStatus[] listStatus(JobConf job) throws IOException {
+ Path[] dirs = getInputPaths(job);
+ // Only the recursive case (the Tez default) takes the parallel path;
non-recursive listing has
+ // subtler sub-directory semantics, so defer to the default.
+ if (dirs.length <= 1
+ || !job.getBoolean(FileInputFormat.INPUT_DIR_RECURSIVE, false)
+ || !BlobStorageUtils.isBlobStorageFileSystem(job,
dirs[0].getFileSystem(job))) {
+ return super.listStatus(job);
+ }
+
+ long start = System.currentTimeMillis();
+ // List as the caller's end-user, not the pool threads' login user;
FileSystem.get is UGI-keyed.
+ UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+ int numThreads = Math.max(2, HiveConf.getIntVar(job,
HiveConf.ConfVars.HIVE_COMPUTE_SPLITS_NUM_THREADS));
+ ExecutorService pool = newWorkerPool(numThreads);
Review Comment:
`IcebergInputFormat` does the same. It allows changing the pool size in
runtime. ORC adds cap on number of listing threads. So it's not uniform even
now. ...
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]