Skip to content
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

improve testing framework #4962

Merged
merged 26 commits into from
Dec 13, 2023
Merged
Show file tree
Hide file tree
Changes from 25 commits
Commits
Show all changes
26 commits
Select commit Hold shift + click to select a range
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
1 change: 1 addition & 0 deletions .scalafmt.conf
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ rewrite {

project.excludeFilters = [
"MultiJoin.scala",
"SMBMultiJoin.scala",
"TupleCoders.scala",
"scalafix/input/*",
"scalafix/output/*"
Expand Down
2 changes: 1 addition & 1 deletion build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -1285,7 +1285,7 @@ lazy val `scio-smb`: Project = project
.in(file("scio-smb"))
.dependsOn(
`scio-core`,
`scio-test` % Test
`scio-test` % "test->test"
)
.settings(commonSettings)
.settings(publishSettings)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,6 @@ class Neo4jIOIT extends PipelineSpec with Eventually with ForAllTestContainer {

val americanPresident: Movie = Movie("American President", 1995)

val options: PipelineOptions = PipelineOptionsFactory.create()
lazy val neo4jOptions: Neo4jOptions = Neo4jOptions(
Neo4jConnectionOptions(container.boltUrl, container.username, container.password)
)
Expand All @@ -107,7 +106,7 @@ class Neo4jIOIT extends PipelineSpec with Eventually with ForAllTestContainer {
Role(martin, Movie("American President", 1995), "A.J. MacInerney")
)

runWithRealContext(options) { sc =>
runWithRealContext() { sc =>
val resultQueryRoles = sc.neo4jCypher[Role](neo4jOptions, queryRoles)
resultQueryRoles should containInAnyOrder(expectedRoles)
}
Expand All @@ -132,7 +131,7 @@ class Neo4jIOIT extends PipelineSpec with Eventually with ForAllTestContainer {
Role(morgan, Movie("The Shawshank Redemption", 1994), "Ellis Boyd 'Red' Redding")
)

runWithRealContext(options) { sc =>
runWithRealContext() { sc =>
val resultQueryMovieYear = sc
.parallelize(queryParams)
.neo4jCypher[Role](neo4jOptions, queryRoles)
Expand All @@ -154,7 +153,7 @@ class Neo4jIOIT extends PipelineSpec with Eventually with ForAllTestContainer {
| (c:Country {name: origin.country})
|CREATE (m)-[:ORIGIN]->(c)
|""".stripMargin
runWithRealContext(options) { sc =>
runWithRealContext() { sc =>
sc
.parallelize(movieOrigins)
.saveAsNeo4j(neo4jOptions, insertOrigins)
Expand Down
10 changes: 6 additions & 4 deletions scio-core/src/main/scala/com/spotify/scio/ScioContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -308,10 +308,12 @@ object ScioContext {
new ScioContext(options, artifacts)

/** Create a new [[ScioContext]] instance for testing. */
def forTest(): ScioContext = {
val opts = PipelineOptionsFactory
.fromArgs("--appName=" + TestUtil.newTestId())
.as(classOf[PipelineOptions])
def forTest(): ScioContext = forTest(TestUtil.newTestId())

/** Create a new [[ScioContext]] instance for testing. */
def forTest(testId: String): ScioContext = {
val opts = PipelineOptionsFactory.create()
opts.as(classOf[ApplicationNameOptions]).setAppName(testId)
new ScioContext(opts, List[String]())
}

Expand Down
8 changes: 8 additions & 0 deletions scio-core/src/main/scala/com/spotify/scio/io/ScioIO.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,10 @@
package com.spotify.scio.io

import com.spotify.scio.ScioContext
import com.spotify.scio.coders.Coder
import com.spotify.scio.testing.TestDataManager
import com.spotify.scio.values.SCollection

import scala.annotation.unused

sealed trait TapT[A] extends Serializable {
Expand Down Expand Up @@ -159,6 +161,12 @@ trait TestIO[T] extends ScioIO[T] {
throw new UnsupportedOperationException(s"$this is for testing purpose only")
}

trait KeyedIO[T] { self: ScioIO[T] =>
type KeyT
def keyBy: T => KeyT
def keyCoder: Coder[KeyT]
}

/**
* Special version of [[ScioIO]] for use with [[ScioContext.customInput]] and
* [[SCollection.saveAsCustomOutput]].
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,8 +64,8 @@ private[scio] class TestInput(val m: Map[String, JobInputSource[_]]) {
val s: MSet[String] =
java.util.concurrent.ConcurrentHashMap.newKeySet[String]().asScala

def apply[T](io: ScioIO[T]): JobInputSource[T] = {
val key = io.testId
def apply[T](io: ScioIO[T]): JobInputSource[T] = apply(io.testId)
def apply[T](key: String): JobInputSource[T] = {
require(
m.contains(key),
s"Missing test input: $key, available: ${m.keys.mkString("[", ", ", "]")}"
Expand All @@ -85,10 +85,9 @@ private[scio] class TestInput(val m: Map[String, JobInputSource[_]]) {
private[scio] class TestOutput(val m: Map[String, SCollection[_] => Any]) {
val s: MSet[String] =
java.util.concurrent.ConcurrentHashMap.newKeySet[String]().asScala

def apply[T](io: ScioIO[T]): SCollection[T] => Any = {
def apply[T](io: ScioIO[T]): SCollection[T] => Any = apply(io.testId)
def apply[T](key: String): SCollection[T] => Any = {
// TODO: support Materialize outputs, maybe Materialized[T]?
val key = io.testId
require(
m.contains(key),
s"Missing test output: $key, available: ${m.keys.mkString("[", ", ", "]")}"
Expand Down
10 changes: 8 additions & 2 deletions scio-core/src/main/scala/com/spotify/scio/testing/TestUtil.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,17 @@ package com.spotify.scio.testing
import java.util.UUID

private[scio] object TestUtil {
def newTestId(className: String = "TestClass"): String = {

def newTestId(clazz: Class[_]): String = {
val className = clazz.getSimpleName
val uuid = UUID.randomUUID().toString.replaceAll("-", "")
s"JobTest-$className-$uuid"
}
def newTestId(): String = {
val uuid = UUID.randomUUID().toString.replaceAll("-", "")
s"JobTest-$uuid"
}

def isTestId(appName: String): Boolean =
"JobTest-[^-]+-[a-z0-9]+".r.pattern.matcher(appName).matches()
"JobTest(-[^-]+)?-[a-z0-9]+".r.pattern.matcher(appName).matches()
}
Original file line number Diff line number Diff line change
Expand Up @@ -1435,7 +1435,7 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] {
)(implicit ev: T <:< String): SCollection[A] =
if (context.isTest) {
val id = context.testId.get
this.flatMap(s => TestDataManager.getInput(id)(ReadIO(ev(s))).asIterable.get)
this.flatMap(s => TestDataManager.getInput(id)(ReadIO[A](ev(s))).asIterable.get)
} else {
this
.covary_[String]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;

/** API for reading and writing Avro sorted-bucket files. */
public class AvroSortedBucketIO {
Expand Down Expand Up @@ -192,11 +191,6 @@ public static <K1, K2, T extends SpecificRecord> TransformOutput<K1, K2, T> tran
/** Reads from Avro sorted-bucket files, to be used with {@link SortedBucketIO.CoGbk}. */
@AutoValue
public abstract static class Read<T extends IndexedRecord> extends SortedBucketIO.Read<T> {
@Nullable
abstract ImmutableList<String> getInputDirectories();

abstract String getFilenameSuffix();

@Nullable
abstract Schema getSchema();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;

/** API for reading and writing BigQuery {@link TableRow} JSON sorted-bucket files. */
public class JsonSortedBucketIO {
Expand Down Expand Up @@ -108,10 +107,6 @@ public static <K1, K2> TransformOutput<K1, K2> transformOutput(
*/
@AutoValue
public abstract static class Read extends SortedBucketIO.Read<TableRow> {
@Nullable
abstract ImmutableList<String> getInputDirectories();

abstract String getFilenameSuffix();

abstract Compression getCompression();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.filter2.predicate.FilterPredicate;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
Expand Down Expand Up @@ -200,11 +199,6 @@ public static <K1, K2, T extends SpecificRecord> TransformOutput<K1, K2, T> tran
/** Reads from Avro sorted-bucket files, to be used with {@link SortedBucketIO.CoGbk}. */
@AutoValue
public abstract static class Read<T extends IndexedRecord> extends SortedBucketIO.Read<T> {
@Nullable
abstract ImmutableList<String> getInputDirectories();

abstract String getFilenameSuffix();

@Nullable
abstract Schema getSchema();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -493,6 +493,12 @@ public abstract static class TransformOutput<K1, K2, V> implements Serializable

/** Represents a single sorted-bucket source written using {@link SortedBucketSink}. */
public abstract static class Read<V> implements Serializable {

@Nullable
abstract ImmutableList<String> getInputDirectories();

abstract String getFilenameSuffix();

public abstract TupleTag<V> getTupleTag();

protected abstract BucketedInput<V> toBucketedInput(SortedBucketSource.Keying keying);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.tensorflow.proto.example.Example;

/**
Expand Down Expand Up @@ -125,10 +124,6 @@ public static <K1, K2> TransformOutput<K1, K2> transformOutput(
*/
@AutoValue
public abstract static class Read extends SortedBucketIO.Read<Example> {
@Nullable
abstract ImmutableList<String> getInputDirectories();

abstract String getFilenameSuffix();

abstract Compression getCompression();

Expand Down
Loading