Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 0 additions & 5 deletions bin/docker-image-tool.sh
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,6 @@ Commands:

Options:
-f file (Optional) Dockerfile to build for JVM based Jobs. By default builds the Dockerfile shipped with Spark.
For Java 17, use `-f kubernetes/dockerfiles/spark/Dockerfile.java17`
-p file (Optional) Dockerfile to build for PySpark Jobs. Builds Python dependencies and ships with Spark.
Skips building PySpark docker image if not specified.
-R file (Optional) Dockerfile to build for SparkR Jobs. Builds R dependencies and ships with Spark.
Expand Down Expand Up @@ -277,10 +276,6 @@ Examples:
# Note: buildx, which does cross building, needs to do the push during build
# So there is no separate push step with -X

- Build and push Java17-based image with tag "v3.3.0" to docker.io/myrepo
$0 -r docker.io/myrepo -t v3.3.0 -f kubernetes/dockerfiles/spark/Dockerfile.java17 build
$0 -r docker.io/myrepo -t v3.3.0 push

EOF
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
case proto.Relation.RelTypeCase.AGGREGATE => transformAggregate(rel.getAggregate)
case proto.Relation.RelTypeCase.SQL => transformSql(rel.getSql)
case proto.Relation.RelTypeCase.LOCAL_RELATION =>
transformLocalRelation(rel.getLocalRelation)
transformLocalRelation(rel.getLocalRelation, common)
case proto.Relation.RelTypeCase.SAMPLE => transformSample(rel.getSample)
case proto.Relation.RelTypeCase.RELTYPE_NOT_SET =>
throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.")
Expand Down Expand Up @@ -122,9 +122,16 @@ class SparkConnectPlanner(plan: proto.Relation, session: SparkSession) {
}
}

private def transformLocalRelation(rel: proto.LocalRelation): LogicalPlan = {
private def transformLocalRelation(
rel: proto.LocalRelation,
common: Option[proto.RelationCommon]): LogicalPlan = {
val attributes = rel.getAttributesList.asScala.map(transformAttribute(_)).toSeq
new org.apache.spark.sql.catalyst.plans.logical.LocalRelation(attributes)
val relation = new org.apache.spark.sql.catalyst.plans.logical.LocalRelation(attributes)
if (common.nonEmpty && common.get.getAlias.nonEmpty) {
logical.SubqueryAlias(identifier = common.get.getAlias, child = relation)
} else {
relation
}
}

private def transformAttribute(exp: proto.Expression.QualifiedAttribute): Attribute = {
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -22,20 +22,18 @@ import scala.collection.JavaConverters._
import org.apache.spark.SparkFunSuite
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.Expression.UnresolvedStar
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.test.SharedSparkSession

/**
* Testing trait for SparkConnect tests with some helper methods to make it easier to create new
* test cases.
*/
trait SparkConnectPlanTest {

def getSession(): SparkSession = None.orNull
trait SparkConnectPlanTest extends SharedSparkSession {

def transform(rel: proto.Relation): LogicalPlan = {
new SparkConnectPlanner(rel, getSession()).transform()
new SparkConnectPlanner(rel, spark).transform()
}

def readRel: proto.Relation =
Expand Down
Loading