Skip to content

[SPARK-52346][SQL] Declarative Pipeline DataflowGraph execution and event logging #51050

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 7 commits into
base: master
Choose a base branch
from
Open
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
84 changes: 84 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,14 @@
],
"sqlState" : "XX000"
},
"APPEND_ONCE_FROM_BATCH_QUERY" : {
"message" : [
"Creating a streaming table from a batch query prevents incremental loading of new data from source. Offending table: '<table>'.",
"Please use the stream() operator. Example usage:",
"CREATE STREAMING TABLE <target table name> ... AS SELECT ... FROM stream(<source table name>) ..."
],
"sqlState" : "42000"
},
"ARITHMETIC_OVERFLOW" : {
"message" : [
"<message>.<alternative> If necessary set <config> to \"false\" to bypass this error."
Expand Down Expand Up @@ -496,6 +504,13 @@
},
"sqlState" : "0A000"
},
"CANNOT_UPDATE_PARTITION_COLUMNS" : {
"message" : [
"Declared partitioning <requestedPartitionColumns> conflicts with existing table partitioning <existingPartitionColumns>.",
"Please delete the table or change the declared partitioning to match its partitions."
],
"sqlState" : "42000"
},
"CANNOT_UP_CAST_DATATYPE" : {
"message" : [
"Cannot up cast <expression> from <sourceType> to <targetType>.",
Expand Down Expand Up @@ -1378,6 +1393,12 @@
},
"sqlState" : "42734"
},
"DUPLICATE_FLOW_SQL_CONF" : {
"message" : [
"Found duplicate sql conf for dataset '<datasetName>': '<key>' is defined by both '<flowName1>' and '<flowName2>'"
],
"sqlState" : "42710"
},
"DUPLICATE_KEY" : {
"message" : [
"Found duplicate keys <keyColumn>."
Expand Down Expand Up @@ -1949,6 +1970,12 @@
],
"sqlState" : "42818"
},
"INCOMPATIBLE_BATCH_VIEW_READ" : {
"message" : [
"View <datasetIdentifier> is a batch view and must be referenced using SparkSession#read. This check can be disabled by setting Spark conf pipelines.incompatibleViewCheck.enabled = false."
],
"sqlState" : "42000"
},
"INCOMPATIBLE_COLUMN_TYPE" : {
"message" : [
"<operator> can only be performed on tables with compatible column types. The <columnOrdinalNumber> column of the <tableOrdinalNumber> table is <dataType1> type which is not compatible with <dataType2> at the same column of the first table.<hint>."
Expand Down Expand Up @@ -2025,6 +2052,12 @@
],
"sqlState" : "42613"
},
"INCOMPATIBLE_STREAMING_VIEW_READ" : {
"message" : [
"View <datasetIdentifier> is a streaming view and must be referenced using SparkSession#readStream. This check can be disabled by setting Spark conf pipelines.incompatibleViewCheck.enabled = false."
],
"sqlState" : "42000"
},
"INCOMPATIBLE_VIEW_SCHEMA_CHANGE" : {
"message" : [
"The SQL query of view <viewName> has an incompatible schema change and column <colName> cannot be resolved. Expected <expectedNum> columns named <colName> but got <actualCols>.",
Expand Down Expand Up @@ -3125,6 +3158,12 @@
},
"sqlState" : "KD002"
},
"INVALID_NAME_IN_USE_COMMAND" : {
"message" : [
"Invalid name '<name>' in <command> command. Reason: <reason>"
],
"sqlState" : "42000"
},
"INVALID_NON_DETERMINISTIC_EXPRESSIONS" : {
"message" : [
"The operator expects a deterministic expression, but the actual expression is <sqlExprs>."
Expand Down Expand Up @@ -3390,6 +3429,12 @@
],
"sqlState" : "22023"
},
"INVALID_RESETTABLE_DEPENDENCY" : {
"message" : [
"Tables <upstreamResettableTables> are resettable but have a non-resettable downstream dependency '<downstreamTable>'. `reset` will fail as Spark Streaming does not support deleted source data. You can either remove the <resetAllowedKey>=false property from '<downstreamTable>' or add it to its upstream dependencies."
],
"sqlState" : "42000"
},
"INVALID_RESET_COMMAND_FORMAT" : {
"message" : [
"Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`."
Expand Down Expand Up @@ -4569,6 +4614,12 @@
],
"sqlState" : "42K03"
},
"PERSISTED_VIEW_READS_FROM_TEMPORARY_VIEW" : {
"message" : [
"Persisted view <persistedViewName> cannot reference temporary view <temporaryViewName> that will not be available outside the pipeline scope. Either make the persisted view temporary or persist the temporary view."
],
"sqlState" : "42K0F"
},
"PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION" : {
"message" : [
"Non-grouping expression <expr> is provided as an argument to the |> AGGREGATE pipe operator but does not contain any aggregate function; please update it to include an aggregate function and then retry the query again."
Expand Down Expand Up @@ -5425,6 +5476,19 @@
],
"sqlState" : "42KD9"
},
"UNABLE_TO_INFER_PIPELINE_TABLE_SCHEMA" : {
"message" : [
"Failed to infer the schema for table <tableName> from its upstream flows.",
"Please modify the flows that write to this table to make their schemas compatible.",
"",
"Inferred schema so far:",
"<inferredDataSchema>",
"",
"Incompatible schema:",
"<incompatibleDataSchema>"
],
"sqlState" : "42KD9"
},
"UNBOUND_SQL_PARAMETER" : {
"message" : [
"Found the unbound parameter: <name>. Please, fix `args` and provide a mapping of the parameter to either a SQL literal or collection constructor functions such as `map()`, `array()`, `struct()`."
Expand Down Expand Up @@ -5590,6 +5654,12 @@
],
"sqlState" : "42883"
},
"UNRESOLVED_TABLE_PATH" : {
"message" : [
"Storage path for table <identifier> cannot be resolved."
],
"sqlState" : "22KD1"
},
"UNRESOLVED_USING_COLUMN_FOR_JOIN" : {
"message" : [
"USING column <colName> cannot be resolved on the <side> side of the join. The <side>-side columns: [<suggestion>]."
Expand Down Expand Up @@ -6566,6 +6636,20 @@
],
"sqlState" : "P0001"
},
"USER_SPECIFIED_AND_INFERRED_SCHEMA_NOT_COMPATIBLE" : {
"message" : [
"Table '<tableName>' has a user-specified schema that is incompatible with the schema",
"inferred from its query.",
"<streamingTableHint>",
"",
"Declared schema:",
"<specifiedSchema>",
"",
"Inferred schema:",
"<inferredDataSchema>"
],
"sqlState" : "42000"
},
"VARIABLE_ALREADY_EXISTS" : {
"message" : [
"Cannot create the variable <variableName> because it already exists.",
Expand Down
53 changes: 53 additions & 0 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -418,6 +418,8 @@ object SparkBuild extends PomBuild {

enable(HiveThriftServer.settings)(hiveThriftServer)

enable(SparkDeclarativePipelines.settings)(pipelines)

enable(SparkConnectCommon.settings)(connectCommon)
enable(SparkConnect.settings)(connect)
enable(SparkConnectClient.settings)(connectClient)
Expand Down Expand Up @@ -884,6 +886,57 @@ object SparkConnectClient {
)
}

object SparkDeclarativePipelines {
import BuildCommons.protoVersion

lazy val settings = Seq(
// For some reason the resolution from the imported Maven build does not work for some
// of these dependendencies that we need to shade later on.
libraryDependencies ++= {
val guavaVersion =
SbtPomKeys.effectivePom.value.getProperties.get(
"connect.guava.version").asInstanceOf[String]
val guavaFailureAccessVersion =
SbtPomKeys.effectivePom.value.getProperties.get(
"guava.failureaccess.version").asInstanceOf[String]
Seq(
"com.google.guava" % "guava" % guavaVersion,
"com.google.guava" % "failureaccess" % guavaFailureAccessVersion,
"com.google.protobuf" % "protobuf-java" % protoVersion % "protobuf"
)
},

(assembly / logLevel) := Level.Info,

// Exclude `scala-library` from assembly.
(assembly / assemblyPackageScala / assembleArtifact) := false,

// SPARK-46733: Include `spark-connect-*.jar`, `unused-*.jar`,`guava-*.jar`,
// `failureaccess-*.jar`, `annotations-*.jar`, `grpc-*.jar`, `protobuf-*.jar`,
// `gson-*.jar`, `error_prone_annotations-*.jar`, `j2objc-annotations-*.jar`,
// `animal-sniffer-annotations-*.jar`, `perfmark-api-*.jar`,
// `proto-google-common-protos-*.jar` in assembly.
// This needs to be consistent with the content of `maven-shade-plugin`.
(assembly / assemblyExcludedJars) := {
val cp = (assembly / fullClasspath).value
val validPrefixes = Set("spark-connect", "unused-", "guava-", "failureaccess-",
"annotations-", "grpc-", "protobuf-", "gson", "error_prone_annotations",
"j2objc-annotations", "animal-sniffer-annotations", "perfmark-api",
"proto-google-common-protos")
cp filterNot { v =>
validPrefixes.exists(v.data.getName.startsWith)
}
},

(assembly / assemblyMergeStrategy) := {
case m if m.toLowerCase(Locale.ROOT).endsWith("manifest.mf") => MergeStrategy.discard
// Drop all proto files that are not needed as artifacts of the build.
case m if m.toLowerCase(Locale.ROOT).endsWith(".proto") => MergeStrategy.discard
case _ => MergeStrategy.first
}
)
}

object SparkProtobuf {
import BuildCommons.protoVersion

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5854,6 +5854,67 @@ object SQLConf {
.booleanConf
.createWithDefault(true)

val PIPELINES_STREAM_STATE_POLLING_INTERVAL: ConfigEntry[Long] = {
buildConf("spark.sql.pipelines.execution.streamstate.pollingInterval")
.doc(
"Interval at which the stream state is polled for changes. This is used to check " +
"if the stream has failed and needs to be restarted."
)
.version("4.1.0")
.timeConf(TimeUnit.SECONDS)
.createWithDefault(1)
}

val PIPELINES_WATCHDOG_MIN_RETRY_TIME_IN_SECONDS: ConfigEntry[Long] = {
buildConf("spark.sql.pipelines.execution.watchdog.minRetryTime")
.doc(
"Initial duration between the time when we notice a flow has failed and when we try to " +
"restart the flow. The interval between flow restarts doubles with every stream " +
"failure up to the maximum value set in `pipelines.execution.watchdog.maxRetryTime`."
)
.version("4.1.0")
.timeConf(TimeUnit.SECONDS)
.createWithDefault(5)
}

val PIPELINES_WATCHDOG_MAX_RETRY_TIME_IN_SECONDS: ConfigEntry[Long] = {
buildConf("spark.sql.pipelines.execution.watchdog.maxRetryTime")
.doc(
"Maximum time interval at which flows will be restarted."
)
.version("4.1.0")
.timeConf(TimeUnit.SECONDS)
.createWithDefault(3600)
}

val PIPELINES_MAX_CONCURRENT_FLOWS: ConfigEntry[Int] = {
buildConf("spark.sql.pipelines.execution.maxConcurrentFlows")
.doc(
"Max number of flows to execute at once. Used to tune performance for triggered " +
"pipelines. Has no effect on continuous pipelines."
)
.version("4.1.0")
.intConf
.createWithDefault(16)
}


val PIPELINES_TIMEOUT_MS_FOR_TERMINATION_JOIN_AND_LOCK: ConfigEntry[Long] = {
buildConf("spark.sql.pipelines.timeoutMsForTerminationJoinAndLock")
.doc("Timeout to grab a lock for stopping update - default is 1hr.")
.version("4.1.0")
.timeConf(TimeUnit.MILLISECONDS)
.createWithDefault(60 * 60 * 1000)
}

val PIPELINES_MAX_FLOW_RETRY_ATTEMPTS: ConfigEntry[Int] = {
buildConf("spark.sql.pipelines.maxFlowRetryAttempts")
.doc("Maximum no. of times a flow can be retried")
.version("4.1.0")
.intConf
.createWithDefault(2)
}

/**
* Holds information about keys that have been deprecated.
*
Expand Down
Loading