Skip to content
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -33,10 +33,10 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability, TableProvider}
import org.apache.spark.sql.connector.read.{Batch, Scan, ScanBuilder}
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, WriteBuilder}
import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, SupportsTruncate, WriteBuilder}
import org.apache.spark.sql.connector.write.streaming.StreamingWrite
import org.apache.spark.sql.execution.streaming.{Sink, Source}
import org.apache.spark.sql.internal.connector.SimpleTableProvider
import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdate}
import org.apache.spark.sql.sources._
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -394,7 +394,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
() => new KafkaScan(options)

override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
new WriteBuilder {
new WriteBuilder with SupportsTruncate with SupportsStreamingUpdate {
private val options = info.options
private val inputSchema: StructType = info.schema()
private val topic = Option(options.get(TOPIC_OPTION_KEY)).map(_.trim)
Expand All @@ -410,6 +410,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
assert(inputSchema != null)
new KafkaStreamingWrite(topic, producerParams, inputSchema)
}

override def truncate(): WriteBuilder = this
override def update(): WriteBuilder = this
}
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* 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.
*/

package org.apache.spark.sql.internal.connector

import org.apache.spark.sql.connector.write.WriteBuilder

// An internal `WriteBuilder` mixin to support UPDATE streaming output mode.
// TODO: design an official API for streaming output mode UPDATE.
trait SupportsStreamingUpdate extends WriteBuilder {
def update(): WriteBuilder
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, LogicalWriteInfo, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.internal.connector.SimpleTableProvider
import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdate}
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
Expand All @@ -52,8 +52,10 @@ private[noop] object NoopTable extends Table with SupportsWrite {
}
}

