Skip to content
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
20 changes: 20 additions & 0 deletions connector/kafka-0-10-sql/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,26 @@
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<version>3.6.1</version>
<executions>
<execution>
<id>write-classpath</id>
<goals>
<goal>build-classpath</goal>
</goals>
<phase>generate-resources</phase>
<configuration>
<includeScope>runtime</includeScope>
<outputFile>${project.build.directory}/classpath.txt</outputFile>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>

</project>
5 changes: 5 additions & 0 deletions dev/requirements.txt
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,11 @@ openpyxl
# PySpark test dependencies (optional)
coverage

# Kafka streaming test dependencies (optional)
# Required for running Kafka integration tests with Docker test containers
testcontainers[kafka]>=3.7.0
kafka-python>=2.0.2

# Linter
mypy==1.8.0
pytest-mypy-plugins==1.9.3
Expand Down
1 change: 1 addition & 0 deletions dev/sparktestsupport/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -659,6 +659,7 @@ def __hash__(self):
"pyspark.sql.tests.streaming.test_streaming",
"pyspark.sql.tests.streaming.test_streaming_foreach",
"pyspark.sql.tests.streaming.test_streaming_foreach_batch",
"pyspark.sql.tests.streaming.test_streaming_kafka_rtm",
"pyspark.sql.tests.streaming.test_streaming_listener",
"pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state",
"pyspark.sql.tests.pandas.streaming.test_pandas_transform_with_state",
Expand Down
19 changes: 16 additions & 3 deletions python/pyspark/sql/connect/streaming/readwriter.py
Original file line number Diff line number Diff line change
Expand Up @@ -493,22 +493,27 @@ def trigger(self, *, continuous: str) -> "DataStreamWriter":
def trigger(self, *, availableNow: bool) -> "DataStreamWriter":
...

@overload
def trigger(self, *, realTime: str) -> "DataStreamWriter":
...

def trigger(
self,
*,
processingTime: Optional[str] = None,
once: Optional[bool] = None,
continuous: Optional[str] = None,
availableNow: Optional[bool] = None,
realTime: Optional[str] = None,
) -> "DataStreamWriter":
params = [processingTime, once, continuous, availableNow]
params = [processingTime, once, continuous, availableNow, realTime]

if params.count(None) == 4:
if params.count(None) == 5:
raise PySparkValueError(
errorClass="ONLY_ALLOW_SINGLE_TRIGGER",
messageParameters={},
)
elif params.count(None) < 3:
elif params.count(None) < 4:
raise PySparkValueError(
errorClass="ONLY_ALLOW_SINGLE_TRIGGER",
messageParameters={},
Expand Down Expand Up @@ -541,6 +546,14 @@ def trigger(
)
self._write_proto.continuous_checkpoint_interval = continuous.strip()

elif realTime is not None:
if type(realTime) != str or len(realTime.strip()) == 0:
raise PySparkValueError(
errorClass="VALUE_NOT_NON_EMPTY_STR",
messageParameters={"arg_name": "realTime", "arg_value": str(realTime)},
)
self._write_proto.real_time_batch_duration = realTime.strip()

else:
if availableNow is not True:
raise PySparkValueError(
Expand Down
26 changes: 23 additions & 3 deletions python/pyspark/sql/streaming/readwriter.py
Original file line number Diff line number Diff line change
Expand Up @@ -1243,6 +1243,7 @@ def trigger(
once: Optional[bool] = None,
continuous: Optional[str] = None,
availableNow: Optional[bool] = None,
realTime: Optional[str] = None
) -> "DataStreamWriter":
"""Set the trigger for the stream query. If this is not set it will run the query as fast
as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``.
Expand All @@ -1268,6 +1269,10 @@ def trigger(
availableNow : bool, optional
if set to True, set a trigger that processes all available data in multiple
batches then terminates the query. Only one trigger can be set.
realTime : str, optional
a batch duration as a string, e.g. '5 seconds', '1 minute'.
Set a trigger that runs a real time mode query with
batch at the specified duration. Only one trigger can be set.

Notes
-----
Expand All @@ -1291,15 +1296,20 @@ def trigger(

>>> df.writeStream.trigger(availableNow=True)
<...streaming.readwriter.DataStreamWriter object ...>

Trigger the query for real time mode execution every 5 seconds.

>>> df.writeStream.trigger(realTime='5 seconds')
<...streaming.readwriter.DataStreamWriter object ...>
"""
params = [processingTime, once, continuous, availableNow]
params = [processingTime, once, continuous, availableNow, realTime]

if params.count(None) == 4:
if params.count(None) == 5:
raise PySparkValueError(
errorClass="ONLY_ALLOW_SINGLE_TRIGGER",
messageParameters={},
)
elif params.count(None) < 3:
elif params.count(None) < 4:
raise PySparkValueError(
errorClass="ONLY_ALLOW_SINGLE_TRIGGER",
messageParameters={},
Expand Down Expand Up @@ -1342,6 +1352,16 @@ def trigger(
jTrigger = getattr(
self._spark._sc._jvm, "org.apache.spark.sql.streaming.Trigger"
).Continuous(interval)
elif realTime is not None:
if type(realTime) != str or len(realTime.strip()) == 0:
raise PySparkValueError(
errorClass="VALUE_NOT_NON_EMPTY_STR",
messageParameters={"arg_name": "realTime", "arg_value": str(realTime)},
)
batch_duration = realTime.strip()
jTrigger = getattr(
self._spark._sc._jvm, "org.apache.spark.sql.streaming.Trigger"
).RealTime(batch_duration)
else:
if availableNow is not True:
raise PySparkValueError(
Expand Down
Loading