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 @@ -380,7 +380,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
* @since 2.4.0
*/
@InterfaceStability.Evolving
def foreachBatch(function: VoidFunction2[Dataset[T], Long]): DataStreamWriter[T] = {
def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): DataStreamWriter[T] = {
foreachBatch((batchDs: Dataset[T], batchId: Long) => function.call(batchDs, batchId))
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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 test.org.apache.spark.sql.streaming;

import java.io.File;

import org.junit.After;
import org.junit.Before;
import org.junit.Test;

import org.apache.spark.api.java.function.VoidFunction2;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.ForeachWriter;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.streaming.StreamingQuery;
import org.apache.spark.sql.test.TestSparkSession;
import org.apache.spark.util.Utils;

public class JavaDataStreamReaderWriterSuite {
private SparkSession spark;
private String input;

@Before
public void setUp() {
spark = new TestSparkSession();
input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "input").toString();
}

@After
public void tearDown() {
try {
Utils.deleteRecursively(new File(input));
} finally {
spark.stop();
spark = null;
}
}

@Test
public void testForeachBatchAPI() {
Copy link
Contributor

Choose a reason for hiding this comment

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

MINOR: I guess it will be duplicated effort on both Scala and Java suite, but IMHO adding sanity check wouldn't hurt much and prevent further possible misses.

StreamingQuery query = spark
.readStream()
.textFile(input)
.writeStream()
.foreachBatch(new VoidFunction2<Dataset<String>, Long>() {
@Override
public void call(Dataset<String> v1, Long v2) throws Exception {}
})
.start();
query.stop();
}

@Test
public void testForeachAPI() {
StreamingQuery query = spark
.readStream()
.textFile(input)
.writeStream()
.foreach(new ForeachWriter<String>() {
@Override
public boolean open(long partitionId, long epochId) {
return true;
}

@Override
public void process(String value) {}

@Override
public void close(Throwable errorOrNull) {}
})
.start();
query.stop();
}
}