-
Notifications
You must be signed in to change notification settings - Fork 322
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Support Trigger in DataStreamWriter (#153)
- Loading branch information
Showing
3 changed files
with
152 additions
and
0 deletions.
There are no files selected for viewing
36 changes: 36 additions & 0 deletions
36
src/csharp/Microsoft.Spark.E2ETest/IpcTests/Sql/Streaming/TriggerTests.cs
This file contains 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,36 @@ | ||
// Licensed to the .NET Foundation under one or more agreements. | ||
// The .NET Foundation licenses this file to you under the MIT license. | ||
// See the LICENSE file in the project root for more information. | ||
|
||
using Microsoft.Spark.Sql; | ||
using Microsoft.Spark.Sql.Streaming; | ||
using Xunit; | ||
|
||
namespace Microsoft.Spark.E2ETest.IpcTests | ||
{ | ||
[Collection("Spark E2E Tests")] | ||
public class TriggerTests | ||
{ | ||
private readonly SparkSession _spark; | ||
|
||
public TriggerTests(SparkFixture fixture) | ||
{ | ||
_spark = fixture.Spark; | ||
} | ||
|
||
/// <summary> | ||
/// Test Trigger's static functions | ||
/// </summary> | ||
[Fact] | ||
public void TestSignatures() | ||
{ | ||
Assert.IsType<Trigger>(Trigger.Once()); | ||
|
||
Assert.IsType<Trigger>(Trigger.Continuous("1 seconds")); | ||
Assert.IsType<Trigger>(Trigger.Continuous(1000)); | ||
|
||
Assert.IsType<Trigger>(Trigger.ProcessingTime("1 seconds")); | ||
Assert.IsType<Trigger>(Trigger.ProcessingTime(1000)); | ||
} | ||
} | ||
} |
This file contains 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 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,91 @@ | ||
// Licensed to the .NET Foundation under one or more agreements. | ||
// The .NET Foundation licenses this file to you under the MIT license. | ||
// See the LICENSE file in the project root for more information. | ||
|
||
using Microsoft.Spark.Interop; | ||
using Microsoft.Spark.Interop.Ipc; | ||
|
||
namespace Microsoft.Spark.Sql.Streaming | ||
{ | ||
/// <summary> | ||
/// Policy used to indicate how often results should be produced by a | ||
/// <see cref="StreamingQuery"/> | ||
/// </summary> | ||
public sealed class Trigger : IJvmObjectReferenceProvider | ||
{ | ||
private static IJvmBridge Jvm { get; } = SparkEnvironment.JvmBridge; | ||
private static readonly string s_triggerClassName = | ||
"org.apache.spark.sql.streaming.Trigger"; | ||
|
||
private readonly JvmObjectReference _jvmObject; | ||
|
||
internal Trigger(JvmObjectReference jvmObject) => _jvmObject = jvmObject; | ||
|
||
JvmObjectReference IJvmObjectReferenceProvider.Reference => _jvmObject; | ||
|
||
/// <summary> | ||
/// A trigger policy that runs a query periodically based on an interval | ||
/// in processing time. | ||
/// If `interval` is 0, the query will run as fast as possible. | ||
/// </summary> | ||
/// <param name="intervalMs">Milliseconds</param> | ||
/// <returns>Trigger Object</returns> | ||
public static Trigger ProcessingTime(long intervalMs) | ||
{ | ||
return Apply("ProcessingTime", intervalMs); | ||
} | ||
|
||
/// <summary> | ||
/// A trigger policy that runs a query periodically based on an interval | ||
/// in processing time. | ||
/// If `interval` is effectively 0, the query will run as fast as possible. | ||
/// </summary> | ||
/// <param name="interval">string representation for interval. eg. "10 seconds"</param> | ||
/// <returns>Trigger Object</returns> | ||
public static Trigger ProcessingTime(string interval) | ||
{ | ||
return Apply("ProcessingTime", interval); | ||
} | ||
|
||
/// <summary> | ||
/// A trigger that process only one batch of data in a streaming query | ||
/// then terminates the query. | ||
/// </summary> | ||
/// <returns>Trigger Object</returns> | ||
public static Trigger Once() | ||
{ | ||
return Apply("Once"); | ||
} | ||
|
||
/// <summary> | ||
/// A trigger that continuously processes streaming data, | ||
/// asynchronously checkpointing at the specified interval. | ||
/// </summary> | ||
/// <param name="intervalMs">Milliseconds</param> | ||
/// <returns>Trigger Object</returns> | ||
public static Trigger Continuous(long intervalMs) | ||
{ | ||
return Apply("Continuous", intervalMs); | ||
} | ||
|
||
/// <summary> | ||
/// A trigger that continuously processes streaming data, | ||
/// asynchronously checkpointing at the specified interval. | ||
/// </summary> | ||
/// <param name="interval">string representation for interval. eg. "10 seconds"</param> | ||
/// <returns>Trigger Object</returns> | ||
public static Trigger Continuous(string interval) | ||
{ | ||
return Apply("Continuous", interval); | ||
} | ||
|
||
private static Trigger Apply(string funcName, params object[] args) | ||
{ | ||
return new Trigger( | ||
(JvmObjectReference)Jvm.CallStaticJavaMethod( | ||
s_triggerClassName, | ||
funcName, | ||
args)); | ||
} | ||
} | ||
} |