milenkovicm opened a new issue, #17297:
URL: https://github.com/apache/datafusion/issues/17297

   ### Is your feature request related to a problem or challenge?
   
   `DataFrame.cache()` does not work in distributed environments as it will 
materialise whole table in memory.
   
   ```rust
   pub async fn cache(self) -> Result<DataFrame> {
       let context = 
SessionContext::new_with_state((*self.session_state).clone());
       // The schema is consistent with the output
       let plan = self.clone().create_physical_plan().await?;
       let schema = plan.schema();
       let task_ctx = Arc::new(self.task_ctx());
       let partitions = collect_partitioned(plan, task_ctx).await?;
       let mem_table = MemTable::try_new(schema, partitions)?;
       context.read_table(Arc::new(mem_table))
   }
   ```
   This does not work in distributed environments such as ballista. 
   
   Also note, cache will eagerly materialise, even there is no downstream 
consumers (which may not be a big problem) but it does not follow semantics of 
spark.cache(..)
   
   ### Describe the solution you'd like
   
   Ideally cache should be represented with logical plan node, which would be 
resolved first time cache is needed. This approach may be a bit complicated to 
implement, and datafusion may not really benefit from it. 
   
   As an easier to implement alternative, we could provide a `cache factory` 
(at session_state maybe0 which would provide same functionality as it is 
currently if not overridden or provide user specified logic, such as returning 
`LogicalPlan::Extension` or similar, and leave query planner/user to deal with 
cache materialisation decision.
   
   ### Describe alternatives you've considered
   
   Users can create DataFrameExt which would provide a new method like 
`distributed_cache` which would return `LogicalPlan::Extension` with 
`DistributedCacheExtension`. This alternative is the simplest to implement (not 
affecting datafusion) but user would need to change code when moving from 
datafusion to ballista 
   
   ### Additional context
   
   _No response_


-- 
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: github-unsubscr...@datafusion.apache.org.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org
For additional commands, e-mail: github-h...@datafusion.apache.org

Reply via email to