From 3822aa914e44eafc2c219feef1d966b9446b61e3 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 17 May 2023 14:00:15 +0200 Subject: [PATCH 01/26] Add prefix param for all IO APIs --- .../scala/com/spotify/scio/avro/AvroIO.scala | 76 +++++++++--------- .../scio/avro/syntax/SCollectionSyntax.scala | 49 ++++++++---- .../scala/com/spotify/scio/io/BinaryIO.scala | 34 ++++---- .../scala/com/spotify/scio/io/TextIO.scala | 44 ++++++----- .../io/dynamic/syntax/SCollectionSyntax.scala | 78 ++++++++++++------- .../scio/util/FilenamePolicySupplier.scala | 41 +++++----- .../com/spotify/scio/util/ScioUtil.scala | 21 +++-- .../com/spotify/scio/values/SCollection.scala | 9 ++- .../com/spotify/scio/extra/csv/CsvIO.scala | 4 +- .../com/spotify/scio/extra/json/JsonIO.scala | 17 ++-- .../com/spotify/scio/extra/json/package.scala | 8 +- .../spotify/scio/datastore/DatastoreIO.scala | 2 +- .../com/spotify/scio/pubsub/PubsubIO.scala | 2 +- .../com/spotify/scio/neo4j/Neo4jIO.scala | 2 +- .../scio/parquet/avro/ParquetAvroIO.scala | 33 ++++---- .../dynamic/syntax/SCollectionSyntax.scala | 12 ++- .../avro/syntax/SCollectionSyntax.scala | 25 +++--- .../parquet/tensorflow/ParquetExampleIO.scala | 33 ++++---- .../dynamic/syntax/SCollectionSyntax.scala | 12 ++- .../tensorflow/syntax/SCollectionSyntax.scala | 8 +- .../scio/parquet/types/ParquetTypeIO.scala | 33 ++++---- .../dynamic/syntax/SCollectionSyntax.scala | 12 ++- .../types/syntax/SCollectionSyntax.scala | 8 +- .../spotify/scio/tensorflow/TFRecordIO.scala | 40 +++++----- .../tensorflow/syntax/SCollectionSyntax.scala | 24 +++--- .../com/spotify/scio/io/ScioIOTest.scala | 10 ++- 26 files changed, 369 insertions(+), 268 deletions(-) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala index 1187a800de..ed84d76327 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala @@ -138,19 +138,19 @@ sealed trait AvroIO[T] extends ScioIO[T] { suffix: String, codec: CodecFactory, metadata: Map[String, AnyRef], - shardNameTemplate: String, - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, - isWindowed: Boolean + prefix: String, + shardNameTemplate: String, + isWindowed: Boolean, + tempDirectory: ResourceId ): beam.AvroIO.Write[U] = { + require(tempDirectory != null, "tempDirectory must not be null") val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed - ) + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(path, suffix) val transform = write .to(fp) .withTempDirectory(tempDirectory) @@ -197,10 +197,11 @@ final case class SpecificRecordIO[T <: SpecificRecord: ClassTag: Coder](path: St params.suffix, params.codec, params.metadata, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, - ScioUtil.isWindowed(data) + params.prefix, + params.shardNameTemplate, + ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) tap(()) @@ -246,10 +247,11 @@ final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[Ge params.suffix, params.codec, params.metadata, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, - ScioUtil.isWindowed(data) + params.prefix, + params.shardNameTemplate, + ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) tap(()) @@ -297,26 +299,25 @@ final case class GenericRecordParseIO[T](path: String, parseFn: GenericRecord => object AvroIO { object WriteParam { private[scio] val DefaultNumShards = 0 - private[scio] val DefaultSuffix = "" + private[scio] val DefaultSuffix = ".avro" private[scio] val DefaultCodec: CodecFactory = CodecFactory.deflateCodec(6) private[scio] val DefaultMetadata: Map[String, AnyRef] = Map.empty + private[scio] val DefaultFilenamePolicySupplier = null + private[scio] val DefaultPrefix = null private[scio] val DefaultShardNameTemplate: String = null private[scio] val DefaultTempDirectory = null - private[scio] val DefaultFilenamePolicySupplier = null } final case class WriteParam private ( numShards: Int = WriteParam.DefaultNumShards, - private val _suffix: String = WriteParam.DefaultSuffix, + suffix: String = WriteParam.DefaultSuffix, codec: CodecFactory = WriteParam.DefaultCodec, metadata: Map[String, AnyRef] = WriteParam.DefaultMetadata, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, - tempDirectory: String = WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier - ) { - // TODO this is kinda weird when compared with the other IOs? - val suffix: String = _suffix + ".avro" - } + tempDirectory: String = WriteParam.DefaultTempDirectory + ) @inline final def apply[T](id: String): AvroIO[T] = new AvroIO[T] with TestIO[T] { @@ -346,19 +347,15 @@ object AvroTyped { suffix: String, codec: CodecFactory, metadata: Map[String, AnyRef], - shardNameTemplate: String, - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, - isWindowed: Boolean + prefix: String, + shardNameTemplate: String, + isWindowed: Boolean, + tempDirectory: ResourceId ) = { - val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed - ) + require(tempDirectory != null, "tempDirectory must not be null") + val fp = FilenamePolicySupplier + .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) val transform = write .to(fp) .withTempDirectory(tempDirectory) @@ -395,10 +392,11 @@ object AvroTyped { params.suffix, params.codec, params.metadata, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, - ScioUtil.isWindowed(data) + params.prefix, + params.shardNameTemplate, + ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) tap(()) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/SCollectionSyntax.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/SCollectionSyntax.scala index 6bf4c263ee..d2a6c3ccf6 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/SCollectionSyntax.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/SCollectionSyntax.scala @@ -48,16 +48,19 @@ final class GenericRecordSCollectionOps(private val self: SCollection[GenericRec metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, shardNameTemplate: String = AvroIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = AvroIO.WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = + AvroIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = AvroIO.WriteParam.DefaultPrefix ): ClosedTap[GenericRecord] = { val param = AvroIO.WriteParam( numShards, suffix, codec, metadata, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.write(GenericRecordIO(path, schema))(param) } @@ -74,21 +77,24 @@ final class ObjectFileSCollectionOps[T](private val self: SCollection[T]) extend def saveAsObjectFile( path: String, numShards: Int = AvroIO.WriteParam.DefaultNumShards, - suffix: String = ".obj", + suffix: String = ".obj.avro", codec: CodecFactory = AvroIO.WriteParam.DefaultCodec, metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, shardNameTemplate: String = AvroIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = AvroIO.WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = + AvroIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = AvroIO.WriteParam.DefaultPrefix )(implicit coder: Coder[T]): ClosedTap[T] = { val param = ObjectFileIO.WriteParam( numShards, suffix, codec, metadata, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.write(ObjectFileIO(path))(param) } @@ -109,16 +115,19 @@ final class SpecificRecordSCollectionOps[T <: SpecificRecord](private val self: metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, shardNameTemplate: String = AvroIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = AvroIO.WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = + AvroIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = AvroIO.WriteParam.DefaultPrefix )(implicit ct: ClassTag[T], coder: Coder[T]): ClosedTap[T] = { val param = AvroIO.WriteParam( numShards, suffix, codec, metadata, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.write(SpecificRecordIO[T](path))(param) } @@ -139,16 +148,19 @@ final class TypedAvroSCollectionOps[T <: HasAvroAnnotation](private val self: SC metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, shardNameTemplate: String = AvroIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = AvroIO.WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = + AvroIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = AvroIO.WriteParam.DefaultPrefix )(implicit ct: ClassTag[T], tt: TypeTag[T], coder: Coder[T]): ClosedTap[T] = { val param = AvroIO.WriteParam( numShards, suffix, codec, metadata, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.write(AvroTyped.AvroIO[T](path))(param) } @@ -165,21 +177,24 @@ final class ProtobufSCollectionOps[T <: Message](private val self: SCollection[T def saveAsProtobufFile( path: String, numShards: Int = AvroIO.WriteParam.DefaultNumShards, - suffix: String = ".protobuf", + suffix: String = ".protobuf.avro", codec: CodecFactory = AvroIO.WriteParam.DefaultCodec, metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, shardNameTemplate: String = AvroIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = AvroIO.WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = + AvroIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = AvroIO.WriteParam.DefaultPrefix )(implicit ct: ClassTag[T], coder: Coder[T]): ClosedTap[T] = { val param = ProtobufIO.WriteParam( numShards, suffix, codec, metadata, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.write(ProtobufIO[T](path))(param) } diff --git a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala index 1b56219c20..d5441a2367 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala @@ -53,30 +53,24 @@ final case class BinaryIO(path: String) extends ScioIO[Array[Byte]] { private def binaryOut( path: String, - prefix: String, suffix: String, numShards: Int, compression: Compression, header: Array[Byte], footer: Array[Byte], - shardNameTemplate: String, framePrefix: Array[Byte] => Array[Byte], frameSuffix: Array[Byte] => Array[Byte], - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, - isWindowed: Boolean + prefix: String, + shardNameTemplate: String, + isWindowed: Boolean, + tempDirectory: ResourceId ): WriteFiles[Array[Byte], Void, Array[Byte]] = { - val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed, - prefix - ) - val dynamicDestinations = - DynamicFileDestinations.constant(fp, SerializableFunctions.identity[Array[Byte]]) + require(tempDirectory != null, "tempDirectory must not be null") + val fp = FilenamePolicySupplier + .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val dynamicDestinations = DynamicFileDestinations + .constant(fp, SerializableFunctions.identity[Array[Byte]]) val sink = new BytesSink( header, footer, @@ -94,18 +88,18 @@ final case class BinaryIO(path: String) extends ScioIO[Array[Byte]] { data.applyInternal( binaryOut( path, - params.prefix, params.suffix, params.numShards, params.compression, params.header, params.footer, - params.shardNameTemplate, params.framePrefix, params.frameSuffix, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, - ScioUtil.isWindowed(data) + params.prefix, + params.shardNameTemplate, + ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) EmptyTap @@ -147,7 +141,7 @@ object BinaryIO { private[scio] val DefaultFilenamePolicySupplier = null } - final case class WriteParam( + final case class WriteParam private ( prefix: String = WriteParam.DefaultPrefix, suffix: String = WriteParam.DefaultSuffix, numShards: Int = WriteParam.DefaultNumShards, diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index 35947404ba..6d33a0d655 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -59,19 +59,19 @@ final case class TextIO(path: String) extends ScioIO[String] { compression: Compression, header: Option[String], footer: Option[String], - shardNameTemplate: String, - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, - isWindowed: Boolean + prefix: String, + shardNameTemplate: String, + isWindowed: Boolean, + tempDirectory: ResourceId ) = { + require(tempDirectory != null, "tempDirectory must not be null") val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed - ) + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(path, suffix) var transform = write .to(fp) .withTempDirectory(tempDirectory) @@ -93,10 +93,11 @@ final case class TextIO(path: String) extends ScioIO[String] { params.compression, params.header, params.footer, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, - ScioUtil.isWindowed(data) + params.prefix, + params.shardNameTemplate, + ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) tap(TextIO.ReadParam()) @@ -112,7 +113,7 @@ object TextIO { private[scio] val DefaultEmptyMatchTreatment = EmptyMatchTreatment.DISALLOW } - final case class ReadParam( + final case class ReadParam private ( compression: Compression = ReadParam.DefaultCompression, emptyMatchTreatment: EmptyMatchTreatment = ReadParam.DefaultEmptyMatchTreatment ) @@ -123,9 +124,10 @@ object TextIO { private[scio] val DefaultSuffix = ".txt" private[scio] val DefaultNumShards = 0 private[scio] val DefaultCompression = Compression.UNCOMPRESSED + private[scio] val DefaultFilenamePolicySupplier = null + private[scio] val DefaultPrefix = null private[scio] val DefaultShardNameTemplate = null private[scio] val DefaultTempDirectory = null - private[scio] val DefaultFilenamePolicySupplier = null } final val DefaultWriteParam: WriteParam = WriteParam( @@ -134,20 +136,22 @@ object TextIO { WriteParam.DefaultCompression, WriteParam.DefaultHeader, WriteParam.DefaultFooter, + WriteParam.DefaultFilenamePolicySupplier, + WriteParam.DefaultPrefix, WriteParam.DefaultShardNameTemplate, - WriteParam.DefaultTempDirectory, - WriteParam.DefaultFilenamePolicySupplier + WriteParam.DefaultTempDirectory ) - final case class WriteParam( + final case class WriteParam private ( suffix: String, numShards: Int, compression: Compression, header: Option[String], footer: Option[String], + filenamePolicySupplier: FilenamePolicySupplier, + prefix: String, shardNameTemplate: String, - tempDirectory: String, - filenamePolicySupplier: FilenamePolicySupplier + tempDirectory: String ) private[scio] def textFile(path: String): Iterator[String] = { diff --git a/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala b/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala index 1d6b6b907a..fec33830b6 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala @@ -20,7 +20,7 @@ package com.spotify.scio.io.dynamic.syntax import com.google.protobuf.Message import com.spotify.scio.io.{ClosedTap, EmptyTap} import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} -import com.spotify.scio.util.{Functions, ProtobufUtil} +import com.spotify.scio.util.{Functions, ProtobufUtil, ScioUtil} import com.spotify.scio.values.SCollection import org.apache.avro.Schema import org.apache.avro.file.CodecFactory @@ -33,27 +33,27 @@ import org.apache.beam.sdk.{io => beam} import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag +import scala.util.chaining._ import java.util.{HashMap => JHashMap} object DynamicSCollectionOps { private[scio] def writeDynamic[A]( path: String, + destinationFn: A => String, numShards: Int, + prefix: String, suffix: String, - destinationFn: A => String, - tempDirectory: String = null + tempDirectory: String ): FileIO.Write[String, A] = { - val transform = FileIO + val naming = ScioUtil.defaultNaming(Option(prefix).getOrElse("part"), suffix) _ + FileIO .writeDynamic[String, A]() .to(path) - .withNumShards(numShards) .by(Functions.serializableFn(destinationFn)) + .withNumShards(numShards) .withDestinationCoder(StringUtf8Coder.of()) - .withNaming(Functions.serializableFn { destination: String => - FileIO.Write.defaultNaming(s"$destination/part", suffix) - }) - - Option(tempDirectory).fold(transform)(transform.withTempDirectory) + .withNaming(Functions.serializableFn(naming)) + .pipe(t => Option(tempDirectory).fold(t)(t.withTempDirectory)) } } @@ -69,7 +69,8 @@ final class DynamicSpecificRecordSCollectionOps[T <: SpecificRecord]( suffix: String = ".avro", codec: CodecFactory = CodecFactory.deflateCodec(6), metadata: Map[String, AnyRef] = Map.empty, - tempDirectory: String = null + tempDirectory: String = null, + prefix: String = null )( destinationFn: T => String )(implicit ct: ClassTag[T]): ClosedTap[Nothing] = { @@ -86,8 +87,14 @@ final class DynamicSpecificRecordSCollectionOps[T <: SpecificRecord]( .withCodec(codec) .withMetadata(nm) val write = - writeDynamic(path, numShards, suffix, destinationFn, tempDirectory) - .via(sink) + writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) self.applyInternal(write) } @@ -112,7 +119,8 @@ final class DynamicGenericRecordSCollectionOps[T <: GenericRecord](private val s suffix: String = ".avro", codec: CodecFactory = CodecFactory.deflateCodec(6), metadata: Map[String, AnyRef] = Map.empty, - tempDirectory: String = null + tempDirectory: String = null, + prefix: String = null )( destinationFn: T => String ): ClosedTap[Nothing] = { @@ -134,8 +142,14 @@ final class DynamicGenericRecordSCollectionOps[T <: GenericRecord](private val s .withCodec(codec) .withMetadata(nm) val write = - writeDynamic(path, numShards, suffix, destinationFn, tempDirectory) - .via(sink) + writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) self.applyInternal(write) } @@ -157,7 +171,8 @@ final class DynamicSCollectionOps[T](private val self: SCollection[T]) extends A numShards: Int = 0, suffix: String = ".txt", compression: Compression = Compression.UNCOMPRESSED, - tempDirectory: String = null + tempDirectory: String = null, + prefix: String = null )(destinationFn: String => String)(implicit ct: ClassTag[T]): ClosedTap[Nothing] = { val s = if (classOf[String] isAssignableFrom ct.runtimeClass) { self.asInstanceOf[SCollection[String]] @@ -169,8 +184,14 @@ final class DynamicSCollectionOps[T](private val self: SCollection[T]) extends A "Text file with dynamic destinations cannot be used in a test context" ) } else { - val write = writeDynamic(path, numShards, suffix, destinationFn, tempDirectory) - .via(beam.TextIO.sink()) + val write = writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(beam.TextIO.sink()) .withCompression(compression) s.applyInternal(write) } @@ -189,7 +210,8 @@ final class DynamicProtobufSCollectionOps[T <: Message](private val self: SColle suffix: String = ".protobuf", codec: CodecFactory = CodecFactory.deflateCodec(6), metadata: Map[String, AnyRef] = Map.empty, - tempDirectory: String = null + tempDirectory: String = null, + prefix: String = null )(destinationFn: T => String)(implicit ct: ClassTag[T]): ClosedTap[Nothing] = { val protoCoder = Coder.protoMessageCoder[T] val elemCoder = CoderMaterializer.beam(self.context, protoCoder) @@ -205,16 +227,18 @@ final class DynamicProtobufSCollectionOps[T <: Message](private val self: SColle val sink = beam.AvroIO .sinkViaGenericRecords( avroSchema, - new RecordFormatter[T] { - override def formatRecord(element: T, schema: Schema): GenericRecord = - AvroBytesUtil.encode(elemCoder, element) - } + (element: T, schema: Schema) => AvroBytesUtil.encode(elemCoder, element) ) .withCodec(codec) .withMetadata(nm) - val write = - writeDynamic(path, numShards, suffix, destinationFn, tempDirectory) - .via(sink) + val write = writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) self.applyInternal(write) } diff --git a/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala b/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala index 59e1da84c7..e6085d7d46 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala @@ -28,35 +28,38 @@ trait FilenamePolicySupplier { } object FilenamePolicySupplier { - def resolve( - path: String, - suffix: String, + def apply( + prefix: String, shardNameTemplate: String, - tempDirectory: ResourceId, + isWindowed: Boolean + ): FilenamePolicySupplier = { + require(shardNameTemplate != null, "shardNameTemplate must not be null") + (path: String, suffix: String) => + ScioUtil.defaultFilenamePolicy( + path, + Option(prefix).getOrElse("part"), + shardNameTemplate, + suffix, + isWindowed + ) + } + + def resolve( filenamePolicySupplier: FilenamePolicySupplier, - isWindowed: Boolean, - defaultPrefix: String = null - ): FilenamePolicy = { - require(tempDirectory != null, "tempDirectory must not be null") + prefix: String, + shardNameTemplate: String, + isWindowed: Boolean + ): FilenamePolicySupplier = { require( shardNameTemplate == null || filenamePolicySupplier == null, "shardNameTemplate and filenamePolicySupplier may not be used together" ) require( - defaultPrefix == null || filenamePolicySupplier == null, + prefix == null || filenamePolicySupplier == null, "prefix and filenamePolicySupplier may not be used together" ) - Option(filenamePolicySupplier) - .map(c => c.apply(ScioUtil.strippedPath(path), suffix)) - .getOrElse( - ScioUtil.defaultFilenamePolicy( - ScioUtil.pathWithPrefix(path, Option(defaultPrefix).getOrElse("part")), - shardNameTemplate, - suffix, - isWindowed - ) - ) + .getOrElse(FilenamePolicySupplier(prefix, shardNameTemplate, isWindowed)) } def filenamePolicySupplierOf( diff --git a/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala b/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala index fefa17a31e..de3907f4fe 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala @@ -26,7 +26,8 @@ import com.spotify.scio.values.SCollection import org.apache.beam.sdk.extensions.gcp.options.GcpOptions import org.apache.beam.sdk.extensions.gcp.util.Transport import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy -import org.apache.beam.sdk.io.{DefaultFilenamePolicy, FileBasedSink, FileSystems} +import org.apache.beam.sdk.io.FileIO.Write.FileNaming +import org.apache.beam.sdk.io.{DefaultFilenamePolicy, FileBasedSink, FileIO, FileSystems} import org.apache.beam.sdk.io.fs.ResourceId import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider import org.apache.beam.sdk.values.WindowingStrategy @@ -94,7 +95,7 @@ private[scio] object ScioUtil { private def stripPath(path: String): String = StringUtils.stripEnd(path, "/") def strippedPath(path: String): String = s"${stripPath(path)}/" def pathWithPrefix(path: String, filePrefix: String): String = s"${stripPath(path)}/${filePrefix}" - def pathWithPartPrefix(path: String): String = s"${stripPath(path)}/part" + def pathWithPartPrefix(path: String): String = pathWithPrefix(path, "part") def consistentHashCode[K](k: K): Int = k match { case key: Array[_] => ArraySeq.unsafeWrapArray(key).## @@ -106,13 +107,23 @@ private[scio] object ScioUtil { def defaultFilenamePolicy( path: String, + prefix: String, shardTemplate: String, suffix: String, isWindowed: Boolean ): FilenamePolicy = { - val resource = FileBasedSink.convertToFileResourceIfPossible(path) - val prefix = StaticValueProvider.of(resource) - DefaultFilenamePolicy.fromStandardParameters(prefix, shardTemplate, suffix, isWindowed) + val prefixedPath = pathWithPrefix(path, prefix) + val resource = FileBasedSink.convertToFileResourceIfPossible(prefixedPath) + val baseFileName = StaticValueProvider.of(resource) + DefaultFilenamePolicy.fromStandardParameters(baseFileName, shardTemplate, suffix, isWindowed) + } + + def defaultNaming( + prefix: String, + suffix: String + )(destination: String): FileNaming = { + val prefixedPath = pathWithPrefix(destination, prefix) + FileIO.Write.defaultNaming(prefixedPath, suffix) } def tempDirOrDefault(tempDirectory: String, sc: ScioContext): ResourceId = { diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala index 1c3c75bafd..cf66da1c1c 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala @@ -1594,7 +1594,9 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] { footer: Option[String] = TextIO.WriteParam.DefaultFooter, shardNameTemplate: String = TextIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = TextIO.WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = TextIO.WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = + TextIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = TextIO.WriteParam.DefaultPrefix )(implicit ct: ClassTag[T]): ClosedTap[String] = { val s = if (classOf[String] isAssignableFrom ct.runtimeClass) { this.asInstanceOf[SCollection[String]] @@ -1608,9 +1610,10 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] { compression, header, footer, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) ) } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala index 519e1dba81..f2a8c80e6b 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala @@ -106,7 +106,7 @@ object CsvIO { WriteParam.DefaultFilenamePolicySupplier ) - final case class ReadParam( + final case class ReadParam private ( compression: beam.Compression = beam.Compression.AUTO, csvConfiguration: CsvConfiguration = CsvIO.DefaultCsvConfig ) @@ -121,7 +121,7 @@ object CsvIO { private[scio] val DefaultFilenamePolicySupplier = null } - final case class WriteParam( + final case class WriteParam private ( compression: beam.Compression, csvConfiguration: CsvConfiguration, suffix: String, diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala index edc903cdb9..5289eb10aa 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala @@ -47,9 +47,10 @@ final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO params.compression, None, None, + params.filenamePolicySupplier, + params.prefix, params.shardNameTemplate, - params.tempDirectory, - params.filenamePolicySupplier + params.tempDirectory ) ) tap(JsonIO.ReadParam(params.compression)) @@ -67,25 +68,27 @@ final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO } object JsonIO { - final case class ReadParam(compression: beam.Compression = beam.Compression.AUTO) + final case class ReadParam private (compression: beam.Compression = beam.Compression.AUTO) object WriteParam { private[scio] val DefaultNumShards = 0 private[scio] val DefaultSuffix = ".json" private[scio] val DefaultCompression = beam.Compression.UNCOMPRESSED private[scio] val DefaultPrinter = Printer.noSpaces + private[scio] val DefaultFilenamePolicySupplier = null private[scio] val DefaultShardNameTemplate: String = null + private[scio] val DefaultPrefix: String = null private[scio] val DefaultTempDirectory = null - private[scio] val DefaultFilenamePolicySupplier = null } - final case class WriteParam( + final case class WriteParam private ( suffix: String, numShards: Int, compression: beam.Compression, printer: Printer, + filenamePolicySupplier: FilenamePolicySupplier, + prefix: String, shardNameTemplate: String, - tempDirectory: String, - filenamePolicySupplier: FilenamePolicySupplier + tempDirectory: String ) } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala index d82ce76241..24a04a80e8 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala @@ -80,7 +80,8 @@ package object json extends AutoDerivation { shardNameTemplate: String = JsonIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = JsonIO.WriteParam.DefaultTempDirectory, filenamePolicySupplier: FilenamePolicySupplier = - JsonIO.WriteParam.DefaultFilenamePolicySupplier + JsonIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = JsonIO.WriteParam.DefaultPrefix ): ClosedTap[T] = self.write(JsonIO[T](path))( JsonIO.WriteParam( @@ -88,9 +89,10 @@ package object json extends AutoDerivation { numShards, compression, printer, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) ) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala index 9cb744bf30..c21d6f9ef9 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala @@ -51,5 +51,5 @@ final case class DatastoreIO(projectId: String) extends ScioIO[Entity] { } object DatastoreIO { - final case class ReadParam(query: Query, namespace: String = null) + final case class ReadParam private (query: Query, namespace: String = null) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala index 027e1adf21..fdce4060b4 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala @@ -47,7 +47,7 @@ object PubsubIO { case object Subscription extends ReadType case object Topic extends ReadType - final case class ReadParam( + final case class ReadParam private ( readType: ReadType, clientFactory: Option[beam.PubsubClient.PubsubClientFactory] = None, deadLetterTopic: Option[String] = None diff --git a/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala b/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala index b232d5990b..5148bf216d 100644 --- a/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala +++ b/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala @@ -32,7 +32,7 @@ object Neo4jIO { object WriteParam { private[neo4j] val BeamDefaultBatchSize = 5000L } - final case class WriteParam(batchSize: Long = WriteParam.BeamDefaultBatchSize) + final case class WriteParam private (batchSize: Long = WriteParam.BeamDefaultBatchSize) implicit private[neo4j] def recordConverter(record: Record): Value = Values.value(record.asMap(identity[Value])) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala index b390468798..45092d3c7d 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala @@ -86,22 +86,18 @@ final case class ParquetAvroIO[T: ClassTag: Coder](path: String) extends ScioIO[ numShards: Int, compression: CompressionCodecName, conf: Configuration, - shardNameTemplate: String, - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, + prefix: String, + shardNameTemplate: String, isWindowed: Boolean, + tempDirectory: ResourceId, isLocalRunner: Boolean ) = { - val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed - ) - val dynamicDestinations = - DynamicFileDestinations.constant(fp, SerializableFunctions.identity[T]) + require(tempDirectory != null, "tempDirectory must not be null") + val fp = FilenamePolicySupplier + .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val dynamicDestinations = DynamicFileDestinations + .constant(fp, SerializableFunctions.identity[T]) val job = Job.getInstance(ParquetConfiguration.ofNullable(conf)) if (isLocalRunner) GcsConnectorUtil.setCredentials(job) @@ -139,10 +135,11 @@ final case class ParquetAvroIO[T: ClassTag: Coder](path: String) extends ScioIO[ params.numShards, params.compression, conf, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, + params.prefix, + params.shardNameTemplate, ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), ScioUtil.isLocalRunner(data.context.options.getRunner) ) ) @@ -305,9 +302,10 @@ object ParquetAvroIO { private[scio] val DefaultSuffix = ".parquet" private[scio] val DefaultCompression = CompressionCodecName.GZIP private[scio] val DefaultConfiguration = null + private[scio] val DefaultFilenamePolicySupplier = null + private[scio] val DefaultPrefix = null private[scio] val DefaultShardNameTemplate = null private[scio] val DefaultTempDirectory = null - private[scio] val DefaultFilenamePolicySupplier = null } final case class WriteParam private ( @@ -316,9 +314,10 @@ object ParquetAvroIO { suffix: String = WriteParam.DefaultSuffix, compression: CompressionCodecName = WriteParam.DefaultCompression, conf: Configuration = WriteParam.DefaultConfiguration, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, - tempDirectory: String = WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier + tempDirectory: String = WriteParam.DefaultTempDirectory ) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/dynamic/syntax/SCollectionSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/dynamic/syntax/SCollectionSyntax.scala index b26ef31f73..a54333e0fd 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/dynamic/syntax/SCollectionSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/dynamic/syntax/SCollectionSyntax.scala @@ -44,7 +44,8 @@ final class DynamicParquetAvroSCollectionOps[T]( suffix: String = ParquetAvroIO.WriteParam.DefaultSuffix, compression: CompressionCodecName = ParquetAvroIO.WriteParam.DefaultCompression, conf: Configuration = ParquetAvroIO.WriteParam.DefaultConfiguration, - tempDirectory: String = ParquetAvroIO.WriteParam.DefaultTempDirectory + tempDirectory: String = ParquetAvroIO.WriteParam.DefaultTempDirectory, + prefix: String = ParquetAvroIO.WriteParam.DefaultPrefix )( destinationFn: T => String )(implicit ct: ClassTag[T], coder: Coder[T]): ClosedTap[Nothing] = { @@ -63,7 +64,14 @@ final class DynamicParquetAvroSCollectionOps[T]( compression, new SerializableConfiguration(ParquetConfiguration.ofNullable(conf)) ) - val write = writeDynamic(path, numShards, suffix, destinationFn, tempDirectory).via(sink) + val write = writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) self.applyInternal(write) } ClosedTap[Nothing](EmptyTap) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/SCollectionSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/SCollectionSyntax.scala index 9e0c1d9a99..b6b776f504 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/SCollectionSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/SCollectionSyntax.scala @@ -47,6 +47,11 @@ class SCollectionOps[T](private val self: SCollection[T]) extends AnyVal { * defaults to .parquet * @param compression * defaults to snappy + * @param conf + * @param shardNameTemplate + * @param tempDirectory + * @param filenamePolicySupplier + * @param prefix */ def saveAsParquetAvroFile( path: String, @@ -57,17 +62,19 @@ class SCollectionOps[T](private val self: SCollection[T]) extends AnyVal { conf: Configuration = WriteParam.DefaultConfiguration, shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, tempDirectory: String = WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix )(implicit ct: ClassTag[T], coder: Coder[T]): ClosedTap[T] = { val param = WriteParam( - schema, - numShards, - suffix, - compression, - conf, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier + schema = schema, + numShards = numShards, + suffix = suffix, + compression = compression, + conf = conf, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + tempDirectory = tempDirectory ) self.write(ParquetAvroIO[T](path))(param) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala index f551bb112e..2b4fc9b7e5 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala @@ -154,22 +154,18 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { numShards: Int, compression: CompressionCodecName, conf: Configuration, - shardNameTemplate: String, - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, + prefix: String, + shardNameTemplate: String, isWindowed: Boolean, + tempDirectory: ResourceId, isLocalRunner: Boolean ) = { - val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed - ) - val dynamicDestinations = - DynamicFileDestinations.constant(fp, SerializableFunctions.identity[Example]) + require(tempDirectory != null, "tempDirectory must not be null") + val fp = FilenamePolicySupplier + .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val dynamicDestinations = DynamicFileDestinations + .constant(fp, SerializableFunctions.identity[Example]) val job = Job.getInstance(ParquetConfiguration.ofNullable(conf)) if (isLocalRunner) GcsConnectorUtil.setCredentials(job) val sink = new ParquetExampleFileBasedSink( @@ -192,10 +188,11 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { params.numShards, params.compression, params.conf, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, + params.prefix, + params.shardNameTemplate, ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), ScioUtil.isLocalRunner(data.context.options.getRunner) ) ) @@ -223,9 +220,10 @@ object ParquetExampleIO { private[tensorflow] val DefaultSuffix = ".parquet" private[tensorflow] val DefaultCompression = CompressionCodecName.GZIP private[tensorflow] val DefaultConfiguration = null + private[tensorflow] val DefaultFilenamePolicySupplier = null + private[tensorflow] val DefaultPrefix = null private[tensorflow] val DefaultShardNameTemplate = null private[tensorflow] val DefaultTempDirectory = null - private[tensorflow] val DefaultFilenamePolicySupplier = null } final case class WriteParam private ( @@ -234,9 +232,10 @@ object ParquetExampleIO { suffix: String = WriteParam.DefaultSuffix, compression: CompressionCodecName = WriteParam.DefaultCompression, conf: Configuration = WriteParam.DefaultConfiguration, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, - tempDirectory: String = WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier + tempDirectory: String = WriteParam.DefaultTempDirectory ) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/dynamic/syntax/SCollectionSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/dynamic/syntax/SCollectionSyntax.scala index 61071e3a5d..9475bf4b3b 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/dynamic/syntax/SCollectionSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/dynamic/syntax/SCollectionSyntax.scala @@ -46,7 +46,8 @@ final class DynamicParquetExampleSCollectionOps( suffix: String = ParquetExampleIO.WriteParam.DefaultSuffix, compression: CompressionCodecName = ParquetExampleIO.WriteParam.DefaultCompression, conf: Configuration = ParquetExampleIO.WriteParam.DefaultConfiguration, - tempDirectory: String = ParquetExampleIO.WriteParam.DefaultTempDirectory + tempDirectory: String = ParquetExampleIO.WriteParam.DefaultTempDirectory, + prefix: String = ParquetExampleIO.WriteParam.DefaultPrefix )( destinationFn: Example => String )(implicit ct: ClassTag[Example], coder: Coder[Example]): ClosedTap[Nothing] = { @@ -60,7 +61,14 @@ final class DynamicParquetExampleSCollectionOps( compression, new SerializableConfiguration(ParquetConfiguration.ofNullable(conf)) ) - val write = writeDynamic(path, numShards, suffix, destinationFn, tempDirectory).via(sink) + val write = writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) self.applyInternal(write) } ClosedTap[Nothing](EmptyTap) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/SCollectionSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/SCollectionSyntax.scala index 689a8e25a6..8f7e38c873 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/SCollectionSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/SCollectionSyntax.scala @@ -40,7 +40,8 @@ final class SCollectionOps(private val self: SCollection[Example]) extends AnyVa conf: Configuration = WriteParam.DefaultConfiguration, shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, tempDirectory: String = WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix ): ClosedTap[Example] = self.write(ParquetExampleIO(path))( WriteParam( @@ -49,9 +50,10 @@ final class SCollectionOps(private val self: SCollection[Example]) extends AnyVa suffix, compression, conf, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) ) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala index 5d7cbc23c4..7a533cecc0 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala @@ -124,22 +124,18 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( numShards: Int, compression: CompressionCodecName, conf: Configuration, - shardNameTemplate: String, - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, + prefix: String, + shardNameTemplate: String, isWindowed: Boolean, + tempDirectory: ResourceId, isLocalRunner: Boolean ) = { - val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed - ) - val dynamicDestinations = - DynamicFileDestinations.constant(fp, SerializableFunctions.identity[T]) + require(tempDirectory != null, "tempDirectory must not be null") + val fp = FilenamePolicySupplier + .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val dynamicDestinations = DynamicFileDestinations + .constant(fp, SerializableFunctions.identity[T]) val job = Job.getInstance(ParquetConfiguration.ofNullable(conf)) if (isLocalRunner) GcsConnectorUtil.setCredentials(job) val sink = new ParquetTypeFileBasedSink[T]( @@ -161,10 +157,11 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( params.numShards, params.compression, params.conf, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, + params.prefix, + params.shardNameTemplate, ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), ScioUtil.isLocalRunner(data.context.options.getRunner) ) ) @@ -190,9 +187,10 @@ object ParquetTypeIO { private[scio] val DefaultSuffix = ".parquet" private[scio] val DefaultCompression = CompressionCodecName.GZIP private[scio] val DefaultConfiguration = null + private[scio] val DefaultFilenamePolicySupplier = null + private[scio] val DefaultPrefix = null private[scio] val DefaultShardNameTemplate = null private[scio] val DefaultTempDirectory = null - private[scio] val DefaultFilenamePolicySupplier = null } final case class WriteParam[T] private ( @@ -200,9 +198,10 @@ object ParquetTypeIO { suffix: String = WriteParam.DefaultSuffix, compression: CompressionCodecName = WriteParam.DefaultCompression, conf: Configuration = WriteParam.DefaultConfiguration, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, - tempDirectory: String = WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier + tempDirectory: String = WriteParam.DefaultTempDirectory ) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/dynamic/syntax/SCollectionSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/dynamic/syntax/SCollectionSyntax.scala index f7f7fc29da..9a86b83222 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/dynamic/syntax/SCollectionSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/dynamic/syntax/SCollectionSyntax.scala @@ -40,7 +40,8 @@ final class DynamicParquetTypeSCollectionOps[T]( suffix: String = ParquetTypeIO.WriteParam.DefaultSuffix, compression: CompressionCodecName = ParquetTypeIO.WriteParam.DefaultCompression, conf: Configuration = ParquetTypeIO.WriteParam.DefaultConfiguration, - tempDirectory: String = ParquetTypeIO.WriteParam.DefaultTempDirectory + tempDirectory: String = ParquetTypeIO.WriteParam.DefaultTempDirectory, + prefix: String = ParquetTypeIO.WriteParam.DefaultPrefix )( destinationFn: T => String )(implicit ct: ClassTag[T], coder: Coder[T], pt: ParquetType[T]): ClosedTap[Nothing] = { @@ -53,7 +54,14 @@ final class DynamicParquetTypeSCollectionOps[T]( compression, new SerializableConfiguration(ParquetConfiguration.ofNullable(conf)) ) - val write = writeDynamic(path, numShards, suffix, destinationFn, tempDirectory).via(sink) + val write = writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) self.applyInternal(write) } ClosedTap[Nothing](EmptyTap) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/SCollectionSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/SCollectionSyntax.scala index 5a5a15f2b1..17f466c9a2 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/SCollectionSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/SCollectionSyntax.scala @@ -41,7 +41,8 @@ final class SCollectionOps[T](private val self: SCollection[T]) extends AnyVal { conf: Configuration = WriteParam.DefaultConfiguration, shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, tempDirectory: String = WriteParam.DefaultTempDirectory, - filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix )(implicit ct: ClassTag[T], coder: Coder[T], pt: ParquetType[T]): ClosedTap[T] = self.write(ParquetTypeIO[T](path))( WriteParam( @@ -49,9 +50,10 @@ final class SCollectionOps[T](private val self: SCollection[T]) extends AnyVal { suffix, compression, conf, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) ) } diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index 2b3a2d43b3..6c265d2cee 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -68,18 +68,20 @@ object TFRecordIO { private[tensorflow] val DefaultSuffix = ".tfrecords" private[tensorflow] val DefaultCompression = Compression.UNCOMPRESSED private[tensorflow] val DefaultNumShards = 0 + private[tensorflow] val DefaultFilenamePolicySupplier = null + private[tensorflow] val DefaultPrefix = null private[tensorflow] val DefaultShardNameTemplate = null private[tensorflow] val DefaultTempDirectory = null - private[tensorflow] val DefaultFilenamePolicySupplier = null } final case class WriteParam private ( suffix: String, compression: Compression, numShards: Int, + filenamePolicySupplier: FilenamePolicySupplier, + prefix: String, shardNameTemplate: String, - tempDirectory: String, - filenamePolicySupplier: FilenamePolicySupplier + tempDirectory: String ) } @@ -145,21 +147,22 @@ private object TFRecordMethods { suffix: String, numShards: Int, compression: Compression, - shardNameTemplate: String, - tempDirectory: ResourceId, filenamePolicySupplier: FilenamePolicySupplier, - isWindowed: Boolean + prefix: String, + shardNameTemplate: String, + isWindowed: Boolean, + tempDirectory: ResourceId ) = { + require(tempDirectory != null, "tempDirectory must not be null") val fp = FilenamePolicySupplier.resolve( - path, - suffix, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier, - isWindowed - ) - val dynamicDestinations = - DynamicFileDestinations.constant(fp, SerializableFunctions.identity[Array[Byte]]) + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(path, suffix) + + val dynamicDestinations = DynamicFileDestinations + .constant(fp, SerializableFunctions.identity[Array[Byte]]) val sink = new TFRecordFileBasedSink( StaticValueProvider.of(tempDirectory), @@ -178,10 +181,11 @@ private object TFRecordMethods { params.suffix, params.numShards, params.compression, - params.shardNameTemplate, - ScioUtil.tempDirOrDefault(params.tempDirectory, data.context), params.filenamePolicySupplier, - ScioUtil.isWindowed(data) + params.prefix, + params.shardNameTemplate, + ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/SCollectionSyntax.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/SCollectionSyntax.scala index cb16b55668..44573b27de 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/SCollectionSyntax.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/SCollectionSyntax.scala @@ -158,15 +158,17 @@ final class ExampleSCollectionOps[T <: Example](private val self: SCollection[T] shardNameTemplate: String = TFExampleIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = TFExampleIO.WriteParam.DefaultTempDirectory, filenamePolicySupplier: FilenamePolicySupplier = - TFExampleIO.WriteParam.DefaultFilenamePolicySupplier + TFExampleIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = TFExampleIO.WriteParam.DefaultPrefix ): ClosedTap[Example] = { val param = TFExampleIO.WriteParam( suffix, compression, numShards, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.covary[Example].write(TFExampleIO(path))(param) } @@ -228,15 +230,17 @@ final class TFRecordSCollectionOps[T <: Array[Byte]](private val self: SCollecti shardNameTemplate: String = TFExampleIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = TFExampleIO.WriteParam.DefaultTempDirectory, filenamePolicySupplier: FilenamePolicySupplier = - TFExampleIO.WriteParam.DefaultFilenamePolicySupplier + TFExampleIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = TFExampleIO.WriteParam.DefaultPrefix )(implicit ev: T <:< Array[Byte]): ClosedTap[Array[Byte]] = { val param = TFRecordIO.WriteParam( suffix, compression, numShards, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.covary[Array[Byte]].write(TFRecordIO(path))(param) } @@ -259,15 +263,17 @@ final class SequenceExampleSCollectionOps[T <: SequenceExample](private val self shardNameTemplate: String = TFExampleIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = TFExampleIO.WriteParam.DefaultTempDirectory, filenamePolicySupplier: FilenamePolicySupplier = - TFExampleIO.WriteParam.DefaultFilenamePolicySupplier + TFExampleIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = TFExampleIO.WriteParam.DefaultPrefix ): ClosedTap[SequenceExample] = { val param = TFExampleIO.WriteParam( suffix, compression, numShards, + filenamePolicySupplier, + prefix, shardNameTemplate, - tempDirectory, - filenamePolicySupplier + tempDirectory ) self.covary[SequenceExample].write(TFSequenceExampleIO(path))(param) } diff --git a/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala index 3c3b337642..1a0dec0940 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala @@ -351,9 +351,10 @@ class ScioIOTest extends ScioIOSpec { codec, metadata, null, - ScioUtil.toResourceId(out2TempDir.getAbsolutePath), null, - false + null, + false, + ScioUtil.toResourceId(out2TempDir.getAbsolutePath) ) // verify @@ -412,9 +413,10 @@ class ScioIOTest extends ScioIOSpec { codec, metadata, null, - ScioUtil.toResourceId(out2TempDir.getAbsolutePath), null, - false + null, + false, + ScioUtil.toResourceId(out2TempDir.getAbsolutePath) ) // verify From 29742ff42b2cd5364f707543601186d18cbf5380 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 17 May 2023 14:19:27 +0200 Subject: [PATCH 02/26] Make scalafix happy --- .../spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala b/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala index fec33830b6..4cc136a424 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala @@ -134,10 +134,7 @@ final class DynamicGenericRecordSCollectionOps[T <: GenericRecord](private val s val sink = beam.AvroIO .sinkViaGenericRecords( schema, - new RecordFormatter[T] { - override def formatRecord(element: T, schema: Schema): GenericRecord = - element - } + (element: T, _: Schema) => element ) .withCodec(codec) .withMetadata(nm) @@ -227,7 +224,7 @@ final class DynamicProtobufSCollectionOps[T <: Message](private val self: SColle val sink = beam.AvroIO .sinkViaGenericRecords( avroSchema, - (element: T, schema: Schema) => AvroBytesUtil.encode(elemCoder, element) + (element: T, _: Schema) => AvroBytesUtil.encode(elemCoder, element) ) .withCodec(codec) .withMetadata(nm) From e422502bb156338427045b94d9f775e0c582105d Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 17 May 2023 14:22:26 +0200 Subject: [PATCH 03/26] Allow null shardNameTemplate --- .../scio/util/FilenamePolicySupplier.scala | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala b/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala index e6085d7d46..d66e1f3ec2 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/FilenamePolicySupplier.scala @@ -32,17 +32,14 @@ object FilenamePolicySupplier { prefix: String, shardNameTemplate: String, isWindowed: Boolean - ): FilenamePolicySupplier = { - require(shardNameTemplate != null, "shardNameTemplate must not be null") - (path: String, suffix: String) => - ScioUtil.defaultFilenamePolicy( - path, - Option(prefix).getOrElse("part"), - shardNameTemplate, - suffix, - isWindowed - ) - } + ): FilenamePolicySupplier = (path: String, suffix: String) => + ScioUtil.defaultFilenamePolicy( + path, + Option(prefix).getOrElse("part"), + shardNameTemplate, + suffix, + isWindowed + ) def resolve( filenamePolicySupplier: FilenamePolicySupplier, From 7aaf4f9d089bcb2b9f798a67893914c2b2e00189 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Tue, 23 May 2023 11:25:34 +0200 Subject: [PATCH 04/26] Allow to match files with suffix --- .../spotify/scio/avro/types/AvroTapIT.scala | 5 +- .../scala/com/spotify/scio/avro/AvroIO.scala | 96 ++++--- .../scio/avro/syntax/ScioContextSyntax.scala | 56 +++- .../scala/com/spotify/scio/avro/taps.scala | 78 +++-- .../avro/types/ConverterProviderTest.scala | 8 +- .../scala/com/spotify/scio/ScioContext.scala | 5 +- .../scala/com/spotify/scio/io/BinaryIO.scala | 32 ++- .../com/spotify/scio/io/FileStorage.scala | 67 +++-- .../main/scala/com/spotify/scio/io/Tap.scala | 20 +- .../main/scala/com/spotify/scio/io/Taps.scala | 11 +- .../scala/com/spotify/scio/io/TextIO.scala | 81 +++--- .../com/spotify/scio/util/ScioUtil.scala | 21 +- .../com/spotify/scio/values/SCollection.scala | 4 +- .../scio/examples/extra/AvroExample.scala | 4 +- .../scio/examples/extra/AvroInOut.scala | 2 +- .../examples/extra/MagnolifyAvroExample.scala | 2 +- .../extra/BackcompatibilityTest.scala | 6 +- .../extra/SortMergeBucketExampleTest.scala | 21 +- .../com/spotify/scio/extra/csv/CsvIO.scala | 75 ++--- .../extra/csv/syntax/SCollectionSyntax.scala | 18 +- .../com/spotify/scio/extra/json/JsonIO.scala | 59 ++-- .../com/spotify/scio/extra/json/package.scala | 5 +- .../spotify/scio/extra/csv/CsvIOTest.scala | 46 ++- .../spotify/scio/extra/json/JsonIOTest.scala | 18 +- .../spotify/scio/bigquery/BigQueryIO.scala | 42 ++- .../bigquery/syntax/ScioContextSyntax.scala | 9 +- .../com/spotify/scio/bigquery/taps.scala | 12 +- .../spotify/scio/jmh/GroupByBenchmark.scala | 4 +- .../scio/parquet/avro/ParquetAvroIO.scala | 63 +++-- .../avro/syntax/ScioContextSyntax.scala | 10 +- .../parquet/tensorflow/ParquetExampleIO.scala | 51 ++-- .../tensorflow/syntax/ScioContextSyntax.scala | 7 +- .../scio/parquet/types/ParquetTypeIO.scala | 57 ++-- .../types/syntax/ScioContextSyntax.scala | 5 +- .../scio/parquet/avro/ParquetAvroIOTest.scala | 267 +++++++++--------- .../scio/parquet/read/ParquetReadFnTest.scala | 42 ++- .../tensorflow/ParquetExampleIOTest.scala | 49 +++- .../parquet/types/ParquetTypeIOTest.scala | 52 ++-- .../scio/smb/SortMergeBucketParityIT.scala | 57 ++-- .../scio/tensorflow/TFRecordFileTap.scala | 4 +- .../spotify/scio/tensorflow/TFRecordIO.scala | 60 ++-- .../tensorflow/syntax/FileStorageSyntax.scala | 3 +- .../scio/tensorflow/TFExampleIOTest.scala | 21 +- .../spotify/scio/tensorflow/TFTapTest.scala | 5 +- .../com/spotify/scio/testing/ScioIOSpec.scala | 18 +- .../spotify/scio/coders/AvroCoderTest.scala | 13 +- .../com/spotify/scio/io/FileStorageTest.scala | 49 ++-- .../com/spotify/scio/io/ScioIOTest.scala | 139 +++++---- .../scala/com/spotify/scio/io/TapTest.scala | 121 ++++---- .../scala/com/spotify/scio/io/TapsTest.scala | 6 +- .../scio/io/dynamic/DynamicFileTest.scala | 97 +++---- .../spotify/scio/testing/JobTestTest.scala | 4 +- .../spotify/scio/util/ProtobufUtilTest.scala | 3 +- site/src/main/paradox/FAQ.md | 7 +- site/src/main/paradox/io/Avro.md | 4 +- 55 files changed, 1167 insertions(+), 854 deletions(-) diff --git a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala index 8416ab970c..df2472c0e6 100644 --- a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala +++ b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala @@ -51,8 +51,9 @@ final class AvroTapIT extends AnyFlatSpec with Matchers { FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create) val tap = AvroTaps(Taps()).avroFile( - "gs://data-integration-test-eu/avro-integration-test/folder-a/folder-b/shakespeare.avro", - schema = schema + "gs://data-integration-test-eu/avro-integration-test/folder-a/folder-b/", + schema = schema, + ".avro" ) val result = Await.result(tap, Duration.Inf) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala index ed84d76327..9b19bebbaf 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala @@ -39,7 +39,7 @@ import scala.reflect.runtime.universe._ import scala.reflect.ClassTag final case class ObjectFileIO[T: Coder](path: String) extends ScioIO[T] { - override type ReadP = Unit + override type ReadP = ObjectFileIO.ReadParam override type WriteP = ObjectFileIO.WriteParam final override val tapT: TapT.Aux[T, T] = TapOf[T] @@ -51,7 +51,7 @@ final case class ObjectFileIO[T: Coder](path: String) extends ScioIO[T] { */ override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = { val coder = CoderMaterializer.beamWithDefault(Coder[T]) - sc.read(GenericRecordIO(path, AvroBytesUtil.schema)) + sc.read(GenericRecordIO(path, AvroBytesUtil.schema))(params) .parDo(new DoFn[GenericRecord, T] { @ProcessElement private[scio] def processElement( @@ -81,20 +81,22 @@ final case class ObjectFileIO[T: Coder](path: String) extends ScioIO[T] { out.output(AvroBytesUtil.encode(elemCoder, element)) }) .write(GenericRecordIO(path, AvroBytesUtil.schema))(params) - tap(()) + tap(AvroIO.ReadParam(params)) } override def tap(read: ReadP): Tap[T] = - ObjectFileTap[T](ScioUtil.addPartSuffix(path)) + ObjectFileTap[T](path, read.suffix) } object ObjectFileIO { + type ReadParam = AvroIO.ReadParam + val ReadParam = AvroIO.ReadParam type WriteParam = AvroIO.WriteParam val WriteParam = AvroIO.WriteParam } final case class ProtobufIO[T <: Message: ClassTag](path: String) extends ScioIO[T] { - override type ReadP = Unit + override type ReadP = ProtobufIO.ReadParam override type WriteP = ProtobufIO.WriteParam final override val tapT: TapT.Aux[T, T] = TapOf[T] private val protoCoder = Coder.protoMessageCoder[T] @@ -106,7 +108,7 @@ final case class ProtobufIO[T <: Message: ClassTag](path: String) extends ScioIO * block file format. */ override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = - sc.read(ObjectFileIO[T](path)(protoCoder)) + sc.read(ObjectFileIO[T](path)(protoCoder))(params) /** * Save this SCollection as a Protobuf file. @@ -120,10 +122,12 @@ final case class ProtobufIO[T <: Message: ClassTag](path: String) extends ScioIO } override def tap(read: ReadP): Tap[T] = - ObjectFileTap[T](ScioUtil.addPartSuffix(path))(protoCoder) + ObjectFileTap[T](path, read.suffix)(protoCoder) } object ProtobufIO { + type ReadParam = AvroIO.ReadParam + val ReadParam = AvroIO.ReadParam type WriteParam = AvroIO.WriteParam val WriteParam = AvroIO.WriteParam } @@ -150,7 +154,7 @@ sealed trait AvroIO[T] extends ScioIO[T] { prefix = prefix, shardNameTemplate = shardNameTemplate, isWindowed = isWindowed - )(path, suffix) + )(ScioUtil.strippedPath(path), suffix) val transform = write .to(fp) .withTempDirectory(tempDirectory) @@ -163,7 +167,7 @@ sealed trait AvroIO[T] extends ScioIO[T] { final case class SpecificRecordIO[T <: SpecificRecord: ClassTag: Coder](path: String) extends AvroIO[T] { - override type ReadP = Unit + override type ReadP = AvroIO.ReadParam override type WriteP = AvroIO.WriteParam override def testId: String = s"AvroIO($path)" @@ -175,7 +179,10 @@ final case class SpecificRecordIO[T <: SpecificRecord: ClassTag: Coder](path: St override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = { val coder = CoderMaterializer.beam(sc, Coder[T]) val cls = ScioUtil.classOf[T] - val t = beam.AvroIO.read(cls).from(path) + val filePattern = ScioUtil.filePattern(path, params.suffix) + val t = beam.AvroIO + .read(cls) + .from(filePattern) sc .applyTransform(t) .setCoder(coder) @@ -204,15 +211,15 @@ final case class SpecificRecordIO[T <: SpecificRecord: ClassTag: Coder](path: St ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) - tap(()) + tap(AvroIO.ReadParam(params)) } override def tap(read: ReadP): Tap[T] = - SpecificRecordTap[T](ScioUtil.addPartSuffix(path)) + SpecificRecordTap[T](path, read.suffix) } final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[GenericRecord] { - override type ReadP = Unit + override type ReadP = AvroIO.ReadParam override type WriteP = AvroIO.WriteParam override def testId: String = s"AvroIO($path)" @@ -223,9 +230,10 @@ final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[Ge */ override protected def read(sc: ScioContext, params: ReadP): SCollection[GenericRecord] = { val coder = CoderMaterializer.beam(sc, Coder.avroGenericRecordCoder(schema)) + val filePattern = ScioUtil.filePattern(path, params.suffix) val t = beam.AvroIO .readGenericRecords(schema) - .from(path) + .from(filePattern) sc .applyTransform(t) .setCoder(coder) @@ -254,11 +262,11 @@ final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[Ge ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) - tap(()) + tap(AvroIO.ReadParam(params)) } override def tap(read: ReadP): Tap[GenericRecord] = - GenericRecordTap(ScioUtil.addPartSuffix(path), schema) + GenericRecordTap(path, schema, read.suffix) } /** @@ -271,7 +279,7 @@ final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[Ge final case class GenericRecordParseIO[T](path: String, parseFn: GenericRecord => T)(implicit coder: Coder[T] ) extends AvroIO[T] { - override type ReadP = Unit + override type ReadP = AvroIO.ReadParam override type WriteP = Nothing // Output is not defined for Avro Generic Parse IO. override def testId: String = s"AvroIO($path)" @@ -280,10 +288,11 @@ final case class GenericRecordParseIO[T](path: String, parseFn: GenericRecord => * Get an SCollection[T] by applying the [[parseFn]] on * [[org.apache.avro.generic.GenericRecord GenericRecord]] from an Avro file. */ - override protected def read(sc: ScioContext, params: Unit): SCollection[T] = { + override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = { + val filePattern = ScioUtil.filePattern(path, params.suffix) val t = beam.AvroIO .parseGenericRecords(Functions.serializableFn(parseFn)) - .from(path) + .from(filePattern) .withCoder(CoderMaterializer.beam(sc, coder)) sc.applyTransform(t) @@ -292,20 +301,33 @@ final case class GenericRecordParseIO[T](path: String, parseFn: GenericRecord => /** Writes are undefined for [[GenericRecordParseIO]] since it is used only for reading. */ override protected def write(data: SCollection[T], params: Nothing): Tap[T] = ??? - override def tap(read: Unit): Tap[T] = - GenericRecordParseTap[T](ScioUtil.addPartSuffix(path), parseFn) + override def tap(read: ReadP): Tap[T] = + GenericRecordParseTap[T](path, read.suffix, parseFn) } object AvroIO { - object WriteParam { - private[scio] val DefaultNumShards = 0 - private[scio] val DefaultSuffix = ".avro" - private[scio] val DefaultCodec: CodecFactory = CodecFactory.deflateCodec(6) - private[scio] val DefaultMetadata: Map[String, AnyRef] = Map.empty - private[scio] val DefaultFilenamePolicySupplier = null - private[scio] val DefaultPrefix = null - private[scio] val DefaultShardNameTemplate: String = null - private[scio] val DefaultTempDirectory = null + + private[scio] object ReadParam { + val DefaultSuffix: String = null + + def apply(params: WriteParam): ReadParam = new ReadParam( + params.suffix + ) + } + + final case class ReadParam private ( + suffix: String = ReadParam.DefaultSuffix + ) + + private[scio] object WriteParam { + val DefaultNumShards = 0 + val DefaultSuffix = ".avro" + val DefaultCodec: CodecFactory = CodecFactory.deflateCodec(6) + val DefaultMetadata: Map[String, AnyRef] = Map.empty + val DefaultFilenamePolicySupplier = null + val DefaultPrefix = null + val DefaultShardNameTemplate: String = null + val DefaultTempDirectory = null } final case class WriteParam private ( @@ -336,7 +358,7 @@ object AvroTyped { } final case class AvroIO[T <: HasAvroAnnotation: TypeTag: Coder](path: String) extends ScioIO[T] { - override type ReadP = Unit + override type ReadP = avro.AvroIO.ReadParam override type WriteP = avro.AvroIO.WriteParam final override val tapT: TapT.Aux[T, T] = TapOf[T] @@ -354,8 +376,12 @@ object AvroTyped { tempDirectory: ResourceId ) = { require(tempDirectory != null, "tempDirectory must not be null") - val fp = FilenamePolicySupplier - .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val fp = FilenamePolicySupplier.resolve( + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(ScioUtil.strippedPath(path), suffix) val transform = write .to(fp) .withTempDirectory(tempDirectory) @@ -399,12 +425,12 @@ object AvroTyped { ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) - tap(()) + tap(avro.AvroIO.ReadParam(params)) } override def tap(read: ReadP): Tap[T] = { val avroT = AvroType[T] - GenericRecordTap(ScioUtil.addPartSuffix(path), avroT.schema) + GenericRecordTap(path, avroT.schema, read.suffix) .map(avroT.fromGenericRecord) } } diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala index a4ca63ae2d..096b228ea8 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala @@ -40,11 +40,25 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { * Serialized objects are stored in Avro files to leverage Avro's block file format. Note that * serialization is not guaranteed to be compatible across Scio releases. */ - def objectFile[T: Coder](path: String): SCollection[T] = - self.read(ObjectFileIO[T](path)) + def objectFile[T: Coder]( + path: String, + suffix: String = ObjectFileIO.ReadParam.DefaultSuffix + ): SCollection[T] = + self.read(ObjectFileIO[T](path))(ObjectFileIO.ReadParam(suffix)) + + @deprecated("Use avroGenericFile instead", since = "0.13.0") + def avroFile( + path: String, + schema: Schema + ): SCollection[GenericRecord] = + avroGenericFile(path, schema) - def avroFile(path: String, schema: Schema): SCollection[GenericRecord] = - self.read(GenericRecordIO(path, schema)) + def avroGenericFile( + path: String, + schema: Schema, + suffix: String = ObjectFileIO.ReadParam.DefaultSuffix + ): SCollection[GenericRecord] = + self.read(GenericRecordIO(path, schema))(AvroIO.ReadParam(suffix)) /** * Get an SCollection of type [[T]] for data stored in Avro format after applying parseFn to map a @@ -70,15 +84,29 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { * }}} */ @experimental - def parseAvroFile[T: Coder](path: String)(parseFn: GenericRecord => T): SCollection[T] = - self.read(GenericRecordParseIO[T](path, parseFn)) + def parseAvroFile[T: Coder]( + path: String, + suffix: String = AvroIO.ReadParam.DefaultSuffix + )( + parseFn: GenericRecord => T + ): SCollection[T] = + self.read(GenericRecordParseIO[T](path, parseFn))(AvroIO.ReadParam(suffix)) + + @deprecated("Use avroSpecificFile instead", since = "0.13.0") + def avroFile[T <: SpecificRecord: ClassTag: Coder]( + path: String + ): SCollection[T] = + avroSpecificFile(path) /** * Get an SCollection of type [[org.apache.avro.specific.SpecificRecord SpecificRecord]] for an * Avro file. */ - def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String): SCollection[T] = - self.read(SpecificRecordIO[T](path)) + def avroSpecificFile[T <: SpecificRecord: ClassTag: Coder]( + path: String, + suffix: String = AvroIO.ReadParam.DefaultSuffix + ): SCollection[T] = + self.read(SpecificRecordIO[T](path))(AvroIO.ReadParam(suffix)) /** * Get a typed SCollection from an Avro schema. @@ -89,9 +117,10 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { * [[com.spotify.scio.avro.types.AvroType AvroType.toSchema]]. */ def typedAvroFile[T <: HasAvroAnnotation: TypeTag: Coder]( - path: String + path: String, + suffix: String = AvroIO.ReadParam.DefaultSuffix ): SCollection[T] = - self.read(AvroTyped.AvroIO[T](path)) + self.read(AvroTyped.AvroIO[T](path))(AvroIO.ReadParam(suffix)) /** * Get an SCollection for a Protobuf file. @@ -99,8 +128,11 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { * Protobuf messages are serialized into `Array[Byte]` and stored in Avro files to leverage Avro's * block file format. */ - def protobufFile[T <: Message: ClassTag](path: String): SCollection[T] = - self.read(ProtobufIO[T](path)) + def protobufFile[T <: Message: ClassTag]( + path: String, + suffix: String = ProtobufIO.ReadParam.DefaultSuffix + ): SCollection[T] = + self.read(ProtobufIO[T](path))(ProtobufIO.ReadParam(suffix)) } /** Enhanced with Avro methods. */ diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala index f380ca034e..2ac05fe102 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala @@ -35,23 +35,28 @@ import scala.reflect.runtime.universe._ /** Tap for [[org.apache.avro.generic.GenericRecord GenericRecord]] Avro files. */ final case class GenericRecordTap( path: String, - @transient private val - schema: Schema + @transient private val schema: Schema, + suffix: String ) extends Tap[GenericRecord] { private lazy val s = Externalizer(schema) - override def value: Iterator[GenericRecord] = FileStorage(path).avroFile[GenericRecord](s.get) + override def value: Iterator[GenericRecord] = + FileStorage(path, suffix).avroFile[GenericRecord](s.get) override def open(sc: ScioContext): SCollection[GenericRecord] = - sc.read(GenericRecordIO(path, s.get)) + sc.avroGenericFile(path, s.get, suffix) } /** Tap for [[org.apache.avro.specific.SpecificRecord SpecificRecord]] Avro files. */ -final case class SpecificRecordTap[T <: SpecificRecord: ClassTag: Coder](path: String) - extends Tap[T] { - override def value: Iterator[T] = FileStorage(path).avroFile[T]() +final case class SpecificRecordTap[T <: SpecificRecord: ClassTag: Coder]( + path: String, + suffix: String +) extends Tap[T] { + override def value: Iterator[T] = + FileStorage(path, suffix).avroFile[T]() - override def open(sc: ScioContext): SCollection[T] = sc.avroFile[T](path) + override def open(sc: ScioContext): SCollection[T] = + sc.avroSpecificFile[T](path, suffix) } /** @@ -60,65 +65,86 @@ final case class SpecificRecordTap[T <: SpecificRecord: ClassTag: Coder](path: S */ final case class GenericRecordParseTap[T: Coder]( path: String, + suffix: String, parseFn: GenericRecord => T ) extends Tap[T] { override def value: Iterator[T] = - FileStorage(path) + FileStorage(path, suffix) // Read Avro GenericRecords, with the writer specified schema .avroFile[GenericRecord](schema = null) .map(parseFn) - override def open(sc: ScioContext): SCollection[T] = sc.parseAvroFile(path)(parseFn) + override def open(sc: ScioContext): SCollection[T] = + sc.parseAvroFile(path, suffix)(parseFn) } /** * Tap for object files. Note that serialization is not guaranteed to be compatible across Scio * releases. */ -case class ObjectFileTap[T: Coder](path: String) extends Tap[T] { +case class ObjectFileTap[T: Coder]( + path: String, + suffix: String +) extends Tap[T] { override def value: Iterator[T] = { val elemCoder = CoderMaterializer.beamWithDefault(Coder[T]) - FileStorage(path).avroFile[GenericRecord](AvroBytesUtil.schema).map { r => - AvroBytesUtil.decode(elemCoder, r) - } + FileStorage(path, suffix) + .avroFile[GenericRecord](AvroBytesUtil.schema) + .map(r => AvroBytesUtil.decode(elemCoder, r)) } - override def open(sc: ScioContext): SCollection[T] = sc.objectFile[T](path) + override def open(sc: ScioContext): SCollection[T] = + sc.objectFile[T](path, suffix) } final case class AvroTaps(self: Taps) { /** Get a `Future[Tap[T]]` of a Protobuf file. */ - def protobufFile[T: Coder](path: String)(implicit ev: T <:< Message): Future[Tap[T]] = - self.mkTap(s"Protobuf: $path", () => self.isPathDone(path), () => ObjectFileTap[T](path)) + def protobufFile[T: Coder](path: String, suffix: String)(implicit + ev: T <:< Message + ): Future[Tap[T]] = + self.mkTap( + s"Protobuf: $path", + () => self.isPathDone(path, suffix), + () => ObjectFileTap[T](path, suffix) + ) /** Get a `Future[Tap[T]]` of an object file. */ - def objectFile[T: Coder](path: String): Future[Tap[T]] = - self.mkTap(s"Object file: $path", () => self.isPathDone(path), () => ObjectFileTap[T](path)) + def objectFile[T: Coder](path: String, suffix: String): Future[Tap[T]] = + self.mkTap( + s"Object file: $path", + () => self.isPathDone(path, suffix), + () => ObjectFileTap[T](path, suffix) + ) /** * Get a `Future[Tap[T]]` for [[org.apache.avro.generic.GenericRecord GenericRecord]] Avro file. */ - def avroFile(path: String, schema: Schema): Future[Tap[GenericRecord]] = + def avroFile(path: String, schema: Schema, suffix: String): Future[Tap[GenericRecord]] = self.mkTap( s"Avro: $path", - () => self.isPathDone(path), - () => GenericRecordTap(path, schema) + () => self.isPathDone(path, suffix), + () => GenericRecordTap(path, schema, suffix) ) /** * Get a `Future[Tap[T]]` for [[org.apache.avro.specific.SpecificRecord SpecificRecord]] Avro * file. */ - def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String): Future[Tap[T]] = - self.mkTap(s"Avro: $path", () => self.isPathDone(path), () => SpecificRecordTap[T](path)) + def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String, suffix: String): Future[Tap[T]] = + self.mkTap( + s"Avro: $path", + () => self.isPathDone(path, suffix), + () => SpecificRecordTap[T](path, suffix) + ) /** Get a `Future[Tap[T]]` for typed Avro source. */ def typedAvroFile[T <: HasAvroAnnotation: TypeTag: Coder]( - path: String + path: String, + suffix: String ): Future[Tap[T]] = { val avroT = AvroType[T] import scala.concurrent.ExecutionContext.Implicits.global - avroFile(path, avroT.schema).map(_.map(avroT.fromGenericRecord)) + avroFile(path, avroT.schema, suffix).map(_.map(avroT.fromGenericRecord)) } } diff --git a/scio-avro/src/test/scala/com/spotify/scio/avro/types/ConverterProviderTest.scala b/scio-avro/src/test/scala/com/spotify/scio/avro/types/ConverterProviderTest.scala index 60768a8118..e8c7b98778 100644 --- a/scio-avro/src/test/scala/com/spotify/scio/avro/types/ConverterProviderTest.scala +++ b/scio-avro/src/test/scala/com/spotify/scio/avro/types/ConverterProviderTest.scala @@ -29,20 +29,20 @@ class ConverterProviderTest extends AnyFlatSpec with Matchers { import ConverterProviderTest._ "ConverterProvider" should "#1831: handle Avro map" in { - val dir = Files.createTempDirectory("avro-") + val dir = Files.createTempDirectory("avro-").toFile val data = Seq(Record(Map("a" -> 1), Some(Map("b" -> 2)), List(Map("c" -> 3)))) val sc1 = ScioContext() - sc1.parallelize(data).saveAsTypedAvroFile(dir.toString) + sc1.parallelize(data).saveAsTypedAvroFile(dir.getAbsolutePath) sc1.run() val sc2 = ScioContext() - val t = sc2.typedAvroFile[Record](s"$dir/*.avro").materialize + val t = sc2.typedAvroFile[Record](dir.getAbsolutePath, ".avro").materialize sc2.run() t.underlying.value.toSeq should contain theSameElementsAs data - FileUtils.deleteDirectory(dir.toFile) + FileUtils.deleteDirectory(dir) } } diff --git a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala index 63aa69662d..6728c377c0 100644 --- a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala +++ b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala @@ -707,9 +707,10 @@ class ScioContext private[scio] ( def textFile( path: String, compression: beam.Compression = TextIO.ReadParam.DefaultCompression, - emptyMatchTreatment: beam.fs.EmptyMatchTreatment = TextIO.ReadParam.DefaultEmptyMatchTreatment + emptyMatchTreatment: beam.fs.EmptyMatchTreatment = TextIO.ReadParam.DefaultEmptyMatchTreatment, + suffix: String = TextIO.ReadParam.DefaultSuffix ): SCollection[String] = - this.read(TextIO(path))(TextIO.ReadParam(compression, emptyMatchTreatment)) + this.read(TextIO(path))(TextIO.ReadParam(compression, emptyMatchTreatment, suffix)) /** * Get an SCollection with a custom input transform. The transform should have a unique name. diff --git a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala index d5441a2367..3815430771 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala @@ -67,8 +67,12 @@ final case class BinaryIO(path: String) extends ScioIO[Array[Byte]] { tempDirectory: ResourceId ): WriteFiles[Array[Byte], Void, Array[Byte]] = { require(tempDirectory != null, "tempDirectory must not be null") - val fp = FilenamePolicySupplier - .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val fp = FilenamePolicySupplier.resolve( + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(ScioUtil.strippedPath(path), suffix) val dynamicDestinations = DynamicFileDestinations .constant(fp, SerializableFunctions.identity[Array[Byte]]) val sink = new BytesSink( @@ -127,18 +131,18 @@ object BinaryIO { private def getObjectInputStream(meta: Metadata): InputStream = Channels.newInputStream(FileSystems.open(meta.resourceId())) - object WriteParam { - private[scio] val DefaultPrefix = null - private[scio] val DefaultSuffix = ".bin" - private[scio] val DefaultNumShards = 0 - private[scio] val DefaultCompression = Compression.UNCOMPRESSED - private[scio] val DefaultHeader = Array.emptyByteArray - private[scio] val DefaultFooter = Array.emptyByteArray - private[scio] val DefaultShardNameTemplate: String = null - private[scio] val DefaultFramePrefix: Array[Byte] => Array[Byte] = _ => Array.emptyByteArray - private[scio] val DefaultFrameSuffix: Array[Byte] => Array[Byte] = _ => Array.emptyByteArray - private[scio] val DefaultTempDirectory = null - private[scio] val DefaultFilenamePolicySupplier = null + private[scio] object WriteParam { + val DefaultPrefix = null + val DefaultSuffix = ".bin" + val DefaultNumShards = 0 + val DefaultCompression = Compression.UNCOMPRESSED + val DefaultHeader = Array.emptyByteArray + val DefaultFooter = Array.emptyByteArray + val DefaultShardNameTemplate: String = null + val DefaultFramePrefix: Array[Byte] => Array[Byte] = _ => Array.emptyByteArray + val DefaultFrameSuffix: Array[Byte] => Array[Byte] = _ => Array.emptyByteArray + val DefaultTempDirectory = null + val DefaultFilenamePolicySupplier = null } final case class WriteParam private ( diff --git a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala index 570a9529e6..b675c42253 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala @@ -34,17 +34,25 @@ import org.apache.commons.compress.compressors.CompressorStreamFactory import org.apache.commons.io.IOUtils import java.nio.charset.StandardCharsets +import java.util.regex.Pattern import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag +import scala.util.Try private[scio] object FileStorage { - @inline final def apply(path: String): FileStorage = new FileStorage(path) + @inline final def apply(path: String, suffix: String): FileStorage = new FileStorage(path, suffix) } -final private[scio] class FileStorage(protected[scio] val path: String) { +final private[scio] class FileStorage(path: String, suffix: String) { + private def listFiles: Seq[Metadata] = - FileSystems.`match`(path, EmptyMatchTreatment.DISALLOW).metadata().iterator.asScala.toSeq + FileSystems + .`match`(ScioUtil.filePattern(path, suffix), EmptyMatchTreatment.DISALLOW) + .metadata() + .iterator + .asScala + .toSeq private def getObjectInputStream(meta: Metadata): InputStream = Channels.newInputStream(FileSystems.open(meta.resourceId())) @@ -92,48 +100,47 @@ final private[scio] class FileStorage(protected[scio] val path: String) { case _: Throwable => buffered } } - val input = getDirectoryInputStream(path, wrapInputStream) + val input = getDirectoryInputStream(wrapInputStream) IOUtils.lineIterator(input, StandardCharsets.UTF_8).asScala } def tableRowJsonFile: Iterator[TableRow] = textFile.map(e => ScioUtil.jsonFactory.fromString(e, classOf[TableRow])) - def isDone: Boolean = { - val partPattern = "([0-9]{5})-of-([0-9]{5})".r - val metadata = - try { - listFiles - } catch { - case _: FileNotFoundException => Seq.empty - } - val nums = metadata.flatMap { meta => - val m = partPattern.findAllIn(meta.resourceId().toString) - if (m.hasNext) { - Some((m.group(1).toInt, m.group(2).toInt)) - } else { - None + def isDone(): Boolean = { + val files = Try(listFiles).recover { case _: FileNotFoundException => Seq.empty }.get + + // best effort matching shardNumber and numShards + val shards = ("(.*)(\\d+)\\D+(\\d+)\\D*" ++ Option(suffix).map(Pattern.quote).getOrElse("")).r + val writtenShards = files + .map(_.resourceId().toString) + .flatMap { + case shards(prefix, shardNumber, numShards) => + val part = for { + idx <- Try(shardNumber.toInt) + total <- Try(numShards.toInt) + key = (prefix, total) + } yield key -> idx + part.toOption + case _ => + None } - } + .groupMap(_._1)(_._2) - if (metadata.isEmpty) { - // empty list + if (files.isEmpty) { + // no files in folder false - } else if (nums.nonEmpty) { - // found xxxxx-of-yyyyy pattern - val parts = nums.map(_._1).sorted - val total = nums.map(_._2).toSet - metadata.size == nums.size && // all paths matched - total.size == 1 && total.head == parts.size && // yyyyy part - parts.head == 0 && parts.last + 1 == parts.size // xxxxx part - } else { + } else if (writtenShards.isEmpty) { + // assume progress is complete when shard info is not retrieved and files are present true + } else { + // we managed to get shard info, verify all of then were written + writtenShards.forall { case ((_, total), idxs) => idxs.size == total } } } @nowarn("msg=parameter value path in method getDirectoryInputStream is never used") private[scio] def getDirectoryInputStream( - path: String, wrapperFn: InputStream => InputStream = identity ): InputStream = { val inputs = listFiles.map(getObjectInputStream).map(wrapperFn).asJava diff --git a/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala b/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala index faa7ff2856..6f73538805 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala @@ -69,10 +69,10 @@ case class UnsupportedTap[T](msg: String) extends Tap[T] { } /** Tap for text files on local file system or GCS. */ -final case class TextTap(path: String) extends Tap[String] { - override def value: Iterator[String] = FileStorage(path).textFile +final case class TextTap(path: String, suffix: String) extends Tap[String] { + override def value: Iterator[String] = FileStorage(path, suffix).textFile - override def open(sc: ScioContext): SCollection[String] = sc.textFile(path) + override def open(sc: ScioContext): SCollection[String] = sc.textFile(path, suffix = suffix) } final private[scio] class InMemoryTap[T: Coder] extends Tap[T] { @@ -82,14 +82,13 @@ final private[scio] class InMemoryTap[T: Coder] extends Tap[T] { sc.parallelize[T](InMemorySink.get(id)) } -private[scio] class MaterializeTap[T: Coder] private (val path: String, coder: BCoder[T]) +private[scio] class MaterializeTap[T: Coder] private (path: String, coder: BCoder[T]) extends Tap[T] { - private val _path = ScioUtil.addPartSuffix(path) override def value: Iterator[T] = { - val storage = FileStorage(_path) + val storage = FileStorage(path, MaterializeTap.Suffix) - if (storage.isDone) { + if (storage.isDone()) { storage .avroFile[GenericRecord](AvroBytesUtil.schema) .map(AvroBytesUtil.decode(coder, _)) @@ -112,12 +111,17 @@ private[scio] class MaterializeTap[T: Coder] private (val path: String, coder: B } override def open(sc: ScioContext): SCollection[T] = sc.requireNotClosed { - val read = AvroIO.readGenericRecords(AvroBytesUtil.schema).from(_path) + val read = AvroIO + .readGenericRecords(AvroBytesUtil.schema) + .from(ScioUtil.filePattern(path, MaterializeTap.Suffix)) sc.applyTransform(read).parDo(dofn) } } object MaterializeTap { + + private val Suffix = ".obj.avro" + def apply[T: Coder](path: String, context: ScioContext): MaterializeTap[T] = new MaterializeTap(path, CoderMaterializer.beam(context, Coder[T])) } diff --git a/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala b/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala index c2f6ca823e..d88eb2892f 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala @@ -32,9 +32,14 @@ trait Taps { /** Get a `Future[Tap[String]]` for a text file. */ def textFile(path: String): Future[Tap[String]] = - mkTap(s"Text: $path", () => isPathDone(path), () => TextTap(path)) - - private[scio] def isPathDone(path: String): Boolean = FileStorage(path).isDone + mkTap( + s"Text: $path", + () => isPathDone(path, null), + () => TextTap(path, null) + ) + + private[scio] def isPathDone(path: String, suffix: String): Boolean = + FileStorage(path, suffix).isDone /** * Make a tap, to be implemented by concrete classes. diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index 6d33a0d655..b1c11e3e9d 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -26,13 +26,15 @@ import com.spotify.scio.util.ScioUtil import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.values.SCollection import org.apache.beam.sdk.io.fs.MatchResult.Metadata -import org.apache.beam.sdk.io.{Compression, FileSystems, TextIO => BTextIO} +import org.apache.beam.sdk.io.{Compression, FileSystems} +import org.apache.beam.sdk.{io => beam} import org.apache.commons.compress.compressors.CompressorStreamFactory import org.apache.commons.io.IOUtils import java.nio.charset.StandardCharsets import scala.jdk.CollectionConverters._ import scala.util.Try +import scala.util.chaining._ import org.apache.beam.sdk.io.fs.{EmptyMatchTreatment, ResourceId} final case class TextIO(path: String) extends ScioIO[String] { @@ -42,17 +44,18 @@ final case class TextIO(path: String) extends ScioIO[String] { override protected def read(sc: ScioContext, params: ReadP): SCollection[String] = { val coder = CoderMaterializer.beam(sc, Coder.stringCoder) - sc.applyTransform( - BTextIO - .read() - .from(path) - .withCompression(params.compression) - .withEmptyMatchTreatment(params.emptyMatchTreatment) - ).setCoder(coder) + val filePattern = ScioUtil.filePattern(path, params.suffix) + val t = beam.TextIO + .read() + .from(filePattern) + .withCompression(params.compression) + .withEmptyMatchTreatment(params.emptyMatchTreatment) + + sc.applyTransform(t) + .setCoder(coder) } private def textOut( - write: BTextIO.Write, path: String, suffix: String, numShards: Int, @@ -72,21 +75,22 @@ final case class TextIO(path: String) extends ScioIO[String] { shardNameTemplate = shardNameTemplate, isWindowed = isWindowed )(path, suffix) - var transform = write + + beam.TextIO + .write() .to(fp) .withTempDirectory(tempDirectory) .withNumShards(numShards) .withCompression(compression) + .pipe(w => header.fold(w)(w.withHeader)) + .pipe(w => footer.fold(w)(w.withFooter)) + .pipe(w => if (!isWindowed) w else w.withWindowedWrites()) - transform = header.fold(transform)(transform.withHeader) - transform = footer.fold(transform)(transform.withFooter) - if (!isWindowed) transform else transform.withWindowedWrites() } override protected def write(data: SCollection[String], params: WriteP): Tap[String] = { data.applyInternal( textOut( - BTextIO.write(), path, params.suffix, params.numShards, @@ -100,34 +104,43 @@ final case class TextIO(path: String) extends ScioIO[String] { ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) ) ) - tap(TextIO.ReadParam()) + tap(TextIO.ReadParam(params)) } override def tap(params: ReadP): Tap[String] = - TextTap(ScioUtil.addPartSuffix(path)) + TextTap(path, params.suffix) } object TextIO { - object ReadParam { + + private[scio] object ReadParam { private[scio] val DefaultCompression = Compression.AUTO private[scio] val DefaultEmptyMatchTreatment = EmptyMatchTreatment.DISALLOW + private[scio] val DefaultSuffix = null + + def apply(params: WriteParam): ReadParam = + new ReadParam( + compression = params.compression, + suffix = params.suffix + params.compression.getSuggestedSuffix + ) } final case class ReadParam private ( compression: Compression = ReadParam.DefaultCompression, - emptyMatchTreatment: EmptyMatchTreatment = ReadParam.DefaultEmptyMatchTreatment + emptyMatchTreatment: EmptyMatchTreatment = ReadParam.DefaultEmptyMatchTreatment, + suffix: String = ReadParam.DefaultSuffix ) - object WriteParam { - private[scio] val DefaultHeader = Option.empty[String] - private[scio] val DefaultFooter = Option.empty[String] - private[scio] val DefaultSuffix = ".txt" - private[scio] val DefaultNumShards = 0 - private[scio] val DefaultCompression = Compression.UNCOMPRESSED - private[scio] val DefaultFilenamePolicySupplier = null - private[scio] val DefaultPrefix = null - private[scio] val DefaultShardNameTemplate = null - private[scio] val DefaultTempDirectory = null + private[scio] object WriteParam { + val DefaultHeader = Option.empty[String] + val DefaultFooter = Option.empty[String] + val DefaultSuffix = ".txt" + val DefaultNumShards = 0 + val DefaultCompression = Compression.UNCOMPRESSED + val DefaultFilenamePolicySupplier = null + val DefaultPrefix = null + val DefaultShardNameTemplate = null + val DefaultTempDirectory = null } final val DefaultWriteParam: WriteParam = WriteParam( @@ -154,7 +167,7 @@ object TextIO { tempDirectory: String ) - private[scio] def textFile(path: String): Iterator[String] = { + private[scio] def textFile(pattern: String): Iterator[String] = { val factory = new CompressorStreamFactory() def wrapInputStream(in: InputStream) = { @@ -162,20 +175,20 @@ object TextIO { Try(factory.createCompressorInputStream(buffered)).getOrElse(buffered) } - val input = getDirectoryInputStream(path, wrapInputStream) + val input = getDirectoryInputStream(pattern, wrapInputStream) IOUtils.lineIterator(input, StandardCharsets.UTF_8).asScala } private def getDirectoryInputStream( - path: String, + pattern: String, wrapperFn: InputStream => InputStream ): InputStream = { - val inputs = listFiles(path).map(getObjectInputStream).map(wrapperFn).asJava + val inputs = listFiles(pattern).map(getObjectInputStream).map(wrapperFn).asJava new SequenceInputStream(Collections.enumeration(inputs)) } - private def listFiles(path: String): Seq[Metadata] = - FileSystems.`match`(path).metadata().iterator().asScala.toSeq + private def listFiles(pattern: String): Seq[Metadata] = + FileSystems.`match`(pattern).metadata().iterator().asScala.toSeq private def getObjectInputStream(meta: Metadata): InputStream = Channels.newInputStream(FileSystems.open(meta.resourceId())) diff --git a/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala b/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala index de3907f4fe..1cd6cf2be2 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala @@ -63,9 +63,6 @@ private[scio] object ScioUtil { def getScalaJsonMapper: ObjectMapper = new ObjectMapper().registerModule(DefaultScalaModule) - def addPartSuffix(path: String, ext: String = ""): String = - if (path.endsWith("/")) s"${path}part-*$ext" else s"$path/part-*$ext" - def getTempFile(context: ScioContext, fileOrPath: String = null): String = { val fop = Option(fileOrPath).getOrElse("scio-materialize-" + UUID.randomUUID().toString) val uri = URI.create(fop) @@ -94,8 +91,22 @@ private[scio] object ScioUtil { private def stripPath(path: String): String = StringUtils.stripEnd(path, "/") def strippedPath(path: String): String = s"${stripPath(path)}/" - def pathWithPrefix(path: String, filePrefix: String): String = s"${stripPath(path)}/${filePrefix}" - def pathWithPartPrefix(path: String): String = pathWithPrefix(path, "part") + def pathWithPrefix(path: String, prefix: String): String = Option(prefix) match { + case Some(p) => s"${stripPath(path)}/$p" + case None => s"${stripPath(path)}/part" + } + + def filePattern(path: String, suffix: String): String = + Option(suffix) match { + case Some(_) if path.contains("*") => + // path is already a pattern + throw new IllegalArgumentException(s"Suffix must be used with a static path but got: $path") + case Some(s) => + // match all file with suffix in path (must be a folder) + s"${stripPath(path)}/*$s" + case None => + path + } def consistentHashCode[K](k: K): Int = k match { case key: Array[_] => ArraySeq.unsafeWrapArray(key).## diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala index cf66da1c1c..03661ba08b 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala @@ -1557,7 +1557,7 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] { val avroCoder = Coder.avroGenericRecordCoder(schema) val write = beam.AvroIO .writeGenericRecords(schema) - .to(ScioUtil.pathWithPartPrefix(path)) + .to(ScioUtil.pathWithPrefix(path, "part")) .withSuffix(".obj.avro") .withCodec(CodecFactory.deflateCodec(6)) .withMetadata(Map.empty[String, AnyRef].asJava) @@ -1576,7 +1576,7 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] { ) = beam.TextIO .write() - .to(ScioUtil.pathWithPartPrefix(path)) + .to(ScioUtil.pathWithPrefix(path, null)) // TODO .withSuffix(suffix) .withNumShards(numShards) .withCompression(compression) diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala index 8d16bd606a..4714191216 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala @@ -100,7 +100,7 @@ object AvroExample { .saveAsAvroFile(args("output")) private def specificIn(sc: ScioContext, args: Args): ClosedTap[String] = - sc.avroFile[Account](args("input")) + sc.avroSpecificFile[Account](args("input")) .map(_.toString) .saveAsTextFile(args("output")) @@ -131,7 +131,7 @@ object AvroExample { .saveAsTypedAvroFile(args("output")) private def genericIn(sc: ScioContext, args: Args): ClosedTap[String] = - sc.avroFile(args("input"), schema) + sc.avroGenericFile(args("input"), schema) .map(_.toString) .saveAsTextFile(args("output")) diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala index 609e490d3c..75e2978260 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala @@ -33,7 +33,7 @@ object AvroInOut { // Open Avro files as a `SCollection[TestRecord]` where `TestRecord` is an Avro specific record // Java class compiled from Avro schema. - sc.avroFile[TestRecord](args("input")) + sc.avroSpecificFile[TestRecord](args("input")) .map { r => // Create a new `Account` Avro specific record. It is recommended to use the builder over // constructor since it's more backwards compatible. diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala index 54a1855f53..6a255e3ce9 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala @@ -73,7 +73,7 @@ object MagnolifyAvroReadExample { import MagnolifyAvroExample._ val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.avroFile(args("input"), wordCountType.schema) + sc.avroGenericFile(args("input"), wordCountType.schema) .map(e => wordCountType(e)) .map(wc => wc.word + ": " + wc.count) .saveAsTextFile(args("output")) diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BackcompatibilityTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BackcompatibilityTest.scala index 5cd7f322eb..b8d618aed3 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BackcompatibilityTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BackcompatibilityTest.scala @@ -19,10 +19,10 @@ package com.spotify.scio.examples.extra import java.nio.file.Paths -import com.spotify.scio.avro.ProtobufIO import com.spotify.scio.ScioContext import com.spotify.scio.proto.SimpleV2.SimplePB import com.spotify.scio.testing._ +import com.spotify.scio.avro._ class BackcompatibilityTest extends PipelineSpec { val input: Seq[SimplePB] = Seq( @@ -46,14 +46,14 @@ class BackcompatibilityTest extends PipelineSpec { // sc.run() "saveAsProtobuf" should "read protobuf files written with Scio 0.7 and above" in { val sc = ScioContext() - val r = sc.read(ProtobufIO[SimplePB](s"$pwd/$path_07/*")) + val r = sc.protobufFile[SimplePB](s"$pwd/$path_07", ".protobuf.avro") r should containInAnyOrder(input) sc.run() } it should "read protobuf files written with Scio 0.6 and below" in { val sc = ScioContext() - val r = sc.read(ProtobufIO[SimplePB](s"$pwd/$path_06/*")) + val r = sc.protobufFile[SimplePB](s"$pwd/$path_06", ".protobuf.avro") r should containInAnyOrder(input) sc.run() } diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala index 1da4f93fb5..4ebd0cf7ef 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala @@ -47,11 +47,15 @@ class SortMergeBucketExampleTest extends AnyFlatSpec with Matchers { ) GenericRecordTap( - s"$userDir/*.avro", - SortMergeBucketExample.UserDataSchema + path = userDir.getAbsolutePath, + schema = SortMergeBucketExample.UserDataSchema, + suffix = ".avro" ).value.size shouldBe 500 - SpecificRecordTap[Account](s"$accountDir/*.avro").value.size shouldBe 500 + SpecificRecordTap[Account]( + path = accountDir.getAbsolutePath, + suffix = ".avro" + ).value.size shouldBe 500 SortMergeBucketJoinExample.main( Array( @@ -61,8 +65,10 @@ class SortMergeBucketExampleTest extends AnyFlatSpec with Matchers { ) ) - TextTap(s"$joinOutputDir/*.txt").value.size shouldBe 100 - () + TextTap( + path = joinOutputDir.getAbsolutePath, + suffix = ".txt" + ).value.size shouldBe 100 } it should "transform user and account data" in withTempFolders { @@ -82,7 +88,10 @@ class SortMergeBucketExampleTest extends AnyFlatSpec with Matchers { ) ) - SpecificRecordTap[Account](s"$joinOutputDir/*.avro").value + SpecificRecordTap[Account]( + joinOutputDir.getAbsolutePath, + ".avro" + ).value .map(account => (account.getId, account.getType.toString)) .toList should contain theSameElementsAs (0 until 500).map((_, "combinedAmount")) () diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala index f2a8c80e6b..04f8518068 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala @@ -88,51 +88,56 @@ import org.apache.beam.sdk.values.PCollection */ object CsvIO { - final private val DefaultCsvConfig: CsvConfiguration = CsvConfiguration( + private val DefaultCsvConfiguration: CsvConfiguration = CsvConfiguration( cellSeparator = ',', quote = '"', quotePolicy = QuotePolicy.WhenNeeded, header = Header.Implicit ) - final val DefaultReadParams: ReadParam = CsvIO.ReadParam(compression = beam.Compression.AUTO) - final val DefaultWriteParams: WriteParam = CsvIO.WriteParam( - WriteParam.DefaultCompression, - WriteParam.DefaultCsvConfig, - WriteParam.DefaultSuffix, - WriteParam.DefaultNumShards, - WriteParam.DefaultShardNameTemplate, - WriteParam.DefaultTempDirectory, - WriteParam.DefaultFilenamePolicySupplier - ) + final val DefaultReadParams: ReadParam = CsvIO.ReadParam() + final val DefaultWriteParams: WriteParam = CsvIO.WriteParam() + + private[scio] object ReadParam { + val DefaultCompression = beam.Compression.AUTO + val DefaultCsvConfiguration = CsvIO.DefaultCsvConfiguration + val DefaultSuffix = null + + def apply(params: WriteParam): ReadParam = + new ReadParam( + compression = params.compression, + csvConfiguration = params.csvConfiguration, + suffix = params.suffix + params.compression.getSuggestedSuffix + ) + } final case class ReadParam private ( - compression: beam.Compression = beam.Compression.AUTO, - csvConfiguration: CsvConfiguration = CsvIO.DefaultCsvConfig + compression: beam.Compression = ReadParam.DefaultCompression, + csvConfiguration: CsvConfiguration = ReadParam.DefaultCsvConfiguration, + suffix: String = ReadParam.DefaultSuffix ) - object WriteParam { - private[scio] val DefaultSuffix = ".csv" - private[scio] val DefaultCsvConfig = CsvIO.DefaultCsvConfig - private[scio] val DefaultNumShards = 1 // put everything in a single file - private[scio] val DefaultCompression = Compression.UNCOMPRESSED - private[scio] val DefaultShardNameTemplate = null - private[scio] val DefaultTempDirectory = null - private[scio] val DefaultFilenamePolicySupplier = null + private[scio] object WriteParam { + val DefaultSuffix = ".csv" + val DefaultCsvConfig = CsvIO.DefaultCsvConfiguration + val DefaultNumShards = 1 // put everything in a single file + val DefaultCompression = Compression.UNCOMPRESSED + val DefaultFilenamePolicySupplier = null + val DefaultPrefix = null + val DefaultShardNameTemplate = null + val DefaultTempDirectory = null } final case class WriteParam private ( - compression: beam.Compression, - csvConfiguration: CsvConfiguration, - suffix: String, - numShards: Int, - shardNameTemplate: String, - tempDirectory: String, - filenamePolicySupplier: FilenamePolicySupplier - ) { - def toReadParam: ReadParam = - ReadParam(compression = compression, csvConfiguration = csvConfiguration) - } + compression: beam.Compression = WriteParam.DefaultCompression, + csvConfiguration: CsvConfiguration = WriteParam.DefaultCsvConfig, + suffix: String = WriteParam.DefaultSuffix, + numShards: Int = WriteParam.DefaultNumShards, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, + shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, + tempDirectory: String = WriteParam.DefaultTempDirectory + ) final case class Read[T: HeaderDecoder: Coder](path: String) extends ScioIO[T] { override type ReadP = CsvIO.ReadParam @@ -176,7 +181,7 @@ object CsvIO { override protected def write(data: SCollection[T], params: WriteP): Tap[T] = { data.applyInternal(csvOut(path, params)) - tap(params.toReadParam) + tap(ReadParam(params)) } override def tap(params: ReadP): Tap[T] = new CsvIO.CsvTap[T](path, params) @@ -192,7 +197,7 @@ object CsvIO { .via(new CsvSink(params.csvConfiguration)) private def read[T: HeaderDecoder: Coder](sc: ScioContext, path: String, params: ReadParam) = - sc.parallelize(Seq(path)) + sc.parallelize(Seq(ScioUtil.filePattern(path, params.suffix))) .withName("Read CSV") .readFiles( ParDo @@ -206,7 +211,7 @@ object CsvIO { extends Tap[T] { override def value: Iterator[T] = BinaryIO - .openInputStreamsFor(ScioUtil.addPartSuffix(path)) + .openInputStreamsFor(ScioUtil.filePattern(path, params.suffix)) .flatMap(_.asUnsafeCsvReader[T](params.csvConfiguration).iterator) override def open(sc: ScioContext): SCollection[T] = CsvIO.read(sc, path, params) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/syntax/SCollectionSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/syntax/SCollectionSyntax.scala index 3eb4f7d566..9000fb55d8 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/syntax/SCollectionSyntax.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/syntax/SCollectionSyntax.scala @@ -40,17 +40,19 @@ trait SCollectionSyntax { shardNameTemplate: String = CsvIO.WriteParam.DefaultShardNameTemplate, tempDirectory: String = CsvIO.WriteParam.DefaultTempDirectory, filenamePolicySupplier: FilenamePolicySupplier = - CsvIO.WriteParam.DefaultFilenamePolicySupplier + CsvIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = CsvIO.WriteParam.DefaultPrefix )(implicit coder: Coder[T], enc: HeaderEncoder[T]): ClosedTap[Nothing] = self.write(CsvIO.Write[T](path))( CsvIO.WriteParam( - compression, - csvConfig, - suffix, - numShards, - shardNameTemplate, - tempDirectory, - filenamePolicySupplier + compression = compression, + csvConfiguration = csvConfig, + suffix = suffix, + numShards = numShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + tempDirectory = tempDirectory ) ) } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala index 5289eb10aa..d678cc66b1 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala @@ -35,7 +35,8 @@ final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO final override val tapT: TapT.Aux[T, T] = TapOf[T] override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = - sc.read(TextIO(path))(TextIO.ReadParam(params.compression)).map(decodeJson) + sc.read(TextIO(path))(TextIO.ReadParam(params.compression, suffix = params.suffix)) + .map(decodeJson) override protected def write(data: SCollection[T], params: WriteP): Tap[T] = { data @@ -53,14 +54,14 @@ final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO params.tempDirectory ) ) - tap(JsonIO.ReadParam(params.compression)) + tap(JsonIO.ReadParam(params)) } override def tap(params: ReadP): Tap[T] = new Tap[T] { override def value: Iterator[T] = - TextIO.textFile(ScioUtil.addPartSuffix(path)).map(decodeJson) + TextIO.textFile(ScioUtil.filePattern(path, params.suffix)).map(decodeJson) override def open(sc: ScioContext): SCollection[T] = - JsonIO(ScioUtil.addPartSuffix(path)).read(sc, params) + JsonIO(ScioUtil.filePattern(path, params.suffix)).read(sc, params) } private def decodeJson(json: String): T = @@ -68,27 +69,41 @@ final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO } object JsonIO { - final case class ReadParam private (compression: beam.Compression = beam.Compression.AUTO) - object WriteParam { - private[scio] val DefaultNumShards = 0 - private[scio] val DefaultSuffix = ".json" - private[scio] val DefaultCompression = beam.Compression.UNCOMPRESSED - private[scio] val DefaultPrinter = Printer.noSpaces - private[scio] val DefaultFilenamePolicySupplier = null - private[scio] val DefaultShardNameTemplate: String = null - private[scio] val DefaultPrefix: String = null - private[scio] val DefaultTempDirectory = null + private[scio] object ReadParam { + val DefaultCompression = beam.Compression.AUTO + val DefaultSuffix = null + + def apply(params: WriteParam): ReadParam = new ReadParam( + params.compression, + params.suffix + params.compression.getSuggestedSuffix + ) + } + + final case class ReadParam private ( + compression: beam.Compression = ReadParam.DefaultCompression, + suffix: String = ReadParam.DefaultSuffix + ) + + private[scio] object WriteParam { + val DefaultNumShards = 0 + val DefaultSuffix = ".json" + val DefaultCompression = beam.Compression.UNCOMPRESSED + val DefaultPrinter = Printer.noSpaces + val DefaultFilenamePolicySupplier = null + val DefaultShardNameTemplate: String = null + val DefaultPrefix: String = null + val DefaultTempDirectory = null } final case class WriteParam private ( - suffix: String, - numShards: Int, - compression: beam.Compression, - printer: Printer, - filenamePolicySupplier: FilenamePolicySupplier, - prefix: String, - shardNameTemplate: String, - tempDirectory: String + suffix: String = WriteParam.DefaultSuffix, + numShards: Int = WriteParam.DefaultNumShards, + compression: beam.Compression = WriteParam.DefaultCompression, + printer: Printer = WriteParam.DefaultPrinter, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, + shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, + tempDirectory: String = WriteParam.DefaultTempDirectory ) } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala index 24a04a80e8..b3100269b4 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/json/package.scala @@ -58,12 +58,13 @@ package object json extends AutoDerivation { @experimental def jsonFile[T: Decoder: Coder]( path: String, - compression: Compression = Compression.AUTO + compression: Compression = JsonIO.ReadParam.DefaultCompression, + suffix: String = JsonIO.ReadParam.DefaultSuffix ): SCollection[T] = { implicit val encoder: Encoder[T] = new Encoder[T] { final override def apply(a: T): io.circe.Json = ??? } - self.read(JsonIO[T](path))(JsonIO.ReadParam(compression)) + self.read(JsonIO[T](path))(JsonIO.ReadParam(compression, suffix)) } } diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala index 1a6d68185b..67305abb06 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala @@ -38,12 +38,7 @@ object CsvIOTest { class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { - var dir: File = _ - override protected def beforeEach(): Unit = dir = tmpDir - override protected def afterEach(): Unit = FileUtils.deleteDirectory(dir) - - "CsvIO.Read" should "read strings" in { - + "CsvIO.Read" should "read strings" in withTempDir { dir => val csv = """header1 |data1 |data2 @@ -52,10 +47,10 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { implicit val decoder: HeaderDecoder[String] = HeaderDecoder.decoder("header1")((str: String) => str) - parse(csv) should containInAnyOrder(Seq("data1", "data2")) + parse(dir)(csv) should containInAnyOrder(Seq("data1", "data2")) } - it should "read tuples" in { + it should "read tuples" in withTempDir { dir => val csv = """ |numericValue, stringValue @@ -66,7 +61,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { implicit val decoder: HeaderDecoder[TestTuple] = HeaderDecoder.decoder("numericValue", "stringValue")(TestTuple.apply) - parse(csv) should containInAnyOrder( + parse(dir)(csv) should containInAnyOrder( Seq( TestTuple(1, "test1"), TestTuple(2, "test2") @@ -74,7 +69,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - it should "read tuples with reversed headers" in { + it should "read tuples with reversed headers" in withTempDir { dir => val csv = """ |stringValue, numericValue @@ -85,7 +80,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { implicit val decoder: HeaderDecoder[TestTuple] = HeaderDecoder.decoder("numericValue", "stringValue")(TestTuple.apply) - parse(csv) should containInAnyOrder( + parse(dir)(csv) should containInAnyOrder( Seq( TestTuple(1, "test1"), TestTuple(2, "test2") @@ -93,7 +88,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - it should "read ordered items without a header" in { + it should "read ordered items without a header" in withTempDir { dir => val csv = """ |test1,1 @@ -104,7 +99,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { TestTuple(i, string) } - parse(csv) should containInAnyOrder( + parse(dir)(csv) should containInAnyOrder( Seq( TestTuple(1, "test1"), TestTuple(2, "test2") @@ -112,7 +107,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - "CsvIO.Write" should "write with headers" in { + "CsvIO.Write" should "write with headers" in withTempDir { dir => implicit val encoder: HeaderEncoder[TestTuple] = HeaderEncoder.caseEncoder("intValue", "stringValue")(TestTuple.unapply) @@ -120,7 +115,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { TestTuple(1, "test1"), TestTuple(2, "test2") ) - val csvLines = writeAsCsvAndReadLines { (sc, path) => + val csvLines = writeAsCsvAndReadLines(dir) { (sc, path) => sc.parallelize(items) .saveAsCsvFile(path) } @@ -131,7 +126,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - it should "write without headers" in { + it should "write without headers" in withTempDir { dir => implicit val encoder: HeaderEncoder[TestTuple] = HeaderEncoder.caseEncoder("intValue", "stringValue")(TestTuple.unapply) val noHeaderConfig = @@ -141,7 +136,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { TestTuple(1, "test1"), TestTuple(2, "test2") ) - val csvLines = writeAsCsvAndReadLines { (sc, path) => + val csvLines = writeAsCsvAndReadLines(dir) { (sc, path) => sc.parallelize(items) .saveAsCsvFile(path, csvConfig = noHeaderConfig) } @@ -151,7 +146,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - it should "write with a row encoder" in { + it should "write with a row encoder" in withTempDir { dir => implicit val encoder: RowEncoder[TestTuple] = RowEncoder.encoder(0, 1)((tup: TestTuple) => (tup.a, tup.string)) val noHeaderConfig = @@ -161,9 +156,8 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { TestTuple(1, "test1"), TestTuple(2, "test2") ) - val csvLines = writeAsCsvAndReadLines { (sc, path) => - sc.parallelize(items) - .saveAsCsvFile(path, csvConfig = noHeaderConfig) + val csvLines = writeAsCsvAndReadLines(dir) { (sc, path) => + sc.parallelize(items).saveAsCsvFile(path, csvConfig = noHeaderConfig) } csvLines should contain allElementsOf Seq( "1,test1", @@ -171,7 +165,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - "Csvio.ReadWrite" should "read and write csv files" in { + "Csvio.ReadWrite" should "read and write csv files" in withTempDir { dir => implicit val codec: HeaderCodec[TestTuple] = HeaderCodec.codec("numericValue", "stringValue")(TestTuple.apply)(TestTuple.unapply(_).get) val csv = """numericValue, stringValue @@ -201,7 +195,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - "CsvIO.ReadDoFn" should "be serialisable" in { + "CsvIO.ReadDoFn" should "be serializable" in withTempDir { dir => implicit val decoder: HeaderDecoder[TestTuple] = HeaderDecoder.decoder("numericValue", "stringValue")(TestTuple.apply) SerializableUtils.serializeToByteArray( @@ -209,11 +203,11 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - private def writeAsCsvAndReadLines[T: HeaderEncoder: Coder]( + private def writeAsCsvAndReadLines[T: HeaderEncoder: Coder](dir: File)( transform: (ScioContext, String) => ClosedTap[Nothing] ): List[String] = { val sc = ScioContext() - transform(sc, dir.getPath) + transform(sc, dir.getAbsolutePath) sc.run().waitUntilFinish() val file: File = getFirstCsvFileFrom(dir) FileUtils.readLines(file, StandardCharsets.UTF_8).asScala.toList @@ -226,7 +220,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { }) .head - private def parse[T: HeaderDecoder: Coder](csv: String): SCollection[T] = { + private def parse[T: HeaderDecoder: Coder](dir: File)(csv: String): SCollection[T] = { val file = new File(dir, "source.csv") FileUtils.write(file, csv, StandardCharsets.UTF_8) ScioContext().csvFile(file.getAbsolutePath) diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala index 157951e682..37cbc4b953 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala @@ -44,11 +44,10 @@ class JsonIOTest extends ScioIOSpec with TapSpec { testJobTest(xs)(JsonIO(_))(_.jsonFile(_))(_.saveAsJsonFile(_)) } - it should "support custom printer" in { - val dir = tmpDir + it should "support custom printer" in withTempDir { dir => val t = runWithFileFuture { _.parallelize(xs) - .saveAsJsonFile(dir.getPath, printer = Printer.noSpaces.copy(dropNullValues = true)) + .saveAsJsonFile(dir.getAbsolutePath, printer = Printer.noSpaces.copy(dropNullValues = true)) } verifyTap(t, xs.toSet) val result = Files @@ -61,10 +60,9 @@ class JsonIOTest extends ScioIOSpec with TapSpec { s"""{"i":$x,"s":"$x"${if (x % 2 == 0) s""","o":$x""" else ""}}""" } result should contain theSameElementsAs expected - FileUtils.deleteDirectory(dir) } - it should "handle invalid JSON" in { + it should "handle invalid JSON" in withTempDir { dir => val badData = Seq( """{"i":1, "s":hello}""", """{"i":1}""", @@ -72,16 +70,16 @@ class JsonIOTest extends ScioIOSpec with TapSpec { """{"i":1, "s":1}""", """{"i":"hello", "s":1}""" ) - val dir = tmpDir runWithFileFuture { - _.parallelize(badData).saveAsTextFile(dir.getPath) + _.parallelize(badData).saveAsTextFile(dir.getAbsolutePath, suffix = ".json") } val sc = ScioContext() - sc.jsonFile[Record](ScioUtil.addPartSuffix(dir.getPath)) + sc.jsonFile[Record]( + path = dir.getAbsolutePath, + suffix = ".json" + ) a[PipelineExecutionException] should be thrownBy { sc.run() } - - FileUtils.deleteDirectory(dir) } } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index 6c02bb5272..db103ad828 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -433,7 +433,7 @@ final case class BigQueryStorageSelect(sqlQuery: Query) extends BigQueryIO[Table /** Get an IO for a BigQuery TableRow JSON file. */ final case class TableRowJsonIO(path: String) extends ScioIO[TableRow] { - override type ReadP = Unit + override type ReadP = TableRowJsonIO.ReadParam override type WriteP = TableRowJsonIO.WriteParam final override val tapT: TapT.Aux[TableRow, TableRow] = TapOf[TableRow] @@ -444,24 +444,50 @@ final case class TableRowJsonIO(path: String) extends ScioIO[TableRow] { override protected def write(data: SCollection[TableRow], params: WriteP): Tap[TableRow] = { data.transform_("BigQuery write") { _.map(ScioUtil.jsonFactory.toString) - .applyInternal(data.textOut(path, ".json", params.numShards, params.compression)) + .applyInternal( + data.textOut( + path = path, + suffix = params.suffix, + numShards = params.numShards, + compression = params.compression + ) + ) } - tap(()) + tap(TableRowJsonIO.ReadParam(params)) } override def tap(read: ReadP): Tap[TableRow] = - TableRowJsonTap(ScioUtil.addPartSuffix(path)) + TableRowJsonTap(path, read.suffix) } object TableRowJsonIO { - object WriteParam { - private[bigquery] val DefaultNumShards = 0 - private[bigquery] val DefaultCompression = Compression.UNCOMPRESSED + + private[bigquery] object ReadParam { + val DefaultCompression = Compression.AUTO + val DefaultSuffix = null + + def apply(params: WriteParam): ReadParam = + new ReadParam( + compression = params.compression, + suffix = params.suffix + params.compression.getSuggestedSuffix + ) + } + + final case class ReadParam private ( + compression: Compression = ReadParam.DefaultCompression, + suffix: String = null + ) + + private[bigquery] object WriteParam { + val DefaultNumShards = 0 + val DefaultCompression = Compression.UNCOMPRESSED + val DefaultSuffix = ".json" } final case class WriteParam private ( numShards: Int = WriteParam.DefaultNumShards, - compression: Compression = WriteParam.DefaultCompression + compression: Compression = WriteParam.DefaultCompression, + suffix: String = WriteParam.DefaultSuffix ) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala index f08c5d5465..0a195918f8 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala @@ -39,6 +39,7 @@ import scala.reflect.runtime.universe._ import com.spotify.scio.bigquery.BigQueryTypedTable import com.spotify.scio.bigquery.BigQueryTypedTable.Format import com.spotify.scio.bigquery.coders.tableRowCoder +import org.apache.beam.sdk.io.Compression /** Enhanced version of [[ScioContext]] with BigQuery methods. */ final class ScioContextOps(private val self: ScioContext) extends AnyVal { @@ -208,8 +209,12 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { ) /** Get an SCollection for a BigQuery TableRow JSON file. */ - def tableRowJsonFile(path: String): SCollection[TableRow] = - self.read(TableRowJsonIO(path)) + def tableRowJsonFile( + path: String, + compression: Compression = TableRowJsonIO.ReadParam.DefaultCompression, + suffix: String = TableRowJsonIO.ReadParam.DefaultSuffix + ): SCollection[TableRow] = + self.read(TableRowJsonIO(path))(TableRowJsonIO.ReadParam(compression, suffix)) } trait ScioContextSyntax { diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala index 7c60297f1f..5af9eeff7b 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala @@ -34,8 +34,8 @@ import com.spotify.scio.bigquery.BigQueryTypedTable.Format import com.twitter.chill.Externalizer /** Tap for BigQuery TableRow JSON files. */ -final case class TableRowJsonTap(path: String) extends Tap[TableRow] { - override def value: Iterator[TableRow] = FileStorage(path).tableRowJsonFile +final case class TableRowJsonTap(path: String, suffix: String) extends Tap[TableRow] { + override def value: Iterator[TableRow] = FileStorage(path, suffix).tableRowJsonFile override def open(sc: ScioContext): SCollection[TableRow] = sc.tableRowJsonFile(path) } @@ -129,8 +129,12 @@ final case class BigQueryTaps(self: Taps) { } /** Get a `Future[Tap[TableRow]]` for a BigQuery TableRow JSON file. */ - def tableRowJsonFile(path: String): Future[Tap[TableRow]] = - mkTap(s"TableRowJson: $path", () => self.isPathDone(path), () => TableRowJsonIO(path).tap(())) + def tableRowJsonFile(path: String, suffix: String): Future[Tap[TableRow]] = + mkTap( + s"TableRowJson: $path", + () => self.isPathDone(path, suffix), + () => TableRowJsonTap(path, suffix) + ) def bigQueryStorage( table: TableReference, diff --git a/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala b/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala index 47add251a9..f03f773628 100644 --- a/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala +++ b/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala @@ -75,7 +75,7 @@ class GroupByBenchmark { @Benchmark def testScioGroupByKey: ScioExecutionContext = runWithContext { sc => - sc.avroFile(source, schema = avroSchema) + sc.avroGenericFile(source, schema = avroSchema) .map(rec => (rec.get("id").toString.head, rec.get("value").asInstanceOf[Double])) .groupByKey } @@ -84,7 +84,7 @@ class GroupByBenchmark { def testBeamGroupByKey: ScioExecutionContext = runWithContext { sc => sc.wrap { - sc.avroFile(source, schema = avroSchema) + sc.avroGenericFile(source, schema = avroSchema) .map { rec => KV.of(rec.get("id").toString.head, rec.get("value").asInstanceOf[Double]) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala index 45092d3c7d..5d552d3d87 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala @@ -94,8 +94,12 @@ final case class ParquetAvroIO[T: ClassTag: Coder](path: String) extends ScioIO[ isLocalRunner: Boolean ) = { require(tempDirectory != null, "tempDirectory must not be null") - val fp = FilenamePolicySupplier - .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val fp = FilenamePolicySupplier.resolve( + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(ScioUtil.strippedPath(path), suffix) val dynamicDestinations = DynamicFileDestinations .constant(fp, SerializableFunctions.identity[T]) val job = Job.getInstance(ParquetConfiguration.ofNullable(conf)) @@ -143,38 +147,37 @@ final case class ParquetAvroIO[T: ClassTag: Coder](path: String) extends ScioIO[ ScioUtil.isLocalRunner(data.context.options.getRunner) ) ) - tap(ParquetAvroIO.ReadParam[T, T](identity[T] _, writerSchema, null)) + tap(ParquetAvroIO.ReadParam(params)) } override def tap(params: ReadP): Tap[T] = - ParquetAvroTap(ScioUtil.addPartSuffix(path), params) + ParquetAvroTap(ScioUtil.filePattern(path, params.suffix), params) } object ParquetAvroIO { private lazy val log = LoggerFactory.getLogger(getClass) - object ReadParam { - private[avro] val DefaultProjection = null - private[avro] val DefaultPredicate = null - private[avro] val DefaultConfiguration = null - - @deprecated( - "Use ReadParam(projectionFn, projection, predicate, conf) instead", - since = "0.10.0" - ) - def apply[A: ClassTag, T: ClassTag]( - projection: Schema, - predicate: FilterPredicate, - projectionFn: A => T - ): ReadParam[A, T] = - ReadParam(projectionFn, projection, predicate) + private[scio] object ReadParam { + val DefaultProjection = null + val DefaultPredicate = null + val DefaultConfiguration = null + val DefaultSuffix = null + + def apply[T: ClassTag](params: WriteParam): ReadParam[T, T] = + new ReadParam[T, T]( + projectionFn = identity, + projection = params.schema, + conf = params.conf, + suffix = params.suffix + ) } final case class ReadParam[A: ClassTag, T: ClassTag] private ( projectionFn: A => T, projection: Schema = ReadParam.DefaultProjection, predicate: FilterPredicate = ReadParam.DefaultPredicate, - conf: Configuration = ReadParam.DefaultConfiguration + conf: Configuration = ReadParam.DefaultConfiguration, + suffix: String = null ) { val avroClass: Class[A] = ScioUtil.classOf[A] val isSpecific: Boolean = classOf[SpecificRecordBase] isAssignableFrom avroClass @@ -296,16 +299,16 @@ object ParquetAvroIO { }) } - object WriteParam { - private[scio] val DefaultSchema = null - private[scio] val DefaultNumShards = 0 - private[scio] val DefaultSuffix = ".parquet" - private[scio] val DefaultCompression = CompressionCodecName.GZIP - private[scio] val DefaultConfiguration = null - private[scio] val DefaultFilenamePolicySupplier = null - private[scio] val DefaultPrefix = null - private[scio] val DefaultShardNameTemplate = null - private[scio] val DefaultTempDirectory = null + private[scio] object WriteParam { + val DefaultSchema = null + val DefaultNumShards = 0 + val DefaultSuffix = ".parquet" + val DefaultCompression = CompressionCodecName.GZIP + val DefaultConfiguration = null + val DefaultFilenamePolicySupplier = null + val DefaultPrefix = null + val DefaultShardNameTemplate = null + val DefaultTempDirectory = null } final case class WriteParam private ( diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/ScioContextSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/ScioContextSyntax.scala index 21ee5e1fb1..fc92399f33 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/ScioContextSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/syntax/ScioContextSyntax.scala @@ -46,10 +46,11 @@ final class ScioContextOps(@transient private val self: ScioContext) extends Any path: String, projection: Schema = ReadParam.DefaultProjection, predicate: FilterPredicate = ReadParam.DefaultPredicate, - conf: Configuration = ReadParam.DefaultConfiguration + conf: Configuration = ReadParam.DefaultConfiguration, + suffix: String = ReadParam.DefaultSuffix ): ParquetAvroFile[T] = self.requireNotClosed { - new ParquetAvroFile[T](self, path, projection, predicate, conf) + new ParquetAvroFile[T](self, path, projection, predicate, conf, suffix) } } @@ -58,7 +59,8 @@ class ParquetAvroFile[T: ClassTag] private[avro] ( path: String, projection: Schema, predicate: FilterPredicate, - conf: Configuration + conf: Configuration, + suffix: String ) { private val logger = LoggerFactory.getLogger(this.getClass) @@ -67,7 +69,7 @@ class ParquetAvroFile[T: ClassTag] private[avro] ( * file. */ def map[U: ClassTag: Coder](f: T => U): SCollection[U] = { - val param = ParquetAvroIO.ReadParam[T, U](f, projection, predicate, conf) + val param = ParquetAvroIO.ReadParam[T, U](f, projection, predicate, conf, suffix) context.read(ParquetAvroIO[U](path))(param) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala index 2b4fc9b7e5..ff503a01a6 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala @@ -91,7 +91,7 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { ParquetRead.read( ReadSupportFactory.example, new SerializableConfiguration(conf), - path, + ScioUtil.filePattern(path, params.suffix), identity[Example] ) ).setCoder(coder) @@ -162,8 +162,12 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { isLocalRunner: Boolean ) = { require(tempDirectory != null, "tempDirectory must not be null") - val fp = FilenamePolicySupplier - .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val fp = FilenamePolicySupplier.resolve( + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(ScioUtil.strippedPath(path), suffix) val dynamicDestinations = DynamicFileDestinations .constant(fp, SerializableFunctions.identity[Example]) val job = Job.getInstance(ParquetConfiguration.ofNullable(conf)) @@ -196,34 +200,43 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { ScioUtil.isLocalRunner(data.context.options.getRunner) ) ) - tap(ParquetExampleIO.ReadParam()) + tap(ParquetExampleIO.ReadParam(params)) } override def tap(params: ReadP): Tap[Example] = - ParquetExampleTap(ScioUtil.addPartSuffix(path), params) + ParquetExampleTap(ScioUtil.filePattern(path, params.suffix), params) } object ParquetExampleIO { - object ReadParam { - private[tensorflow] val DefaultProjection = null - private[tensorflow] val DefaultPredicate = null - private[tensorflow] val DefaultConfiguration = null + + private[tensorflow] object ReadParam { + val DefaultProjection = null + val DefaultPredicate = null + val DefaultConfiguration = null + val DefaultSuffix = null + + def apply(params: WriteParam): ReadParam = + new ReadParam( + conf = params.conf, + suffix = params.suffix + ) } final case class ReadParam private ( projection: Seq[String] = ReadParam.DefaultProjection, predicate: FilterPredicate = ReadParam.DefaultPredicate, - conf: Configuration = ReadParam.DefaultConfiguration + conf: Configuration = ReadParam.DefaultConfiguration, + suffix: String = ReadParam.DefaultSuffix ) - object WriteParam { - private[tensorflow] val DefaultNumShards = 0 - private[tensorflow] val DefaultSuffix = ".parquet" - private[tensorflow] val DefaultCompression = CompressionCodecName.GZIP - private[tensorflow] val DefaultConfiguration = null - private[tensorflow] val DefaultFilenamePolicySupplier = null - private[tensorflow] val DefaultPrefix = null - private[tensorflow] val DefaultShardNameTemplate = null - private[tensorflow] val DefaultTempDirectory = null + private[tensorflow] object WriteParam { + val DefaultNumShards = 0 + val DefaultSuffix = ".parquet" + val DefaultCompression = CompressionCodecName.GZIP + val DefaultConfiguration = null + val DefaultFilenamePolicySupplier = null + val DefaultPrefix = null + val DefaultShardNameTemplate = null + val DefaultTempDirectory = null } final case class WriteParam private ( diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/ScioContextSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/ScioContextSyntax.scala index 0d009dd027..7d67c5ac22 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/ScioContextSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/syntax/ScioContextSyntax.scala @@ -33,9 +33,12 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { path: String, projection: Seq[String] = ReadParam.DefaultProjection, predicate: FilterPredicate = ReadParam.DefaultPredicate, - conf: Configuration = ReadParam.DefaultConfiguration + conf: Configuration = ReadParam.DefaultConfiguration, + suffix: String = ReadParam.DefaultSuffix ): SCollection[Example] = - self.read(ParquetExampleIO(path))(ParquetExampleIO.ReadParam(projection, predicate, conf)) + self.read(ParquetExampleIO(path))( + ParquetExampleIO.ReadParam(projection, predicate, conf, suffix) + ) } trait ScioContextSyntax { diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala index 7a533cecc0..98e66340f4 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala @@ -47,8 +47,8 @@ import scala.reflect.ClassTag final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( path: String ) extends ScioIO[T] { - override type ReadP = ParquetTypeIO.ReadParam[T] - override type WriteP = ParquetTypeIO.WriteParam[T] + override type ReadP = ParquetTypeIO.ReadParam + override type WriteP = ParquetTypeIO.WriteParam override val tapT: TapT.Aux[T, T] = TapOf[T] private val tpe: ParquetType[T] = implicitly[ParquetType[T]] @@ -132,8 +132,12 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( isLocalRunner: Boolean ) = { require(tempDirectory != null, "tempDirectory must not be null") - val fp = FilenamePolicySupplier - .resolve(filenamePolicySupplier, prefix, shardNameTemplate, isWindowed)(path, suffix) + val fp = FilenamePolicySupplier.resolve( + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + isWindowed = isWindowed + )(ScioUtil.strippedPath(path), suffix) val dynamicDestinations = DynamicFileDestinations .constant(fp, SerializableFunctions.identity[T]) val job = Job.getInstance(ParquetConfiguration.ofNullable(conf)) @@ -165,35 +169,44 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( ScioUtil.isLocalRunner(data.context.options.getRunner) ) ) - tap(ParquetTypeIO.ReadParam[T]()) + tap(ParquetTypeIO.ReadParam(params)) } override def tap(params: ReadP): Tap[tapT.T] = - ParquetTypeTap(ScioUtil.addPartSuffix(path), params) + ParquetTypeTap(ScioUtil.filePattern(path, params.suffix), params) } object ParquetTypeIO { - object ReadParam { - private[scio] val DefaultPredicate = null - private[scio] val DefaultConfiguration = null + + private[scio] object ReadParam { + val DefaultPredicate = null + val DefaultConfiguration = null + val DefaultSuffix = null + + def apply[T](params: WriteParam): ReadParam = + new ReadParam( + conf = params.conf, + suffix = params.suffix + ) } - final case class ReadParam[T] private ( + final case class ReadParam private ( predicate: FilterPredicate = null, - conf: Configuration = ReadParam.DefaultConfiguration + conf: Configuration = ReadParam.DefaultConfiguration, + suffix: String = ReadParam.DefaultSuffix ) - object WriteParam { - private[scio] val DefaultNumShards = 0 - private[scio] val DefaultSuffix = ".parquet" - private[scio] val DefaultCompression = CompressionCodecName.GZIP - private[scio] val DefaultConfiguration = null - private[scio] val DefaultFilenamePolicySupplier = null - private[scio] val DefaultPrefix = null - private[scio] val DefaultShardNameTemplate = null - private[scio] val DefaultTempDirectory = null + private[scio] object WriteParam { + val DefaultNumShards = 0 + val DefaultSuffix = ".parquet" + val DefaultCompression = CompressionCodecName.GZIP + val DefaultConfiguration = null + val DefaultFilenamePolicySupplier = null + val DefaultPrefix = null + val DefaultShardNameTemplate = null + val DefaultTempDirectory = null } - final case class WriteParam[T] private ( + final case class WriteParam private ( numShards: Int = WriteParam.DefaultNumShards, suffix: String = WriteParam.DefaultSuffix, compression: CompressionCodecName = WriteParam.DefaultCompression, @@ -207,7 +220,7 @@ object ParquetTypeIO { case class ParquetTypeTap[T: ClassTag: Coder: ParquetType]( path: String, - params: ParquetTypeIO.ReadParam[T] + params: ParquetTypeIO.ReadParam ) extends Tap[T] { override def value: Iterator[T] = { val tpe = implicitly[ParquetType[T]] diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/ScioContextSyntax.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/ScioContextSyntax.scala index c3b5882d37..c9646e0253 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/ScioContextSyntax.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/syntax/ScioContextSyntax.scala @@ -35,9 +35,10 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { def typedParquetFile[T: ClassTag: Coder: ParquetType]( path: String, predicate: FilterPredicate = ReadParam.DefaultPredicate, - conf: Configuration = ReadParam.DefaultConfiguration + conf: Configuration = ReadParam.DefaultConfiguration, + suffix: String = ReadParam.DefaultSuffix ): SCollection[T] = - self.read(ParquetTypeIO[T](path))(ParquetTypeIO.ReadParam(predicate, conf)) + self.read(ParquetTypeIO[T](path))(ParquetTypeIO.ReadParam(predicate, conf, suffix)) } trait ScioContextSyntax { implicit def parquetTypeScioContext(c: ScioContext): ScioContextOps = new ScioContextOps(c) diff --git a/scio-parquet/src/test/scala/com/spotify/scio/parquet/avro/ParquetAvroIOTest.scala b/scio-parquet/src/test/scala/com/spotify/scio/parquet/avro/ParquetAvroIOTest.scala index 8c3a2b3da5..c57881faca 100644 --- a/scio-parquet/src/test/scala/com/spotify/scio/parquet/avro/ParquetAvroIOTest.scala +++ b/scio-parquet/src/test/scala/com/spotify/scio/parquet/avro/ParquetAvroIOTest.scala @@ -21,7 +21,7 @@ import java.io.File import com.spotify.scio._ import com.spotify.scio.coders.Coder import com.spotify.scio.avro._ -import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec, TapSpec, TextIO} +import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec, ScioIOTest, TapSpec, TextIO} import com.spotify.scio.parquet.ParquetConfiguration import com.spotify.scio.testing._ import com.spotify.scio.util.FilenamePolicySupplier @@ -39,18 +39,23 @@ import org.joda.time.{DateTime, DateTimeFieldType, Duration, Instant} import org.scalatest.BeforeAndAfterAll import java.lang +import java.nio.file.Files class ParquetAvroIOFileNamePolicyTest extends FileNamePolicySpec[TestRecord] { - val extension: String = ".parquet" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override val suffix: String = ".parquet" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[TestRecord] = { in.map(AvroUtils.newSpecificRecord) .saveAsParquetAvroFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -64,32 +69,33 @@ class ParquetAvroIOFileNamePolicyTest extends FileNamePolicySpec[TestRecord] { } class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { - private val dir = tmpDir + private val testDir = Files.createTempDirectory("scio-test-").toFile private val specificRecords = (1 to 10).map(AvroUtils.newSpecificRecord) override protected def beforeAll(): Unit = { val sc = ScioContext() - sc.parallelize(specificRecords) - .saveAsParquetAvroFile(dir.toString) + sc.parallelize(specificRecords).saveAsParquetAvroFile(testDir.getAbsolutePath) sc.run() - () } - override protected def afterAll(): Unit = FileUtils.deleteDirectory(dir) + override protected def afterAll(): Unit = FileUtils.deleteDirectory(testDir) "ParquetAvroIO" should "work with specific records" in { val xs = (1 to 100).map(AvroUtils.newSpecificRecord) testTap(xs)(_.saveAsParquetAvroFile(_))(".parquet") - testJobTest(xs)(ParquetAvroIO(_))(_.parquetAvroFile[TestRecord](_).map(identity))( - _.saveAsParquetAvroFile(_) - ) + testJobTest(xs)(ParquetAvroIO(_))( + _.parquetAvroFile[TestRecord](_).map(identity) + )(_.saveAsParquetAvroFile(_)) } it should "read specific records with projection" in { val sc = ScioContext() val projection = Projection[TestRecord](_.getIntField) - val data = sc - .parquetAvroFile[TestRecord](s"$dir/*.parquet", projection = projection) + val data = sc.parquetAvroFile[TestRecord]( + path = testDir.getAbsolutePath, + projection = projection, + suffix = ".parquet" + ) data.map(_.getIntField.toInt) should containInAnyOrder(1 to 10) data.map(identity) should forAll[TestRecord] { r => r.getLongField == null && r.getFloatField == null && r.getDoubleField == null && @@ -97,74 +103,78 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { .size() == 0 } sc.run() - () } it should "read specific records with predicate" in { val sc = ScioContext() val predicate = Predicate[TestRecord](_.getIntField <= 5) - val data = - sc.parquetAvroFile[TestRecord](s"$dir/*.parquet", predicate = predicate) - data.map(identity) should containInAnyOrder(specificRecords.filter(_.getIntField <= 5)) + val data = sc.parquetAvroFile[TestRecord]( + path = testDir.getAbsolutePath, + predicate = predicate, + suffix = ".parquet" + ) + val expected = specificRecords.filter(_.getIntField <= 5) + data.map(identity) should containInAnyOrder(expected) sc.run() - () } it should "read specific records with projection and predicate" in { val sc = ScioContext() val projection = Projection[TestRecord](_.getIntField) val predicate = Predicate[TestRecord](_.getIntField <= 5) - val data = - sc.parquetAvroFile[TestRecord](s"$dir/*.parquet", projection, predicate) + val data = sc.parquetAvroFile[TestRecord]( + path = testDir.getAbsolutePath, + projection = projection, + predicate = predicate, + suffix = ".parquet" + ) data.map(_.getIntField.toInt) should containInAnyOrder(1 to 5) data.map(identity) should forAll[TestRecord] { r => - r.getLongField == null && r.getFloatField == null && r.getDoubleField == null && - r.getBooleanField == null && r.getStringField == null && r.getArrayField - .size() == 0 + r.getLongField == null && + r.getFloatField == null && + r.getDoubleField == null && + r.getBooleanField == null && + r.getStringField == null && + r.getArrayField.size() == 0 } sc.run() - () } - it should "write and read SpecificRecords with default logical types" in { - val records = - (1 to 10).map(_ => - TestLogicalTypes - .newBuilder() - .setTimestamp(DateTime.now()) - .setDecimal(BigDecimal.decimal(1.0).setScale(2).bigDecimal) - .build() - ) - val path = dir.toPath.resolve("logicalTypesSr").toString + it should "write and read SpecificRecords with default logical types" in withTempDir { dir => + val records = (1 to 10).map(_ => + TestLogicalTypes + .newBuilder() + .setTimestamp(DateTime.now()) + .setDecimal(BigDecimal.decimal(1.0).setScale(2).bigDecimal) + .build() + ) val sc1 = ScioContext() sc1 .parallelize(records) .saveAsParquetAvroFile( - path, + path = dir.getAbsolutePath, conf = ParquetConfiguration.of( AvroWriteSupport.AVRO_DATA_SUPPLIER -> classOf[LogicalTypeSupplier] ) ) sc1.run() - () val sc2 = ScioContext() sc2 .parquetAvroFile[TestLogicalTypes]( - s"$path/*.parquet", + path = dir.getAbsolutePath, conf = ParquetConfiguration.of( AvroReadSupport.AVRO_DATA_SUPPLIER -> classOf[LogicalTypeSupplier] - ) + ), + suffix = ".parquet" ) .map(identity) should containInAnyOrder(records) sc2.run() - () } - it should "write and read GenericRecords with default logical types" in { - + it should "write and read GenericRecords with default logical types" in withTempDir { dir => val records: Seq[GenericRecord] = (1 to 10).map { _ => val gr = new GenericRecordBuilder(TestLogicalTypes.SCHEMA$) gr.set("timestamp", DateTime.now()) @@ -174,7 +184,6 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { ) gr.build() } - val path = dir.toPath.resolve("logicalTypesGr").toString implicit val coder = { GenericData.get().addLogicalTypeConversion(new TimeConversions.TimestampConversion) @@ -186,31 +195,30 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { sc1 .parallelize(records) .saveAsParquetAvroFile( - path, + path = dir.getAbsolutePath, schema = TestLogicalTypes.SCHEMA$, conf = ParquetConfiguration.of( AvroWriteSupport.AVRO_DATA_SUPPLIER -> classOf[LogicalTypeSupplier] ) ) sc1.run() - () val sc2 = ScioContext() sc2 .parquetAvroFile[GenericRecord]( - s"$path/*.parquet", + path = dir.getAbsolutePath, projection = TestLogicalTypes.SCHEMA$, conf = ParquetConfiguration.of( AvroReadSupport.AVRO_DATA_SUPPLIER -> classOf[LogicalTypeSupplier] - ) + ), + suffix = ".parquet" ) .map(identity) should containInAnyOrder(records) sc2.run() - () } - it should "write and read SpecificRecords with custom logical types" in { + it should "write and read SpecificRecords with custom logical types" in withTempDir { dir => val records = (1 to 10).map(_ => TestLogicalTypes @@ -219,27 +227,26 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { .setDecimal(BigDecimal.decimal(1.0).setScale(2).bigDecimal) .build() ) - val path = dir.toPath.resolve("logicalTypesCustom").toString val sc1 = ScioContext() sc1 .parallelize(records) .saveAsParquetAvroFile( - path, + path = dir.getAbsolutePath, conf = ParquetConfiguration.of( AvroWriteSupport.AVRO_DATA_SUPPLIER -> classOf[CustomLogicalTypeSupplier] ) ) sc1.run() - () val sc2 = ScioContext() sc2 .parquetAvroFile[TestLogicalTypes]( - s"$path/*.parquet", + path = dir.getAbsolutePath, conf = ParquetConfiguration.of( AvroReadSupport.AVRO_DATA_SUPPLIER -> classOf[CustomLogicalTypeSupplier] - ) + ), + suffix = ".parquet" ) .map(identity) should containInAnyOrder(records) @@ -247,56 +254,52 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { () } - it should "read with incomplete projection" in { - val dir = tmpDir - + it should "read with incomplete projection" in withTempDir { dir => val sc1 = ScioContext() val nestedRecords = (1 to 10).map(x => new Account(x, x.toString, x.toString, x.toDouble, AccountStatus.Active)) sc1 .parallelize(nestedRecords) - .saveAsParquetAvroFile(dir.toString) + .saveAsParquetAvroFile(dir.getAbsolutePath) sc1.run() val sc2 = ScioContext() val projection = Projection[Account](_.getName) - val data = - sc2.parquetAvroFile[Account](s"$dir/*.parquet", projection = projection) + val data = sc2.parquetAvroFile[Account]( + path = dir.getAbsolutePath, + projection = projection, + suffix = ".parquet" + ) val expected = nestedRecords.map(_.getName.toString) data.map(_.getName.toString) should containInAnyOrder(expected) data.flatMap(a => Some(a.getName.toString)) should containInAnyOrder(expected) sc2.run() - - FileUtils.deleteDirectory(dir) } - it should "read/write generic records" in { - val dir = tmpDir - + it should "read/write generic records" in withTempDir { dir => val genericRecords = (1 to 100).map(AvroUtils.newGenericRecord) val sc1 = ScioContext() implicit val coder = Coder.avroGenericRecordCoder(AvroUtils.schema) sc1 .parallelize(genericRecords) - .saveAsParquetAvroFile(dir.toString, numShards = 1, schema = AvroUtils.schema) + .saveAsParquetAvroFile(dir.getAbsolutePath, numShards = 1, schema = AvroUtils.schema) sc1.run() val files = dir.listFiles() files.length shouldBe 1 val sc2 = ScioContext() - val data: SCollection[GenericRecord] = - sc2.parquetAvroFile[GenericRecord](s"$dir/*.parquet", AvroUtils.schema) + val data: SCollection[GenericRecord] = sc2.parquetAvroFile[GenericRecord]( + path = dir.getAbsolutePath, + projection = AvroUtils.schema, + suffix = ".parquet" + ) data should containInAnyOrder(genericRecords) sc2.run() - - FileUtils.deleteDirectory(dir) } - it should "write windowed generic records to dynamic destinations" in { + it should "write windowed generic records to dynamic destinations" in withTempDir { dir => // This test follows the same pattern as com.spotify.scio.io.dynamic.DynamicFileTest - - val dir = tmpDir val genericRecords = (0 until 10).map(AvroUtils.newGenericRecord) val options = PipelineOptionsFactory.fromArgs("--streaming=true").create() val sc1 = ScioContext(options) @@ -309,7 +312,7 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { .timestampBy(x => new Instant(x.get("int_field").asInstanceOf[Int] * 1800000L), Duration.ZERO) .withFixedWindows(Duration.standardHours(1), Duration.ZERO, WindowOptions()) .saveAsParquetAvroFile( - dir.toString, + dir.getAbsolutePath, numShards = 1, schema = AvroUtils.schema, filenamePolicySupplier = FilenamePolicySupplier.filenamePolicySupplierOf( @@ -336,7 +339,7 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { val params = ParquetAvroIO.ReadParam[GenericRecord, GenericRecord]( - identity[GenericRecord] _, + identity[GenericRecord], AvroUtils.schema, null ) @@ -344,36 +347,32 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { (0 until 10) .sliding(2, 2) .zipWithIndex - .map(t => + .map { case (window, idx) => ( - "y=1970/m=01/d=01/h=%02d/part-0-of-1.parquet".format(t._2), - t._1.map(AvroUtils.newGenericRecord) + f"y=1970/m=01/d=01/h=$idx%02d/part-0-of-1.parquet", + window.map(AvroUtils.newGenericRecord) ) - ) + } .foreach { case (filename, records) => { val tap = ParquetAvroTap(s"$dir/$filename", params) tap.value.toList should contain theSameElementsAs records } } - - FileUtils.deleteDirectory(dir) } - it should "write generic records to dynamic destinations" in { - val dir = tmpDir - + it should "write generic records to dynamic destinations" in withTempDir { dir => val genericRecords = (1 to 100).map(AvroUtils.newGenericRecord) val sc = ScioContext() implicit val coder = Coder.avroGenericRecordCoder(AvroUtils.schema) sc.parallelize(genericRecords) .saveAsParquetAvroFile( - dir.toString, + dir.getAbsolutePath, numShards = 1, schema = AvroUtils.schema, filenamePolicySupplier = FilenamePolicySupplier.filenamePolicySupplierOf( unwindowed = (shardNumber: Int, numShards: Int) => - "part-%s-of-%s-with-custom-naming".format(shardNumber, numShards) + s"part-$shardNumber-of-$numShards-with-custom-naming" ) ) sc.run() @@ -382,66 +381,58 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { files.length shouldBe 1 files.head.getAbsolutePath should include("part-0-of-1-with-custom-naming.parquet") - val params = - ParquetAvroIO.ReadParam[GenericRecord, GenericRecord]( - identity[GenericRecord] _, - AvroUtils.schema, - null - ) + val params = ParquetAvroIO.ReadParam[GenericRecord, GenericRecord]( + identity[GenericRecord], + AvroUtils.schema, + null + ) val tap = ParquetAvroTap(files.head.getAbsolutePath, params) tap.value.toList should contain theSameElementsAs genericRecords - - FileUtils.deleteDirectory(dir) } - it should "throw exception when filename functions not correctly defined for dynamic destinations" in { - val dir = tmpDir - - val genericRecords = (1 to 100).map(AvroUtils.newGenericRecord) - implicit val coder = Coder.avroGenericRecordCoder(AvroUtils.schema) - - an[NotImplementedError] should be thrownBy { - val sc = ScioContext() - sc.parallelize(genericRecords) - .saveAsParquetAvroFile( - dir.toString, - numShards = 1, - schema = AvroUtils.schema, - filenamePolicySupplier = FilenamePolicySupplier.filenamePolicySupplierOf( - windowed = (_, _, _, _) => "test for exception handling" + it should "throw exception when filename functions not correctly defined for dynamic destinations" in withTempDir { + dir => + val genericRecords = (1 to 100).map(AvroUtils.newGenericRecord) + implicit val coder = Coder.avroGenericRecordCoder(AvroUtils.schema) + + an[NotImplementedError] should be thrownBy { + val sc = ScioContext() + sc.parallelize(genericRecords) + .saveAsParquetAvroFile( + dir.getAbsolutePath, + numShards = 1, + schema = AvroUtils.schema, + filenamePolicySupplier = FilenamePolicySupplier.filenamePolicySupplierOf( + windowed = (_, _, _, _) => "test for exception handling" + ) ) - ) - try { - sc.run() - } catch { - case e: PipelineExecutionException => - throw e.getCause + try { + sc.run() + } catch { + case e: PipelineExecutionException => + throw e.getCause + } } - } - an[NotImplementedError] should be thrownBy { - val sc = ScioContext() - sc.parallelize(genericRecords) - .timestampBy( - x => new Instant(x.get("int_field").asInstanceOf[Int] * 1800000L), - Duration.ZERO - ) - .withFixedWindows(Duration.standardHours(1), Duration.ZERO, WindowOptions()) - .saveAsParquetAvroFile( - dir.toString, - numShards = 1, - schema = AvroUtils.schema, - filenamePolicySupplier = FilenamePolicySupplier.filenamePolicySupplierOf( - unwindowed = (_, _) => "test for exception handling" + val e = the[PipelineExecutionException] thrownBy { + val sc = ScioContext() + sc.parallelize(genericRecords) + .timestampBy( + x => new Instant(x.get("int_field").asInstanceOf[Int] * 1800000L), + Duration.ZERO + ) + .withFixedWindows(Duration.standardHours(1), Duration.ZERO, WindowOptions()) + .saveAsParquetAvroFile( + dir.getAbsolutePath, + numShards = 1, + schema = AvroUtils.schema, + filenamePolicySupplier = FilenamePolicySupplier.filenamePolicySupplierOf( + unwindowed = (_, _) => "test for exception handling" + ) ) - ) - try { sc.run() - } catch { - case e: PipelineExecutionException => - throw e.getCause } - } + e.getCause shouldBe a[NotImplementedError] } it should "apply map functions to test input" in { diff --git a/scio-parquet/src/test/scala/com/spotify/scio/parquet/read/ParquetReadFnTest.scala b/scio-parquet/src/test/scala/com/spotify/scio/parquet/read/ParquetReadFnTest.scala index 78a624e5dd..fbf0d5b20f 100644 --- a/scio-parquet/src/test/scala/com/spotify/scio/parquet/read/ParquetReadFnTest.scala +++ b/scio-parquet/src/test/scala/com/spotify/scio/parquet/read/ParquetReadFnTest.scala @@ -19,24 +19,23 @@ package com.spotify.scio.parquet.read import com.spotify.scio.ScioContext import com.spotify.scio.parquet.ParquetConfiguration import com.spotify.scio.parquet.types._ +import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfterAll import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import java.io.File import java.nio.file.Files import java.util.UUID case class Record(strField: String) class ParquetReadFnTest extends AnyFlatSpec with Matchers with BeforeAndAfterAll { + private val testDir = Files.createTempDirectory("scio-test").toFile + private val testMultiDir = new File(testDir, "multi") + private val testSingleDir = new File(testDir, "single") private val records = (1 to 500).map(_ => Record(UUID.randomUUID().toString)).toList - private val directory = { - val d = Files.createTempDirectory("parquet") - d.toFile.deleteOnExit() - d.toString - } - override def beforeAll(): Unit = { // Multiple row-groups val multiRowGroupConf = ParquetConfiguration.of("parquet.block.size" -> 16) @@ -46,13 +45,14 @@ class ParquetReadFnTest extends AnyFlatSpec with Matchers with BeforeAndAfterAll val sc = ScioContext() val data = sc.parallelize(records) - - data.saveAsTypedParquetFile(s"$directory/multi", conf = multiRowGroupConf) - data.saveAsTypedParquetFile(s"$directory/single", conf = singleRowGroupConf) + data.saveAsTypedParquetFile(testMultiDir.getAbsolutePath, conf = multiRowGroupConf) + data.saveAsTypedParquetFile(testSingleDir.getAbsolutePath, conf = singleRowGroupConf) sc.run() } + override def afterAll(): Unit = FileUtils.deleteDirectory(testDir) + "Parquet ReadFn" should "read at file-level granularity for files with multiple row groups" in { val granularityConf = ParquetConfiguration.of( ParquetReadConfiguration.SplitGranularity -> ParquetReadConfiguration.SplitGranularityFile, @@ -61,7 +61,11 @@ class ParquetReadFnTest extends AnyFlatSpec with Matchers with BeforeAndAfterAll val sc = ScioContext() val tap = sc - .typedParquetFile[Record](s"$directory/multi/*.parquet", conf = granularityConf) + .typedParquetFile[Record]( + path = testMultiDir.getAbsolutePath, + conf = granularityConf, + suffix = ".parquet" + ) .materialize val readElements = sc.run().waitUntilDone().tap(tap).value.toList @@ -77,7 +81,11 @@ class ParquetReadFnTest extends AnyFlatSpec with Matchers with BeforeAndAfterAll val sc = ScioContext() val tap = sc - .typedParquetFile[Record](s"$directory/single/*.parquet", conf = granularityConf) + .typedParquetFile[Record]( + path = testSingleDir.getAbsolutePath, + conf = granularityConf, + suffix = ".parquet" + ) .materialize val readElements = sc.run().waitUntilDone().tap(tap).value.toList @@ -93,7 +101,11 @@ class ParquetReadFnTest extends AnyFlatSpec with Matchers with BeforeAndAfterAll val sc = ScioContext() val tap = sc - .typedParquetFile[Record](s"$directory/multi/*.parquet", conf = granularityConf) + .typedParquetFile[Record]( + path = testMultiDir.getAbsolutePath, + conf = granularityConf, + suffix = ".parquet" + ) .materialize val readElements = sc.run().waitUntilDone().tap(tap).value.toList @@ -109,7 +121,11 @@ class ParquetReadFnTest extends AnyFlatSpec with Matchers with BeforeAndAfterAll val sc = ScioContext() val tap = sc - .typedParquetFile[Record](s"$directory/single/*.parquet", conf = granularityConf) + .typedParquetFile[Record]( + path = testSingleDir.getAbsolutePath, + conf = granularityConf, + suffix = ".parquet" + ) .materialize val readElements = sc.run().waitUntilDone().tap(tap).value.toList diff --git a/scio-parquet/src/test/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIOTest.scala b/scio-parquet/src/test/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIOTest.scala index 81ba2aff9b..04f27a1401 100644 --- a/scio-parquet/src/test/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIOTest.scala +++ b/scio-parquet/src/test/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIOTest.scala @@ -19,7 +19,7 @@ package com.spotify.scio.parquet.tensorflow import com.google.protobuf.ByteString import com.spotify.scio.ScioContext -import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec, TapSpec} +import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec, ScioIOTest, TapSpec} import com.spotify.scio.testing.ScioIOSpec import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.values.SCollection @@ -29,6 +29,7 @@ import org.apache.parquet.filter2.predicate.FilterApi import org.scalatest.BeforeAndAfterAll import org.tensorflow.proto.example.{BytesList, Example, Feature, Features, FloatList, Int64List} +import java.nio.file.Files import scala.jdk.CollectionConverters._ object ParquetExampleHelper { @@ -92,17 +93,21 @@ object ParquetExampleHelper { class ParquetExampleIOFileNamePolicyTest extends FileNamePolicySpec[Example] { import ParquetExampleHelper._ - val extension: String = ".parquet" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override val suffix: String = ".parquet" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[Example] = { in.map(newExample) .saveAsParquetExampleFile( tmpDir, schema, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -118,18 +123,16 @@ class ParquetExampleIOFileNamePolicyTest extends FileNamePolicySpec[Example] { class ParquetExampleIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { import ParquetExampleHelper._ - private val dir = tmpDir + private val testDir = Files.createTempDirectory("scio-test-").toFile private val examples = (1 to 10).map(newExample) override protected def beforeAll(): Unit = { val sc = ScioContext() - sc.parallelize(examples) - .saveAsParquetExampleFile(dir.toString, schema) + sc.parallelize(examples).saveAsParquetExampleFile(testDir.getAbsolutePath, schema) sc.run() - () } - override protected def afterAll(): Unit = FileUtils.deleteDirectory(dir) + override protected def afterAll(): Unit = FileUtils.deleteDirectory(testDir) private val projection = Seq( "int64_req_1", @@ -169,7 +172,10 @@ class ParquetExampleIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAl it should "read Examples" in { val sc = ScioContext() - val data = sc.parquetExampleFile(s"$dir/*.parquet") + val data = sc.parquetExampleFile( + path = testDir.getAbsolutePath, + suffix = ".parquet" + ) data should containInAnyOrder(examples) sc.run() () @@ -177,7 +183,11 @@ class ParquetExampleIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAl it should "read Examples with projection" in { val sc = ScioContext() - val data = sc.parquetExampleFile(s"$dir/*.parquet", projection) + val data = sc.parquetExampleFile( + path = testDir.getAbsolutePath, + projection = projection, + suffix = ".parquet" + ) data should containInAnyOrder(examples.map(projectFields(projection))) sc.run() () @@ -185,7 +195,11 @@ class ParquetExampleIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAl it should "read Examples with predicate" in { val sc = ScioContext() - val data = sc.parquetExampleFile(s"$dir/*.parquet", predicate = predicate) + val data = sc.parquetExampleFile( + path = testDir.getAbsolutePath, + predicate = predicate, + suffix = ".parquet" + ) val expected = examples.filter { e => e.getFeatures.getFeatureOrThrow("int64_req_1").getInt64List.getValue(0) <= 5L && e.getFeatures.getFeatureOrThrow("float_req_2").getFloatList.getValue(0) >= 2.5f @@ -197,7 +211,12 @@ class ParquetExampleIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAl it should "read Examples with projection and predicate in non-test context" in { val sc = ScioContext() - val data = sc.parquetExampleFile(s"$dir/*.parquet", projection, predicate) + val data = sc.parquetExampleFile( + path = testDir.getAbsolutePath, + projection = projection, + predicate = predicate, + suffix = ".parquet" + ) val expected = examples .filter { e => e.getFeatures.getFeatureOrThrow("int64_req_1").getInt64List.getValue(0) <= 5L && diff --git a/scio-parquet/src/test/scala/com/spotify/scio/parquet/types/ParquetTypeIOTest.scala b/scio-parquet/src/test/scala/com/spotify/scio/parquet/types/ParquetTypeIOTest.scala index 6d5cc67cb4..e401ab94a1 100644 --- a/scio-parquet/src/test/scala/com/spotify/scio/parquet/types/ParquetTypeIOTest.scala +++ b/scio-parquet/src/test/scala/com/spotify/scio/parquet/types/ParquetTypeIOTest.scala @@ -19,7 +19,7 @@ package com.spotify.scio.parquet.types import java.{lang => jl} import com.spotify.scio.ScioContext -import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec, TapSpec} +import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec, ScioIOTest, TapSpec} import com.spotify.scio.testing.ScioIOSpec import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.values.SCollection @@ -27,17 +27,23 @@ import org.apache.commons.io.FileUtils import org.apache.parquet.filter2.predicate.FilterApi import org.scalatest.BeforeAndAfterAll +import java.nio.file.Files + class ParquetTypeIOFileNamePolicyTest extends FileNamePolicySpec[Wide] { - val extension: String = ".parquet" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override val suffix: String = ".parquet" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[Wide] = { in.map(i => Wide(i, i.toString, Some(i), (1 to i).toList)) .saveAsTypedParquetFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -51,18 +57,16 @@ class ParquetTypeIOFileNamePolicyTest extends FileNamePolicySpec[Wide] { } class ParquetTypeIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { - private val dir = tmpDir + private val testDir = Files.createTempDirectory("scio-test-").toFile private val records = (1 to 10).map(newRecord) override protected def beforeAll(): Unit = { val sc = ScioContext() - sc.parallelize(records) - .saveAsTypedParquetFile(dir.toString) + sc.parallelize(records).saveAsTypedParquetFile(testDir.getAbsolutePath) sc.run() - () } - override protected def afterAll(): Unit = FileUtils.deleteDirectory(dir) + override protected def afterAll(): Unit = FileUtils.deleteDirectory(testDir) private def newRecord(i: Int): Wide = Wide(i, i.toString, Some(i), (1 to i).toList) @@ -81,35 +85,45 @@ class ParquetTypeIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { it should "read case classes" in { val sc = ScioContext() - val data = sc.typedParquetFile[Wide](s"$dir/*.parquet") + val data = sc.typedParquetFile[Wide]( + path = testDir.getAbsolutePath, + suffix = ".parquet" + ) data should containInAnyOrder(records) sc.run() - () } it should "read case classes with projection" in { val sc = ScioContext() - val data = sc.typedParquetFile[Narrow](s"$dir/*.parquet") + val data = sc.typedParquetFile[Narrow]( + path = testDir.getAbsolutePath, + suffix = ".parquet" + ) data should containInAnyOrder(records.map(r => Narrow(r.i, r.r))) sc.run() - () } it should "read case classes with predicate" in { val sc = ScioContext() - val data = sc.typedParquetFile[Wide](s"$dir/*.parquet", predicate = predicate) + val data = sc.typedParquetFile[Wide]( + path = testDir.getAbsolutePath, + predicate = predicate, + suffix = ".parquet" + ) data should containInAnyOrder(records.filter(t => t.i <= 5 || t.o.exists(_ >= 95))) sc.run() - () } it should "read case classes with projection and predicate" in { val sc = ScioContext() - val data = sc.typedParquetFile[Narrow](s"$dir/*.parquet", predicate = predicate) + val data = sc.typedParquetFile[Narrow]( + path = testDir.getAbsolutePath, + predicate = predicate, + suffix = ".parquet" + ) val expected = records.filter(t => t.i <= 5 || t.o.exists(_ >= 95)).map(t => Narrow(t.i, t.r)) data should containInAnyOrder(expected) sc.run() - () } } diff --git a/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala b/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala index 39a254e0a7..699d29eb56 100644 --- a/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala +++ b/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala @@ -34,6 +34,7 @@ import org.scalatest.Assertion import org.scalatest.matchers.should.Matchers import org.scalatest.flatspec.AnyFlatSpec +import java.io.File import scala.jdk.CollectionConverters._ import scala.util.Random @@ -64,8 +65,8 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB) = ( - sc.avroFile(s"${inputs(0)}/*.avro", schema), - sc.avroFile(s"${inputs(1)}/*.avro", schema) + sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).cogroup(avroB.keyBy(keyFn)) @@ -89,14 +90,14 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { val (lhs, rhs) = ( SCollection.unionAll( List( - sc.avroFile(s"${inputs(0)}/*.avro", schema), - sc.avroFile(s"${inputs(1)}/*.avro", schema) + sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro") ) ), SCollection.unionAll( List( - sc.avroFile(s"${inputs(2)}/*.avro", schema), - sc.avroFile(s"${inputs(3)}/*.avro", schema) + sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(3).getAbsolutePath, schema, ".avro") ) ) ) @@ -116,9 +117,9 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB, avroC) = ( - sc.avroFile(s"${inputs(0)}/*.avro", schema), - sc.avroFile(s"${inputs(1)}/*.avro", schema), - sc.avroFile(s"${inputs(2)}/*.avro", schema) + sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).cogroup(avroB.keyBy(keyFn), avroC.keyBy(keyFn)) @@ -137,10 +138,10 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB, avroC, avroD) = ( - sc.avroFile(s"${inputs(0)}/*.avro", schema), - sc.avroFile(s"${inputs(1)}/*.avro", schema), - sc.avroFile(s"${inputs(2)}/*.avro", schema), - sc.avroFile(s"${inputs(3)}/*.avro", schema) + sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(3).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).cogroup(avroB.keyBy(keyFn), avroC.keyBy(keyFn), avroD.keyBy(keyFn)) @@ -150,7 +151,7 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { "sortMergeGroupByKey" should "have parity with Scio's groupBy" in withNumSources(1) { inputs => compareResults( _.sortMergeGroupByKey(classOf[Integer], mkRead(inputs(0))) - )(sc => sc.avroFile(s"${inputs(0)}/*.avro", schema).groupBy(keyFn)) + )(sc => sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro").groupBy(keyFn)) } "sortMergeJoin" should "have parity with a 2-way Join" in withNumSources(2) { inputs => @@ -163,8 +164,8 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB) = ( - sc.avroFile(s"${inputs(0)}/*.avro", schema), - sc.avroFile(s"${inputs(1)}/*.avro", schema) + sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).join(avroB.keyBy(keyFn)) @@ -185,11 +186,11 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB, avroC, avroD, avroE) = ( - sc.avroFile(s"${inputs(0)}/*.avro", schema), - sc.avroFile(s"${inputs(1)}/*.avro", schema), - sc.avroFile(s"${inputs(2)}/*.avro", schema), - sc.avroFile(s"${inputs(3)}/*.avro", schema), - sc.avroFile(s"${inputs(4)}/*.avro", schema) + sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(3).getAbsolutePath, schema, ".avro"), + sc.avroGenericFile(inputs(4).getAbsolutePath, schema, ".avro") ) MultiJoin.cogroup( @@ -205,9 +206,9 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { // Write randomly generated Avro records in SMB fashion to `numSources` destinations // in the local file system private def withNumSources(numSources: Int)( - testFn: Map[Int, Path] => Any + testFn: Map[Int, File] => Any ): Unit = { - val tempFolder = Files.createTempDirectory("smb") + val tempFolder = Files.createTempDirectory("smb").toFile val sc = ScioContext() val outputPaths = (0 until numSources).map { n => @@ -218,7 +219,7 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { gr } - val outputPath = tempFolder.resolve(s"source$n") + val outputPath = new File(tempFolder, s"source$n") sc.parallelize(data) .saveAsSortedBucket( @@ -237,14 +238,14 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { try { testFn(outputPaths) } finally { - FileUtils.deleteDirectory(tempFolder.toFile) + FileUtils.deleteDirectory(tempFolder) } } - private def mkRead(path: Path): SortedBucketIO.Read[GenericRecord] = + private def mkRead(path: File): SortedBucketIO.Read[GenericRecord] = AvroSortedBucketIO - .read(new TupleTag[GenericRecord](path.toString), schema) - .from(path.toString) + .read(new TupleTag[GenericRecord](path.getAbsolutePath), schema) + .from(path.getAbsolutePath) private def compareResults[T: Coder]( smbOp: ScioContext => SCollection[T] diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala index bcd0531169..7fbe5c4d4a 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala @@ -22,8 +22,8 @@ import com.spotify.scio.io.{FileStorage, Tap} import com.spotify.scio.values.SCollection /** Tap for Tensorflow TFRecord files. */ -final case class TFRecordFileTap(path: String) extends Tap[Array[Byte]] { - override def value: Iterator[Array[Byte]] = FileStorage(path).tfRecordFile +final case class TFRecordFileTap(path: String, suffix: String) extends Tap[Array[Byte]] { + override def value: Iterator[Array[Byte]] = FileStorage(path, suffix).tfRecordFile override def open(sc: ScioContext): SCollection[Array[Byte]] = sc.tfRecordFile(path) } diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index 6c265d2cee..6d10602dfa 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -50,28 +50,39 @@ final case class TFRecordIO(path: String) extends ScioIO[Array[Byte]] { override protected def write(data: SCollection[Array[Byte]], params: WriteP): Tap[Array[Byte]] = { TFRecordMethods.write(data, path, params) - tap(TFRecordIO.ReadParam(params.compression)) + tap(TFRecordIO.ReadParam(params)) } override def tap(params: ReadP): Tap[Array[Byte]] = - TFRecordMethods.tap(params, path) + TFRecordMethods.tap(path, params) } object TFRecordIO { - object ReadParam { - private[tensorflow] val DefaultCompression = Compression.AUTO + + private[tensorflow] object ReadParam { + val DefaultCompression = Compression.AUTO + val DefaultSuffix = null + + def apply(params: WriteParam): ReadParam = + new ReadParam( + compression = params.compression, + suffix = params.suffix + params.compression.getSuggestedSuffix + ) } - final case class ReadParam private (compression: Compression = ReadParam.DefaultCompression) + final case class ReadParam private ( + compression: Compression = ReadParam.DefaultCompression, + suffix: String = ReadParam.DefaultSuffix + ) - object WriteParam { - private[tensorflow] val DefaultSuffix = ".tfrecords" - private[tensorflow] val DefaultCompression = Compression.UNCOMPRESSED - private[tensorflow] val DefaultNumShards = 0 - private[tensorflow] val DefaultFilenamePolicySupplier = null - private[tensorflow] val DefaultPrefix = null - private[tensorflow] val DefaultShardNameTemplate = null - private[tensorflow] val DefaultTempDirectory = null + private[tensorflow] object WriteParam { + val DefaultSuffix = ".tfrecords" + val DefaultCompression = Compression.UNCOMPRESSED + val DefaultNumShards = 0 + val DefaultFilenamePolicySupplier = null + val DefaultPrefix = null + val DefaultShardNameTemplate = null + val DefaultTempDirectory = null } final case class WriteParam private ( @@ -97,17 +108,17 @@ final case class TFExampleIO(path: String) extends ScioIO[Example] { override protected def write(data: SCollection[Example], params: WriteP): Tap[Example] = { TFRecordMethods.write(data.map(_.toByteArray), path, params) - tap(TFExampleIO.ReadParam(params.compression)) + tap(TFExampleIO.ReadParam(params)) } override def tap(params: ReadP): Tap[Example] = - TFRecordMethods.tap(params, path).map(Example.parseFrom) + TFRecordMethods.tap(path, params).map(Example.parseFrom) } object TFExampleIO { type ReadParam = TFRecordIO.ReadParam - type WriteParam = TFRecordIO.WriteParam val ReadParam = TFRecordIO.ReadParam + type WriteParam = TFRecordIO.WriteParam val WriteParam = TFRecordIO.WriteParam } @@ -126,11 +137,16 @@ final case class TFSequenceExampleIO(path: String) extends ScioIO[SequenceExampl params: WriteP ): Tap[SequenceExample] = { TFRecordMethods.write(data.map(_.toByteArray), path, params) - tap(TFExampleIO.ReadParam(params.compression)) + tap( + TFExampleIO.ReadParam( + params.compression, + params.suffix + params.compression.getSuggestedSuffix + ) + ) } override def tap(params: ReadP): Tap[SequenceExample] = - TFRecordMethods.tap(params, path).map(SequenceExample.parseFrom) + TFRecordMethods.tap(path, params).map(SequenceExample.parseFrom) } private object TFRecordMethods { @@ -138,7 +154,7 @@ private object TFRecordMethods { sc.applyTransform( beam.TFRecordIO .read() - .from(path) + .from(ScioUtil.filePattern(path, params.suffix)) .withCompression(params.compression) ) @@ -159,7 +175,7 @@ private object TFRecordMethods { prefix = prefix, shardNameTemplate = shardNameTemplate, isWindowed = isWindowed - )(path, suffix) + )(ScioUtil.strippedPath(path), suffix) val dynamicDestinations = DynamicFileDestinations .constant(fp, SerializableFunctions.identity[Array[Byte]]) @@ -192,6 +208,6 @@ private object TFRecordMethods { () } - def tap(@unused read: TFRecordIO.ReadParam, path: String): Tap[Array[Byte]] = - TFRecordFileTap(ScioUtil.addPartSuffix(path)) + def tap(path: String, read: TFRecordIO.ReadParam): Tap[Array[Byte]] = + TFRecordFileTap(path, read.suffix) } diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/FileStorageSyntax.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/FileStorageSyntax.scala index 68f52d23d6..5c85dad21b 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/FileStorageSyntax.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/FileStorageSyntax.scala @@ -29,8 +29,7 @@ final class FileStorageOps(private val self: FileStorage) extends AnyVal { new Iterator[Array[Byte]] { private def wrapInputStream(in: InputStream) = TFRecordCodec.wrapInputStream(in, Compression.AUTO) - private val input = - self.getDirectoryInputStream(self.path, wrapInputStream) + private val input = self.getDirectoryInputStream(wrapInputStream) private var current: Array[Byte] = TFRecordCodec.read(input) override def hasNext: Boolean = current != null override def next(): Array[Byte] = { diff --git a/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFExampleIOTest.scala b/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFExampleIOTest.scala index 77733a3ea3..c7849bc733 100644 --- a/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFExampleIOTest.scala +++ b/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFExampleIOTest.scala @@ -17,8 +17,7 @@ package com.spotify.scio.tensorflow -import com.google.protobuf.ByteString -import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec} +import com.spotify.scio.io.{ClosedTap, FileNamePolicySpec, ScioIOTest} import com.spotify.scio.testing._ import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.values.SCollection @@ -28,9 +27,7 @@ import org.tensorflow.proto.example.Example object TFExampleIOTest { case class Record(i: Int, s: String) - implicit val efInt: ExampleField.Primitive[Int] = ExampleField.from[Long](_.toInt)(_.toLong) - implicit val efString: ExampleField.Primitive[String] = - ExampleField.from[ByteString](_.toStringUtf8)(ByteString.copyFromUtf8) + import magnolify.tensorflow.unsafe.{efInt, efString} val recordT: ExampleType[Record] = ExampleType[Record] } @@ -47,16 +44,20 @@ class TFExampleIOTest extends ScioIOSpec { class TFExampleIOFileNamePolicyTest extends FileNamePolicySpec[Example] { import TFExampleIOTest._ - val extension: String = ".tfrecords" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override val suffix: String = ".tfrecords" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[Example] = { in.map(x => recordT(Record(x, x.toString))) .saveAsTfRecordFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } diff --git a/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFTapTest.scala b/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFTapTest.scala index 2972f17749..144ced3c10 100644 --- a/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFTapTest.scala +++ b/scio-tensorflow/src/test/scala/com/spotify/scio/tensorflow/TFTapTest.scala @@ -23,15 +23,14 @@ import com.spotify.scio.io.TapSpec import org.apache.commons.io.FileUtils class TFTapTest extends TapSpec { - "SCollection" should "support saveAsTFRecordFile" in { + "SCollection" should "support saveAsTFRecordFile" in withTempDir { dir => val data = Seq.fill(100)(UUID.randomUUID().toString) import org.apache.beam.sdk.io.{Compression => CType} for (compressionType <- Seq(CType.UNCOMPRESSED, CType.DEFLATE, CType.GZIP)) { - val dir = tmpDir val t = runWithFileFuture { _.parallelize(data) .map(_.getBytes) - .saveAsTfRecordFile(dir.getPath, compression = compressionType) + .saveAsTfRecordFile(dir.getAbsolutePath, compression = compressionType) } verifyTap(t.map(new String(_)), data.toSet) FileUtils.deleteDirectory(dir) diff --git a/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala b/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala index 4d998b1d59..4970cad7e6 100644 --- a/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala +++ b/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala @@ -35,10 +35,13 @@ import org.apache.beam.sdk.transforms.windowing.{ } import org.apache.commons.io.FileUtils +import java.nio.file.Files import scala.reflect.ClassTag /** Trait for unit testing [[ScioIO]]. */ trait ScioIOSpec extends PipelineSpec { + def testPrefix: String = "foo-shard-" + def testShardNameTemplate: String = "SS-of-num-shards-NN" def testFilenamePolicySupplier( path: String, @@ -62,7 +65,7 @@ trait ScioIOSpec extends PipelineSpec { val unitary = paneInfo.isFirst && paneInfo.isLast if (unitary) "" else s"-pane${paneInfo.getTiming}-index${paneInfo.getIndex}" } - val filename = s"foo-shard-${shardNumber}-of-numShards-${numShards}${w}${p}" + val filename = s"foo-shard-$shardNumber-of-num-shards-$numShards$w$p" resource.getCurrentDirectory.resolve( filename + suffix + outputFileHints.getSuggestedFilenameSuffix, StandardResolveOptions.RESOLVE_FILE @@ -74,7 +77,7 @@ trait ScioIOSpec extends PipelineSpec { numShards: Int, outputFileHints: FileBasedSink.OutputFileHints ): ResourceId = { - val filename = s"foo-shard-${shardNumber}-of-numShards-${numShards}" + val filename = s"foo-shard-$shardNumber-of-num-shards-$numShards" resource.getCurrentDirectory.resolve( filename + suffix + outputFileHints.getSuggestedFilenameSuffix, StandardResolveOptions.RESOLVE_FILE @@ -83,18 +86,17 @@ trait ScioIOSpec extends PipelineSpec { } } - def listFiles(tmpDir: File): Array[String] = { + def listFiles(tmpDir: File): Seq[File] = { tmpDir .listFiles() - .filterNot(_.getName.startsWith("_")) - .filterNot(_.getName.startsWith(".")) - .map(_.toString) + .filterNot(x => x.getName.startsWith("_")) + .filterNot(x => x.getName.startsWith(".")) } def testTap[T: Coder]( xs: Seq[T] )(writeFn: (SCollection[T], String) => ClosedTap[T])(suffix: String): Unit = { - val tmpDir = new File(new File(CoreSysProps.TmpDir.value), "scio-test-" + UUID.randomUUID()) + val tmpDir = Files.createTempDirectory("scio-test-").toFile val sc = ScioContext() val data = sc.parallelize(xs) @@ -102,7 +104,7 @@ trait ScioIOSpec extends PipelineSpec { val scioResult = sc.run().waitUntilDone() val tap = scioResult.tap(closedTap) - val files = listFiles(tmpDir) + val files = listFiles(tmpDir).map(_.getName) tap.value.toSeq should contain theSameElementsAs xs tap.open(ScioContext()) should containInAnyOrder(xs) all(files) should endWith(suffix) diff --git a/scio-test/src/test/scala/com/spotify/scio/coders/AvroCoderTest.scala b/scio-test/src/test/scala/com/spotify/scio/coders/AvroCoderTest.scala index a3603b357b..345f30d70b 100644 --- a/scio-test/src/test/scala/com/spotify/scio/coders/AvroCoderTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/coders/AvroCoderTest.scala @@ -23,6 +23,7 @@ import com.spotify.scio.avro._ import com.spotify.scio.avro.StringFieldTest import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.values.SCollection +import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfterAll import java.nio.file.Files @@ -30,11 +31,7 @@ import scala.jdk.CollectionConverters._ class AvroCoderTest extends PipelineSpec with BeforeAndAfterAll { - private lazy val tempFolder = { - val dir = Files.createTempDirectory(getClass.getSimpleName) - dir.toFile.deleteOnExit() - dir - } + private val testDir = Files.createTempDirectory("scio-test-").toFile // Write Avro source to local file system override protected def beforeAll(): Unit = { @@ -49,10 +46,12 @@ class AvroCoderTest extends PipelineSpec with BeforeAndAfterAll { .setArrayField(ImmutableList.of("someListVal")) .build() ) - .saveAsAvroFile(tempFolder.toString) + .saveAsAvroFile(testDir.getAbsolutePath) sc.run() } + override def afterAll(): Unit = FileUtils.deleteDirectory(testDir) + // Verifies fix for BEAM-12628 Avro string encoding issue "Avro SpecificRecords" should "be read and serialized using java String field representations" in { def isJavaStr(t: Any): Boolean = { @@ -66,7 +65,7 @@ class AvroCoderTest extends PipelineSpec with BeforeAndAfterAll { val sc = ScioContext() val data: SCollection[StringFieldTest] = sc - .avroFile[StringFieldTest](tempFolder.resolve("*.avro").toString) + .avroSpecificFile[StringFieldTest](testDir.getAbsolutePath, ".avro") .map(identity) data should satisfy[StringFieldTest] { mappedData => diff --git a/scio-test/src/test/scala/com/spotify/scio/io/FileStorageTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/FileStorageTest.scala index 1b415ac49f..40465d6ff8 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/FileStorageTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/FileStorageTest.scala @@ -17,31 +17,44 @@ package com.spotify.scio.io -import java.nio.file.Files - +import org.apache.commons.io.FileUtils import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import java.io.File +import java.nio.file.Files + class FileStorageTest extends AnyFlatSpec with Matchers { - "FileStorage.isDone" should "return true on an empty directory" in { - val dir = Files.createTempDirectory("file-storage-") - dir.toFile.deleteOnExit() - FileStorage(dir.toFile.getAbsolutePath).isDone shouldBe true + + def withTempDirectory(test: File => Any): Unit = { + val dir = Files.createTempDirectory("file-storage-").toFile + try { + test(dir) + } finally { + FileUtils.deleteDirectory(dir) + } + } + + "FileStorage.isDone" should "return false on an empty directory" in withTempDirectory { dir => + FileStorage(dir.getAbsolutePath, ".txt").isDone() shouldBe false + } + + it should "return true for non sharded files" in withTempDirectory { dir => + new File(dir, "result.txt").createNewFile() + FileStorage(dir.getAbsolutePath, ".txt").isDone() shouldBe true } - it should "return false on non existing files" in { - val dir = Files.createTempDirectory("file-storage-") - dir.toFile.deleteOnExit() - FileStorage(dir.toFile.getAbsolutePath + "/*").isDone shouldBe false + it should "return true on existing files" in withTempDirectory { dir => + new File(dir, "part-01-of-02.txt").createNewFile() + new File(dir, "part-02-of-02.txt").createNewFile() + FileStorage(dir.getAbsolutePath, ".txt").isDone() shouldBe true } - it should "return true on existing files" in { - val dir = Files.createTempDirectory("file-storage-") - val f1 = Files.createTempFile(dir, "part", ".avro") - val f2 = Files.createTempFile(dir, "part", ".avro") - dir.toFile.deleteOnExit() - f1.toFile.deleteOnExit() - f2.toFile.deleteOnExit() - FileStorage(dir.toFile.getAbsolutePath + "/*.avro").isDone shouldBe true + it should "return false when shards are missing" in withTempDirectory { dir => + new File(dir, "pane-1-part-01-of-02.txt").createNewFile() + new File(dir, "pane-1-part-02-of-02.txt").createNewFile() + // missing pane-1-part-01-of-02.txt + new File(dir, "pane-2-part-02-of-02.txt").createNewFile() + FileStorage(dir.getAbsolutePath, ".txt").isDone() shouldBe false } } diff --git a/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala index 1a0dec0940..e3c1807efc 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala @@ -39,12 +39,12 @@ import java.util.UUID import scala.jdk.CollectionConverters._ object ScioIOTest { + val TestNumShards = 10 @AvroType.toSchema case class AvroRecord(i: Int, s: String, r: List[String]) } trait FileNamePolicySpec[T] extends ScioIOSpec { - val TestNumShards = 10 def mkIn( sc: ScioContext, @@ -66,7 +66,7 @@ trait FileNamePolicySpec[T] extends ScioIOSpec { // (windowed input, tmpDir, isBounded) write: (SCollection[Int], String, Boolean) => ClosedTap[T] )( - fileFn: Array[String] => Unit = _ => () + fileFn: Seq[String] => Unit = _ => () ): Unit = { val tmpDir = new File(new File(CoreSysProps.TmpDir.value), "scio-test-" + UUID.randomUUID()) @@ -75,7 +75,7 @@ trait FileNamePolicySpec[T] extends ScioIOSpec { write(in, tmpDir.getAbsolutePath, in.internal.isBounded == IsBounded.BOUNDED) sc.run().waitUntilDone() - fileFn(listFiles(tmpDir)) + fileFn(listFiles(tmpDir).map(_.getName)) FileUtils.deleteDirectory(tmpDir) } @@ -92,10 +92,12 @@ trait FileNamePolicySpec[T] extends ScioIOSpec { } } - def extension: String + def suffix: String def failSaves: Seq[SCollection[Int] => ClosedTap[T]] = Seq.empty def save( - filenamePolicySupplier: FilenamePolicySupplier = null + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[T] it should "throw when incompatible save parameters are used" in { @@ -105,7 +107,22 @@ trait FileNamePolicySpec[T] extends ScioIOSpec { it should "work with an unwindowed collection" in { testWindowingFilenames(_.parallelize(1 to 100), false, save()) { files => assert(files.length >= 1) - all(files) should (include("/part-") and include("-of-") and include(extension)) + all(files) should (startWith("part-") and + include("-of-") and + endWith(suffix)) + } + } + + it should "work with an unwindowed collection with custom prefix and shardNameTemplate" in { + testWindowingFilenames( + _.parallelize(1 to 100), + false, + save(prefix = testPrefix, shardNameTemplate = testShardNameTemplate) + ) { files => + assert(files.length >= 1) + all(files) should (startWith("foo-shard-") and + include("-of-num-shards-") and + endWith(suffix)) } } @@ -113,17 +130,19 @@ trait FileNamePolicySpec[T] extends ScioIOSpec { testWindowingFilenames(_.parallelize(1 to 100), false, save(testFilenamePolicySupplier)) { files => assert(files.length >= 1) - all(files) should (include("/foo-shard-") and include("-of-numShards-") and include( - extension - )) + all(files) should (startWith("foo-shard-") and + include("-of-num-shards-") and + endWith(suffix)) } } it should "work with a windowed collection" in { testWindowingFilenames(_.parallelize(1 to 100), true, save()) { files => assert(files.length >= 1) - all(files) should - (include("/part") and include("-of-") and include("-pane-") and include(extension)) + all(files) should (startWith("part") and + include("-of-") and + include("-pane-") and + endWith(suffix)) } } @@ -132,9 +151,11 @@ trait FileNamePolicySpec[T] extends ScioIOSpec { .addElements(1, (2 to 10): _*) .advanceWatermarkToInfinity() testWindowingFilenames(_.testStream(xxx), true, save()) { files => - assert(files.length == TestNumShards) - all(files) should - (include("/part") and include("-of-") and include("-pane-") and include(extension)) + assert(files.length == ScioIOTest.TestNumShards) + all(files) should (startWith("part") and + include("-of-") and + include("-pane-") and + endWith(suffix)) } } @@ -143,26 +164,30 @@ trait FileNamePolicySpec[T] extends ScioIOSpec { .addElements(1, (2 to 10): _*) .advanceWatermarkToInfinity() testWindowingFilenames(_.testStream(xxx), true, save(testFilenamePolicySupplier)) { files => - assert(files.length == TestNumShards) - all(files) should - (include("/foo-shard-") and include("-of-numShards-") and include("-window") and include( - extension - )) + assert(files.length == ScioIOTest.TestNumShards) + all(files) should (startWith("foo-shard-") and + include("-of-num-shards-") and + include("-window") and + endWith(suffix)) } } } class AvroIOFileNamePolicyTest extends FileNamePolicySpec[TestRecord] { - val extension: String = ".avro" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override def suffix: String = ".avro" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[TestRecord] = { in.map(AvroUtils.newSpecificRecord) .saveAsAvroFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -176,16 +201,20 @@ class AvroIOFileNamePolicyTest extends FileNamePolicySpec[TestRecord] { } class TextIOFileNamePolicyTest extends FileNamePolicySpec[String] { - val extension: String = ".txt" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override def suffix: String = ".txt" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[String] = { in.map(_.toString) .saveAsTextFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -201,16 +230,20 @@ class TextIOFileNamePolicyTest extends FileNamePolicySpec[String] { class ObjectIOFileNamePolicyTest extends FileNamePolicySpec[ScioIOTest.AvroRecord] { import ScioIOTest._ - val extension: String = ".obj.avro" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override def suffix: String = ".obj.avro" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[AvroRecord] = { in.map(x => AvroRecord(x, x.toString, (1 to x).map(_.toString).toList)) .saveAsObjectFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -224,16 +257,20 @@ class ObjectIOFileNamePolicyTest extends FileNamePolicySpec[ScioIOTest.AvroRecor } class ProtobufIOFileNamePolicyTest extends FileNamePolicySpec[TrackPB] { - val extension: String = ".protobuf.avro" - def save( - filenamePolicySupplier: FilenamePolicySupplier = null + override def suffix: String = ".protobuf.avro" + override def save( + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[TrackPB] = { in.map(x => TrackPB.newBuilder().setTrackId(x.toString).build()) .saveAsProtobufFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -247,16 +284,20 @@ class ProtobufIOFileNamePolicyTest extends FileNamePolicySpec[TrackPB] { } class BinaryIOFileNamePolicyTest extends FileNamePolicySpec[Nothing] { - val extension: String = ".bin" + override def suffix: String = ".bin" def save( - filenamePolicySupplier: FilenamePolicySupplier = null + filenamePolicySupplier: FilenamePolicySupplier = null, + prefix: String = null, + shardNameTemplate: String = null )(in: SCollection[Int], tmpDir: String, isBounded: Boolean): ClosedTap[Nothing] = { in.map(x => ByteBuffer.allocate(4).putInt(x).array) .saveAsBinaryFile( tmpDir, // TODO there is an exception with auto-sharding that fails for unbounded streams due to a GBK so numShards must be specified - numShards = if (isBounded) 0 else TestNumShards, - filenamePolicySupplier = filenamePolicySupplier + numShards = if (isBounded) 0 else ScioIOTest.TestNumShards, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate ) } @@ -277,11 +318,13 @@ class BinaryIOFileNamePolicyTest extends FileNamePolicySpec[Nothing] { class ScioIOTest extends ScioIOSpec { import ScioIOTest._ val TestNumShards = 10 + def relativeFiles(dir: File): Seq[String] = + listFiles(dir).map(_.toPath.relativize(dir.toPath)).sorted.map(_.toString) "AvroIO" should "work with SpecificRecord" in { val xs = (1 to 100).map(AvroUtils.newSpecificRecord) testTap(xs)(_.saveAsAvroFile(_))(".avro") - testJobTest(xs)(AvroIO[TestRecord](_))(_.avroFile(_))(_.saveAsAvroFile(_)) + testJobTest(xs)(AvroIO[TestRecord](_))(_.avroSpecificFile(_))(_.saveAsAvroFile(_)) } it should "work with GenericRecord" in { @@ -289,7 +332,7 @@ class ScioIOTest extends ScioIOSpec { implicit val coder = Coder.avroGenericRecordCoder(schema) val xs = (1 to 100).map(AvroUtils.newGenericRecord) testTap(xs)(_.saveAsAvroFile(_, schema = schema))(".avro") - testJobTest(xs)(AvroIO(_))(_.avroFile(_, schema))(_.saveAsAvroFile(_, schema = schema)) + testJobTest(xs)(AvroIO(_))(_.avroGenericFile(_, schema))(_.saveAsAvroFile(_, schema = schema)) } it should "work with typed Avro" in { @@ -325,7 +368,7 @@ class ScioIOTest extends ScioIOSpec { val out1TempDir = new File(new File(CoreSysProps.TmpDir.value), "scio-test-" + UUID.randomUUID()) var previousTransform: BAvroIO.Write[TestRecord] = write1 - .to(ScioUtil.pathWithPartPrefix(out1.getAbsolutePath)) + .to(ScioUtil.pathWithPrefix(out1.getAbsolutePath, null)) .withSuffix(suffix) .withNumShards(numShards) .withCodec(codec) @@ -364,8 +407,6 @@ class ScioIOTest extends ScioIOSpec { data.applyInternal(currentTransform) sc.run().waitUntilDone() - def relativeFiles(dir: File) = listFiles(dir).map(_.stripPrefix(dir.getAbsolutePath)).sorted - relativeFiles(out1) should contain theSameElementsAs relativeFiles(out2) FileUtils.deleteDirectory(out1TempDir) @@ -387,7 +428,7 @@ class ScioIOTest extends ScioIOSpec { val write1 = AvroTyped.writeTransform[AvroRecord]() var previousTransform = write1 - .to(ScioUtil.pathWithPartPrefix(out1.getAbsolutePath)) + .to(ScioUtil.pathWithPrefix(out1.getAbsolutePath, null)) .withSuffix(suffix) .withNumShards(numShards) .withCodec(codec) @@ -429,8 +470,6 @@ class ScioIOTest extends ScioIOSpec { data.applyInternal(currentTransform) sc.run().waitUntilDone() - def relativeFiles(dir: File) = listFiles(dir).map(_.stripPrefix(dir.getAbsolutePath)).sorted - relativeFiles(out1) should contain theSameElementsAs relativeFiles(out2) FileUtils.deleteDirectory(out1TempDir) diff --git a/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala index 9ccfca0fa7..ae05543c75 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala @@ -17,26 +17,25 @@ package com.spotify.scio.io -import java.io._ -import java.nio.ByteBuffer -import java.util.UUID import com.spotify.scio._ -import com.spotify.scio.avro._ import com.spotify.scio.avro.AvroUtils._ +import com.spotify.scio.avro._ +import com.spotify.scio.coders.Coder +import com.spotify.scio.options.ScioOptions import com.spotify.scio.proto.SimpleV2.{SimplePB => SimplePBV2} import com.spotify.scio.proto.SimpleV3.{SimplePB => SimplePBV3} import com.spotify.scio.testing.PipelineSpec -import com.spotify.scio.util.ScioUtil +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericData, GenericRecord} import org.apache.beam.sdk.util.SerializableUtils import org.apache.commons.compress.compressors.CompressorStreamFactory import org.apache.commons.io.{FileUtils, IOUtils} -import com.spotify.scio.coders.Coder -import com.spotify.scio.options.ScioOptions -import org.apache.avro.generic.GenericRecord -import org.apache.avro.generic.GenericData -import org.apache.avro.Schema +import java.io._ +import java.nio.ByteBuffer import java.nio.charset.StandardCharsets +import java.nio.file.Files +import java.util.UUID trait TapSpec extends PipelineSpec { def verifyTap[T: Coder](tap: Tap[T], expected: Set[T]): Unit = { @@ -60,8 +59,14 @@ trait TapSpec extends PipelineSpec { scioResult.tap(f) } - def tmpDir: File = - new File(new File(sys.props("java.io.tmpdir")), "scio-test-" + UUID.randomUUID()) + def withTempDir(test: File => Any): Unit = { + val dir = Files.createTempDirectory("scio-test-").toFile + try { + test(dir) + } finally { + FileUtils.deleteDirectory(dir) + } + } } class TapTest extends TapSpec { @@ -85,33 +90,28 @@ class TapTest extends TapSpec { verifyTap(t, expectedRecords) } - it should "support saveAsAvroFile with SpecificRecord" in { - val dir = tmpDir + it should "support saveAsAvroFile with SpecificRecord" in withTempDir { dir => val t = runWithFileFuture { _.parallelize(Seq(1, 2, 3)) .map(newSpecificRecord) - .saveAsAvroFile(dir.getPath) + .saveAsAvroFile(dir.getAbsolutePath) } verifyTap(t, Set(1, 2, 3).map(newSpecificRecord)) - FileUtils.deleteDirectory(dir) } - it should "support saveAsAvroFile with GenericRecord" in { - val dir = tmpDir + it should "support saveAsAvroFile with GenericRecord" in withTempDir { dir => val t = runWithFileFuture { _.parallelize(Seq(1, 2, 3)) .map(newGenericRecord) - .saveAsAvroFile(dir.getPath, schema = schema) + .saveAsAvroFile(dir.getAbsolutePath, schema = schema) } verifyTap(t, Set(1, 2, 3).map(newGenericRecord)) - FileUtils.deleteDirectory(dir) } - it should "support saveAsAvroFile with reflect record" in { + it should "support saveAsAvroFile with reflect record" in withTempDir { dir => import com.spotify.scio.coders.AvroBytesUtil implicit val coder = Coder.avroGenericRecordCoder(AvroBytesUtil.schema) - val dir = tmpDir val tap = runWithFileFuture { _.parallelize(Seq("a", "b", "c")) .map { s => @@ -119,7 +119,7 @@ class TapTest extends TapSpec { record.put("bytes", ByteBuffer.wrap(s.getBytes)) record } - .saveAsAvroFile(dir.getPath, schema = AvroBytesUtil.schema) + .saveAsAvroFile(dir.getAbsolutePath, schema = AvroBytesUtil.schema) } val result = tap @@ -129,41 +129,41 @@ class TapTest extends TapSpec { } verifyTap(result, Set("a", "b", "c")) - FileUtils.deleteDirectory(dir) } - it should "support saveAsTextFile" in { - val dir = tmpDir + it should "support saveAsTextFile" in withTempDir { dir => val t = runWithFileFuture { _.parallelize(Seq("a", "b", "c")) - .saveAsTextFile(dir.getPath) + .saveAsTextFile(dir.getAbsolutePath) } verifyTap(t, Set("a", "b", "c")) - FileUtils.deleteDirectory(dir) } - it should "support reading compressed text files" in { + it should "support reading compressed text files" in withTempDir { dir => val nFiles = 10 val nLines = 100 - val data = - Array.fill(nFiles)(Array.fill(nLines)(UUID.randomUUID().toString)) - for ((cType, ext) <- Seq(("gz", "gz"), ("bzip2", "bz2"))) { - val dir = tmpDir - dir.mkdir() - for (i <- 0 until nFiles) { - val file = new File(dir, "part-%05d-%05d.%s".format(i, nFiles, ext)) + val data = Array.fill(nFiles)(Array.fill(nLines)(UUID.randomUUID().toString)) + + Seq( + CompressorStreamFactory.GZIP -> ".gz", + CompressorStreamFactory.BZIP2 -> ".bz2" + ).map { case (cType, ext) => + val compressDir = new File(dir, cType) + compressDir.mkdir() + val suffix = ".txt" + ext + (0 until nFiles).foreach { f => + val file = new File(compressDir, f"part-$f%05d-of-$nFiles%05d$suffix%s") val os = new CompressorStreamFactory() .createCompressorOutputStream(cType, new FileOutputStream(file)) - data(i).foreach(l => IOUtils.write(l + "\n", os, StandardCharsets.UTF_8)) + IOUtils.write(data(f).mkString("", "\n", "\n"), os, StandardCharsets.UTF_8) os.close() } - verifyTap(TextTap(ScioUtil.addPartSuffix(dir.getPath, ext)), data.flatten.toSet) - FileUtils.deleteDirectory(dir) + + verifyTap(TextTap(compressDir.getAbsolutePath, suffix), data.flatten.toSet) } } - it should "support saveAsProtobuf proto version 2" in { - val dir = tmpDir + it should "support saveAsProtobuf proto version 2" in withTempDir { dir => val data = Seq(("a", 1L), ("b", 2L), ("c", 3L)) // use java protos otherwise we would have to pull in pb-scala def mkProto(t: (String, Long)): SimplePBV2 = @@ -175,11 +175,10 @@ class TapTest extends TapSpec { val t = runWithFileFuture { _.parallelize(data) .map(mkProto) - .saveAsProtobufFile(dir.getPath) + .saveAsProtobufFile(dir.getAbsolutePath) } val expected = data.map(mkProto).toSet verifyTap(t, expected) - FileUtils.deleteDirectory(dir) } // use java protos otherwise we would have to pull in pb-scala @@ -190,67 +189,63 @@ class TapTest extends TapSpec { .setTrackId(t._1) .build() - it should "support saveAsProtobuf proto version 3" in { - val dir = tmpDir + it should "support saveAsProtobuf proto version 3" in withTempDir { dir => val data = Seq(("a", 1L), ("b", 2L), ("c", 3L)) val t = runWithFileFuture { _.parallelize(data) .map(mkProto3) - .saveAsProtobufFile(dir.getPath) + .saveAsProtobufFile(dir.getAbsolutePath) } val expected = data.map(mkProto3).toSet verifyTap(t, expected) - FileUtils.deleteDirectory(dir) } - it should "support saveAsProtobuf write with nullableCoders" in { - val dir = tmpDir + it should "support saveAsProtobuf write with nullableCoders" in withTempDir { dir => val data = Seq(("a", 1L), ("b", 2L), ("c", 3L)) val actual = data.map(mkProto3) val t = runWithFileFuture { sc => sc.optionsAs[ScioOptions].setNullableCoders(true) sc.parallelize(actual) - .saveAsProtobufFile(dir.getPath) + .saveAsProtobufFile(dir.getAbsolutePath) } val expected = actual.toSet verifyTap(t, expected) val sc = ScioContext() - sc.protobufFile[SimplePBV3](s"$dir/*.avro") should containInAnyOrder(expected) + sc.protobufFile[SimplePBV3]( + path = dir.getAbsolutePath, + suffix = ".protobuf.avro" + ) should containInAnyOrder(expected) sc.run() - - FileUtils.deleteDirectory(dir) } - it should "support saveAsProtobuf read with nullableCoders" in { - val dir = tmpDir + it should "support saveAsProtobuf read with nullableCoders" in withTempDir { dir => val data = Seq(("a", 1L), ("b", 2L), ("c", 3L)) val actual = data.map(mkProto3) val t = runWithFileFuture { _.parallelize(actual) - .saveAsProtobufFile(dir.getPath) + .saveAsProtobufFile(dir.getAbsolutePath) } val expected = actual.toSet verifyTap(t, expected) val sc = ScioContext() sc.optionsAs[ScioOptions].setNullableCoders(true) - sc.protobufFile[SimplePBV3](s"$dir/*.avro") should containInAnyOrder(expected) + sc.protobufFile[SimplePBV3]( + path = dir.getAbsolutePath, + suffix = ".protobuf.avro" + ) should containInAnyOrder(expected) sc.run() - - FileUtils.deleteDirectory(dir) } - it should "keep parent after Tap.map" in { - val dir = tmpDir + it should "keep parent after Tap.map" in withTempDir { dir => val t = runWithFileFuture { _.parallelize(Seq(1, 2, 3)) - .saveAsTextFile(dir.getPath) + .saveAsTextFile(dir.getAbsolutePath) }.map(_.toInt) verifyTap(t, Set(1, 2, 3)) t.isInstanceOf[Tap[Int]] shouldBe true t.parent.get.isInstanceOf[Tap[_]] shouldBe true - FileUtils.deleteDirectory(dir) } it should "support waitForResult" in { diff --git a/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala index 4882fb34a5..d0ad78834a 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala @@ -17,11 +17,8 @@ package com.spotify.scio.io -import java.io.File import java.nio.file.{Files, Path} -import java.util.UUID -import com.spotify.scio.CoreSysProps import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers @@ -31,8 +28,7 @@ import scala.concurrent.duration._ class TapsTest extends AnyFlatSpec with Matchers { val data: Seq[String] = Seq("a", "b", "c") - private def tmpFile: Path = - new File(new File(CoreSysProps.TmpDir.value), "taps-test-" + UUID.randomUUID()).toPath + private def tmpFile: Path = Files.createTempFile("taps-test-", ".txt") private def writeText(p: Path, data: Seq[String]): Unit = { val writer = Files.newBufferedWriter(p) diff --git a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala index 8832f97531..118e30c119 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala @@ -18,11 +18,11 @@ package com.spotify.scio.io.dynamic import java.nio.file.{Files, Path} - import com.spotify.scio._ import com.spotify.scio.avro.AvroUtils._ import com.spotify.scio.avro._ import com.spotify.scio.coders.Coder +import com.spotify.scio.io.TapSpec import com.spotify.scio.proto.SimpleV2.SimplePB import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.values.WindowOptions @@ -30,40 +30,38 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory import org.apache.commons.io.FileUtils import org.joda.time.{Duration, Instant} +import java.io.File import scala.jdk.CollectionConverters._ -class DynamicFileTest extends PipelineSpec { - private def verifyOutput(path: Path, expected: String*): Unit = { +class DynamicFileTest extends PipelineSpec with TapSpec { + private def verifyOutput(path: File, expected: String*): Unit = { + val p = path.toPath val actual = Files - .list(path) + .list(p) .iterator() .asScala .filterNot(_.toFile.getName.startsWith(".")) .toSet - actual shouldBe expected.map(path.resolve).toSet - () + actual shouldBe expected.map(p.resolve).toSet } - "Dynamic File" should "support text files" in { - val tmpDir = Files.createTempDirectory("dynamic-io-") + "Dynamic File" should "support text files" in withTempDir { dir => val sc1 = ScioContext() sc1 .parallelize(1 to 10) - .saveAsDynamicTextFile(tmpDir.toString)(s => (s.toInt % 2).toString) + .saveAsDynamicTextFile(dir.getAbsolutePath)(s => (s.toInt % 2).toString) sc1.run() - verifyOutput(tmpDir, "0", "1") + verifyOutput(dir, "0", "1") val sc2 = ScioContext() - val lines0 = sc2.textFile(s"$tmpDir/0/*.txt") - val lines1 = sc2.textFile(s"$tmpDir/1/*.txt") + val lines0 = sc2.textFile(s"$dir/0/*.txt") + val lines1 = sc2.textFile(s"$dir/1/*.txt") lines0 should containInAnyOrder((1 to 10).filter(_ % 2 == 0).map(_.toString)) lines1 should containInAnyOrder((1 to 10).filter(_ % 2 == 1).map(_.toString)) sc2.run() - FileUtils.deleteDirectory(tmpDir.toFile) } - it should "support text files with windowing" in { - val tmpDir = Files.createTempDirectory("dynamic-io-") + it should "support text files with windowing" in withTempDir { dir => val options = PipelineOptionsFactory.fromArgs("--streaming=true").create() val sc1 = ScioContext(options) sc1 @@ -72,71 +70,65 @@ class DynamicFileTest extends PipelineSpec { // mysterious "Could not find proxy for val sc1" compiler error .timestampBy(x => new Instant(x * 60000L), Duration.ZERO) .withFixedWindows(Duration.standardMinutes(1), Duration.ZERO, WindowOptions()) - .saveAsDynamicTextFile(tmpDir.toString, 1)(s => (s.toInt % 2).toString) + .saveAsDynamicTextFile(dir.toString, 1)(s => (s.toInt % 2).toString) sc1.run() - verifyOutput(tmpDir, "0", "1") - Files.list(tmpDir.resolve("0")).iterator().asScala.size shouldBe 5 - Files.list(tmpDir.resolve("1")).iterator().asScala.size shouldBe 5 + verifyOutput(dir, "0", "1") + Files.list(dir.toPath.resolve("0")).iterator().asScala.size shouldBe 5 + Files.list(dir.toPath.resolve("1")).iterator().asScala.size shouldBe 5 val sc2 = ScioContext() - val lines0 = sc2.textFile(s"$tmpDir/0/*.txt") - val lines1 = sc2.textFile(s"$tmpDir/1/*.txt") + val lines0 = sc2.textFile(s"$dir/0/*.txt") + val lines1 = sc2.textFile(s"$dir/1/*.txt") lines0 should containInAnyOrder((1 to 10).filter(_ % 2 == 0).map(_.toString)) lines1 should containInAnyOrder((1 to 10).filter(_ % 2 == 1).map(_.toString)) (1 to 10).foreach { x => val p = x % 2 val t1 = new Instant(x * 60000L) val t2 = t1.plus(60000L) - val lines = sc2.textFile(s"$tmpDir/$p/part-$t1-$t2-*.txt") + val lines = sc2.textFile(s"$dir/$p/part-$t1-$t2-*.txt") lines should containSingleValue(x.toString) } sc2.run() - FileUtils.deleteDirectory(tmpDir.toFile) } - it should "support generic Avro files" in { - val tmpDir = Files.createTempDirectory("dynamic-io-") + it should "support generic Avro files" in withTempDir { dir => val sc1 = ScioContext() implicit val coder = Coder.avroGenericRecordCoder(schema) sc1 .parallelize(1 to 10) .map(newGenericRecord) - .saveAsDynamicAvroFile(tmpDir.toString, schema = schema) { r => + .saveAsDynamicAvroFile(dir.toString, schema = schema) { r => (r.get("int_field").toString.toInt % 2).toString } sc1.run() - verifyOutput(tmpDir, "0", "1") + verifyOutput(dir, "0", "1") val sc2 = ScioContext() - val lines0 = sc2.avroFile(s"$tmpDir/0/*.avro", schema) - val lines1 = sc2.avroFile(s"$tmpDir/1/*.avro", schema) + val lines0 = sc2.avroGenericFile(s"$dir/0/*.avro", schema) + val lines1 = sc2.avroGenericFile(s"$dir/1/*.avro", schema) lines0 should containInAnyOrder((1 to 10).filter(_ % 2 == 0).map(newGenericRecord)) lines1 should containInAnyOrder((1 to 10).filter(_ % 2 == 1).map(newGenericRecord)) sc2.run() - FileUtils.deleteDirectory(tmpDir.toFile) } - it should "support specific Avro files" in { - val tmpDir = Files.createTempDirectory("dynamic-io-") + it should "support specific Avro files" in withTempDir { dir => val sc1 = ScioContext() sc1 .parallelize(1 to 10) .map(newSpecificRecord) - .saveAsDynamicAvroFile(tmpDir.toString)(r => (r.getIntField % 2).toString) + .saveAsDynamicAvroFile(dir.toString)(r => (r.getIntField % 2).toString) sc1.run() - verifyOutput(tmpDir, "0", "1") + verifyOutput(dir, "0", "1") val sc2 = ScioContext() - val lines0 = sc2.avroFile[TestRecord](s"$tmpDir/0/*.avro") - val lines1 = sc2.avroFile[TestRecord](s"$tmpDir/1/*.avro") + val lines0 = sc2.avroSpecificFile[TestRecord](s"$dir/0/*.avro") + val lines1 = sc2.avroSpecificFile[TestRecord](s"$dir/1/*.avro") lines0 should containInAnyOrder((1 to 10).filter(_ % 2 == 0).map(newSpecificRecord)) lines1 should containInAnyOrder((1 to 10).filter(_ % 2 == 1).map(newSpecificRecord)) sc2.run() - FileUtils.deleteDirectory(tmpDir.toFile) } - it should "support Avro files with windowing" in { - val tmpDir = Files.createTempDirectory("dynamic-io-") + it should "support Avro files with windowing" in withTempDir { dir => val options = PipelineOptionsFactory.fromArgs("--streaming=true").create() val sc1 = ScioContext(options) sc1 @@ -146,46 +138,43 @@ class DynamicFileTest extends PipelineSpec { // mysterious "Could not find proxy for val sc1" compiler error .timestampBy(x => new Instant(x.getIntField * 60000L), Duration.ZERO) .withFixedWindows(Duration.standardMinutes(1), Duration.ZERO, WindowOptions()) - .saveAsDynamicAvroFile(tmpDir.toString, 1)(r => (r.getIntField % 2).toString) + .saveAsDynamicAvroFile(dir.toString, 1)(r => (r.getIntField % 2).toString) sc1.run() - verifyOutput(tmpDir, "0", "1") - Files.list(tmpDir.resolve("0")).iterator().asScala.size shouldBe 5 - Files.list(tmpDir.resolve("1")).iterator().asScala.size shouldBe 5 + verifyOutput(dir, "0", "1") + Files.list(dir.toPath.resolve("0")).iterator().asScala.size shouldBe 5 + Files.list(dir.toPath.resolve("1")).iterator().asScala.size shouldBe 5 val sc2 = ScioContext() - val records0 = sc2.avroFile[TestRecord](s"$tmpDir/0/*.avro") - val records1 = sc2.avroFile[TestRecord](s"$tmpDir/1/*.avro") + val records0 = sc2.avroSpecificFile[TestRecord](s"$dir/0/*.avro") + val records1 = sc2.avroSpecificFile[TestRecord](s"$dir/1/*.avro") records0 should containInAnyOrder((1 to 10).filter(_ % 2 == 0).map(newSpecificRecord)) records1 should containInAnyOrder((1 to 10).filter(_ % 2 == 1).map(newSpecificRecord)) (1 to 10).foreach { x => val p = x % 2 val t1 = new Instant(x * 60000L) val t2 = t1.plus(60000L) - val records = sc2.avroFile[TestRecord](s"$tmpDir/$p/part-$t1-$t2-*.avro") + val records = sc2.avroSpecificFile[TestRecord](s"$dir/$p/part-$t1-$t2-*.avro") records should containSingleValue(newSpecificRecord(x)) } sc2.run() - FileUtils.deleteDirectory(tmpDir.toFile) } - it should "support Proto files" in { - val tmpDir = Files.createTempDirectory("dynamic-io-") + it should "support Proto files" in withTempDir { dir => val sc1 = ScioContext() val mkProto = (x: Long) => SimplePB.newBuilder().setPlays(x).setTrackId(s"track$x").build() sc1 .parallelize(1L to 10L) .map(mkProto) - .saveAsDynamicProtobufFile(tmpDir.toString)(r => (r.getPlays % 2).toString) + .saveAsDynamicProtobufFile(dir.toString)(r => (r.getPlays % 2).toString) sc1.run() - verifyOutput(tmpDir, "0", "1") + verifyOutput(dir, "0", "1") val sc2 = ScioContext() - val lines0 = sc2.protobufFile[SimplePB](s"$tmpDir/0/*.protobuf") - val lines1 = sc2.protobufFile[SimplePB](s"$tmpDir/1/*.protobuf") + val lines0 = sc2.protobufFile[SimplePB](s"$dir/0/*.protobuf") + val lines1 = sc2.protobufFile[SimplePB](s"$dir/1/*.protobuf") lines0 should containInAnyOrder((1L to 10L).filter(_ % 2 == 0).map(mkProto)) lines1 should containInAnyOrder((1L to 10L).filter(_ % 2 == 1).map(mkProto)) sc2.run() - FileUtils.deleteDirectory(tmpDir.toFile) } } diff --git a/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala b/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala index e8d94df4bc..5a26c84a13 100644 --- a/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala @@ -52,7 +52,7 @@ object ObjectFileJob { object SpecificAvroFileJob { def main(cmdlineArgs: Array[String]): Unit = { val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.avroFile[TestRecord](args("input")) + sc.avroSpecificFile[TestRecord](args("input")) .saveAsAvroFile(args("output")) sc.run() () @@ -62,7 +62,7 @@ object SpecificAvroFileJob { object GenericAvroFileJob { def main(cmdlineArgs: Array[String]): Unit = { val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.avroFile(args("input"), AvroUtils.schema) + sc.avroGenericFile(args("input"), AvroUtils.schema) .saveAsAvroFile(args("output"), schema = AvroUtils.schema) sc.run() () diff --git a/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala b/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala index 792286debf..92cfe5cedc 100644 --- a/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala @@ -37,7 +37,6 @@ class ProtobufUtilTest extends AnyFlatSpec with Matchers { "ProtobufUtil" should "convert Message -> GenericRecords that can be written and read" in { val sc = ScioContext() - val dir = Files.createTempDirectory("protobuf-util-") val (path1, path2) = (new File(s"$dir/1"), new File(s"$dir/2")) path1.deleteOnExit() @@ -65,7 +64,7 @@ class ProtobufUtilTest extends AnyFlatSpec with Matchers { val result = sc.run().waitUntilDone() val (tapFromAvroWrite, tapFromProtoWrite) = ( - ObjectFileTap[TrackPB](ScioUtil.addPartSuffix(path1.getPath)), + ObjectFileTap[TrackPB](path1.getPath, ".protobuf"), protoWriteTap.get(result) ) diff --git a/site/src/main/paradox/FAQ.md b/site/src/main/paradox/FAQ.md index dc4e88f2e6..88b8b43bf4 100644 --- a/site/src/main/paradox/FAQ.md +++ b/site/src/main/paradox/FAQ.md @@ -94,9 +94,10 @@ object MyJob { def main(cmdlineArgs: Array[String]): Unit = { val (sc, args) = ContextAndArgs(cmdlineArgs) - val collections = - Seq("gs://bucket1/data/*.avro", "gs://bucket2/data/*.avro") - .map(sc.avroFile[TestRecord](_)) + val collections = Seq( + "gs://bucket1/data/", + "gs://bucket2/data/" + ).map(path => sc.avroSpecificFile[TestRecord](path, suffix=".avro")) val all = SCollection.unionAll(collections) } diff --git a/site/src/main/paradox/io/Avro.md b/site/src/main/paradox/io/Avro.md index acf1c18f72..307e5a49a0 100644 --- a/site/src/main/paradox/io/Avro.md +++ b/site/src/main/paradox/io/Avro.md @@ -15,7 +15,7 @@ import org.apache.avro.specific.SpecificRecord def sc: ScioContext = ??? // SpecificRecordClass is compiled from Avro schema files -def result = sc.avroFile[SpecificRecord]("gs://path-to-data/lake/part-*.avro") +def result = sc.avroSpecificFile[SpecificRecord]("gs://path-to-data/lake/part-*.avro") ``` ### Read Generic records @@ -31,7 +31,7 @@ def yourAvroSchema: Schema = ??? def sc: ScioContext = ??? -def result = sc.avroFile("gs://path-to-data/lake/part-*.avro", yourAvroSchema) +def result = sc.avroGenericFile("gs://path-to-data/lake/part-*.avro", yourAvroSchema) // `record` is of GenericRecord type ``` From a4cab234e7a9e2674f110b5f112511a2585192cb Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Tue, 23 May 2023 14:35:22 +0200 Subject: [PATCH 05/26] Apply format --- .../scio/elasticsearch/ElasticsearchIOBehavior.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/scio-elasticsearch/common/src/it/scala/com/spotify/scio/elasticsearch/ElasticsearchIOBehavior.scala b/scio-elasticsearch/common/src/it/scala/com/spotify/scio/elasticsearch/ElasticsearchIOBehavior.scala index 98456089de..21c485af44 100644 --- a/scio-elasticsearch/common/src/it/scala/com/spotify/scio/elasticsearch/ElasticsearchIOBehavior.scala +++ b/scio-elasticsearch/common/src/it/scala/com/spotify/scio/elasticsearch/ElasticsearchIOBehavior.scala @@ -30,10 +30,10 @@ object ElasticsearchIOBehavior { val Password = "changeme" final case class Person( - name: String, - lastname: String, - job_description: String - ) + name: String, + lastname: String, + job_description: String + ) val ImageName: DockerImageName = { // get the elasticsearch version for the java client properties From 9e94e5435305f88b7baf3423ab7da9dd054d861c Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Tue, 23 May 2023 14:35:41 +0200 Subject: [PATCH 06/26] Remove unused nowarn --- scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala index b675c42253..8d79c05fb7 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala @@ -139,7 +139,6 @@ final private[scio] class FileStorage(path: String, suffix: String) { } } - @nowarn("msg=parameter value path in method getDirectoryInputStream is never used") private[scio] def getDirectoryInputStream( wrapperFn: InputStream => InputStream = identity ): InputStream = { From 8f76e963502b87c9cd6101852d31cfcd754f4d1e Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Tue, 23 May 2023 14:51:24 +0200 Subject: [PATCH 07/26] Streamline file taps --- .../scala/com/spotify/scio/avro/taps.scala | 27 ++++++++++++++----- .../com/spotify/scio/io/FileStorage.scala | 3 +-- .../main/scala/com/spotify/scio/io/Taps.scala | 10 +++---- 3 files changed, 27 insertions(+), 13 deletions(-) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala index 2ac05fe102..81b3d9893f 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala @@ -99,7 +99,7 @@ case class ObjectFileTap[T: Coder]( final case class AvroTaps(self: Taps) { /** Get a `Future[Tap[T]]` of a Protobuf file. */ - def protobufFile[T: Coder](path: String, suffix: String)(implicit + def protobufFile[T: Coder](path: String, suffix: String = null)(implicit ev: T <:< Message ): Future[Tap[T]] = self.mkTap( @@ -109,28 +109,43 @@ final case class AvroTaps(self: Taps) { ) /** Get a `Future[Tap[T]]` of an object file. */ - def objectFile[T: Coder](path: String, suffix: String): Future[Tap[T]] = + def objectFile[T: Coder](path: String, suffix: String = null): Future[Tap[T]] = self.mkTap( s"Object file: $path", () => self.isPathDone(path, suffix), () => ObjectFileTap[T](path, suffix) ) + @deprecated("Use avroGenericFile instead", since = "0.13.0") + def avroFile(path: String, schema: Schema): Future[Tap[GenericRecord]] = + avroGenericFile(path, schema) + /** * Get a `Future[Tap[T]]` for [[org.apache.avro.generic.GenericRecord GenericRecord]] Avro file. */ - def avroFile(path: String, schema: Schema, suffix: String): Future[Tap[GenericRecord]] = + def avroGenericFile( + path: String, + schema: Schema, + suffix: String = null + ): Future[Tap[GenericRecord]] = self.mkTap( s"Avro: $path", () => self.isPathDone(path, suffix), () => GenericRecordTap(path, schema, suffix) ) + @deprecated("Use avroSpecificFile instead", since = "0.13.0") + def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String): Future[Tap[T]] = + avroSpecificFile[T](path) + /** * Get a `Future[Tap[T]]` for [[org.apache.avro.specific.SpecificRecord SpecificRecord]] Avro * file. */ - def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String, suffix: String): Future[Tap[T]] = + def avroSpecificFile[T <: SpecificRecord: ClassTag: Coder]( + path: String, + suffix: String = null + ): Future[Tap[T]] = self.mkTap( s"Avro: $path", () => self.isPathDone(path, suffix), @@ -140,11 +155,11 @@ final case class AvroTaps(self: Taps) { /** Get a `Future[Tap[T]]` for typed Avro source. */ def typedAvroFile[T <: HasAvroAnnotation: TypeTag: Coder]( path: String, - suffix: String + suffix: String = null ): Future[Tap[T]] = { val avroT = AvroType[T] import scala.concurrent.ExecutionContext.Implicits.global - avroFile(path, avroT.schema, suffix).map(_.map(avroT.fromGenericRecord)) + avroGenericFile(path, avroT.schema, suffix).map(_.map(avroT.fromGenericRecord)) } } diff --git a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala index 8d79c05fb7..7369dbe824 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala @@ -35,7 +35,6 @@ import org.apache.commons.io.IOUtils import java.nio.charset.StandardCharsets import java.util.regex.Pattern -import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import scala.util.Try @@ -111,7 +110,7 @@ final private[scio] class FileStorage(path: String, suffix: String) { val files = Try(listFiles).recover { case _: FileNotFoundException => Seq.empty }.get // best effort matching shardNumber and numShards - val shards = ("(.*)(\\d+)\\D+(\\d+)\\D*" ++ Option(suffix).map(Pattern.quote).getOrElse("")).r + val shards = ("(.*)(\\d+)\\D+(\\d+)\\D*" + Option(suffix).map(Pattern.quote).getOrElse("")).r val writtenShards = files .map(_.resourceId().toString) .flatMap { diff --git a/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala b/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala index d88eb2892f..5807b00962 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala @@ -31,15 +31,15 @@ class TapNotAvailableException(msg: String) extends Exception(msg) trait Taps { /** Get a `Future[Tap[String]]` for a text file. */ - def textFile(path: String): Future[Tap[String]] = + def textFile(path: String, suffix: String = null): Future[Tap[String]] = mkTap( s"Text: $path", - () => isPathDone(path, null), - () => TextTap(path, null) + () => isPathDone(path, suffix), + () => TextTap(path, suffix) ) - private[scio] def isPathDone(path: String, suffix: String): Boolean = - FileStorage(path, suffix).isDone + private[scio] def isPathDone(path: String, suffix: String = null): Boolean = + FileStorage(path, suffix).isDone() /** * Make a tap, to be implemented by concrete classes. From 525992cc6ce69242dd1b28a0d49540975d40e0e0 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Tue, 23 May 2023 15:32:13 +0200 Subject: [PATCH 08/26] Use more stable shard pattern --- .../scala/com/spotify/scio/io/FileStorage.scala | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala index 7369dbe824..799112b595 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala @@ -34,17 +34,21 @@ import org.apache.commons.compress.compressors.CompressorStreamFactory import org.apache.commons.io.IOUtils import java.nio.charset.StandardCharsets -import java.util.regex.Pattern import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import scala.util.Try private[scio] object FileStorage { + + private val ShardPattern = "(.*)-(\\d+)-of-(\\d+)(.*)".r + @inline final def apply(path: String, suffix: String): FileStorage = new FileStorage(path, suffix) } final private[scio] class FileStorage(path: String, suffix: String) { + import FileStorage._ + private def listFiles: Seq[Metadata] = FileSystems .`match`(ScioUtil.filePattern(path, suffix), EmptyMatchTreatment.DISALLOW) @@ -110,15 +114,16 @@ final private[scio] class FileStorage(path: String, suffix: String) { val files = Try(listFiles).recover { case _: FileNotFoundException => Seq.empty }.get // best effort matching shardNumber and numShards - val shards = ("(.*)(\\d+)\\D+(\\d+)\\D*" + Option(suffix).map(Pattern.quote).getOrElse("")).r + // relies of the used shardNameTemplate to be '$prefix-$shardNumber-of-$numShards$suffix' format val writtenShards = files .map(_.resourceId().toString) .flatMap { - case shards(prefix, shardNumber, numShards) => + case ShardPattern(prefix, shardNumber, numShards, suffix) => val part = for { idx <- Try(shardNumber.toInt) total <- Try(numShards.toInt) - key = (prefix, total) + // prefix or suffix may contain pane/window info + key = (prefix, suffix, total) } yield key -> idx part.toOption case _ => @@ -133,8 +138,8 @@ final private[scio] class FileStorage(path: String, suffix: String) { // assume progress is complete when shard info is not retrieved and files are present true } else { - // we managed to get shard info, verify all of then were written - writtenShards.forall { case ((_, total), idxs) => idxs.size == total } + // we managed to get shard info, verify that all of them were written + writtenShards.forall { case ((_, _, total), idxs) => idxs.size == total } } } From e6167cd22650f2d6f68a9f6032c83af126ec77a6 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 11:20:00 +0200 Subject: [PATCH 09/26] Scalafix --- .../scala/com/spotify/scio/io/FileStorage.scala | 13 +++++++------ .../src/main/scala/com/spotify/scio/io/TextIO.scala | 4 ++-- .../scio/io/dynamic/syntax/SCollectionSyntax.scala | 1 - .../com/spotify/scio/tensorflow/TFRecordIO.scala | 1 - .../scala/com/spotify/scio/testing/ScioIOSpec.scala | 1 - 5 files changed, 9 insertions(+), 11 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala index 799112b595..41d204033b 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala @@ -17,10 +17,6 @@ package com.spotify.scio.io -import java.io._ -import java.nio.ByteBuffer -import java.nio.channels.{Channels, SeekableByteChannel} -import java.util.Collections import com.google.api.services.bigquery.model.TableRow import com.spotify.scio.util.ScioUtil import org.apache.avro.Schema @@ -33,7 +29,12 @@ import org.apache.beam.sdk.io.fs.MatchResult.Metadata import org.apache.commons.compress.compressors.CompressorStreamFactory import org.apache.commons.io.IOUtils +import java.io._ +import java.nio.ByteBuffer +import java.nio.channels.{Channels, SeekableByteChannel} import java.nio.charset.StandardCharsets +import java.util.Collections +import scala.collection.compat._ // scalafix:ok import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import scala.util.Try @@ -114,7 +115,7 @@ final private[scio] class FileStorage(path: String, suffix: String) { val files = Try(listFiles).recover { case _: FileNotFoundException => Seq.empty }.get // best effort matching shardNumber and numShards - // relies of the used shardNameTemplate to be '$prefix-$shardNumber-of-$numShards$suffix' format + // relies on the shardNameTemplate to be of '$prefix-$shardNumber-of-$numShards$suffix' format val writtenShards = files .map(_.resourceId().toString) .flatMap { @@ -132,7 +133,7 @@ final private[scio] class FileStorage(path: String, suffix: String) { .groupMap(_._1)(_._2) if (files.isEmpty) { - // no files in folder + // no file matched false } else if (writtenShards.isEmpty) { // assume progress is complete when shard info is not retrieved and files are present diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index b1c11e3e9d..e578c450fa 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -132,8 +132,8 @@ object TextIO { ) private[scio] object WriteParam { - val DefaultHeader = Option.empty[String] - val DefaultFooter = Option.empty[String] + val DefaultHeader: Option[String] = None + val DefaultFooter: Option[String] = None val DefaultSuffix = ".txt" val DefaultNumShards = 0 val DefaultCompression = Compression.UNCOMPRESSED diff --git a/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala b/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala index b9f56a0f4e..fb5caa5266 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/dynamic/syntax/SCollectionSyntax.scala @@ -27,7 +27,6 @@ import org.apache.avro.file.CodecFactory import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificRecord import org.apache.beam.sdk.coders.StringUtf8Coder -import org.apache.beam.sdk.io.AvroIO.RecordFormatter import org.apache.beam.sdk.io.{Compression, FileIO} import org.apache.beam.sdk.{io => beam} diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index 6d10602dfa..154094ee10 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -36,7 +36,6 @@ import org.apache.beam.sdk.io.fs.ResourceId import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider import org.apache.beam.sdk.transforms.SerializableFunctions -import scala.annotation.unused final case class TFRecordIO(path: String) extends ScioIO[Array[Byte]] { override type ReadP = TFRecordIO.ReadParam diff --git a/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala b/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala index 4970cad7e6..ded1019d31 100644 --- a/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala +++ b/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala @@ -18,7 +18,6 @@ package com.spotify.scio.testing import java.io.File -import java.util.UUID import com.spotify.scio._ import com.spotify.scio.io._ import com.spotify.scio.values.SCollection From 4c90b5afe32f6a83ff6ae5c7b14b8bce1c47eca2 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 13:54:35 +0200 Subject: [PATCH 10/26] Streamline pattern parameter --- .../scala/com/spotify/scio/io/BinaryIO.scala | 8 +++---- .../main/scala/com/spotify/scio/io/Tap.scala | 3 ++- .../com/spotify/scio/extra/csv/CsvIO.scala | 24 ++++++++++++------- .../com/spotify/scio/extra/json/JsonIO.scala | 9 ++++--- .../scio/parquet/avro/ParquetAvroIO.scala | 5 ++-- .../parquet/tensorflow/ParquetExampleIO.scala | 8 ++++--- .../scio/parquet/types/ParquetTypeIO.scala | 5 ++-- .../spotify/scio/tensorflow/TFRecordIO.scala | 11 ++++++--- 8 files changed, 46 insertions(+), 27 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala index 3815430771..60f1f20421 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala @@ -114,7 +114,7 @@ final case class BinaryIO(path: String) extends ScioIO[Array[Byte]] { object BinaryIO { - private[scio] def openInputStreamsFor(path: String): Iterator[InputStream] = { + private[scio] def openInputStreamsFor(pattern: String): Iterator[InputStream] = { val factory = new CompressorStreamFactory() def wrapInputStream(in: InputStream) = { @@ -122,11 +122,11 @@ object BinaryIO { Try(factory.createCompressorInputStream(buffered)).getOrElse(buffered) } - listFiles(path).map(getObjectInputStream).map(wrapInputStream).iterator + listFiles(pattern).map(getObjectInputStream).map(wrapInputStream).iterator } - private def listFiles(path: String): Seq[Metadata] = - FileSystems.`match`(path).metadata().iterator.asScala.toSeq + private def listFiles(pattern: String): Seq[Metadata] = + FileSystems.`match`(pattern).metadata().iterator.asScala.toSeq private def getObjectInputStream(meta: Metadata): InputStream = Channels.newInputStream(FileSystems.open(meta.resourceId())) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala b/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala index 6f73538805..1e09e2ef52 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala @@ -111,9 +111,10 @@ private[scio] class MaterializeTap[T: Coder] private (path: String, coder: BCode } override def open(sc: ScioContext): SCollection[T] = sc.requireNotClosed { + val filePattern = ScioUtil.filePattern(path, MaterializeTap.Suffix) val read = AvroIO .readGenericRecords(AvroBytesUtil.schema) - .from(ScioUtil.filePattern(path, MaterializeTap.Suffix)) + .from(filePattern) sc.applyTransform(read).parDo(dofn) } } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala index 04f8518068..194cf0d2a0 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala @@ -196,23 +196,29 @@ object CsvIO { .withCompression(params.compression) .via(new CsvSink(params.csvConfiguration)) - private def read[T: HeaderDecoder: Coder](sc: ScioContext, path: String, params: ReadParam) = - sc.parallelize(Seq(ScioUtil.filePattern(path, params.suffix))) + private def read[T: HeaderDecoder: Coder](sc: ScioContext, path: String, params: ReadParam) = { + val filePattern = ScioUtil.filePattern(path, params.suffix) + val read = ParDo + .of(CsvIO.ReadDoFn[T](params.csvConfiguration)) + .asInstanceOf[PTransform[PCollection[beam.FileIO.ReadableFile], PCollection[T]]] + + sc.parallelize(Seq(filePattern)) .withName("Read CSV") .readFiles( - ParDo - .of(CsvIO.ReadDoFn[T](params.csvConfiguration)) - .asInstanceOf[PTransform[PCollection[beam.FileIO.ReadableFile], PCollection[T]]], - DirectoryTreatment.PROHIBIT, - params.compression + filesTransform = read, + directoryTreatment = DirectoryTreatment.PROHIBIT, + compression = params.compression ) + } final private class CsvTap[T: HeaderDecoder: Coder](path: String, params: ReadParam) extends Tap[T] { - override def value: Iterator[T] = + override def value: Iterator[T] = { + val filePattern = ScioUtil.filePattern(path, params.suffix) BinaryIO - .openInputStreamsFor(ScioUtil.filePattern(path, params.suffix)) + .openInputStreamsFor(filePattern) .flatMap(_.asUnsafeCsvReader[T](params.csvConfiguration).iterator) + } override def open(sc: ScioContext): SCollection[T] = CsvIO.read(sc, path, params) } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala index d678cc66b1..a1165a2ce8 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala @@ -58,10 +58,13 @@ final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO } override def tap(params: ReadP): Tap[T] = new Tap[T] { - override def value: Iterator[T] = - TextIO.textFile(ScioUtil.filePattern(path, params.suffix)).map(decodeJson) + override def value: Iterator[T] = { + val filePattern = ScioUtil.filePattern(path, params.suffix) + TextIO.textFile(filePattern).map(decodeJson) + } + override def open(sc: ScioContext): SCollection[T] = - JsonIO(ScioUtil.filePattern(path, params.suffix)).read(sc, params) + JsonIO(path).read(sc, params) } private def decodeJson(json: String): T = diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala index 5d552d3d87..de6a6d0019 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala @@ -151,7 +151,7 @@ final case class ParquetAvroIO[T: ClassTag: Coder](path: String) extends ScioIO[ } override def tap(params: ReadP): Tap[T] = - ParquetAvroTap(ScioUtil.filePattern(path, params.suffix), params) + ParquetAvroTap(path, params) } object ParquetAvroIO { @@ -329,7 +329,8 @@ case class ParquetAvroTap[A, T: ClassTag: Coder]( params: ParquetAvroIO.ReadParam[A, T] ) extends Tap[T] { override def value: Iterator[T] = { - val xs = FileSystems.`match`(path).metadata().asScala.toList + val filePattern = ScioUtil.filePattern(path, params.suffix) + val xs = FileSystems.`match`(filePattern).metadata().asScala.toList xs.iterator.flatMap { metadata => val reader = AvroParquetReader .builder[A](BeamInputFile.of(metadata.resourceId())) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala index ff503a01a6..6a419ab7ae 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala @@ -75,6 +75,7 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { params: ReadP ): SCollection[Example] = { val job = Job.getInstance(conf) + val filePattern = ScioUtil.filePattern(path, params.suffix) Option(params.projection).foreach { projection => ExampleParquetInputFormat.setFields(job, projection.asJava) @@ -91,7 +92,7 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { ParquetRead.read( ReadSupportFactory.example, new SerializableConfiguration(conf), - ScioUtil.filePattern(path, params.suffix), + filePattern, identity[Example] ) ).setCoder(coder) @@ -204,7 +205,7 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { } override def tap(params: ReadP): Tap[Example] = - ParquetExampleTap(ScioUtil.filePattern(path, params.suffix), params) + ParquetExampleTap(path, params) } object ParquetExampleIO { @@ -255,7 +256,8 @@ object ParquetExampleIO { final case class ParquetExampleTap(path: String, params: ParquetExampleIO.ReadParam) extends Tap[Example] { override def value: Iterator[Example] = { - val xs = FileSystems.`match`(path).metadata().asScala.toList + val filePattern = ScioUtil.filePattern(path, params.suffix) + val xs = FileSystems.`match`(filePattern).metadata().asScala.toList xs.iterator.flatMap { metadata => val reader = ExampleParquetReader .builder(BeamInputFile.of(metadata.resourceId())) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala index 98e66340f4..22b406b4e9 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala @@ -173,7 +173,7 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( } override def tap(params: ReadP): Tap[tapT.T] = - ParquetTypeTap(ScioUtil.filePattern(path, params.suffix), params) + ParquetTypeTap(path, params) } object ParquetTypeIO { @@ -224,7 +224,8 @@ case class ParquetTypeTap[T: ClassTag: Coder: ParquetType]( ) extends Tap[T] { override def value: Iterator[T] = { val tpe = implicitly[ParquetType[T]] - val xs = FileSystems.`match`(path).metadata().asScala.toList + val filePattern = ScioUtil.filePattern(path, params.suffix) + val xs = FileSystems.`match`(filePattern).metadata().asScala.toList xs.iterator.flatMap { metadata => val reader = tpe.readBuilder(BeamInputFile.of(metadata.resourceId())).build() new Iterator[T] { diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index 154094ee10..77574a9d62 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -36,7 +36,6 @@ import org.apache.beam.sdk.io.fs.ResourceId import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider import org.apache.beam.sdk.transforms.SerializableFunctions - final case class TFRecordIO(path: String) extends ScioIO[Array[Byte]] { override type ReadP = TFRecordIO.ReadParam override type WriteP = TFRecordIO.WriteParam @@ -149,13 +148,19 @@ final case class TFSequenceExampleIO(path: String) extends ScioIO[SequenceExampl } private object TFRecordMethods { - def read(sc: ScioContext, path: String, params: TFRecordIO.ReadParam): SCollection[Array[Byte]] = + def read( + sc: ScioContext, + path: String, + params: TFRecordIO.ReadParam + ): SCollection[Array[Byte]] = { + val filePattern = ScioUtil.filePattern(path, params.suffix) sc.applyTransform( beam.TFRecordIO .read() - .from(ScioUtil.filePattern(path, params.suffix)) + .from(filePattern) .withCompression(params.compression) ) + } private def tfWrite( path: String, From 5f4ef78f70b6e0fe200859eac4285d9edf04f123 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 14:13:15 +0200 Subject: [PATCH 11/26] Scalafix --- .../src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala | 2 +- .../scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala index 67305abb06..914ac0e5cb 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/csv/CsvIOTest.scala @@ -195,7 +195,7 @@ class CsvIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterEach { ) } - "CsvIO.ReadDoFn" should "be serializable" in withTempDir { dir => + "CsvIO.ReadDoFn" should "be serializable" in { implicit val decoder: HeaderDecoder[TestTuple] = HeaderDecoder.decoder("numericValue", "stringValue")(TestTuple.apply) SerializableUtils.serializeToByteArray( diff --git a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala index c37dbfc413..b8c848a301 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala @@ -17,7 +17,7 @@ package com.spotify.scio.io.dynamic -import java.nio.file.{Files, Path} +import java.nio.file.Files import com.spotify.scio._ import com.spotify.scio.avro.AvroUtils._ import com.spotify.scio.avro._ @@ -27,7 +27,6 @@ import com.spotify.scio.proto.SimpleV2.SimplePB import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.values.WindowOptions import org.apache.beam.sdk.options.PipelineOptionsFactory -import org.apache.commons.io.FileUtils import org.joda.time.{Duration, Instant} import java.io.File From 4c7abf34b0ae5fbc8cee9a2112c446217871b8fb Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 14:13:21 +0200 Subject: [PATCH 12/26] Use file path in tap without suffix --- .../src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala index df2472c0e6..c5757286b6 100644 --- a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala +++ b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala @@ -50,10 +50,9 @@ final class AvroTapIT extends AnyFlatSpec with Matchers { it should "read avro file" in { FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create) - val tap = AvroTaps(Taps()).avroFile( - "gs://data-integration-test-eu/avro-integration-test/folder-a/folder-b/", + val tap = AvroTaps(Taps()).avroGenericFile( + "gs://data-integration-test-eu/avro-integration-test/folder-a/folder-b/shakespeare.avro", schema = schema, - ".avro" ) val result = Await.result(tap, Duration.Inf) From 4e08ee508084b244f2cd49607c55bb1a40fa6849 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 15:09:19 +0200 Subject: [PATCH 13/26] Normalize all taps --- .../spotify/scio/avro/types/AvroTapIT.scala | 2 +- .../scala/com/spotify/scio/avro/AvroIO.scala | 13 ++-- .../scala/com/spotify/scio/avro/taps.scala | 61 ++++++++++--------- .../main/scala/com/spotify/scio/io/Tap.scala | 12 ++-- .../main/scala/com/spotify/scio/io/Taps.scala | 6 +- .../scala/com/spotify/scio/io/TextIO.scala | 2 +- .../extra/SortMergeBucketExampleTest.scala | 13 ++-- .../com/spotify/scio/extra/csv/CsvIO.scala | 5 +- .../spotify/scio/bigquery/BigQueryIO.scala | 2 +- .../com/spotify/scio/bigquery/taps.scala | 31 ++++++---- .../scio/parquet/avro/ParquetAvroIO.scala | 2 +- .../parquet/tensorflow/ParquetExampleIO.scala | 3 +- .../scio/parquet/types/ParquetTypeIO.scala | 5 +- .../scio/tensorflow/TFRecordFileTap.scala | 8 ++- .../spotify/scio/tensorflow/TFRecordIO.scala | 2 +- .../scala/com/spotify/scio/io/TapTest.scala | 4 +- .../spotify/scio/util/ProtobufUtilTest.scala | 2 +- 17 files changed, 95 insertions(+), 78 deletions(-) diff --git a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala index c5757286b6..796f5ce786 100644 --- a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala +++ b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala @@ -52,7 +52,7 @@ final class AvroTapIT extends AnyFlatSpec with Matchers { val tap = AvroTaps(Taps()).avroGenericFile( "gs://data-integration-test-eu/avro-integration-test/folder-a/folder-b/shakespeare.avro", - schema = schema, + schema = schema ) val result = Await.result(tap, Duration.Inf) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala index 9b19bebbaf..1b02278359 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala @@ -85,7 +85,7 @@ final case class ObjectFileIO[T: Coder](path: String) extends ScioIO[T] { } override def tap(read: ReadP): Tap[T] = - ObjectFileTap[T](path, read.suffix) + ObjectFileTap[T](path, read) } object ObjectFileIO { @@ -122,7 +122,7 @@ final case class ProtobufIO[T <: Message: ClassTag](path: String) extends ScioIO } override def tap(read: ReadP): Tap[T] = - ObjectFileTap[T](path, read.suffix)(protoCoder) + ObjectFileTap[T](path, read)(protoCoder) } object ProtobufIO { @@ -215,7 +215,7 @@ final case class SpecificRecordIO[T <: SpecificRecord: ClassTag: Coder](path: St } override def tap(read: ReadP): Tap[T] = - SpecificRecordTap[T](path, read.suffix) + SpecificRecordTap[T](path, read) } final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[GenericRecord] { @@ -266,7 +266,7 @@ final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[Ge } override def tap(read: ReadP): Tap[GenericRecord] = - GenericRecordTap(path, schema, read.suffix) + GenericRecordTap(path, schema, read) } /** @@ -302,7 +302,7 @@ final case class GenericRecordParseIO[T](path: String, parseFn: GenericRecord => override protected def write(data: SCollection[T], params: Nothing): Tap[T] = ??? override def tap(read: ReadP): Tap[T] = - GenericRecordParseTap[T](path, read.suffix, parseFn) + GenericRecordParseTap[T](path, parseFn, read) } object AvroIO { @@ -430,8 +430,7 @@ object AvroTyped { override def tap(read: ReadP): Tap[T] = { val avroT = AvroType[T] - GenericRecordTap(path, avroT.schema, read.suffix) - .map(avroT.fromGenericRecord) + GenericRecordTap(path, avroT.schema, read).map(avroT.fromGenericRecord) } } } diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala index 81b3d9893f..e88d09bf34 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala @@ -36,27 +36,27 @@ import scala.reflect.runtime.universe._ final case class GenericRecordTap( path: String, @transient private val schema: Schema, - suffix: String + params: AvroIO.ReadParam ) extends Tap[GenericRecord] { private lazy val s = Externalizer(schema) override def value: Iterator[GenericRecord] = - FileStorage(path, suffix).avroFile[GenericRecord](s.get) + FileStorage(path, params.suffix).avroFile[GenericRecord](s.get) override def open(sc: ScioContext): SCollection[GenericRecord] = - sc.avroGenericFile(path, s.get, suffix) + sc.read(GenericRecordIO(path, s.get))(params) } /** Tap for [[org.apache.avro.specific.SpecificRecord SpecificRecord]] Avro files. */ final case class SpecificRecordTap[T <: SpecificRecord: ClassTag: Coder]( path: String, - suffix: String + params: AvroIO.ReadParam ) extends Tap[T] { override def value: Iterator[T] = - FileStorage(path, suffix).avroFile[T]() + FileStorage(path, params.suffix).avroFile[T]() override def open(sc: ScioContext): SCollection[T] = - sc.avroSpecificFile[T](path, suffix) + sc.read(SpecificRecordIO[T](path))(params) } /** @@ -65,55 +65,58 @@ final case class SpecificRecordTap[T <: SpecificRecord: ClassTag: Coder]( */ final case class GenericRecordParseTap[T: Coder]( path: String, - suffix: String, - parseFn: GenericRecord => T + parseFn: GenericRecord => T, + params: AvroIO.ReadParam ) extends Tap[T] { override def value: Iterator[T] = - FileStorage(path, suffix) + FileStorage(path, params.suffix) // Read Avro GenericRecords, with the writer specified schema .avroFile[GenericRecord](schema = null) .map(parseFn) override def open(sc: ScioContext): SCollection[T] = - sc.parseAvroFile(path, suffix)(parseFn) + sc.read(GenericRecordParseIO[T](path, parseFn))(params) } /** * Tap for object files. Note that serialization is not guaranteed to be compatible across Scio * releases. */ -case class ObjectFileTap[T: Coder]( +final case class ObjectFileTap[T: Coder]( path: String, - suffix: String + params: AvroIO.ReadParam ) extends Tap[T] { override def value: Iterator[T] = { val elemCoder = CoderMaterializer.beamWithDefault(Coder[T]) - FileStorage(path, suffix) + FileStorage(path, params.suffix) .avroFile[GenericRecord](AvroBytesUtil.schema) .map(r => AvroBytesUtil.decode(elemCoder, r)) } override def open(sc: ScioContext): SCollection[T] = - sc.objectFile[T](path, suffix) + sc.read(ObjectFileIO[T](path))(params) } final case class AvroTaps(self: Taps) { /** Get a `Future[Tap[T]]` of a Protobuf file. */ - def protobufFile[T: Coder](path: String, suffix: String = null)(implicit - ev: T <:< Message + def protobufFile[T: Coder](path: String, params: ProtobufIO.ReadParam = ProtobufIO.ReadParam())( + implicit ev: T <:< Message ): Future[Tap[T]] = self.mkTap( s"Protobuf: $path", - () => self.isPathDone(path, suffix), - () => ObjectFileTap[T](path, suffix) + () => self.isPathDone(path, params.suffix), + () => ObjectFileTap[T](path, params) ) /** Get a `Future[Tap[T]]` of an object file. */ - def objectFile[T: Coder](path: String, suffix: String = null): Future[Tap[T]] = + def objectFile[T: Coder]( + path: String, + params: ObjectFileIO.ReadParam = ObjectFileIO.ReadParam() + ): Future[Tap[T]] = self.mkTap( s"Object file: $path", - () => self.isPathDone(path, suffix), - () => ObjectFileTap[T](path, suffix) + () => self.isPathDone(path, params.suffix), + () => ObjectFileTap[T](path, params) ) @deprecated("Use avroGenericFile instead", since = "0.13.0") @@ -126,12 +129,12 @@ final case class AvroTaps(self: Taps) { def avroGenericFile( path: String, schema: Schema, - suffix: String = null + params: AvroIO.ReadParam = AvroIO.ReadParam() ): Future[Tap[GenericRecord]] = self.mkTap( s"Avro: $path", - () => self.isPathDone(path, suffix), - () => GenericRecordTap(path, schema, suffix) + () => self.isPathDone(path, params.suffix), + () => GenericRecordTap(path, schema, params) ) @deprecated("Use avroSpecificFile instead", since = "0.13.0") @@ -144,22 +147,22 @@ final case class AvroTaps(self: Taps) { */ def avroSpecificFile[T <: SpecificRecord: ClassTag: Coder]( path: String, - suffix: String = null + params: AvroIO.ReadParam = AvroIO.ReadParam() ): Future[Tap[T]] = self.mkTap( s"Avro: $path", - () => self.isPathDone(path, suffix), - () => SpecificRecordTap[T](path, suffix) + () => self.isPathDone(path, params.suffix), + () => SpecificRecordTap[T](path, params) ) /** Get a `Future[Tap[T]]` for typed Avro source. */ def typedAvroFile[T <: HasAvroAnnotation: TypeTag: Coder]( path: String, - suffix: String = null + params: AvroIO.ReadParam = AvroIO.ReadParam() ): Future[Tap[T]] = { val avroT = AvroType[T] import scala.concurrent.ExecutionContext.Implicits.global - avroGenericFile(path, avroT.schema, suffix).map(_.map(avroT.fromGenericRecord)) + avroGenericFile(path, avroT.schema, params).map(_.map(avroT.fromGenericRecord)) } } diff --git a/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala b/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala index 1e09e2ef52..7d32f92b85 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/Tap.scala @@ -63,16 +63,18 @@ case object EmptyTap extends Tap[Nothing] { override def open(sc: ScioContext): SCollection[Nothing] = sc.empty[Nothing]() } -case class UnsupportedTap[T](msg: String) extends Tap[T] { +final case class UnsupportedTap[T](msg: String) extends Tap[T] { override def value: Iterator[T] = throw new UnsupportedOperationException(msg) override def open(sc: ScioContext): SCollection[T] = throw new UnsupportedOperationException(msg) } /** Tap for text files on local file system or GCS. */ -final case class TextTap(path: String, suffix: String) extends Tap[String] { - override def value: Iterator[String] = FileStorage(path, suffix).textFile +final case class TextTap(path: String, params: TextIO.ReadParam) extends Tap[String] { + override def value: Iterator[String] = + FileStorage(path, params.suffix).textFile - override def open(sc: ScioContext): SCollection[String] = sc.textFile(path, suffix = suffix) + override def open(sc: ScioContext): SCollection[String] = + sc.read(TextIO(path))(params) } final private[scio] class InMemoryTap[T: Coder] extends Tap[T] { @@ -82,7 +84,7 @@ final private[scio] class InMemoryTap[T: Coder] extends Tap[T] { sc.parallelize[T](InMemorySink.get(id)) } -private[scio] class MaterializeTap[T: Coder] private (path: String, coder: BCoder[T]) +final private[scio] class MaterializeTap[T: Coder] private (path: String, coder: BCoder[T]) extends Tap[T] { override def value: Iterator[T] = { diff --git a/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala b/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala index 5807b00962..7af541e70a 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/Taps.scala @@ -31,11 +31,11 @@ class TapNotAvailableException(msg: String) extends Exception(msg) trait Taps { /** Get a `Future[Tap[String]]` for a text file. */ - def textFile(path: String, suffix: String = null): Future[Tap[String]] = + def textFile(path: String, params: TextIO.ReadParam = TextIO.ReadParam()): Future[Tap[String]] = mkTap( s"Text: $path", - () => isPathDone(path, suffix), - () => TextTap(path, suffix) + () => isPathDone(path, params.suffix), + () => TextTap(path, params) ) private[scio] def isPathDone(path: String, suffix: String = null): Boolean = diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index e578c450fa..97f6530873 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -108,7 +108,7 @@ final case class TextIO(path: String) extends ScioIO[String] { } override def tap(params: ReadP): Tap[String] = - TextTap(path, params.suffix) + TextTap(path, params) } object TextIO { diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala index 4ebd0cf7ef..cc028f3929 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala @@ -19,9 +19,8 @@ package com.spotify.scio.examples.extra import java.io.File import java.nio.file.Files - -import com.spotify.scio.avro.{Account, GenericRecordTap, SpecificRecordTap} -import com.spotify.scio.io.TextTap +import com.spotify.scio.avro.{Account, AvroIO, GenericRecordTap, SpecificRecordTap} +import com.spotify.scio.io.{TextIO, TextTap} import org.scalatest.matchers.should.Matchers import org.scalatest.flatspec.AnyFlatSpec @@ -49,12 +48,12 @@ class SortMergeBucketExampleTest extends AnyFlatSpec with Matchers { GenericRecordTap( path = userDir.getAbsolutePath, schema = SortMergeBucketExample.UserDataSchema, - suffix = ".avro" + params = AvroIO.ReadParam(".avro") ).value.size shouldBe 500 SpecificRecordTap[Account]( path = accountDir.getAbsolutePath, - suffix = ".avro" + params = AvroIO.ReadParam(".avro") ).value.size shouldBe 500 SortMergeBucketJoinExample.main( @@ -67,7 +66,7 @@ class SortMergeBucketExampleTest extends AnyFlatSpec with Matchers { TextTap( path = joinOutputDir.getAbsolutePath, - suffix = ".txt" + params = TextIO.ReadParam(suffix = ".avro") ).value.size shouldBe 100 } @@ -90,7 +89,7 @@ class SortMergeBucketExampleTest extends AnyFlatSpec with Matchers { SpecificRecordTap[Account]( joinOutputDir.getAbsolutePath, - ".avro" + AvroIO.ReadParam(".avro") ).value .map(account => (account.getId, account.getType.toString)) .toList should contain theSameElementsAs (0 until 500).map((_, "combinedAmount")) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala index 194cf0d2a0..42be0ad088 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala @@ -211,7 +211,7 @@ object CsvIO { ) } - final private class CsvTap[T: HeaderDecoder: Coder](path: String, params: ReadParam) + final private case class CsvTap[T: HeaderDecoder: Coder](path: String, params: ReadParam) extends Tap[T] { override def value: Iterator[T] = { val filePattern = ScioUtil.filePattern(path, params.suffix) @@ -220,7 +220,8 @@ object CsvIO { .flatMap(_.asUnsafeCsvReader[T](params.csvConfiguration).iterator) } - override def open(sc: ScioContext): SCollection[T] = CsvIO.read(sc, path, params) + override def open(sc: ScioContext): SCollection[T] = + CsvIO.read(sc, path, params) } final private[scio] case class ReadDoFn[T: HeaderDecoder]( diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index db103ad828..a57cec0dc6 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -457,7 +457,7 @@ final case class TableRowJsonIO(path: String) extends ScioIO[TableRow] { } override def tap(read: ReadP): Tap[TableRow] = - TableRowJsonTap(path, read.suffix) + TableRowJsonTap(path, read) } object TableRowJsonIO { diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala index 5af9eeff7b..ab37470823 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/taps.scala @@ -34,10 +34,12 @@ import com.spotify.scio.bigquery.BigQueryTypedTable.Format import com.twitter.chill.Externalizer /** Tap for BigQuery TableRow JSON files. */ -final case class TableRowJsonTap(path: String, suffix: String) extends Tap[TableRow] { - override def value: Iterator[TableRow] = FileStorage(path, suffix).tableRowJsonFile +final case class TableRowJsonTap(path: String, params: TableRowJsonIO.ReadParam) + extends Tap[TableRow] { + override def value: Iterator[TableRow] = + FileStorage(path, params.suffix).tableRowJsonFile override def open(sc: ScioContext): SCollection[TableRow] = - sc.tableRowJsonFile(path) + sc.read(TableRowJsonIO(path))(params) } final case class BigQueryTypedTap[T: Coder](table: Table, fn: (GenericRecord, TableSchema) => T) @@ -50,7 +52,7 @@ final case class BigQueryTypedTap[T: Coder](table: Table, fn: (GenericRecord, Ta override def open(sc: ScioContext): SCollection[T] = { val ser = Externalizer(ts) - sc.bigQueryTable(table, Format.GenericRecord).map(gr => fn(gr, ser.get)) + sc.read(BigQueryTypedTable(table, Format.GenericRecord)).map(gr => fn(gr, ser.get)) } } @@ -59,7 +61,7 @@ final case class BigQueryTap(table: TableReference) extends Tap[TableRow] { override def value: Iterator[TableRow] = BigQuery.defaultInstance().tables.rows(Table.Ref(table)) override def open(sc: ScioContext): SCollection[TableRow] = - sc.bigQueryTable(Table.Ref(table)) + sc.read(BigQueryTypedTable(Table.Ref(table), Format.TableRow)) } /** Tap for BigQuery tables using storage api. */ @@ -68,10 +70,12 @@ final case class BigQueryStorageTap(table: Table, readOptions: TableReadOptions) override def value: Iterator[TableRow] = BigQuery.defaultInstance().tables.storageRows(table, readOptions) override def open(sc: ScioContext): SCollection[TableRow] = - sc.bigQueryStorage( - table, - readOptions.getSelectedFieldsList.asScala.toList, - readOptions.getRowRestriction + sc.read( + BigQueryStorage( + table, + readOptions.getSelectedFieldsList.asScala.toList, + Option(readOptions.getRowRestriction) + ) ) } @@ -129,11 +133,14 @@ final case class BigQueryTaps(self: Taps) { } /** Get a `Future[Tap[TableRow]]` for a BigQuery TableRow JSON file. */ - def tableRowJsonFile(path: String, suffix: String): Future[Tap[TableRow]] = + def tableRowJsonFile( + path: String, + params: TableRowJsonIO.ReadParam = TableRowJsonIO.ReadParam() + ): Future[Tap[TableRow]] = mkTap( s"TableRowJson: $path", - () => self.isPathDone(path, suffix), - () => TableRowJsonTap(path, suffix) + () => self.isPathDone(path, params.suffix), + () => TableRowJsonTap(path, params) ) def bigQueryStorage( diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala index de6a6d0019..92d5ee2677 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala @@ -324,7 +324,7 @@ object ParquetAvroIO { ) } -case class ParquetAvroTap[A, T: ClassTag: Coder]( +final case class ParquetAvroTap[A, T: ClassTag: Coder]( path: String, params: ParquetAvroIO.ReadParam[A, T] ) extends Tap[T] { diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala index 6a419ab7ae..187c4c495a 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala @@ -275,5 +275,6 @@ final case class ParquetExampleTap(path: String, params: ParquetExampleIO.ReadPa } } - override def open(sc: ScioContext): SCollection[Example] = sc.read(ParquetExampleIO(path))(params) + override def open(sc: ScioContext): SCollection[Example] = + sc.read(ParquetExampleIO(path))(params) } diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala index 22b406b4e9..47e91421d0 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala @@ -218,7 +218,7 @@ object ParquetTypeIO { ) } -case class ParquetTypeTap[T: ClassTag: Coder: ParquetType]( +final case class ParquetTypeTap[T: ClassTag: Coder: ParquetType]( path: String, params: ParquetTypeIO.ReadParam ) extends Tap[T] { @@ -240,5 +240,6 @@ case class ParquetTypeTap[T: ClassTag: Coder: ParquetType]( } } - override def open(sc: ScioContext): SCollection[T] = sc.read(ParquetTypeIO[T](path))(params) + override def open(sc: ScioContext): SCollection[T] = + sc.read(ParquetTypeIO[T](path))(params) } diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala index 7fbe5c4d4a..9693ecf64b 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordFileTap.scala @@ -22,8 +22,10 @@ import com.spotify.scio.io.{FileStorage, Tap} import com.spotify.scio.values.SCollection /** Tap for Tensorflow TFRecord files. */ -final case class TFRecordFileTap(path: String, suffix: String) extends Tap[Array[Byte]] { - override def value: Iterator[Array[Byte]] = FileStorage(path, suffix).tfRecordFile +final case class TFRecordFileTap(path: String, params: TFRecordIO.ReadParam) + extends Tap[Array[Byte]] { + override def value: Iterator[Array[Byte]] = + FileStorage(path, params.suffix).tfRecordFile override def open(sc: ScioContext): SCollection[Array[Byte]] = - sc.tfRecordFile(path) + sc.read(TFRecordIO(path))(params) } diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index 77574a9d62..303e22af82 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -213,5 +213,5 @@ private object TFRecordMethods { } def tap(path: String, read: TFRecordIO.ReadParam): Tap[Array[Byte]] = - TFRecordFileTap(path, read.suffix) + TFRecordFileTap(path, read) } diff --git a/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala index ae05543c75..2cbde8960e 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/TapTest.scala @@ -27,6 +27,7 @@ import com.spotify.scio.proto.SimpleV3.{SimplePB => SimplePBV3} import com.spotify.scio.testing.PipelineSpec import org.apache.avro.Schema import org.apache.avro.generic.{GenericData, GenericRecord} +import org.apache.beam.sdk.io.Compression import org.apache.beam.sdk.util.SerializableUtils import org.apache.commons.compress.compressors.CompressorStreamFactory import org.apache.commons.io.{FileUtils, IOUtils} @@ -159,7 +160,8 @@ class TapTest extends TapSpec { os.close() } - verifyTap(TextTap(compressDir.getAbsolutePath, suffix), data.flatten.toSet) + val params = TextIO.ReadParam(compression = Compression.detect(suffix), suffix = suffix) + verifyTap(TextTap(compressDir.getAbsolutePath, params), data.flatten.toSet) } } diff --git a/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala b/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala index 92cfe5cedc..8adbc9e607 100644 --- a/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/util/ProtobufUtilTest.scala @@ -64,7 +64,7 @@ class ProtobufUtilTest extends AnyFlatSpec with Matchers { val result = sc.run().waitUntilDone() val (tapFromAvroWrite, tapFromProtoWrite) = ( - ObjectFileTap[TrackPB](path1.getPath, ".protobuf"), + ObjectFileTap[TrackPB](path1.getPath, ObjectFileIO.ReadParam(".protobuf")), protoWriteTap.get(result) ) From 310adb4f3b975ac71042148413a22cb7c379a9cc Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 16:44:23 +0200 Subject: [PATCH 14/26] Create missing filePattern in IOs read --- scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala | 3 ++- .../test/scala/com/spotify/scio/extra/json/JsonIOTest.scala | 2 -- .../main/scala/com/spotify/scio/bigquery/BigQueryIO.scala | 6 ++++-- .../com/spotify/scio/parquet/types/ParquetTypeIO.scala | 6 ++++-- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala index 1b02278359..ac65360d9c 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala @@ -401,7 +401,8 @@ object AvroTyped { */ override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = { val avroT = AvroType[T] - val t = beam.AvroIO.readGenericRecords(avroT.schema).from(path) + val filePattern = ScioUtil.filePattern(path, params.suffix) + val t = beam.AvroIO.readGenericRecords(avroT.schema).from(filePattern) sc.applyTransform(t).map(avroT.fromGenericRecord) } diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala index 37cbc4b953..5ed7037fb7 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/json/JsonIOTest.scala @@ -23,9 +23,7 @@ import io.circe.Printer import com.spotify.scio._ import com.spotify.scio.io.TapSpec import com.spotify.scio.testing._ -import com.spotify.scio.util.ScioUtil import org.apache.beam.sdk.Pipeline.PipelineExecutionException -import org.apache.commons.io.FileUtils import scala.jdk.CollectionConverters._ import scala.io.Source diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index a57cec0dc6..a74d24b335 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -437,9 +437,11 @@ final case class TableRowJsonIO(path: String) extends ScioIO[TableRow] { override type WriteP = TableRowJsonIO.WriteParam final override val tapT: TapT.Aux[TableRow, TableRow] = TapOf[TableRow] - override protected def read(sc: ScioContext, params: ReadP): SCollection[TableRow] = - sc.applyTransform(TextIO.read().from(path)) + override protected def read(sc: ScioContext, params: ReadP): SCollection[TableRow] = { + val filePattern = ScioUtil.filePattern(path, params.suffix) + sc.applyTransform(TextIO.read().from(filePattern)) .map(e => ScioUtil.jsonFactory.fromString(e, classOf[TableRow])) + } override protected def write(data: SCollection[TableRow], params: WriteP): Tap[TableRow] = { data.transform_("BigQuery write") { diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala index 47e91421d0..f64bd3dd3c 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala @@ -72,6 +72,7 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( conf: Configuration, params: ReadP ): SCollection[T] = { + val filePattern = ScioUtil.filePattern(path, params.suffix) if (params.predicate != null) { ParquetInputFormat.setFilterPredicate(conf, params.predicate) } @@ -82,7 +83,7 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( ParquetRead.read( ReadSupportFactory.typed, new SerializableConfiguration(conf), - path, + filePattern, identity[T] ) ).setCoder(coder) @@ -91,7 +92,8 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( private def readLegacy(sc: ScioContext, conf: Configuration, params: ReadP): SCollection[T] = { val cls = ScioUtil.classOf[T] val job = Job.getInstance(conf) - GcsConnectorUtil.setInputPaths(sc, job, path) + val filePattern = ScioUtil.filePattern(path, params.suffix) + GcsConnectorUtil.setInputPaths(sc, job, filePattern) tpe.setupInput(job) job.getConfiguration.setClass("key.class", classOf[Void], classOf[Void]) job.getConfiguration.setClass("value.class", cls, cls) From f1a78c8a97193e854d26a34868f9559313741460 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 17:30:17 +0200 Subject: [PATCH 15/26] WIP --- .../scala/com/spotify/scio/io/TextIO.scala | 30 ++++------- .../com/spotify/scio/values/SCollection.scala | 13 ----- .../spotify/scio/bigquery/BigQueryIO.scala | 51 ++++--------------- .../bigquery/syntax/ScioContextSyntax.scala | 17 ++----- 4 files changed, 24 insertions(+), 87 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index 97f6530873..55d50f5c96 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -143,28 +143,18 @@ object TextIO { val DefaultTempDirectory = null } - final val DefaultWriteParam: WriteParam = WriteParam( - WriteParam.DefaultSuffix, - WriteParam.DefaultNumShards, - WriteParam.DefaultCompression, - WriteParam.DefaultHeader, - WriteParam.DefaultFooter, - WriteParam.DefaultFilenamePolicySupplier, - WriteParam.DefaultPrefix, - WriteParam.DefaultShardNameTemplate, - WriteParam.DefaultTempDirectory - ) + val DefaultWriteParam: WriteParam = WriteParam() final case class WriteParam private ( - suffix: String, - numShards: Int, - compression: Compression, - header: Option[String], - footer: Option[String], - filenamePolicySupplier: FilenamePolicySupplier, - prefix: String, - shardNameTemplate: String, - tempDirectory: String + suffix: String = WriteParam.DefaultSuffix, + numShards: Int = WriteParam.DefaultNumShards, + compression: Compression = WriteParam.DefaultCompression, + header: Option[String] = WriteParam.DefaultHeader, + footer: Option[String] = WriteParam.DefaultFooter, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, + shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, + tempDirectory: String = WriteParam.DefaultTempDirectory ) private[scio] def textFile(pattern: String): Iterator[String] = { diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala index 03661ba08b..067a0bc8d4 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala @@ -1568,19 +1568,6 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] { ClosedTap(MaterializeTap[T](path, context)) } - private[scio] def textOut( - path: String, - suffix: String, - numShards: Int, - compression: Compression - ) = - beam.TextIO - .write() - .to(ScioUtil.pathWithPrefix(path, null)) // TODO - .withSuffix(suffix) - .withNumShards(numShards) - .withCompression(compression) - /** * Save this SCollection as a text file. Note that elements must be of type `String`. * @group output diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index a74d24b335..5a8d475d27 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -25,10 +25,9 @@ import com.spotify.scio.bigquery.ExtendedErrorInfo._ import com.spotify.scio.bigquery.client.BigQuery import com.spotify.scio.bigquery.types.BigQueryType.HasAnnotation import com.spotify.scio.coders._ -import com.spotify.scio.io.{ScioIO, Tap, TapOf, TestIO} -import com.spotify.scio.util.{Functions, ScioUtil} +import com.spotify.scio.io.{ScioIO, Tap, TapOf, TapT, TestIO, TextIO} +import com.spotify.scio.util.{FilenamePolicySupplier, Functions, ScioUtil} import com.spotify.scio.values.SCollection -import com.spotify.scio.io.TapT import com.twitter.chill.ClosureCleaner import org.apache.avro.generic.GenericRecord import org.apache.beam.sdk.extensions.gcp.options.GcpOptions @@ -37,7 +36,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.{CreateDisposition, import org.apache.beam.sdk.io.gcp.bigquery.BigQueryAvroUtilsWrapper import org.apache.beam.sdk.io.gcp.bigquery.{BigQueryUtils, SchemaAndRecord} import org.apache.beam.sdk.io.gcp.{bigquery => beam} -import org.apache.beam.sdk.io.{Compression, TextIO} +import org.apache.beam.sdk.io.Compression import org.apache.beam.sdk.transforms.SerializableFunction import scala.jdk.CollectionConverters._ @@ -435,25 +434,16 @@ final case class BigQueryStorageSelect(sqlQuery: Query) extends BigQueryIO[Table final case class TableRowJsonIO(path: String) extends ScioIO[TableRow] { override type ReadP = TableRowJsonIO.ReadParam override type WriteP = TableRowJsonIO.WriteParam - final override val tapT: TapT.Aux[TableRow, TableRow] = TapOf[TableRow] + override val tapT: TapT.Aux[TableRow, TableRow] = TapOf[TableRow] override protected def read(sc: ScioContext, params: ReadP): SCollection[TableRow] = { - val filePattern = ScioUtil.filePattern(path, params.suffix) - sc.applyTransform(TextIO.read().from(filePattern)) + sc.read(TextIO(path))(params) .map(e => ScioUtil.jsonFactory.fromString(e, classOf[TableRow])) } override protected def write(data: SCollection[TableRow], params: WriteP): Tap[TableRow] = { data.transform_("BigQuery write") { - _.map(ScioUtil.jsonFactory.toString) - .applyInternal( - data.textOut( - path = path, - suffix = params.suffix, - numShards = params.numShards, - compression = params.compression - ) - ) + _.map(ScioUtil.jsonFactory.toString).write(TextIO(path))(params) } tap(TableRowJsonIO.ReadParam(params)) } @@ -464,33 +454,12 @@ final case class TableRowJsonIO(path: String) extends ScioIO[TableRow] { object TableRowJsonIO { - private[bigquery] object ReadParam { - val DefaultCompression = Compression.AUTO - val DefaultSuffix = null + type ReadParam = TextIO.ReadParam + val ReadParam = TextIO.ReadParam - def apply(params: WriteParam): ReadParam = - new ReadParam( - compression = params.compression, - suffix = params.suffix + params.compression.getSuggestedSuffix - ) - } - - final case class ReadParam private ( - compression: Compression = ReadParam.DefaultCompression, - suffix: String = null - ) + type WriteParam = TextIO.WriteParam + val WriteParam = TextIO.WriteParam - private[bigquery] object WriteParam { - val DefaultNumShards = 0 - val DefaultCompression = Compression.UNCOMPRESSED - val DefaultSuffix = ".json" - } - - final case class WriteParam private ( - numShards: Int = WriteParam.DefaultNumShards, - compression: Compression = WriteParam.DefaultCompression, - suffix: String = WriteParam.DefaultSuffix - ) } object BigQueryTyped { diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala index 0a195918f8..67e90c00e6 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala @@ -19,18 +19,7 @@ package com.spotify.scio.bigquery.syntax import com.spotify.scio.ScioContext import com.spotify.scio.bigquery.types.BigQueryType.HasAnnotation -import com.spotify.scio.bigquery.{ - BigQuerySelect, - BigQueryStorage, - BigQueryStorageSelect, - BigQueryType, - BigQueryTyped, - Query, - Source, - Table, - TableRow, - TableRowJsonIO -} +import com.spotify.scio.bigquery.{BigQuerySelect, BigQueryStorage, BigQueryStorageSelect, BigQueryType, BigQueryTyped, Query, Source, Table, TableRow, TableRowJsonIO} import com.spotify.scio.coders.Coder import com.spotify.scio.values._ @@ -40,6 +29,7 @@ import com.spotify.scio.bigquery.BigQueryTypedTable import com.spotify.scio.bigquery.BigQueryTypedTable.Format import com.spotify.scio.bigquery.coders.tableRowCoder import org.apache.beam.sdk.io.Compression +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment /** Enhanced version of [[ScioContext]] with BigQuery methods. */ final class ScioContextOps(private val self: ScioContext) extends AnyVal { @@ -212,9 +202,10 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { def tableRowJsonFile( path: String, compression: Compression = TableRowJsonIO.ReadParam.DefaultCompression, + emptyMatchTreatment: EmptyMatchTreatment = TableRowJsonIO.ReadParam.DefaultEmptyMatchTreatment, suffix: String = TableRowJsonIO.ReadParam.DefaultSuffix ): SCollection[TableRow] = - self.read(TableRowJsonIO(path))(TableRowJsonIO.ReadParam(compression, suffix)) + self.read(TableRowJsonIO(path))(TableRowJsonIO.ReadParam(compression, emptyMatchTreatment, suffix)) } trait ScioContextSyntax { From 3edc05be289f51e4582f87de98c6de62306bb791 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 18:50:27 +0200 Subject: [PATCH 16/26] Fix taps tests --- .../scala/com/spotify/scio/io/TextIO.scala | 2 +- .../extra/SortMergeBucketExampleTest.scala | 2 +- .../spotify/scio/bigquery/BigQueryIO.scala | 10 ++++---- .../bigquery/syntax/SCollectionSyntax.scala | 25 ++++++++++++++++--- .../bigquery/syntax/ScioContextSyntax.scala | 17 +++++++++++-- .../scala/com/spotify/scio/io/TapsTest.scala | 9 ++++--- .../scio/io/dynamic/DynamicFileTest.scala | 4 +-- 7 files changed, 51 insertions(+), 18 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index 55d50f5c96..82f2d309f7 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -74,7 +74,7 @@ final case class TextIO(path: String) extends ScioIO[String] { prefix = prefix, shardNameTemplate = shardNameTemplate, isWindowed = isWindowed - )(path, suffix) + )(ScioUtil.strippedPath(path), suffix) beam.TextIO .write() diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala index cc028f3929..7ba1a819b6 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/SortMergeBucketExampleTest.scala @@ -66,7 +66,7 @@ class SortMergeBucketExampleTest extends AnyFlatSpec with Matchers { TextTap( path = joinOutputDir.getAbsolutePath, - params = TextIO.ReadParam(suffix = ".avro") + params = TextIO.ReadParam(suffix = ".txt") ).value.size shouldBe 100 } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index 5a8d475d27..3a15f6ecbc 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -436,15 +436,15 @@ final case class TableRowJsonIO(path: String) extends ScioIO[TableRow] { override type WriteP = TableRowJsonIO.WriteParam override val tapT: TapT.Aux[TableRow, TableRow] = TapOf[TableRow] - override protected def read(sc: ScioContext, params: ReadP): SCollection[TableRow] = { + override protected def read(sc: ScioContext, params: ReadP): SCollection[TableRow] = sc.read(TextIO(path))(params) .map(e => ScioUtil.jsonFactory.fromString(e, classOf[TableRow])) - } override protected def write(data: SCollection[TableRow], params: WriteP): Tap[TableRow] = { - data.transform_("BigQuery write") { - _.map(ScioUtil.jsonFactory.toString).write(TextIO(path))(params) - } + data + .map(ScioUtil.jsonFactory.toString) + .withName("BigQuery write") + .write(TextIO(path))(params) tap(TableRowJsonIO.ReadParam(params)) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/SCollectionSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/SCollectionSyntax.scala index 9d7a9d581d..b5619893d6 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/SCollectionSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/SCollectionSyntax.scala @@ -31,10 +31,10 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.{CreateDisposition, import scala.reflect.ClassTag import scala.reflect.runtime.universe._ - import com.spotify.scio.bigquery.Table import com.spotify.scio.bigquery.BigQueryTypedTable import com.spotify.scio.bigquery.BigQueryTypedTable.Format +import com.spotify.scio.util.FilenamePolicySupplier import org.apache.avro.generic.GenericRecord /** Enhanced version of [[SCollection]] with BigQuery methods. */ @@ -73,10 +73,27 @@ final class SCollectionTableRowOps[T <: TableRow](private val self: SCollection[ def saveAsTableRowJsonFile( path: String, numShards: Int = TableRowJsonWriteParam.DefaultNumShards, - compression: Compression = TableRowJsonWriteParam.DefaultCompression + suffix: String = TableRowJsonWriteParam.DefaultSuffix, + compression: Compression = TableRowJsonWriteParam.DefaultCompression, + shardNameTemplate: String = TableRowJsonWriteParam.DefaultShardNameTemplate, + tempDirectory: String = TableRowJsonWriteParam.DefaultTempDirectory, + filenamePolicySupplier: FilenamePolicySupplier = + TableRowJsonWriteParam.DefaultFilenamePolicySupplier, + prefix: String = TableRowJsonWriteParam.DefaultPrefix ): ClosedTap[TableRow] = { - val param = TableRowJsonWriteParam(numShards, compression) - self.covary[TableRow].write(TableRowJsonIO(path))(param) + self + .covary[TableRow] + .write(TableRowJsonIO(path))( + TableRowJsonWriteParam( + suffix = suffix, + numShards = numShards, + compression = compression, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + tempDirectory = tempDirectory + ) + ) } } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala index 67e90c00e6..1c03da5c48 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/ScioContextSyntax.scala @@ -19,7 +19,18 @@ package com.spotify.scio.bigquery.syntax import com.spotify.scio.ScioContext import com.spotify.scio.bigquery.types.BigQueryType.HasAnnotation -import com.spotify.scio.bigquery.{BigQuerySelect, BigQueryStorage, BigQueryStorageSelect, BigQueryType, BigQueryTyped, Query, Source, Table, TableRow, TableRowJsonIO} +import com.spotify.scio.bigquery.{ + BigQuerySelect, + BigQueryStorage, + BigQueryStorageSelect, + BigQueryType, + BigQueryTyped, + Query, + Source, + Table, + TableRow, + TableRowJsonIO +} import com.spotify.scio.coders.Coder import com.spotify.scio.values._ @@ -205,7 +216,9 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { emptyMatchTreatment: EmptyMatchTreatment = TableRowJsonIO.ReadParam.DefaultEmptyMatchTreatment, suffix: String = TableRowJsonIO.ReadParam.DefaultSuffix ): SCollection[TableRow] = - self.read(TableRowJsonIO(path))(TableRowJsonIO.ReadParam(compression, emptyMatchTreatment, suffix)) + self.read(TableRowJsonIO(path))( + TableRowJsonIO.ReadParam(compression, emptyMatchTreatment, suffix) + ) } trait ScioContextSyntax { diff --git a/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala index d0ad78834a..130abcb9f6 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala @@ -18,10 +18,10 @@ package com.spotify.scio.io import java.nio.file.{Files, Path} - import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import java.io.File import scala.concurrent.Await import scala.concurrent.duration._ @@ -53,6 +53,7 @@ class TapsTest extends AnyFlatSpec with Matchers { it should "fail missing text file" in { TapsSysProps.Algorithm.value = "immediate" val f = tmpFile + Files.delete(f) val future = Taps().textFile(f.toString) future.isCompleted shouldBe true future.value.get.isSuccess shouldBe false @@ -78,9 +79,11 @@ class TapsTest extends AnyFlatSpec with Matchers { TapsSysProps.PollingInitialInterval.value = "1000" TapsSysProps.PollingMaximumAttempts.value = "1" val f = tmpFile + Files.delete(f) val future = Taps().textFile(f.toString) future.isCompleted shouldBe false - - Await.ready(future, 10.seconds) + val error = Await.result(future.failed, 10.seconds) + error shouldBe a[TapNotAvailableException] + error.getMessage shouldBe s"Text: $f" } } diff --git a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala index b8c848a301..6ed4862479 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala @@ -57,7 +57,7 @@ class DynamicFileTest extends PipelineSpec with TapSpec { .parallelize(1 to 10) .saveAsDynamicTextFile(dir.getAbsolutePath)(partitionIntegers) sc1.run() - verifyOutput(dir, "0", "1") + verifyOutput(dir, "even", "odd") val sc2 = ScioContext() val even = sc2.textFile(s"$dir/even/*.txt") @@ -203,7 +203,7 @@ class DynamicFileTest extends PipelineSpec with TapSpec { .map(mkProto) .saveAsDynamicProtobufFile(dir.getAbsolutePath)(r => partitionIntegers(r.getPlays.toInt % 2)) sc1.run() - verifyOutput(dir, "0", "1") + verifyOutput(dir, "even", "odd") val sc2 = ScioContext() val even = sc2.protobufFile[SimplePB](s"$dir/even/*.protobuf") From f0121682ee10ecc17a8b5a0bf66b32a81926bedb Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 20:40:12 +0200 Subject: [PATCH 17/26] Scalafix --- .../src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala | 1 - scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala | 1 - 2 files changed, 2 deletions(-) diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index 3a15f6ecbc..1db5be9872 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -36,7 +36,6 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.{CreateDisposition, import org.apache.beam.sdk.io.gcp.bigquery.BigQueryAvroUtilsWrapper import org.apache.beam.sdk.io.gcp.bigquery.{BigQueryUtils, SchemaAndRecord} import org.apache.beam.sdk.io.gcp.{bigquery => beam} -import org.apache.beam.sdk.io.Compression import org.apache.beam.sdk.transforms.SerializableFunction import scala.jdk.CollectionConverters._ diff --git a/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala index 130abcb9f6..6701099f25 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/TapsTest.scala @@ -21,7 +21,6 @@ import java.nio.file.{Files, Path} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -import java.io.File import scala.concurrent.Await import scala.concurrent.duration._ From c11fb5e24e5efb92458808df595a800cb2bf0b5c Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 24 May 2023 20:49:52 +0200 Subject: [PATCH 18/26] Fix factorized TableRowJson params --- .../spotify/scio/bigquery/BigQueryIO.scala | 31 ++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index 1db5be9872..5e055e3161 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -31,6 +31,7 @@ import com.spotify.scio.values.SCollection import com.twitter.chill.ClosureCleaner import org.apache.avro.generic.GenericRecord import org.apache.beam.sdk.extensions.gcp.options.GcpOptions +import org.apache.beam.sdk.io.Compression import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.{CreateDisposition, WriteDisposition} import org.apache.beam.sdk.io.gcp.bigquery.BigQueryAvroUtilsWrapper @@ -457,7 +458,35 @@ object TableRowJsonIO { val ReadParam = TextIO.ReadParam type WriteParam = TextIO.WriteParam - val WriteParam = TextIO.WriteParam + private[scio] object WriteParam { + val DefaultSuffix = ".json" + val DefaultNumShards = TextIO.WriteParam.DefaultNumShards + val DefaultCompression = TextIO.WriteParam.DefaultCompression + val DefaultFilenamePolicySupplier = TextIO.WriteParam.DefaultFilenamePolicySupplier + val DefaultPrefix = TextIO.WriteParam.DefaultPrefix + val DefaultShardNameTemplate = TextIO.WriteParam.DefaultShardNameTemplate + val DefaultTempDirectory = TextIO.WriteParam.DefaultTempDirectory + + def apply( + suffix: String = DefaultSuffix, + numShards: Int = DefaultNumShards, + compression: Compression = DefaultCompression, + filenamePolicySupplier: FilenamePolicySupplier = DefaultFilenamePolicySupplier, + prefix: String = DefaultPrefix, + shardNameTemplate: String = DefaultShardNameTemplate, + tempDirectory: String = DefaultTempDirectory + ): WriteParam = { + TextIO.WriteParam( + suffix = suffix, + numShards = numShards, + compression = compression, + filenamePolicySupplier = filenamePolicySupplier, + prefix = prefix, + shardNameTemplate = shardNameTemplate, + tempDirectory = tempDirectory + ) + } + } } From d269e45e6bee1dbf39dbeeb0ddf0547feea3fb88 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 31 May 2023 10:22:19 +0200 Subject: [PATCH 19/26] Open IO default params --- .../scala/com/spotify/scio/avro/AvroIO.scala | 14 ++++---- .../spotify/scio/cassandra/CassandraIO.scala | 4 +-- .../scala/com/spotify/scio/io/BinaryIO.scala | 18 +++++------ .../scala/com/spotify/scio/io/TextIO.scala | 26 +++++++-------- .../scio/elasticsearch/ElasticsearchIO.scala | 17 +++++----- .../com/spotify/scio/extra/csv/CsvIO.scala | 32 +++++++++---------- .../com/spotify/scio/extra/json/JsonIO.scala | 25 ++++++++------- .../spotify/scio/bigquery/BigQueryIO.scala | 29 +++++++++-------- .../spotify/scio/bigtable/BigTableIO.scala | 4 +-- .../com/spotify/scio/spanner/SpannerIO.scala | 8 ++--- .../com/spotify/scio/neo4j/Neo4jIO.scala | 4 +-- .../scio/parquet/avro/ParquetAvroIO.scala | 32 +++++++++---------- .../parquet/tensorflow/ParquetExampleIO.scala | 30 ++++++++--------- .../scio/parquet/types/ParquetTypeIO.scala | 30 ++++++++--------- .../com/spotify/scio/redis/RedisIO.scala | 4 +-- .../spotify/scio/tensorflow/TFRecordIO.scala | 24 +++++++------- 16 files changed, 151 insertions(+), 150 deletions(-) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala index ac65360d9c..6b97caf5f7 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala @@ -307,7 +307,7 @@ final case class GenericRecordParseIO[T](path: String, parseFn: GenericRecord => object AvroIO { - private[scio] object ReadParam { + object ReadParam { val DefaultSuffix: String = null def apply(params: WriteParam): ReadParam = new ReadParam( @@ -319,15 +319,15 @@ object AvroIO { suffix: String = ReadParam.DefaultSuffix ) - private[scio] object WriteParam { - val DefaultNumShards = 0 - val DefaultSuffix = ".avro" + object WriteParam { + val DefaultNumShards: Int = 0 + val DefaultSuffix: String = ".avro" val DefaultCodec: CodecFactory = CodecFactory.deflateCodec(6) val DefaultMetadata: Map[String, AnyRef] = Map.empty - val DefaultFilenamePolicySupplier = null - val DefaultPrefix = null + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null + val DefaultPrefix: String = null val DefaultShardNameTemplate: String = null - val DefaultTempDirectory = null + val DefaultTempDirectory: String = null } final case class WriteParam private ( diff --git a/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/CassandraIO.scala b/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/CassandraIO.scala index 02042a3630..4a7a1e9495 100644 --- a/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/CassandraIO.scala +++ b/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/CassandraIO.scala @@ -54,11 +54,11 @@ final case class CassandraIO[T](opts: CassandraOptions) extends ScioIO[T] { object CassandraIO { object WriteParam { - private[cassandra] val DefaultPar = 0 + val DefaultParallelism = 0 } final case class WriteParam[T] private ( outputFn: T => Seq[Any], - parallelism: Int = WriteParam.DefaultPar + parallelism: Int = WriteParam.DefaultParallelism ) } diff --git a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala index 60f1f20421..7d5c7f22d8 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/BinaryIO.scala @@ -131,18 +131,18 @@ object BinaryIO { private def getObjectInputStream(meta: Metadata): InputStream = Channels.newInputStream(FileSystems.open(meta.resourceId())) - private[scio] object WriteParam { - val DefaultPrefix = null - val DefaultSuffix = ".bin" - val DefaultNumShards = 0 - val DefaultCompression = Compression.UNCOMPRESSED - val DefaultHeader = Array.emptyByteArray - val DefaultFooter = Array.emptyByteArray + object WriteParam { + val DefaultPrefix: String = null + val DefaultSuffix: String = ".bin" + val DefaultNumShards: Int = 0 + val DefaultCompression: Compression = Compression.UNCOMPRESSED + val DefaultHeader: Array[Byte] = Array.emptyByteArray + val DefaultFooter: Array[Byte] = Array.emptyByteArray val DefaultShardNameTemplate: String = null val DefaultFramePrefix: Array[Byte] => Array[Byte] = _ => Array.emptyByteArray val DefaultFrameSuffix: Array[Byte] => Array[Byte] = _ => Array.emptyByteArray - val DefaultTempDirectory = null - val DefaultFilenamePolicySupplier = null + val DefaultTempDirectory: String = null + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null } final case class WriteParam private ( diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index 82f2d309f7..f6ee078199 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -113,12 +113,12 @@ final case class TextIO(path: String) extends ScioIO[String] { object TextIO { - private[scio] object ReadParam { - private[scio] val DefaultCompression = Compression.AUTO - private[scio] val DefaultEmptyMatchTreatment = EmptyMatchTreatment.DISALLOW - private[scio] val DefaultSuffix = null + object ReadParam { + val DefaultCompression: Compression = Compression.AUTO + val DefaultEmptyMatchTreatment: EmptyMatchTreatment = EmptyMatchTreatment.DISALLOW + val DefaultSuffix: String = null - def apply(params: WriteParam): ReadParam = + private[io] def apply(params: WriteParam): ReadParam = new ReadParam( compression = params.compression, suffix = params.suffix + params.compression.getSuggestedSuffix @@ -131,16 +131,16 @@ object TextIO { suffix: String = ReadParam.DefaultSuffix ) - private[scio] object WriteParam { + object WriteParam { val DefaultHeader: Option[String] = None val DefaultFooter: Option[String] = None - val DefaultSuffix = ".txt" - val DefaultNumShards = 0 - val DefaultCompression = Compression.UNCOMPRESSED - val DefaultFilenamePolicySupplier = null - val DefaultPrefix = null - val DefaultShardNameTemplate = null - val DefaultTempDirectory = null + val DefaultSuffix: String = ".txt" + val DefaultNumShards: Int = 0 + val DefaultCompression: Compression = Compression.UNCOMPRESSED + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null + val DefaultPrefix: String = null + val DefaultShardNameTemplate: String = null + val DefaultTempDirectory: String = null } val DefaultWriteParam: WriteParam = WriteParam() diff --git a/scio-elasticsearch/common/src/main/scala/com/spotify/scio/elasticsearch/ElasticsearchIO.scala b/scio-elasticsearch/common/src/main/scala/com/spotify/scio/elasticsearch/ElasticsearchIO.scala index 5401c52926..23b3507371 100644 --- a/scio-elasticsearch/common/src/main/scala/com/spotify/scio/elasticsearch/ElasticsearchIO.scala +++ b/scio-elasticsearch/common/src/main/scala/com/spotify/scio/elasticsearch/ElasticsearchIO.scala @@ -75,19 +75,18 @@ final case class ElasticsearchIO[T](esOptions: ElasticsearchOptions) extends Sci object ElasticsearchIO { object WriteParam { - private[elasticsearch] val DefaultErrorFn: BulkExecutionException => Unit = m => throw m - private[elasticsearch] val DefaultFlushInterval = Duration.standardSeconds(1) - private[elasticsearch] val DefaultNumShards = -1L - private[elasticsearch] val DefaultMaxBulkRequestOperations = 3000 - private[elasticsearch] val DefaultMaxBulkRequestBytes = 5L * 1024L * 1024L - private[elasticsearch] val DefaultMaxRetries = 3 - private[elasticsearch] val DefaultRetryPause = Duration.millis(35000) - private[elasticsearch] val DefaultRetryConfig = RetryConfig( + val DefaultErrorFn: BulkExecutionException => Unit = m => throw m + val DefaultFlushInterval: Duration = Duration.standardSeconds(1) + val DefaultNumShards: Long = -1L + val DefaultMaxBulkRequestOperations: Int = 3000 + val DefaultMaxBulkRequestBytes: Long = 5L * 1024L * 1024L + val DefaultMaxRetries: Int = 3 + val DefaultRetryPause: Duration = Duration.millis(35000) + val DefaultRetryConfig: RetryConfig = RetryConfig( maxRetries = WriteParam.DefaultMaxRetries, retryPause = WriteParam.DefaultRetryPause ) } - final case class WriteParam[T] private ( f: T => Iterable[BulkOperation], errorFn: BulkExecutionException => Unit = WriteParam.DefaultErrorFn, diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala index 42be0ad088..c551f1d3d8 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala @@ -95,15 +95,15 @@ object CsvIO { header = Header.Implicit ) - final val DefaultReadParams: ReadParam = CsvIO.ReadParam() - final val DefaultWriteParams: WriteParam = CsvIO.WriteParam() + val DefaultReadParams: ReadParam = CsvIO.ReadParam() + val DefaultWriteParams: WriteParam = CsvIO.WriteParam() - private[scio] object ReadParam { - val DefaultCompression = beam.Compression.AUTO - val DefaultCsvConfiguration = CsvIO.DefaultCsvConfiguration - val DefaultSuffix = null + object ReadParam { + val DefaultCompression: Compression = beam.Compression.AUTO + val DefaultCsvConfiguration: CsvConfiguration = CsvIO.DefaultCsvConfiguration + val DefaultSuffix: String = null - def apply(params: WriteParam): ReadParam = + private[csv] def apply(params: WriteParam): ReadParam = new ReadParam( compression = params.compression, csvConfiguration = params.csvConfiguration, @@ -117,15 +117,15 @@ object CsvIO { suffix: String = ReadParam.DefaultSuffix ) - private[scio] object WriteParam { - val DefaultSuffix = ".csv" - val DefaultCsvConfig = CsvIO.DefaultCsvConfiguration - val DefaultNumShards = 1 // put everything in a single file - val DefaultCompression = Compression.UNCOMPRESSED - val DefaultFilenamePolicySupplier = null - val DefaultPrefix = null - val DefaultShardNameTemplate = null - val DefaultTempDirectory = null + object WriteParam { + val DefaultSuffix: String = ".csv" + val DefaultCsvConfig: CsvConfiguration = CsvIO.DefaultCsvConfiguration + val DefaultNumShards: Int = 1 // put everything in a single file + val DefaultCompression: Compression = Compression.UNCOMPRESSED + val DefaultFilenamePolicySupplier: Null = null + val DefaultPrefix: String = null + val DefaultShardNameTemplate: String = null + val DefaultTempDirectory: String = null } final case class WriteParam private ( diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala index a1165a2ce8..c39e878b0c 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala @@ -28,6 +28,7 @@ import io.circe.syntax._ import org.apache.beam.sdk.{io => beam} import com.spotify.scio.io.TapT import com.spotify.scio.util.FilenamePolicySupplier +import org.apache.beam.sdk.io.Compression final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO[T] { override type ReadP = JsonIO.ReadParam @@ -73,11 +74,11 @@ final case class JsonIO[T: Encoder: Decoder: Coder](path: String) extends ScioIO object JsonIO { - private[scio] object ReadParam { - val DefaultCompression = beam.Compression.AUTO - val DefaultSuffix = null + object ReadParam { + val DefaultCompression: Compression = beam.Compression.AUTO + val DefaultSuffix: String = null - def apply(params: WriteParam): ReadParam = new ReadParam( + private[json] def apply(params: WriteParam): ReadParam = new ReadParam( params.compression, params.suffix + params.compression.getSuggestedSuffix ) @@ -88,21 +89,21 @@ object JsonIO { suffix: String = ReadParam.DefaultSuffix ) - private[scio] object WriteParam { - val DefaultNumShards = 0 - val DefaultSuffix = ".json" - val DefaultCompression = beam.Compression.UNCOMPRESSED - val DefaultPrinter = Printer.noSpaces - val DefaultFilenamePolicySupplier = null + object WriteParam { + val DefaultNumShards: Int = 0 + val DefaultSuffix: String = ".json" + val DefaultCompression: Compression = Compression.UNCOMPRESSED + val DefaultPrinter: Printer = Printer.noSpaces + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null val DefaultShardNameTemplate: String = null val DefaultPrefix: String = null - val DefaultTempDirectory = null + val DefaultTempDirectory: String = null } final case class WriteParam private ( suffix: String = WriteParam.DefaultSuffix, numShards: Int = WriteParam.DefaultNumShards, - compression: beam.Compression = WriteParam.DefaultCompression, + compression: Compression = WriteParam.DefaultCompression, printer: Printer = WriteParam.DefaultPrinter, filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, prefix: String = WriteParam.DefaultPrefix, diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala index 5e055e3161..ef17b2e619 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/BigQueryIO.scala @@ -92,7 +92,7 @@ private object Reads { } private[bigquery] object Writes { - trait WriteParamDefauls { + trait WriteParamDefaults { val DefaultSchema: TableSchema = null val DefaultWriteDisposition: WriteDisposition = null val DefaultCreateDisposition: CreateDisposition = null @@ -139,7 +139,7 @@ object BigQueryIO { object BigQueryTypedSelect { object ReadParam { - private[bigquery] val DefaultFlattenResults = false + val DefaultFlattenResults: Boolean = false } final case class ReadParam private (flattenResults: Boolean = ReadParam.DefaultFlattenResults) @@ -224,7 +224,7 @@ object BigQueryTypedTable { val insertErrorTransform: SCollection[extendedErrorInfo.Info] => Unit } - object WriteParam extends Writes.WriteParamDefauls { + object WriteParam extends Writes.WriteParamDefaults { @inline final def apply( s: TableSchema, wd: WriteDisposition, @@ -403,8 +403,8 @@ final case class BigQueryStorage( object BigQueryStorage { object ReadParam { - private[bigquery] val DefaultSelectFields: List[String] = Nil - private[bigquery] val DefaultRowRestriction: Option[String] = None + val DefaultSelectFields: List[String] = Nil + val DefaultRowRestriction: Option[String] = None } } @@ -458,14 +458,15 @@ object TableRowJsonIO { val ReadParam = TextIO.ReadParam type WriteParam = TextIO.WriteParam - private[scio] object WriteParam { - val DefaultSuffix = ".json" - val DefaultNumShards = TextIO.WriteParam.DefaultNumShards - val DefaultCompression = TextIO.WriteParam.DefaultCompression - val DefaultFilenamePolicySupplier = TextIO.WriteParam.DefaultFilenamePolicySupplier - val DefaultPrefix = TextIO.WriteParam.DefaultPrefix - val DefaultShardNameTemplate = TextIO.WriteParam.DefaultShardNameTemplate - val DefaultTempDirectory = TextIO.WriteParam.DefaultTempDirectory + object WriteParam { + val DefaultSuffix: String = ".json" + val DefaultNumShards: Int = TextIO.WriteParam.DefaultNumShards + val DefaultCompression: Compression = TextIO.WriteParam.DefaultCompression + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = + TextIO.WriteParam.DefaultFilenamePolicySupplier + val DefaultPrefix: String = TextIO.WriteParam.DefaultPrefix + val DefaultShardNameTemplate: String = TextIO.WriteParam.DefaultShardNameTemplate + val DefaultTempDirectory: String = TextIO.WriteParam.DefaultTempDirectory def apply( suffix: String = DefaultSuffix, @@ -641,7 +642,7 @@ object BigQueryTyped { val insertErrorTransform: SCollection[extendedErrorInfo.Info] => Unit } - object WriteParam extends Writes.WriteParamDefauls { + object WriteParam extends Writes.WriteParamDefaults { @inline final def apply( wd: WriteDisposition, cd: CreateDisposition, diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala index 968b134d39..2c654e2100 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala @@ -87,8 +87,8 @@ final case class BigtableRead(bigtableOptions: BigtableOptions, tableId: String) object BigtableRead { object ReadParam { - private[bigtable] val DefaultKeyRanges: Seq[ByteKeyRange] = Seq.empty[ByteKeyRange] - private[bigtable] val DefaultRowFilter: RowFilter = null + val DefaultKeyRanges: Seq[ByteKeyRange] = Seq.empty[ByteKeyRange] + val DefaultRowFilter: RowFilter = null def apply(keyRange: ByteKeyRange) = new ReadParam(Seq(keyRange)) diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/spanner/SpannerIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/spanner/SpannerIO.scala index c16cbef66d..31ea99a069 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/spanner/SpannerIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/spanner/SpannerIO.scala @@ -39,8 +39,8 @@ sealed trait SpannerIO[T] extends ScioIO[T] { object SpannerRead { object ReadParam { - private[spanner] val DefaultWithTransaction = false - private[spanner] val DefaultWithBatching = true + val DefaultWithTransaction: Boolean = false + val DefaultWithBatching: Boolean = true } sealed trait ReadMethod @@ -88,8 +88,8 @@ final case class SpannerRead(config: SpannerConfig) extends SpannerIO[Struct] { object SpannerWrite { object WriteParam { - private[spanner] val DefaultFailureMode = FailureMode.FAIL_FAST - private[spanner] val DefaultBatchSizeBytes = 1024L * 1024L + val DefaultFailureMode: FailureMode = FailureMode.FAIL_FAST + val DefaultBatchSizeBytes: Long = 1024L * 1024L } final case class WriteParam private ( diff --git a/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala b/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala index 5148bf216d..6a33067183 100644 --- a/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala +++ b/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala @@ -30,9 +30,9 @@ import scala.util.matching.Regex object Neo4jIO { object WriteParam { - private[neo4j] val BeamDefaultBatchSize = 5000L + val DefaultBatchSize: Long = 5000L } - final case class WriteParam private (batchSize: Long = WriteParam.BeamDefaultBatchSize) + final case class WriteParam private (batchSize: Long = WriteParam.DefaultBatchSize) implicit private[neo4j] def recordConverter(record: Record): Value = Values.value(record.asMap(identity[Value])) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala index 9a3984d64e..855966c6db 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala @@ -157,13 +157,13 @@ final case class ParquetAvroIO[T: ClassTag: Coder](path: String) extends ScioIO[ object ParquetAvroIO { private lazy val log = LoggerFactory.getLogger(getClass) - private[scio] object ReadParam { - val DefaultProjection = null - val DefaultPredicate = null - val DefaultConfiguration = null - val DefaultSuffix = null + object ReadParam { + val DefaultProjection: Schema = null + val DefaultPredicate: FilterPredicate = null + val DefaultConfiguration: Configuration = null + val DefaultSuffix: String = null - def apply[T: ClassTag](params: WriteParam): ReadParam[T, T] = + private[parquet] def apply[T: ClassTag](params: WriteParam): ReadParam[T, T] = new ReadParam[T, T]( projectionFn = identity, projection = params.schema, @@ -299,16 +299,16 @@ object ParquetAvroIO { }) } - private[scio] object WriteParam { - val DefaultSchema = null - val DefaultNumShards = 0 - val DefaultSuffix = ".parquet" - val DefaultCompression = CompressionCodecName.GZIP - val DefaultConfiguration = null - val DefaultFilenamePolicySupplier = null - val DefaultPrefix = null - val DefaultShardNameTemplate = null - val DefaultTempDirectory = null + object WriteParam { + val DefaultSchema: Schema = null + val DefaultNumShards: Int = 0 + val DefaultSuffix: String = ".parquet" + val DefaultCompression: CompressionCodecName = CompressionCodecName.GZIP + val DefaultConfiguration: Configuration = null + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null + val DefaultPrefix: String = null + val DefaultShardNameTemplate: String = null + val DefaultTempDirectory: String = null } final case class WriteParam private ( diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala index 187c4c495a..7a38a09b6a 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala @@ -210,13 +210,13 @@ final case class ParquetExampleIO(path: String) extends ScioIO[Example] { object ParquetExampleIO { - private[tensorflow] object ReadParam { - val DefaultProjection = null - val DefaultPredicate = null - val DefaultConfiguration = null - val DefaultSuffix = null + object ReadParam { + val DefaultProjection: Seq[String] = null + val DefaultPredicate: FilterPredicate = null + val DefaultConfiguration: Configuration = null + val DefaultSuffix: String = null - def apply(params: WriteParam): ReadParam = + private[parquet] def apply(params: WriteParam): ReadParam = new ReadParam( conf = params.conf, suffix = params.suffix @@ -229,15 +229,15 @@ object ParquetExampleIO { suffix: String = ReadParam.DefaultSuffix ) - private[tensorflow] object WriteParam { - val DefaultNumShards = 0 - val DefaultSuffix = ".parquet" - val DefaultCompression = CompressionCodecName.GZIP - val DefaultConfiguration = null - val DefaultFilenamePolicySupplier = null - val DefaultPrefix = null - val DefaultShardNameTemplate = null - val DefaultTempDirectory = null + object WriteParam { + val DefaultNumShards: Int = 0 + val DefaultSuffix: String = ".parquet" + val DefaultCompression: CompressionCodecName = CompressionCodecName.GZIP + val DefaultConfiguration: Configuration = null + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null + val DefaultPrefix: String = null + val DefaultShardNameTemplate: String = null + val DefaultTempDirectory: String = null } final case class WriteParam private ( diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala index f64bd3dd3c..0202b77141 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala @@ -180,32 +180,32 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType]( object ParquetTypeIO { - private[scio] object ReadParam { - val DefaultPredicate = null - val DefaultConfiguration = null - val DefaultSuffix = null + object ReadParam { + val DefaultPredicate: FilterPredicate = null + val DefaultConfiguration: Configuration = null + val DefaultSuffix: String = null - def apply[T](params: WriteParam): ReadParam = + private[parquet] def apply(params: WriteParam): ReadParam = new ReadParam( conf = params.conf, suffix = params.suffix ) } final case class ReadParam private ( - predicate: FilterPredicate = null, + predicate: FilterPredicate = ReadParam.DefaultPredicate, conf: Configuration = ReadParam.DefaultConfiguration, suffix: String = ReadParam.DefaultSuffix ) - private[scio] object WriteParam { - val DefaultNumShards = 0 - val DefaultSuffix = ".parquet" - val DefaultCompression = CompressionCodecName.GZIP - val DefaultConfiguration = null - val DefaultFilenamePolicySupplier = null - val DefaultPrefix = null - val DefaultShardNameTemplate = null - val DefaultTempDirectory = null + object WriteParam { + val DefaultNumShards: Int = 0 + val DefaultSuffix: String = ".parquet" + val DefaultCompression: CompressionCodecName = CompressionCodecName.GZIP + val DefaultConfiguration: Configuration = null + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null + val DefaultPrefix: String = null + val DefaultShardNameTemplate: String = null + val DefaultTempDirectory: String = null } final case class WriteParam private ( diff --git a/scio-redis/src/main/scala/com/spotify/scio/redis/RedisIO.scala b/scio-redis/src/main/scala/com/spotify/scio/redis/RedisIO.scala index a44e209887..6ba790088b 100644 --- a/scio-redis/src/main/scala/com/spotify/scio/redis/RedisIO.scala +++ b/scio-redis/src/main/scala/com/spotify/scio/redis/RedisIO.scala @@ -93,8 +93,8 @@ final case class RedisRead(connectionOptions: RedisConnectionOptions, keyPattern object RedisRead { object ReadParam { - private[redis] val DefaultBatchSize: Int = 1000 - private[redis] val DefaultOutputParallelization: Boolean = true + val DefaultBatchSize: Int = 1000 + val DefaultOutputParallelization: Boolean = true } final case class ReadParam private ( diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index 303e22af82..d84c6de84d 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -57,11 +57,11 @@ final case class TFRecordIO(path: String) extends ScioIO[Array[Byte]] { object TFRecordIO { - private[tensorflow] object ReadParam { - val DefaultCompression = Compression.AUTO - val DefaultSuffix = null + object ReadParam { + val DefaultCompression: Compression = Compression.AUTO + val DefaultSuffix: String = null - def apply(params: WriteParam): ReadParam = + private[tensorflow] def apply(params: WriteParam): ReadParam = new ReadParam( compression = params.compression, suffix = params.suffix + params.compression.getSuggestedSuffix @@ -73,14 +73,14 @@ object TFRecordIO { suffix: String = ReadParam.DefaultSuffix ) - private[tensorflow] object WriteParam { - val DefaultSuffix = ".tfrecords" - val DefaultCompression = Compression.UNCOMPRESSED - val DefaultNumShards = 0 - val DefaultFilenamePolicySupplier = null - val DefaultPrefix = null - val DefaultShardNameTemplate = null - val DefaultTempDirectory = null + object WriteParam { + val DefaultSuffix: String = ".tfrecords" + val DefaultCompression: Compression = Compression.UNCOMPRESSED + val DefaultNumShards: Int = 0 + val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null + val DefaultPrefix: String = null + val DefaultShardNameTemplate: String = null + val DefaultTempDirectory: String = null } final case class WriteParam private ( From 03ddcbc2f0e96828aa32326d71b2d3c58e03f4d7 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 31 May 2023 10:36:40 +0200 Subject: [PATCH 20/26] Change ReadParam from WriteParam constructor visibility --- scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala | 5 ++--- scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala | 2 +- .../src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala | 2 +- .../src/main/scala/com/spotify/scio/extra/json/JsonIO.scala | 2 +- .../scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala | 2 +- .../spotify/scio/parquet/tensorflow/ParquetExampleIO.scala | 2 +- .../scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala | 2 +- .../main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala | 2 +- 8 files changed, 9 insertions(+), 10 deletions(-) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala index 6b97caf5f7..4a206ae239 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroIO.scala @@ -310,9 +310,8 @@ object AvroIO { object ReadParam { val DefaultSuffix: String = null - def apply(params: WriteParam): ReadParam = new ReadParam( - params.suffix - ) + private[scio] def apply(params: WriteParam): ReadParam = + new ReadParam(params.suffix) } final case class ReadParam private ( diff --git a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala index f6ee078199..1429ccb430 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/TextIO.scala @@ -118,7 +118,7 @@ object TextIO { val DefaultEmptyMatchTreatment: EmptyMatchTreatment = EmptyMatchTreatment.DISALLOW val DefaultSuffix: String = null - private[io] def apply(params: WriteParam): ReadParam = + private[scio] def apply(params: WriteParam): ReadParam = new ReadParam( compression = params.compression, suffix = params.suffix + params.compression.getSuggestedSuffix diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala index c551f1d3d8..ed59e8c477 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/csv/CsvIO.scala @@ -103,7 +103,7 @@ object CsvIO { val DefaultCsvConfiguration: CsvConfiguration = CsvIO.DefaultCsvConfiguration val DefaultSuffix: String = null - private[csv] def apply(params: WriteParam): ReadParam = + private[scio] def apply(params: WriteParam): ReadParam = new ReadParam( compression = params.compression, csvConfiguration = params.csvConfiguration, diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala index c39e878b0c..4a4a3086fd 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/json/JsonIO.scala @@ -78,7 +78,7 @@ object JsonIO { val DefaultCompression: Compression = beam.Compression.AUTO val DefaultSuffix: String = null - private[json] def apply(params: WriteParam): ReadParam = new ReadParam( + private[scio] def apply(params: WriteParam): ReadParam = new ReadParam( params.compression, params.suffix + params.compression.getSuggestedSuffix ) diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala index 855966c6db..4a1caf7f23 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/avro/ParquetAvroIO.scala @@ -163,7 +163,7 @@ object ParquetAvroIO { val DefaultConfiguration: Configuration = null val DefaultSuffix: String = null - private[parquet] def apply[T: ClassTag](params: WriteParam): ReadParam[T, T] = + private[scio] def apply[T: ClassTag](params: WriteParam): ReadParam[T, T] = new ReadParam[T, T]( projectionFn = identity, projection = params.schema, diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala index 7a38a09b6a..3e7a0c6ae7 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/tensorflow/ParquetExampleIO.scala @@ -216,7 +216,7 @@ object ParquetExampleIO { val DefaultConfiguration: Configuration = null val DefaultSuffix: String = null - private[parquet] def apply(params: WriteParam): ReadParam = + private[scio] def apply(params: WriteParam): ReadParam = new ReadParam( conf = params.conf, suffix = params.suffix diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala index 0202b77141..01c1f024c3 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/types/ParquetTypeIO.scala @@ -185,7 +185,7 @@ object ParquetTypeIO { val DefaultConfiguration: Configuration = null val DefaultSuffix: String = null - private[parquet] def apply(params: WriteParam): ReadParam = + private[scio] def apply(params: WriteParam): ReadParam = new ReadParam( conf = params.conf, suffix = params.suffix diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index d84c6de84d..09568357b3 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -61,7 +61,7 @@ object TFRecordIO { val DefaultCompression: Compression = Compression.AUTO val DefaultSuffix: String = null - private[tensorflow] def apply(params: WriteParam): ReadParam = + private[scio] def apply(params: WriteParam): ReadParam = new ReadParam( compression = params.compression, suffix = params.suffix + params.compression.getSuggestedSuffix From 33b198e3a7d37deb494bb83c2f93720f41108e32 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 31 May 2023 10:51:28 +0200 Subject: [PATCH 21/26] Fix compile issues --- .../src/main/scala/com/spotify/scio/cassandra/package.scala | 2 +- .../scala/com/spotify/scio/neo4j/syntax/SCollectionSyntax.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/package.scala b/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/package.scala index 61a4232971..bd281e5580 100644 --- a/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/package.scala +++ b/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/package.scala @@ -65,7 +65,7 @@ package object cassandra { */ def saveAsCassandra( opts: CassandraOptions, - parallelism: Int = CassandraIO.WriteParam.DefaultPar + parallelism: Int = CassandraIO.WriteParam.DefaultParallelism )(f: T => Seq[Any]): ClosedTap[Nothing] = self.write(CassandraIO[T](opts))(CassandraIO.WriteParam(f, parallelism)) } diff --git a/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/syntax/SCollectionSyntax.scala b/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/syntax/SCollectionSyntax.scala index 6153f80833..84dd1ab42a 100644 --- a/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/syntax/SCollectionSyntax.scala +++ b/scio-neo4j/src/main/scala/com/spotify/scio/neo4j/syntax/SCollectionSyntax.scala @@ -91,7 +91,7 @@ final class Neo4jSCollectionOps[T](private val self: SCollection[T]) extends Any def saveAsNeo4j( neo4jOptions: Neo4jOptions, unwindCypher: String, - batchSize: Long = WriteParam.BeamDefaultBatchSize + batchSize: Long = WriteParam.DefaultBatchSize )(implicit neo4jType: ValueType[T], coder: Coder[T]): ClosedTap[Nothing] = self.write(Neo4jIO[T](neo4jOptions, unwindCypher))(WriteParam(batchSize)) } From d31e6ffa7f541dfb54dbd727a6e676e3c902abf5 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 31 May 2023 15:49:11 +0200 Subject: [PATCH 22/26] Overload avroFile API --- .../spotify/scio/avro/types/AvroTapIT.scala | 2 +- .../scio/avro/syntax/ScioContextSyntax.scala | 32 ++++++-------- .../scala/com/spotify/scio/avro/taps.scala | 27 ++++++------ .../scio/examples/extra/AvroExample.scala | 4 +- .../scio/examples/extra/AvroInOut.scala | 2 +- .../examples/extra/MagnolifyAvroExample.scala | 2 +- .../spotify/scio/jmh/GroupByBenchmark.scala | 4 +- .../scio/smb/SortMergeBucketParityIT.scala | 42 +++++++++---------- .../com/spotify/scio/io/ScioIOTest.scala | 4 +- .../scio/io/dynamic/DynamicFileTest.scala | 14 +++---- .../spotify/scio/testing/JobTestTest.scala | 4 +- site/src/main/paradox/FAQ.md | 2 +- site/src/main/paradox/io/Avro.md | 4 +- 13 files changed, 69 insertions(+), 74 deletions(-) diff --git a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala index 796f5ce786..8416ab970c 100644 --- a/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala +++ b/scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala @@ -50,7 +50,7 @@ final class AvroTapIT extends AnyFlatSpec with Matchers { it should "read avro file" in { FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create) - val tap = AvroTaps(Taps()).avroGenericFile( + val tap = AvroTaps(Taps()).avroFile( "gs://data-integration-test-eu/avro-integration-test/folder-a/folder-b/shakespeare.avro", schema = schema ) diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala index 096b228ea8..8e50c9ae69 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/syntax/ScioContextSyntax.scala @@ -46,18 +46,11 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { ): SCollection[T] = self.read(ObjectFileIO[T](path))(ObjectFileIO.ReadParam(suffix)) - @deprecated("Use avroGenericFile instead", since = "0.13.0") - def avroFile( - path: String, - schema: Schema - ): SCollection[GenericRecord] = - avroGenericFile(path, schema) + def avroFile(path: String, schema: Schema): SCollection[GenericRecord] = + self.read(GenericRecordIO(path, schema))(AvroIO.ReadParam()) - def avroGenericFile( - path: String, - schema: Schema, - suffix: String = ObjectFileIO.ReadParam.DefaultSuffix - ): SCollection[GenericRecord] = + // overloaded API. We can't use default params + def avroFile(path: String, schema: Schema, suffix: String): SCollection[GenericRecord] = self.read(GenericRecordIO(path, schema))(AvroIO.ReadParam(suffix)) /** @@ -92,20 +85,19 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { ): SCollection[T] = self.read(GenericRecordParseIO[T](path, parseFn))(AvroIO.ReadParam(suffix)) - @deprecated("Use avroSpecificFile instead", since = "0.13.0") - def avroFile[T <: SpecificRecord: ClassTag: Coder]( - path: String - ): SCollection[T] = - avroSpecificFile(path) + /** + * Get an SCollection of type [[org.apache.avro.specific.SpecificRecord SpecificRecord]] for an + * Avro file. + */ + def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String): SCollection[T] = + self.read(SpecificRecordIO[T](path))(AvroIO.ReadParam()) + // overloaded API. We can't use default params /** * Get an SCollection of type [[org.apache.avro.specific.SpecificRecord SpecificRecord]] for an * Avro file. */ - def avroSpecificFile[T <: SpecificRecord: ClassTag: Coder]( - path: String, - suffix: String = AvroIO.ReadParam.DefaultSuffix - ): SCollection[T] = + def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String, suffix: String): SCollection[T] = self.read(SpecificRecordIO[T](path))(AvroIO.ReadParam(suffix)) /** diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala index e88d09bf34..6276d19811 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/taps.scala @@ -119,35 +119,38 @@ final case class AvroTaps(self: Taps) { () => ObjectFileTap[T](path, params) ) - @deprecated("Use avroGenericFile instead", since = "0.13.0") + /** + * Get a `Future[Tap[T]]` for [[org.apache.avro.generic.GenericRecord GenericRecord]] Avro file. + */ def avroFile(path: String, schema: Schema): Future[Tap[GenericRecord]] = - avroGenericFile(path, schema) + avroFile(path, schema, AvroIO.ReadParam()) + // overloaded API. We can't use default params /** * Get a `Future[Tap[T]]` for [[org.apache.avro.generic.GenericRecord GenericRecord]] Avro file. */ - def avroGenericFile( - path: String, - schema: Schema, - params: AvroIO.ReadParam = AvroIO.ReadParam() - ): Future[Tap[GenericRecord]] = + def avroFile(path: String, schema: Schema, params: AvroIO.ReadParam): Future[Tap[GenericRecord]] = self.mkTap( s"Avro: $path", () => self.isPathDone(path, params.suffix), () => GenericRecordTap(path, schema, params) ) - @deprecated("Use avroSpecificFile instead", since = "0.13.0") + /** + * Get a `Future[Tap[T]]` for [[org.apache.avro.specific.SpecificRecord SpecificRecord]] Avro + * file. + */ def avroFile[T <: SpecificRecord: ClassTag: Coder](path: String): Future[Tap[T]] = - avroSpecificFile[T](path) + avroFile[T](path, AvroIO.ReadParam()) + // overloaded API. We can't use default params /** * Get a `Future[Tap[T]]` for [[org.apache.avro.specific.SpecificRecord SpecificRecord]] Avro * file. */ - def avroSpecificFile[T <: SpecificRecord: ClassTag: Coder]( + def avroFile[T <: SpecificRecord: ClassTag: Coder]( path: String, - params: AvroIO.ReadParam = AvroIO.ReadParam() + params: AvroIO.ReadParam ): Future[Tap[T]] = self.mkTap( s"Avro: $path", @@ -163,6 +166,6 @@ final case class AvroTaps(self: Taps) { val avroT = AvroType[T] import scala.concurrent.ExecutionContext.Implicits.global - avroGenericFile(path, avroT.schema, params).map(_.map(avroT.fromGenericRecord)) + avroFile(path, avroT.schema, params).map(_.map(avroT.fromGenericRecord)) } } diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala index 4714191216..8d16bd606a 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroExample.scala @@ -100,7 +100,7 @@ object AvroExample { .saveAsAvroFile(args("output")) private def specificIn(sc: ScioContext, args: Args): ClosedTap[String] = - sc.avroSpecificFile[Account](args("input")) + sc.avroFile[Account](args("input")) .map(_.toString) .saveAsTextFile(args("output")) @@ -131,7 +131,7 @@ object AvroExample { .saveAsTypedAvroFile(args("output")) private def genericIn(sc: ScioContext, args: Args): ClosedTap[String] = - sc.avroGenericFile(args("input"), schema) + sc.avroFile(args("input"), schema) .map(_.toString) .saveAsTextFile(args("output")) diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala index 75e2978260..609e490d3c 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/AvroInOut.scala @@ -33,7 +33,7 @@ object AvroInOut { // Open Avro files as a `SCollection[TestRecord]` where `TestRecord` is an Avro specific record // Java class compiled from Avro schema. - sc.avroSpecificFile[TestRecord](args("input")) + sc.avroFile[TestRecord](args("input")) .map { r => // Create a new `Account` Avro specific record. It is recommended to use the builder over // constructor since it's more backwards compatible. diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala index 6a255e3ce9..54a1855f53 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyAvroExample.scala @@ -73,7 +73,7 @@ object MagnolifyAvroReadExample { import MagnolifyAvroExample._ val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.avroGenericFile(args("input"), wordCountType.schema) + sc.avroFile(args("input"), wordCountType.schema) .map(e => wordCountType(e)) .map(wc => wc.word + ": " + wc.count) .saveAsTextFile(args("output")) diff --git a/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala b/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala index f03f773628..47add251a9 100644 --- a/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala +++ b/scio-jmh/src/test/scala/com/spotify/scio/jmh/GroupByBenchmark.scala @@ -75,7 +75,7 @@ class GroupByBenchmark { @Benchmark def testScioGroupByKey: ScioExecutionContext = runWithContext { sc => - sc.avroGenericFile(source, schema = avroSchema) + sc.avroFile(source, schema = avroSchema) .map(rec => (rec.get("id").toString.head, rec.get("value").asInstanceOf[Double])) .groupByKey } @@ -84,7 +84,7 @@ class GroupByBenchmark { def testBeamGroupByKey: ScioExecutionContext = runWithContext { sc => sc.wrap { - sc.avroGenericFile(source, schema = avroSchema) + sc.avroFile(source, schema = avroSchema) .map { rec => KV.of(rec.get("id").toString.head, rec.get("value").asInstanceOf[Double]) } diff --git a/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala b/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala index 699d29eb56..382b6f9310 100644 --- a/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala +++ b/scio-smb/src/it/scala/com/spotify/scio/smb/SortMergeBucketParityIT.scala @@ -65,8 +65,8 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB) = ( - sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro") + sc.avroFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(1).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).cogroup(avroB.keyBy(keyFn)) @@ -90,14 +90,14 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { val (lhs, rhs) = ( SCollection.unionAll( List( - sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro") + sc.avroFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(1).getAbsolutePath, schema, ".avro") ) ), SCollection.unionAll( List( - sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(3).getAbsolutePath, schema, ".avro") + sc.avroFile(inputs(2).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(3).getAbsolutePath, schema, ".avro") ) ) ) @@ -117,9 +117,9 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB, avroC) = ( - sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro") + sc.avroFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(1).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(2).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).cogroup(avroB.keyBy(keyFn), avroC.keyBy(keyFn)) @@ -138,10 +138,10 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB, avroC, avroD) = ( - sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(3).getAbsolutePath, schema, ".avro") + sc.avroFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(1).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(2).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(3).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).cogroup(avroB.keyBy(keyFn), avroC.keyBy(keyFn), avroD.keyBy(keyFn)) @@ -151,7 +151,7 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { "sortMergeGroupByKey" should "have parity with Scio's groupBy" in withNumSources(1) { inputs => compareResults( _.sortMergeGroupByKey(classOf[Integer], mkRead(inputs(0))) - )(sc => sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro").groupBy(keyFn)) + )(sc => sc.avroFile(inputs(0).getAbsolutePath, schema, ".avro").groupBy(keyFn)) } "sortMergeJoin" should "have parity with a 2-way Join" in withNumSources(2) { inputs => @@ -164,8 +164,8 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB) = ( - sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro") + sc.avroFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(1).getAbsolutePath, schema, ".avro") ) avroA.keyBy(keyFn).join(avroB.keyBy(keyFn)) @@ -186,11 +186,11 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { ) ) { sc => val (avroA, avroB, avroC, avroD, avroE) = ( - sc.avroGenericFile(inputs(0).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(1).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(2).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(3).getAbsolutePath, schema, ".avro"), - sc.avroGenericFile(inputs(4).getAbsolutePath, schema, ".avro") + sc.avroFile(inputs(0).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(1).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(2).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(3).getAbsolutePath, schema, ".avro"), + sc.avroFile(inputs(4).getAbsolutePath, schema, ".avro") ) MultiJoin.cogroup( diff --git a/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala index 93c541a369..2ac971d8c9 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/ScioIOTest.scala @@ -324,7 +324,7 @@ class ScioIOTest extends ScioIOSpec { "AvroIO" should "work with SpecificRecord" in { val xs = (1 to 100).map(AvroUtils.newSpecificRecord) testTap(xs)(_.saveAsAvroFile(_))(".avro") - testJobTest(xs)(AvroIO[TestRecord](_))(_.avroSpecificFile(_))(_.saveAsAvroFile(_)) + testJobTest(xs)(AvroIO[TestRecord](_))(_.avroFile(_))(_.saveAsAvroFile(_)) } it should "work with GenericRecord" in { @@ -332,7 +332,7 @@ class ScioIOTest extends ScioIOSpec { implicit val coder = Coder.avroGenericRecordCoder(schema) val xs = (1 to 100).map(AvroUtils.newGenericRecord) testTap(xs)(_.saveAsAvroFile(_, schema = schema))(".avro") - testJobTest(xs)(AvroIO(_))(_.avroGenericFile(_, schema))(_.saveAsAvroFile(_, schema = schema)) + testJobTest(xs)(AvroIO(_))(_.avroFile(_, schema))(_.saveAsAvroFile(_, schema = schema)) } it should "work with typed Avro" in { diff --git a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala index 6ed4862479..ae95a183f4 100644 --- a/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/io/dynamic/DynamicFileTest.scala @@ -135,8 +135,8 @@ class DynamicFileTest extends PipelineSpec with TapSpec { verifyOutput(dir, "even", "odd") val sc2 = ScioContext() - val even = sc2.avroGenericFile(s"$dir/even/*.avro", schema) - val odd = sc2.avroGenericFile(s"$dir/odd/*.avro", schema) + val even = sc2.avroFile(s"$dir/even/*.avro", schema) + val odd = sc2.avroFile(s"$dir/odd/*.avro", schema) val (expectedEven, expectedOdd) = (1 to 10).partition(_ % 2 == 0) even should containInAnyOrder(expectedEven.map(newGenericRecord)) odd should containInAnyOrder(expectedOdd.map(newGenericRecord)) @@ -153,8 +153,8 @@ class DynamicFileTest extends PipelineSpec with TapSpec { verifyOutput(dir, "even", "odd") val sc2 = ScioContext() - val even = sc2.avroSpecificFile[TestRecord](s"$dir/even/*.avro") - val odd = sc2.avroSpecificFile[TestRecord](s"$dir/odd/*.avro") + val even = sc2.avroFile[TestRecord](s"$dir/even/*.avro") + val odd = sc2.avroFile[TestRecord](s"$dir/odd/*.avro") val (expectedEven, expectedOdd) = (1 to 10).partition(_ % 2 == 0) even should containInAnyOrder(expectedEven.map(newSpecificRecord)) odd should containInAnyOrder(expectedOdd.map(newSpecificRecord)) @@ -178,8 +178,8 @@ class DynamicFileTest extends PipelineSpec with TapSpec { Files.list(dir.toPath.resolve("odd")).iterator().asScala.size shouldBe 5 val sc2 = ScioContext() - val even = sc2.avroSpecificFile[TestRecord](s"$dir/even/*.avro") - val odd = sc2.avroSpecificFile[TestRecord](s"$dir/odd/*.avro") + val even = sc2.avroFile[TestRecord](s"$dir/even/*.avro") + val odd = sc2.avroFile[TestRecord](s"$dir/odd/*.avro") val (expectedEven, expectedOdd) = (1 to 10).partition(_ % 2 == 0) even should containInAnyOrder(expectedEven.map(newSpecificRecord)) odd should containInAnyOrder(expectedOdd.map(newSpecificRecord)) @@ -187,7 +187,7 @@ class DynamicFileTest extends PipelineSpec with TapSpec { val p = partitionIntegers(x % 2) val t1 = new Instant(x * 60000L) val t2 = t1.plus(60000L) - val records = sc2.avroSpecificFile[TestRecord](s"$dir/$p/part-$t1-$t2-*.avro") + val records = sc2.avroFile[TestRecord](s"$dir/$p/part-$t1-$t2-*.avro") records should containSingleValue(newSpecificRecord(x)) } sc2.run() diff --git a/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala b/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala index 5a26c84a13..e8d94df4bc 100644 --- a/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/testing/JobTestTest.scala @@ -52,7 +52,7 @@ object ObjectFileJob { object SpecificAvroFileJob { def main(cmdlineArgs: Array[String]): Unit = { val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.avroSpecificFile[TestRecord](args("input")) + sc.avroFile[TestRecord](args("input")) .saveAsAvroFile(args("output")) sc.run() () @@ -62,7 +62,7 @@ object SpecificAvroFileJob { object GenericAvroFileJob { def main(cmdlineArgs: Array[String]): Unit = { val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.avroGenericFile(args("input"), AvroUtils.schema) + sc.avroFile(args("input"), AvroUtils.schema) .saveAsAvroFile(args("output"), schema = AvroUtils.schema) sc.run() () diff --git a/site/src/main/paradox/FAQ.md b/site/src/main/paradox/FAQ.md index 88b8b43bf4..1c035ef254 100644 --- a/site/src/main/paradox/FAQ.md +++ b/site/src/main/paradox/FAQ.md @@ -97,7 +97,7 @@ object MyJob { val collections = Seq( "gs://bucket1/data/", "gs://bucket2/data/" - ).map(path => sc.avroSpecificFile[TestRecord](path, suffix=".avro")) + ).map(path => sc.avroFile[TestRecord](path, suffix=".avro")) val all = SCollection.unionAll(collections) } diff --git a/site/src/main/paradox/io/Avro.md b/site/src/main/paradox/io/Avro.md index 307e5a49a0..acf1c18f72 100644 --- a/site/src/main/paradox/io/Avro.md +++ b/site/src/main/paradox/io/Avro.md @@ -15,7 +15,7 @@ import org.apache.avro.specific.SpecificRecord def sc: ScioContext = ??? // SpecificRecordClass is compiled from Avro schema files -def result = sc.avroSpecificFile[SpecificRecord]("gs://path-to-data/lake/part-*.avro") +def result = sc.avroFile[SpecificRecord]("gs://path-to-data/lake/part-*.avro") ``` ### Read Generic records @@ -31,7 +31,7 @@ def yourAvroSchema: Schema = ??? def sc: ScioContext = ??? -def result = sc.avroGenericFile("gs://path-to-data/lake/part-*.avro", yourAvroSchema) +def result = sc.avroFile("gs://path-to-data/lake/part-*.avro", yourAvroSchema) // `record` is of GenericRecord type ``` From 209e5e8eb8ef737cb90db6aef20ee8d7b484824a Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 7 Jun 2023 10:04:05 +0200 Subject: [PATCH 23/26] Standardize io param default values --- .../spotify/scio/datastore/DatastoreIO.scala | 16 ++++++-- .../datastore/syntax/SCollectionSyntax.scala | 2 +- .../datastore/syntax/ScioContextSyntax.scala | 4 +- .../com/spotify/scio/pubsub/PubsubIO.scala | 21 ++++++++--- .../scala/com/spotify/scio/jdbc/JdbcIO.scala | 37 +++++++++++-------- .../scio/jdbc/syntax/SCollectionSyntax.scala | 13 ++++--- .../scio/jdbc/syntax/ScioContextSyntax.scala | 11 +++--- .../spotify/scio/tensorflow/TFRecordIO.scala | 14 +++---- 8 files changed, 72 insertions(+), 46 deletions(-) diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala index 94df2dd8ec..2977d464d8 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/DatastoreIO.scala @@ -55,13 +55,23 @@ final case class DatastoreIO(projectId: String) extends ScioIO[Entity] { } object DatastoreIO { + + object ReadParam { + val DefaultNamespace: String = null + val DefaultConfigOverride: BDatastore.Read => BDatastore.Read = identity + } + final case class ReadParam private ( query: Query, - namespace: String = null, - configOverride: BDatastore.Read => BDatastore.Read = identity + namespace: String = ReadParam.DefaultNamespace, + configOverride: BDatastore.Read => BDatastore.Read = ReadParam.DefaultConfigOverride ) + object WriteParam { + val DefaultConfigOverride: BDatastore.Write => BDatastore.Write = identity + } + final case class WriteParam private ( - configOverride: BDatastore.Write => BDatastore.Write = identity + configOverride: BDatastore.Write => BDatastore.Write = WriteParam.DefaultConfigOverride ) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/SCollectionSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/SCollectionSyntax.scala index dc0af533fb..8ed685b3bf 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/SCollectionSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/SCollectionSyntax.scala @@ -33,7 +33,7 @@ final class SCollectionEntityOps[T <: Entity](private val coll: SCollection[T]) */ def saveAsDatastore( projectId: String, - configOverride: BDatastore.Write => BDatastore.Write = identity + configOverride: BDatastore.Write => BDatastore.Write = WriteParam.DefaultConfigOverride ): ClosedTap[Nothing] = coll.covary_[Entity].write(DatastoreIO(projectId))(WriteParam(configOverride)) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/ScioContextSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/ScioContextSyntax.scala index f6399941fb..bb50a920a1 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/ScioContextSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/datastore/syntax/ScioContextSyntax.scala @@ -33,8 +33,8 @@ final class ScioContextOps(private val sc: ScioContext) extends AnyVal { def datastore( projectId: String, query: Query, - namespace: String = null, - configOverride: BDatastore.Read => BDatastore.Read = identity + namespace: String = ReadParam.DefaultNamespace, + configOverride: BDatastore.Read => BDatastore.Read = ReadParam.DefaultConfigOverride ): SCollection[Entity] = sc.read(DatastoreIO(projectId))(ReadParam(query, namespace, configOverride)) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala index dc345a6f52..209d9ff3f8 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/pubsub/PubsubIO.scala @@ -49,8 +49,8 @@ object PubsubIO { final case class ReadParam private ( readType: ReadType, - clientFactory: Option[beam.PubsubClient.PubsubClientFactory] = None, - deadLetterTopic: Option[String] = None + clientFactory: Option[beam.PubsubClient.PubsubClientFactory] = ReadParam.DefaultClientFactory, + deadLetterTopic: Option[String] = ReadParam.DefaultDeadLetterTopic ) { val isSubscription: Boolean = readType match { case Subscription => true @@ -58,17 +58,26 @@ object PubsubIO { } } object ReadParam { + val DefaultClientFactory: Option[beam.PubsubClient.PubsubClientFactory] = None + val DefaultDeadLetterTopic: Option[String] = None + // required for back compatibility def apply(isSubscription: Boolean): ReadParam = if (isSubscription) ReadParam(Subscription) else ReadParam(Topic) } - final case class WriteParam( - maxBatchSize: Option[Int] = None, - maxBatchBytesSize: Option[Int] = None, - clientFactory: Option[beam.PubsubClient.PubsubClientFactory] = None + final case class WriteParam private ( + maxBatchSize: Option[Int] = WriteParam.DefaultMaxBatchSize, + maxBatchBytesSize: Option[Int] = WriteParam.DefaultMaxBatchBytesSize, + clientFactory: Option[beam.PubsubClient.PubsubClientFactory] = WriteParam.DefaultClientFactory ) + object WriteParam { + val DefaultMaxBatchSize: Option[Int] = None + val DefaultMaxBatchBytesSize: Option[Int] = None + val DefaultClientFactory: Option[beam.PubsubClient.PubsubClientFactory] = None + } + def string( name: String, idAttribute: String = null, diff --git a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala index d1079de75c..b2bce808a2 100644 --- a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala +++ b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala @@ -65,42 +65,47 @@ object JdbcIO { } object ReadParam { - private[jdbc] val BeamDefaultFetchSize = -1 - private[jdbc] val DefaultOutputParallelization = true + val BeamDefaultFetchSize: Int = -1 + val DefaultOutputParallelization: Boolean = true + val DefaultStatementPreparator: PreparedStatement => Unit = null + val DefaultDataSourceProviderFn: () => DataSource = null + def dfaultConfigOverride[T]: BJdbcIO.Read[T] => BJdbcIO.Read[T] = identity } - final case class ReadParam[T]( + final case class ReadParam[T] private ( rowMapper: ResultSet => T, - statementPreparator: PreparedStatement => Unit = null, + statementPreparator: PreparedStatement => Unit = ReadParam.DefaultStatementPreparator, fetchSize: Int = ReadParam.BeamDefaultFetchSize, outputParallelization: Boolean = ReadParam.DefaultOutputParallelization, - dataSourceProviderFn: () => DataSource = null, - configOverride: BJdbcIO.Read[T] => BJdbcIO.Read[T] = identity[BJdbcIO.Read[T]] _ + dataSourceProviderFn: () => DataSource = ReadParam.DefaultDataSourceProviderFn, + configOverride: BJdbcIO.Read[T] => BJdbcIO.Read[T] = ReadParam.dfaultConfigOverride ) object WriteParam { - private[jdbc] val BeamDefaultBatchSize = -1L - private[jdbc] val BeamDefaultMaxRetryAttempts = 5 - private[jdbc] val BeamDefaultInitialRetryDelay = org.joda.time.Duration.ZERO - private[jdbc] val BeamDefaultMaxRetryDelay = org.joda.time.Duration.ZERO - private[jdbc] val BeamDefaultRetryConfiguration = BJdbcIO.RetryConfiguration.create( + val BeamDefaultBatchSize = -1L + val BeamDefaultMaxRetryAttempts = 5 + val BeamDefaultInitialRetryDelay = org.joda.time.Duration.ZERO + val BeamDefaultMaxRetryDelay = org.joda.time.Duration.ZERO + val BeamDefaultRetryConfiguration = BJdbcIO.RetryConfiguration.create( BeamDefaultMaxRetryAttempts, BeamDefaultMaxRetryDelay, BeamDefaultInitialRetryDelay ) - private[jdbc] val DefaultRetryStrategy: SQLException => Boolean = + val DefaultRetryStrategy: SQLException => Boolean = new BJdbcIO.DefaultRetryStrategy().apply - private[jdbc] val DefaultAutoSharding: Boolean = false + val DefaultAutoSharding: Boolean = false + val DefaultDataSourceProviderFn: () => DataSource = null + def defaultConfigOverride[T]: BJdbcIO.Write[T] => BJdbcIO.Write[T] = identity } - final case class WriteParam[T]( + final case class WriteParam[T] private ( preparedStatementSetter: (T, PreparedStatement) => Unit, batchSize: Long = WriteParam.BeamDefaultBatchSize, retryConfiguration: BJdbcIO.RetryConfiguration = WriteParam.BeamDefaultRetryConfiguration, retryStrategy: SQLException => Boolean = WriteParam.DefaultRetryStrategy, autoSharding: Boolean = WriteParam.DefaultAutoSharding, - dataSourceProviderFn: () => DataSource = null, - configOverride: BJdbcIO.Write[T] => BJdbcIO.Write[T] = identity[BJdbcIO.Write[T]] _ + dataSourceProviderFn: () => DataSource = WriteParam.DefaultDataSourceProviderFn, + configOverride: BJdbcIO.Write[T] => BJdbcIO.Write[T] = WriteParam.defaultConfigOverride ) } diff --git a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala index 319c0bd6dd..542f75d9c8 100644 --- a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala +++ b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala @@ -19,6 +19,7 @@ package com.spotify.scio.jdbc.syntax import com.spotify.scio.io.ClosedTap import com.spotify.scio.jdbc.{JdbcConnectionOptions, JdbcIO, JdbcWrite, JdbcWriteOptions} +import com.spotify.scio.jdbc.JdbcIO.WriteParam import com.spotify.scio.values.SCollection import org.apache.beam.sdk.io.jdbc.JdbcIO.{RetryConfiguration, Write} @@ -69,12 +70,12 @@ final class JdbcSCollectionOps[T](private val self: SCollection[T]) extends AnyV def saveAsJdbc( connectionOptions: JdbcConnectionOptions, statement: String, - batchSize: Long = JdbcIO.WriteParam.BeamDefaultBatchSize, - retryConfiguration: RetryConfiguration = JdbcIO.WriteParam.BeamDefaultRetryConfiguration, - retryStrategy: SQLException => Boolean = JdbcIO.WriteParam.DefaultRetryStrategy, - autoSharding: Boolean = JdbcIO.WriteParam.DefaultAutoSharding, - dataSourceProviderFn: () => DataSource = null, - configOverride: Write[T] => Write[T] = identity + batchSize: Long = WriteParam.BeamDefaultBatchSize, + retryConfiguration: RetryConfiguration = WriteParam.BeamDefaultRetryConfiguration, + retryStrategy: SQLException => Boolean = WriteParam.DefaultRetryStrategy, + autoSharding: Boolean = WriteParam.DefaultAutoSharding, + dataSourceProviderFn: () => DataSource = WriteParam.DefaultDataSourceProviderFn, + configOverride: Write[T] => Write[T] = WriteParam.defaultConfigOverride )(preparedStatementSetter: (T, PreparedStatement) => Unit): ClosedTap[Nothing] = self.write(JdbcWrite[T](connectionOptions, statement))( JdbcIO.WriteParam( diff --git a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala index 486189d10f..1e2ba77b85 100644 --- a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala +++ b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala @@ -21,6 +21,7 @@ import com.spotify.scio.ScioContext import com.spotify.scio.coders.Coder import com.spotify.scio.jdbc.sharded.{JdbcShardedReadOptions, JdbcShardedSelect} import com.spotify.scio.jdbc.{JdbcConnectionOptions, JdbcIO, JdbcReadOptions, JdbcSelect} +import com.spotify.scio.jdbc.JdbcIO.ReadParam import com.spotify.scio.values.SCollection import org.apache.beam.sdk.io.jdbc.JdbcIO.Read @@ -65,11 +66,11 @@ final class JdbcScioContextOps(private val self: ScioContext) extends AnyVal { def jdbcSelect[T: ClassTag: Coder]( connectionOptions: JdbcConnectionOptions, query: String, - statementPreparator: PreparedStatement => Unit = null, - fetchSize: Int = JdbcIO.ReadParam.BeamDefaultFetchSize, - outputParallelization: Boolean = JdbcIO.ReadParam.DefaultOutputParallelization, - dataSourceProviderFn: () => DataSource = null, - configOverride: Read[T] => Read[T] = identity[Read[T]] _ + statementPreparator: PreparedStatement => Unit = ReadParam.DefaultStatementPreparator, + fetchSize: Int = ReadParam.BeamDefaultFetchSize, + outputParallelization: Boolean = ReadParam.DefaultOutputParallelization, + dataSourceProviderFn: () => DataSource = ReadParam.DefaultDataSourceProviderFn, + configOverride: Read[T] => Read[T] = ReadParam.dfaultConfigOverride )(rowMapper: ResultSet => T): SCollection[T] = self.read(JdbcSelect(connectionOptions, query))( JdbcIO.ReadParam( diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala index 09568357b3..7d157c756e 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/TFRecordIO.scala @@ -84,13 +84,13 @@ object TFRecordIO { } final case class WriteParam private ( - suffix: String, - compression: Compression, - numShards: Int, - filenamePolicySupplier: FilenamePolicySupplier, - prefix: String, - shardNameTemplate: String, - tempDirectory: String + suffix: String = WriteParam.DefaultSuffix, + compression: Compression = WriteParam.DefaultCompression, + numShards: Int = WriteParam.DefaultNumShards, + filenamePolicySupplier: FilenamePolicySupplier = WriteParam.DefaultFilenamePolicySupplier, + prefix: String = WriteParam.DefaultPrefix, + shardNameTemplate: String = WriteParam.DefaultShardNameTemplate, + tempDirectory: String = WriteParam.DefaultTempDirectory ) } From 3a89081ff89d7995ce43442262af084e9dddf85c Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 7 Jun 2023 10:24:49 +0200 Subject: [PATCH 24/26] Add checks for null paths --- .../scala/com/spotify/scio/util/ScioUtil.scala | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala b/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala index 1cd6cf2be2..1f20daa7ca 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/ScioUtil.scala @@ -90,23 +90,28 @@ private[scio] object ScioUtil { } private def stripPath(path: String): String = StringUtils.stripEnd(path, "/") - def strippedPath(path: String): String = s"${stripPath(path)}/" - def pathWithPrefix(path: String, prefix: String): String = Option(prefix) match { - case Some(p) => s"${stripPath(path)}/$p" - case None => s"${stripPath(path)}/part" + def strippedPath(path: String): String = { + require(path != null, "Path must not be null") + s"${stripPath(path)}/" + } + def pathWithPrefix(path: String, prefix: String): String = { + require(path != null, "Path must not be null") + stripPath(path) + "/" + Option(prefix).getOrElse("part") } - def filePattern(path: String, suffix: String): String = + def filePattern(path: String, suffix: String): String = { + require(path != null, "Path must not be null") Option(suffix) match { case Some(_) if path.contains("*") => // path is already a pattern throw new IllegalArgumentException(s"Suffix must be used with a static path but got: $path") case Some(s) => // match all file with suffix in path (must be a folder) - s"${stripPath(path)}/*$s" + stripPath(path) + "/*" + s case None => path } + } def consistentHashCode[K](k: K): Int = k match { case key: Array[_] => ArraySeq.unsafeWrapArray(key).## From 28fca66d5a211d3c0618f6626922e78ef137aeca Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 7 Jun 2023 10:34:00 +0200 Subject: [PATCH 25/26] FIx typo and type param --- scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala | 6 +++--- .../com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala | 2 +- .../com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala index b2bce808a2..9f2a4895e7 100644 --- a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala +++ b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala @@ -69,7 +69,7 @@ object JdbcIO { val DefaultOutputParallelization: Boolean = true val DefaultStatementPreparator: PreparedStatement => Unit = null val DefaultDataSourceProviderFn: () => DataSource = null - def dfaultConfigOverride[T]: BJdbcIO.Read[T] => BJdbcIO.Read[T] = identity + def defaultConfigOverride[T]: BJdbcIO.Read[T] => BJdbcIO.Read[T] = identity } final case class ReadParam[T] private ( @@ -78,7 +78,7 @@ object JdbcIO { fetchSize: Int = ReadParam.BeamDefaultFetchSize, outputParallelization: Boolean = ReadParam.DefaultOutputParallelization, dataSourceProviderFn: () => DataSource = ReadParam.DefaultDataSourceProviderFn, - configOverride: BJdbcIO.Read[T] => BJdbcIO.Read[T] = ReadParam.dfaultConfigOverride + configOverride: BJdbcIO.Read[T] => BJdbcIO.Read[T] = ReadParam.defaultConfigOverride[T] ) object WriteParam { @@ -105,7 +105,7 @@ object JdbcIO { retryStrategy: SQLException => Boolean = WriteParam.DefaultRetryStrategy, autoSharding: Boolean = WriteParam.DefaultAutoSharding, dataSourceProviderFn: () => DataSource = WriteParam.DefaultDataSourceProviderFn, - configOverride: BJdbcIO.Write[T] => BJdbcIO.Write[T] = WriteParam.defaultConfigOverride + configOverride: BJdbcIO.Write[T] => BJdbcIO.Write[T] = WriteParam.defaultConfigOverride[T] ) } diff --git a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala index 542f75d9c8..89a3253da0 100644 --- a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala +++ b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/SCollectionSyntax.scala @@ -75,7 +75,7 @@ final class JdbcSCollectionOps[T](private val self: SCollection[T]) extends AnyV retryStrategy: SQLException => Boolean = WriteParam.DefaultRetryStrategy, autoSharding: Boolean = WriteParam.DefaultAutoSharding, dataSourceProviderFn: () => DataSource = WriteParam.DefaultDataSourceProviderFn, - configOverride: Write[T] => Write[T] = WriteParam.defaultConfigOverride + configOverride: Write[T] => Write[T] = WriteParam.defaultConfigOverride[T] )(preparedStatementSetter: (T, PreparedStatement) => Unit): ClosedTap[Nothing] = self.write(JdbcWrite[T](connectionOptions, statement))( JdbcIO.WriteParam( diff --git a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala index 1e2ba77b85..472890f1f3 100644 --- a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala +++ b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/syntax/ScioContextSyntax.scala @@ -70,7 +70,7 @@ final class JdbcScioContextOps(private val self: ScioContext) extends AnyVal { fetchSize: Int = ReadParam.BeamDefaultFetchSize, outputParallelization: Boolean = ReadParam.DefaultOutputParallelization, dataSourceProviderFn: () => DataSource = ReadParam.DefaultDataSourceProviderFn, - configOverride: Read[T] => Read[T] = ReadParam.dfaultConfigOverride + configOverride: Read[T] => Read[T] = ReadParam.defaultConfigOverride[T] )(rowMapper: ResultSet => T): SCollection[T] = self.read(JdbcSelect(connectionOptions, query))( JdbcIO.ReadParam( From b501af187e2d0a0d1a6a1e93acd92589ef79f263 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Wed, 7 Jun 2023 14:00:57 +0200 Subject: [PATCH 26/26] Scalafix --- .../scala/com/spotify/scio/jdbc/JdbcIO.scala | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala index 9f2a4895e7..036d8ed1e3 100644 --- a/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala +++ b/scio-jdbc/src/main/scala/com/spotify/scio/jdbc/JdbcIO.scala @@ -22,6 +22,7 @@ import com.spotify.scio.coders.{Coder, CoderMaterializer} import com.spotify.scio.io._ import com.spotify.scio.values.SCollection import org.apache.beam.sdk.io.jdbc.{JdbcIO => BJdbcIO} +import org.joda.time.Duration import java.sql.{PreparedStatement, ResultSet, SQLException} import javax.sql.DataSource @@ -82,15 +83,16 @@ object JdbcIO { ) object WriteParam { - val BeamDefaultBatchSize = -1L - val BeamDefaultMaxRetryAttempts = 5 - val BeamDefaultInitialRetryDelay = org.joda.time.Duration.ZERO - val BeamDefaultMaxRetryDelay = org.joda.time.Duration.ZERO - val BeamDefaultRetryConfiguration = BJdbcIO.RetryConfiguration.create( - BeamDefaultMaxRetryAttempts, - BeamDefaultMaxRetryDelay, - BeamDefaultInitialRetryDelay - ) + val BeamDefaultBatchSize: Long = -1L + val BeamDefaultMaxRetryAttempts: Int = 5 + val BeamDefaultInitialRetryDelay: Duration = org.joda.time.Duration.ZERO + val BeamDefaultMaxRetryDelay: Duration = org.joda.time.Duration.ZERO + val BeamDefaultRetryConfiguration: BJdbcIO.RetryConfiguration = + BJdbcIO.RetryConfiguration.create( + BeamDefaultMaxRetryAttempts, + BeamDefaultMaxRetryDelay, + BeamDefaultInitialRetryDelay + ) val DefaultRetryStrategy: SQLException => Boolean = new BJdbcIO.DefaultRetryStrategy().apply val DefaultAutoSharding: Boolean = false