private[noop] object NoopWriteBuilder extends WriteBuilder with SupportsTruncate {
private[noop] object NoopWriteBuilder extends WriteBuilder
with SupportsTruncate with SupportsStreamingUpdate {
override def truncate(): WriteBuilder = this
override def update(): WriteBuilder = this
override def buildForBatch(): BatchWrite = NoopBatchWrite
override def buildForStreaming(): StreamingWrite = NoopStreamingWrite
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.QueryExecution
import org.apache.spark.sql.execution.command.StreamingExplainCommand
import org.apache.spark.sql.execution.datasources.v2.StreamWriterCommitProgress
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.connector.SupportsStreamingUpdate
import org.apache.spark.sql.streaming._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.util.{Clock, UninterruptibleThread, Utils}
Expand Down Expand Up @@ -629,14 +630,9 @@ abstract class StreamExecution(
writeBuilder.asInstanceOf[SupportsTruncate].truncate().buildForStreaming()

case Update =>
// Although no v2 sinks really support Update mode now, but during tests we do want them
// to pretend to support Update mode, and treat Update mode same as Append mode.
if (Utils.isTesting) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan thank you for removing this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this what prevented tests from catching that v2 sinks didn't support update?

I find it very concerning that the default implementations were changed to v2 when sinks don't actually support update.

How do we plan to verify that the v2 path works, since we don't have reliable test coverage? Maybe we should move back to v1 after all.

Copy link
Contributor

@tdas tdas May 14, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Spark 2.4.x does not have this condition, and therefore all of Spark 2.4.x since Nov 2018 has been using v2 code paths. That is lot of production hours in v2 sinks and not in v1 sinks, and that dwarf the testing we can do with unit test coverage.

Copy link
Contributor

@HeartSaVioR HeartSaVioR May 14, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just to confirm my understanding, does DSV1 even support update mode in sink natively? I guess not. If then even moving back to v1 is not a solution.

As I commented earlier in other PR, the issue is that streaming output mode is not 100% tied with the output mode in sink. The streaming output mode is applied on the "result table" on aggregation which is "logical" and not materialized. The streaming output mode is working properly for the result table - it might be possible to say tests itself are correct, except the sink side.

I think the issue may be complicated than it seems, because of the fact the result table is flat (no information on keys), and end users can apply arbitrary operations on the result table to produce any form of data before putting to the sink. The semantic of streaming output mode can be lost in any operation between - we don't restrict anything and it's up to the end users to respect the semantic to shape the final output.
(flatMapGroupsWithState is even one of things we don't restrict anything w.r.t streaming output mode even for the result table. End users can emit multiple outputs for a key in append mode across batches - it's up to the end users to implement their logic correctly.)

If we call this as "freedom" and don't want to restrict the end users applying their logic, then it's not weird to have different semantic for streaming output mode and sink output mode, say, update mode for streaming output mode and append for sink output mode. It's up to end users to deal with upsert and that's what Spark has been providing for whole lifetime of Spark 2.x. AFAIK, in fact, sink output mode has been always append mode, regardless of streaming output mode.
(Maybe we want to match the representation of sink output mode as same as batch query when we'd like to separate both.)

So the boundary of streaming output mode is actually misleading and I don't think just adding a method in interface to define the key is enough to address this. (It's much more complicated than that.) I'm not sure there's a plan to force respecting the semantic between mode on result table and mode on sink, but at least as of now, streaming output mode is only effective on creating result table. The result table itself doesn't have the information of the semantic, and any further operation can break the semantic.

writeBuilder.buildForStreaming()
} else {
throw new IllegalArgumentException(
"Data source v2 streaming sinks does not support Update mode.")
}
require(writeBuilder.isInstanceOf[SupportsStreamingUpdate],
table.name + " does not support Update mode.")
writeBuilder.asInstanceOf[SupportsStreamingUpdate].update().buildForStreaming()
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapabi
import org.apache.spark.sql.connector.write.{LogicalWriteInfo, SupportsTruncate, WriteBuilder}
import org.apache.spark.sql.connector.write.streaming.StreamingWrite
import org.apache.spark.sql.execution.streaming.sources.ConsoleWrite
import org.apache.spark.sql.internal.connector.SimpleTableProvider
import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdate}
import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
Expand Down Expand Up @@ -73,11 +73,12 @@ object ConsoleTable extends Table with SupportsWrite {
}

override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
new WriteBuilder with SupportsTruncate {
new WriteBuilder with SupportsTruncate with SupportsStreamingUpdate {
private val inputSchema: StructType = info.schema()

// Do nothing for truncate. Console sink is special that it just prints all the records.
// Do nothing for truncate/update. Console sink is special and it just prints all the records.
override def truncate(): WriteBuilder = this
override def update(): WriteBuilder = this

override def buildForStreaming(): StreamingWrite = {
assert(inputSchema != null)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapabi
import org.apache.spark.sql.connector.write.{DataWriter, LogicalWriteInfo, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.execution.python.PythonForeachWriter
import org.apache.spark.sql.internal.connector.SupportsStreamingUpdate
import org.apache.spark.sql.types.StructType

/**
Expand All @@ -54,12 +55,13 @@ case class ForeachWriterTable[T](
}

override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
new WriteBuilder with SupportsTruncate {
new WriteBuilder with SupportsTruncate with SupportsStreamingUpdate {
private var inputSchema: StructType = info.schema()

// Do nothing for truncate. Foreach sink is special that it just forwards all the records to
// ForeachWriter.
// Do nothing for truncate/update. Foreach sink is special and it just forwards all the
// records to ForeachWriter.
override def truncate(): WriteBuilder = this
override def update(): WriteBuilder = this

override def buildForStreaming(): StreamingWrite = {
new StreamingWrite {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapabi
import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, LogicalWriteInfo, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.execution.streaming.Sink
import org.apache.spark.sql.internal.connector.SupportsStreamingUpdate
import org.apache.spark.sql.types.StructType

/**
Expand All @@ -53,7 +54,7 @@ class MemorySink extends Table with SupportsWrite with Logging {
}

override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
new WriteBuilder with SupportsTruncate {
new WriteBuilder with SupportsTruncate with SupportsStreamingUpdate {
private var needTruncate: Boolean = false
private val inputSchema: StructType = info.schema()

Expand All @@ -62,6 +63,9 @@ class MemorySink extends Table with SupportsWrite with Logging {
this
}

// The in-memory sink treats update as append.
override def update(): WriteBuilder = this

override def buildForStreaming(): StreamingWrite = {
new MemoryStreamingWrite(MemorySink.this, inputSchema, needTruncate)
}
Expand Down