-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-30669][SS] Introduce AdmissionControl APIs for StructuredStreaming #27380
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
Closed
Closed
Changes from 9 commits
Commits
Show all changes
17 commits
Select commit
Hold shift + click to select a range
cfeb6f8
save so far
brkyvz 980719a
Implement API in Kafka
brkyvz 18314a1
pull master
brkyvz 9563c5b
fix issues
brkyvz 4ed77e6
Forgot to implement old code path
brkyvz 62d12ed
Kafka test passes
brkyvz 06256bb
add file source test as well
brkyvz 6d64106
run a second batch as well
brkyvz a9c6897
update comment
brkyvz 0548ba9
Update FileStreamSourceSuite.scala
brkyvz 579b851
Update ReadAllAvailable.java
brkyvz d383daf
Update ReadLimit.java
brkyvz d3f8cd1
Update ReadMaxFiles.java
brkyvz 2deef07
Update ReadMaxRows.java
brkyvz 895fb87
Update SupportsAdmissionControl.java
brkyvz bdbfa11
address
brkyvz de5f486
fix indent
brkyvz File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,6 +32,8 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation | |
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap | ||
| import org.apache.spark.sql.connector.read.streaming | ||
| import org.apache.spark.sql.connector.read.streaming.{ReadAllAvailable, ReadLimit, ReadMaxRows, SupportsAdmissionControl} | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.kafka010.KafkaSource._ | ||
| import org.apache.spark.sql.kafka010.KafkaSourceProvider._ | ||
|
|
@@ -79,7 +81,7 @@ private[kafka010] class KafkaSource( | |
| metadataPath: String, | ||
| startingOffsets: KafkaOffsetRangeLimit, | ||
| failOnDataLoss: Boolean) | ||
| extends Source with Logging { | ||
| extends SupportsAdmissionControl with Source with Logging { | ||
|
|
||
| private val sc = sqlContext.sparkContext | ||
|
|
||
|
|
@@ -114,6 +116,10 @@ private[kafka010] class KafkaSource( | |
| }.partitionToOffsets | ||
| } | ||
|
|
||
| override def getDefaultReadLimit: ReadLimit = { | ||
| maxOffsetsPerTrigger.map(ReadLimit.maxRows).getOrElse(super.getDefaultReadLimit) | ||
| } | ||
|
|
||
| private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None | ||
|
|
||
| private val converter = new KafkaRecordToRowConverter() | ||
|
|
@@ -122,23 +128,30 @@ private[kafka010] class KafkaSource( | |
|
|
||
| /** Returns the maximum available offset for this source. */ | ||
| override def getOffset: Option[Offset] = { | ||
| throw new IllegalStateException( | ||
|
||
| "latestOffset(Offset, ReadLimit) should be called instead of this method") | ||
| } | ||
|
|
||
| override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = { | ||
| // Make sure initialPartitionOffsets is initialized | ||
| initialPartitionOffsets | ||
|
|
||
| val latest = kafkaReader.fetchLatestOffsets( | ||
| currentPartitionOffsets.orElse(Some(initialPartitionOffsets))) | ||
| val offsets = maxOffsetsPerTrigger match { | ||
| case None => | ||
| val offsets = limit match { | ||
| case rows: ReadMaxRows => | ||
| if (currentPartitionOffsets.isEmpty) { | ||
| rateLimit(rows.maxRows(), initialPartitionOffsets, latest) | ||
| } else { | ||
| rateLimit(rows.maxRows(), currentPartitionOffsets.get, latest) | ||
| } | ||
| case _: ReadAllAvailable => | ||
| latest | ||
| case Some(limit) if currentPartitionOffsets.isEmpty => | ||
| rateLimit(limit, initialPartitionOffsets, latest) | ||
| case Some(limit) => | ||
| rateLimit(limit, currentPartitionOffsets.get, latest) | ||
| } | ||
|
|
||
| currentPartitionOffsets = Some(offsets) | ||
| logDebug(s"GetOffset: ${offsets.toSeq.map(_.toString).sorted}") | ||
| Some(KafkaSourceOffset(offsets)) | ||
| KafkaSourceOffset(offsets) | ||
| } | ||
|
|
||
| /** Proportionally distribute limit number of offsets among topicpartitions */ | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
41 changes: 41 additions & 0 deletions
41
...atalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadAllAvailable.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,41 @@ | ||
| /* | ||
| * 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.connector.read.streaming; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
|
|
||
| /** | ||
| * Represents a {@link ReadLimit} where the {@link MicroBatchStream} must scan all the data | ||
| * available at the streaming source. This is meant to be a hard specification as being able | ||
| * to return all available data is necessary for Trigger.Once() to work correctly. | ||
| * If a source is unable to scan all available data, then it must throw an error. | ||
| * | ||
| * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) | ||
| * @since 3.0.0 | ||
| */ | ||
| @Evolving | ||
| public final class ReadAllAvailable implements ReadLimit { | ||
| static final ReadAllAvailable SINGLETON = new ReadAllAvailable(); | ||
|
|
||
| private ReadAllAvailable() {} | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "All Available"; | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
55 changes: 55 additions & 0 deletions
55
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxFiles.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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.connector.read.streaming; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
|
|
||
| /** | ||
| * Represents a {@link ReadLimit} where the {@link MicroBatchStream} should scan approximately the | ||
| * given maximum number of files. | ||
| * | ||
| * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) | ||
| * @since 3.0.0 | ||
| */ | ||
| @Evolving | ||
| public class ReadMaxFiles implements ReadLimit { | ||
| private int files; | ||
|
|
||
| ReadMaxFiles(int maxFiles) { | ||
| this.files = maxFiles; | ||
| } | ||
|
|
||
| /** Approximate maximum rows to scan. */ | ||
| public int maxFiles() { return this.files; } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "MaxFiles: " + maxFiles(); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { | ||
| if (this == o) return true; | ||
| if (o == null || getClass() != o.getClass()) return false; | ||
| ReadMaxFiles other = (ReadMaxFiles) o; | ||
| return other.maxFiles() == maxFiles(); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { return files; } | ||
| } |
55 changes: 55 additions & 0 deletions
55
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxRows.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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.connector.read.streaming; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
|
|
||
| /** | ||
| * Represents a {@link ReadLimit} where the {@link MicroBatchStream} should scan approximately the | ||
| * given maximum number of rows. | ||
| * | ||
| * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) | ||
| * @since 3.0.0 | ||
| */ | ||
| @Evolving | ||
| public final class ReadMaxRows implements ReadLimit { | ||
| private long rows; | ||
|
|
||
| ReadMaxRows(long rows) { | ||
| this.rows = rows; | ||
| } | ||
|
|
||
| /** Approximate maximum rows to scan. */ | ||
| public long maxRows() { return this.rows; } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "MaxRows: " + maxRows(); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { | ||
| if (this == o) return true; | ||
| if (o == null || getClass() != o.getClass()) return false; | ||
| ReadMaxRows other = (ReadMaxRows) o; | ||
| return other.maxRows() == maxRows(); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { return Long.hashCode(this.rows); } | ||
| } |
58 changes: 58 additions & 0 deletions
58
...src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsAdmissionControl.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,58 @@ | ||
| /* | ||
| * 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.connector.read.streaming; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
|
|
||
| /** | ||
| * A mix-in interface for {@link SparkDataStream} streaming sources to signal that they can control | ||
| * the rate of data ingested into the system. These rate limits can come implicitly from the | ||
| * contract of triggers, e.g. Trigger.Once() requires that a micro-batch process all data | ||
| * available to the system at the start of the micro-batch. Alternatively, sources can decide to | ||
| * limit ingest through data source options. | ||
| * | ||
| * Through this interface, a MicroBatchStream should be able to return the next offset that it will | ||
| * process until given a {@link ReadLimit}. | ||
| * | ||
| * @since 3.0.0 | ||
| */ | ||
| @Evolving | ||
| public interface SupportsAdmissionControl extends SparkDataStream { | ||
|
|
||
| /** | ||
| * Returns the read limits potentially passed to the data source through options when creating | ||
| * the data source. | ||
| */ | ||
| default ReadLimit getDefaultReadLimit() { | ||
| return ReadLimit.allAvailable(); | ||
| } | ||
|
|
||
| /** | ||
| * Returns the most recent offset available given a read limit. The start offset can be used | ||
| * to figure out how much new data should be read given the limit. Users should implement this | ||
| * method instead of latestOffset for a MicroBatchStream or getOffset for Source. | ||
| * | ||
| * When this method is called on a `Source`, the source can return `null` if there is no | ||
| * data to process. In addition, for the very first micro-batch, the `startOffset` will be | ||
| * null as well. | ||
| * | ||
| * When this method is called on a MicroBatchStream, the `startOffset` will be `initialOffset` | ||
| * for the very first micro-batch. The source can return `null` if there is no data to process. | ||
| */ | ||
| Offset latestOffset(Offset startOffset, ReadLimit limit); | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is
UnsupportedOperationExceptionbetter in this context?