-
Notifications
You must be signed in to change notification settings - Fork 150
Expand file tree
/
Copy pathphysical_plan.rs
More file actions
124 lines (107 loc) · 3.72 KB
/
physical_plan.rs
File metadata and controls
124 lines (107 loc) · 3.72 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
// 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.
use std::sync::Arc;
use datafusion::physical_plan::{ExecutionPlan, ExecutionPlanProperties, displayable};
use datafusion_proto::physical_plan::{AsExecutionPlan, DefaultPhysicalExtensionCodec};
use prost::Message;
use pyo3::exceptions::PyRuntimeError;
use pyo3::prelude::*;
use pyo3::types::PyBytes;
use crate::context::PySessionContext;
use crate::errors::PyDataFusionResult;
use crate::metrics::PyMetricsSet;
#[pyclass(
from_py_object,
frozen,
name = "ExecutionPlan",
module = "datafusion",
subclass
)]
#[derive(Debug, Clone)]
pub struct PyExecutionPlan {
pub plan: Arc<dyn ExecutionPlan>,
}
impl PyExecutionPlan {
/// creates a new PyPhysicalPlan
pub fn new(plan: Arc<dyn ExecutionPlan>) -> Self {
Self { plan }
}
}
#[pymethods]
impl PyExecutionPlan {
/// Get the inputs to this plan
pub fn children(&self) -> Vec<PyExecutionPlan> {
self.plan
.children()
.iter()
.map(|&p| p.to_owned().into())
.collect()
}
pub fn display(&self) -> String {
let d = displayable(self.plan.as_ref());
format!("{}", d.one_line())
}
pub fn display_indent(&self) -> String {
let d = displayable(self.plan.as_ref());
format!("{}", d.indent(false))
}
pub fn to_proto<'py>(&'py self, py: Python<'py>) -> PyDataFusionResult<Bound<'py, PyBytes>> {
let codec = DefaultPhysicalExtensionCodec {};
let proto = datafusion_proto::protobuf::PhysicalPlanNode::try_from_physical_plan(
self.plan.clone(),
&codec,
)?;
let bytes = proto.encode_to_vec();
Ok(PyBytes::new(py, &bytes))
}
#[staticmethod]
pub fn from_proto(
ctx: PySessionContext,
proto_msg: Bound<'_, PyBytes>,
) -> PyDataFusionResult<Self> {
let bytes: &[u8] = proto_msg.extract().map_err(Into::<PyErr>::into)?;
let proto_plan =
datafusion_proto::protobuf::PhysicalPlanNode::decode(bytes).map_err(|e| {
PyRuntimeError::new_err(format!(
"Unable to decode logical node from serialized bytes: {e}"
))
})?;
let codec = DefaultPhysicalExtensionCodec {};
let plan = proto_plan.try_into_physical_plan(ctx.ctx.task_ctx().as_ref(), &codec)?;
Ok(Self::new(plan))
}
pub fn metrics(&self) -> Option<PyMetricsSet> {
self.plan.metrics().map(PyMetricsSet::new)
}
fn __repr__(&self) -> String {
self.display_indent()
}
#[getter]
pub fn partition_count(&self) -> usize {
self.plan.output_partitioning().partition_count()
}
}
impl From<PyExecutionPlan> for Arc<dyn ExecutionPlan> {
fn from(plan: PyExecutionPlan) -> Arc<dyn ExecutionPlan> {
plan.plan.clone()
}
}
impl From<Arc<dyn ExecutionPlan>> for PyExecutionPlan {
fn from(plan: Arc<dyn ExecutionPlan>) -> PyExecutionPlan {
PyExecutionPlan { plan: plan.clone() }
}
}