jtmzheng opened a new issue, #5514:
URL: https://github.com/apache/hudi/issues/5514

   **_Tips before filing an issue_**
   
   - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)?
   
   - Join the mailing list to engage in conversations and get faster support at 
[email protected].
   
   - If you have triaged this as a bug, then file an 
[issue](https://issues.apache.org/jira/projects/HUDI/issues) directly.
   
   I'm seeing some unexpected behavior where a `read_optimized` Spark query on 
a MOR table is taking ~30 minutes without any action (this is on Hudi 0.9.0 
without metadata table enabled) :
   ```
   start_time = datetime.now()
   read_options = {"hoodie.datasource.query.type": "read_optimized"}
   df = (
       spark.read.format("hudi")
       .options(**read_options)
       .load("{table_s3_path}")
   )
   print(f"Elapsed: {datetime.now() - start_time}")
   ```
   
   ```
   Elapsed: 0:34:38.293859
   ```
   
   A snapshot query returns in ~ 5s (as expected) since there is no action like 
count, collect, show, etc. This also doesn't seem to affect COW tables.
   
   Looking at the Spark UI curiously showed jobs being created referencing 
https://github.com/apache/hudi/blob/release-0.9.0/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java#L73.
   
   I got help from a user on Hudi Slack: 
https://apache-hudi.slack.com/archives/C4D716NPQ/p1651784954682329 who pointed 
to:
   
   ```
   int parallelism = Math.min(DEFAULT_LISTING_PARALLELISM, 
partitionPaths.size());
   
       List<Pair<String, FileStatus[]>> partitionToFiles = 
engineContext.map(partitionPaths, partitionPathStr -> {
         Path partitionPath = new Path(partitionPathStr);
         FileSystem fs = partitionPath.getFileSystem(hadoopConf.get());
         return Pair.of(partitionPathStr, 
FSUtils.getAllDataFilesInPartition(fs, partitionPath));
       }, parallelism);
   ```
   
   being the culprit where the read optimized query was listing the files in 
the table (there are a lot of files so it's not surprising this takes a while 
since it's not doing any partition pruning). Link: 
https://github.com/apache/hudi/blob/release-0.9.0/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java#L119
 
   
   Can anyone provide insight on what's going on? What can I do to work around 
this?
   
   
   Steps to reproduce the behavior:
   
   1. Create a MOR table with some test data
   2. Query the table through Spark using a read optimized query **without** 
any action
   3. Verify Spark jobs are created that listed the files through the Spark UI
   
   **Expected behavior**
   
   The read optimized query does not list the files until an action (eg. if you 
query a specific partition it should only list the files in that partition).
   
   **Environment Description**
   
   * Hudi version : 0.9.0 (EMR)
   
   * Spark version : 3.1.2
   
   * Hive version : 3.1.2
   
   * Hadoop version : Amazon 3.2.1
   
   * Storage (HDFS/S3/GCS..) : S3
   
   * Running on Docker? (yes/no) : no
   
   
   **Additional context**
   
   N/A
   
   **Stacktrace**
   
   N/A
   


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

Reply via email to