diff --git a/build.sbt b/build.sbt index 063acd141f..9804d4f7b6 100644 --- a/build.sbt +++ b/build.sbt @@ -209,10 +209,12 @@ def previousVersion(currentVersion: String): Option[String] = { lazy val mimaSettings = Def.settings( mimaBinaryIssueFilters := Seq.empty, - mimaPreviousArtifacts := previousVersion(version.value) - .filter(_ => publishArtifact.value) - .map(organization.value % s"${normalizedName.value}_${scalaBinaryVersion.value}" % _) - .toSet + // TODO enable back after 0.14 + mimaPreviousArtifacts := Set.empty +// mimaPreviousArtifacts := previousVersion(version.value) +// .filter(_ => publishArtifact.value) +// .map(organization.value % s"${normalizedName.value}_${scalaBinaryVersion.value}" % _) +// .toSet ) lazy val formatSettings = Def.settings(scalafmtOnCompile := false, javafmtOnCompile := false) @@ -554,11 +556,8 @@ lazy val `scio-core`: Project = project "commons-io" % "commons-io" % commonsIoVersion, "io.grpc" % "grpc-api" % grpcVersion, "joda-time" % "joda-time" % jodaTimeVersion, - "me.lyh" %% "protobuf-generic" % protobufGenericVersion, - "org.apache.avro" % "avro" % avroVersion, // TODO remove from core "org.apache.beam" % "beam-runners-core-construction-java" % beamVersion, "org.apache.beam" % "beam-sdks-java-core" % beamVersion, - "org.apache.beam" % "beam-sdks-java-extensions-avro" % beamVersion, "org.apache.beam" % "beam-sdks-java-extensions-protobuf" % beamVersion, "org.apache.beam" % "beam-vendor-guava-26_0-jre" % beamVendorVersion, "org.apache.commons" % "commons-compress" % commonsCompressVersion, @@ -678,6 +677,7 @@ lazy val `scio-avro`: Project = project "com.google.protobuf" % "protobuf-java" % protobufVersion, "com.twitter" %% "chill" % chillVersion, "com.twitter" % "chill-java" % chillVersion, + "me.lyh" %% "protobuf-generic" % protobufGenericVersion, "org.apache.avro" % "avro" % avroVersion excludeAll ( "com.thoughtworks.paranamer" % "paranamer" ), @@ -704,7 +704,7 @@ lazy val `scio-google-cloud-platform`: Project = project .in(file("scio-google-cloud-platform")) .dependsOn( `scio-core` % "compile;it->it", - `scio-avro` % "test", + `scio-avro`, `scio-test` % "test->test;it" ) .configs(IntegrationTest) 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 2cbab6327d..5adb3afac8 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 @@ -20,11 +20,13 @@ package com.spotify.scio.avro import com.google.protobuf.Message import com.spotify.scio.avro.types.AvroType.HasAvroAnnotation import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} +import com.spotify.scio.avro._ +import com.spotify.scio.{avro, ScioContext} import com.spotify.scio.io._ +import com.spotify.scio.protobuf.util.ProtobufUtil import com.spotify.scio.util.FilenamePolicySupplier -import com.spotify.scio.util.{Functions, ProtobufUtil, ScioUtil} +import com.spotify.scio.util.{Functions, ScioUtil} import com.spotify.scio.values._ -import com.spotify.scio.{avro, ScioContext} import org.apache.avro.Schema import org.apache.avro.file.CodecFactory import org.apache.avro.generic.GenericRecord @@ -70,7 +72,7 @@ final case class ObjectFileIO[T: Coder](path: String) extends ScioIO[T] { */ override protected def write(data: SCollection[T], params: WriteP): Tap[T] = { val elemCoder = CoderMaterializer.beamWithDefault(Coder[T]) - implicit val bcoder = Coder.avroGenericRecordCoder(AvroBytesUtil.schema) + implicit val bcoder = avroGenericRecordCoder(AvroBytesUtil.schema) data .parDo(new DoFn[T, GenericRecord] { @ProcessElement @@ -229,7 +231,7 @@ final case class GenericRecordIO(path: String, schema: Schema) extends AvroIO[Ge * file. */ override protected def read(sc: ScioContext, params: ReadP): SCollection[GenericRecord] = { - val coder = CoderMaterializer.beam(sc, Coder.avroGenericRecordCoder(schema)) + val coder = CoderMaterializer.beam(sc, avroGenericRecordCoder(schema)) val filePattern = ScioUtil.filePattern(path, params.suffix) val t = BAvroIO .readGenericRecords(schema) @@ -321,6 +323,8 @@ object AvroIO { object WriteParam { val DefaultNumShards: Int = 0 val DefaultSuffix: String = ".avro" + val DefaultSuffixProtobuf: String = ".protobuf.avro" + val DefaultSuffixObjectFile: String = ".obj.avro" val DefaultCodec: CodecFactory = CodecFactory.deflateCodec(6) val DefaultMetadata: Map[String, AnyRef] = Map.empty val DefaultFilenamePolicySupplier: FilenamePolicySupplier = null diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/dynamic/package.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/dynamic/package.scala new file mode 100644 index 0000000000..b5a9b88118 --- /dev/null +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/dynamic/package.scala @@ -0,0 +1,27 @@ +/* + * Copyright 2023 Spotify AB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.spotify.scio.avro + +/** + * IO package for avro dynamic destinations. Import All. + * + * {{{ + * import com.spotify.scio.avro.dynamic._ + * }}} + */ +package object dynamic extends syntax.AvroDynamicSCollectionSyntax diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/dynamic/syntax/AvroDynamicSCollectionSyntax.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/dynamic/syntax/AvroDynamicSCollectionSyntax.scala new file mode 100644 index 0000000000..3664e8d7f9 --- /dev/null +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/dynamic/syntax/AvroDynamicSCollectionSyntax.scala @@ -0,0 +1,196 @@ +/* + * Copyright 2019 Spotify AB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.spotify.scio.avro.dynamic.syntax + +import com.google.protobuf.Message +import com.spotify.scio.avro.AvroIO +import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} +import com.spotify.scio.io.{ClosedTap, EmptyTap} +import com.spotify.scio.io.dynamic.syntax.DynamicSCollectionOps +import com.spotify.scio.protobuf.util.ProtobufUtil +import com.spotify.scio.values.SCollection +import org.apache.avro.Schema +import org.apache.avro.file.CodecFactory +import org.apache.avro.generic.GenericRecord +import org.apache.avro.specific.SpecificRecord +import org.apache.beam.sdk.extensions.avro.io.{AvroIO => BAvroIO} + +import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag +import java.util.{HashMap => JHashMap} + +final class DynamicSpecificRecordSCollectionOps[T <: SpecificRecord]( + private val self: SCollection[T] +) extends AnyVal { + import DynamicSCollectionOps.writeDynamic + + /** Save this SCollection as Avro files specified by the destination function. */ + def saveAsDynamicAvroFile( + path: String, + numShards: Int = AvroIO.WriteParam.DefaultNumShards, + suffix: String = AvroIO.WriteParam.DefaultSuffix, + codec: CodecFactory = AvroIO.WriteParam.DefaultCodec, + metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, + tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, + prefix: String = AvroIO.WriteParam.DefaultPrefix + )( + destinationFn: T => String + )(implicit ct: ClassTag[T]): ClosedTap[Nothing] = { + if (self.context.isTest) { + throw new NotImplementedError( + "Avro file with dynamic destinations cannot be used in a test context" + ) + } else { + val cls = ct.runtimeClass.asInstanceOf[Class[T]] + val nm = new JHashMap[String, AnyRef]() + nm.putAll(metadata.asJava) + val sink = BAvroIO + .sink(cls) + .withCodec(codec) + .withMetadata(nm) + val write = + writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) + + self.applyInternal(write) + } + + ClosedTap[Nothing](EmptyTap) + } +} + +/** + * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with dynamic destinations + * methods. + */ +final class DynamicGenericRecordSCollectionOps(private val self: SCollection[GenericRecord]) + extends AnyVal { + import DynamicSCollectionOps.writeDynamic + + /** Save this SCollection as Avro files specified by the destination function. */ + def saveAsDynamicAvroFile( + path: String, + schema: Schema, + numShards: Int = AvroIO.WriteParam.DefaultNumShards, + suffix: String = AvroIO.WriteParam.DefaultSuffix, + codec: CodecFactory = AvroIO.WriteParam.DefaultCodec, + metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, + tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, + prefix: String = AvroIO.WriteParam.DefaultPrefix + )( + destinationFn: GenericRecord => String + ): ClosedTap[Nothing] = { + if (self.context.isTest) { + throw new NotImplementedError( + "Avro file with dynamic destinations cannot be used in a test context" + ) + } else { + val nm = new JHashMap[String, AnyRef]() + nm.putAll(metadata.asJava) + val sink = BAvroIO + .sinkViaGenericRecords( + schema, + (element: GenericRecord, _: Schema) => element + ) + .withCodec(codec) + .withMetadata(nm) + val write = + writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) + + self.applyInternal(write) + } + + ClosedTap[Nothing](EmptyTap) + } +} + +final class DynamicProtobufSCollectionOps[T <: Message](private val self: SCollection[T]) + extends AnyVal { + import DynamicSCollectionOps.writeDynamic + + def saveAsDynamicProtobufFile( + path: String, + numShards: Int = AvroIO.WriteParam.DefaultNumShards, + suffix: String = AvroIO.WriteParam.DefaultSuffixProtobuf, + codec: CodecFactory = AvroIO.WriteParam.DefaultCodec, + metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, + tempDirectory: String = AvroIO.WriteParam.DefaultTempDirectory, + prefix: String = AvroIO.WriteParam.DefaultPrefix + )(destinationFn: T => String)(implicit ct: ClassTag[T]): ClosedTap[Nothing] = { + val protoCoder = Coder.protoMessageCoder[T] + val elemCoder = CoderMaterializer.beam(self.context, protoCoder) + val avroSchema = AvroBytesUtil.schema + val nm = new JHashMap[String, AnyRef]() + nm.putAll((metadata ++ ProtobufUtil.schemaMetadataOf(ct)).asJava) + + if (self.context.isTest) { + throw new NotImplementedError( + "Protobuf file with dynamic destinations cannot be used in a test context" + ) + } else { + val sink = BAvroIO + .sinkViaGenericRecords( + avroSchema, + (element: T, _: Schema) => AvroBytesUtil.encode(elemCoder, element) + ) + .withCodec(codec) + .withMetadata(nm) + val write = writeDynamic( + path = path, + destinationFn = destinationFn, + numShards = numShards, + prefix = prefix, + suffix = suffix, + tempDirectory = tempDirectory + ).via(sink) + + self.applyInternal(write) + } + + ClosedTap[Nothing](EmptyTap) + } +} + +trait AvroDynamicSCollectionSyntax { + implicit def dynamicSpecificRecordSCollectionOps[T <: SpecificRecord]( + sc: SCollection[T] + ): DynamicSpecificRecordSCollectionOps[T] = + new DynamicSpecificRecordSCollectionOps(sc) + + implicit def dynamicGenericRecordSCollectionOps( + sc: SCollection[GenericRecord] + ): DynamicGenericRecordSCollectionOps = + new DynamicGenericRecordSCollectionOps(sc) + + implicit def dynamicProtobufSCollectionOps[T <: Message]( + sc: SCollection[T] + ): DynamicProtobufSCollectionOps[T] = new DynamicProtobufSCollectionOps(sc) +} diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/io/AvroFileStorage.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/io/AvroFileStorage.scala new file mode 100644 index 0000000000..da9379bcef --- /dev/null +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/io/AvroFileStorage.scala @@ -0,0 +1,80 @@ +/* + * Copyright 2019 Spotify AB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.spotify.scio.avro.io + +import com.spotify.scio.io.FileStorage +import org.apache.avro.Schema +import org.apache.avro.file.{DataFileReader, SeekableInput} +import org.apache.avro.generic.GenericDatumReader +import org.apache.avro.specific.SpecificDatumReader +import com.spotify.scio.util.ScioUtil +import org.apache.beam.sdk.io.FileSystems +import org.apache.beam.sdk.io.fs.MatchResult.Metadata + +import java.nio.ByteBuffer +import java.nio.channels.SeekableByteChannel +import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag + +object AvroFileStorage { + @inline final def apply(path: String, suffix: String): AvroFileStorage = + new AvroFileStorage(path, suffix) +} + +final private[scio] class AvroFileStorage(path: String, suffix: String) { + + private def getAvroSeekableInput(meta: Metadata): SeekableInput = + new SeekableInput { + require(meta.isReadSeekEfficient) + private val in = { + val channel = FileSystems.open(meta.resourceId()).asInstanceOf[SeekableByteChannel] + // metadata is lazy loaded on GCS FS and only triggered upon first read + channel.read(ByteBuffer.allocate(1)) + // reset position + channel.position(0) + } + + override def read(b: Array[Byte], off: Int, len: Int): Int = + in.read(ByteBuffer.wrap(b, off, len)) + + override def tell(): Long = in.position() + + override def length(): Long = in.size() + + override def seek(p: Long): Unit = { + in.position(p) + () + } + + override def close(): Unit = in.close() + } + + def avroFile[T](schema: Schema): Iterator[T] = + avroFile(new GenericDatumReader[T](schema)) + + def avroFile[T: ClassTag](): Iterator[T] = + avroFile(new SpecificDatumReader[T](ScioUtil.classOf[T])) + + def avroFile[T](reader: GenericDatumReader[T]): Iterator[T] = + FileStorage + .listFiles(path, suffix) + .map(m => DataFileReader.openReader(getAvroSeekableInput(m), reader)) + .map(_.iterator().asScala) + .reduce(_ ++ _) + +} diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/package.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/package.scala index 64f27bf883..0e715f8bc8 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/package.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/package.scala @@ -18,6 +18,7 @@ package com.spotify.scio import com.spotify.scio.avro.syntax.Syntax +import com.spotify.scio.coders.avro.AvroCoders /** * Main package for Avro APIs. Import all. @@ -26,7 +27,7 @@ import com.spotify.scio.avro.syntax.Syntax * import com.spotify.scio.avro._ * }}} */ -package object avro extends Syntax { +package object avro extends Syntax with AvroCoders { /** Typed Avro annotations and converters. */ val AvroType = com.spotify.scio.avro.types.AvroType diff --git a/scio-core/src/main/scala/com/spotify/scio/schemas/instances/AvroInstances.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/schemas/instances/AvroInstances.scala similarity index 98% rename from scio-core/src/main/scala/com/spotify/scio/schemas/instances/AvroInstances.scala rename to scio-avro/src/main/scala/com/spotify/scio/avro/schemas/instances/AvroInstances.scala index ae811d58b4..41d59cab72 100644 --- a/scio-core/src/main/scala/com/spotify/scio/schemas/instances/AvroInstances.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/schemas/instances/AvroInstances.scala @@ -15,13 +15,13 @@ * under the License. */ -package com.spotify.scio.schemas.instances +package com.spotify.scio.avro.schemas.instances import com.spotify.scio.schemas.{RawRecord, Schema} -import org.apache.avro.specific.SpecificRecord import org.apache.avro.generic.{GenericRecord, IndexedRecord} -import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils +import org.apache.avro.specific.SpecificRecord import org.apache.beam.sdk.extensions.avro.schemas.AvroRecordSchema +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils import org.apache.beam.sdk.schemas.{Schema => BSchema} import org.apache.beam.sdk.transforms.SerializableFunction import org.apache.beam.sdk.values.{Row, TypeDescriptor} diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/schemas/package.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/schemas/package.scala new file mode 100644 index 0000000000..23649df6ad --- /dev/null +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/schemas/package.scala @@ -0,0 +1,20 @@ +/* + * Copyright 2023 Spotify AB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.spotify.scio.avro + +package object schemas extends instances.AvroInstances 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 d2a6c3ccf6..674d55e9b3 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 @@ -18,6 +18,7 @@ package com.spotify.scio.avro.syntax import com.google.protobuf.Message +//import com.spotify.scio.avro.{AvroIO, GenericRecordIO, ObjectFileIO, AvroTyped, SpecificRecordIO, ProtobufIO} import com.spotify.scio.avro._ import com.spotify.scio.avro.types.AvroType.HasAvroAnnotation import com.spotify.scio.coders.Coder @@ -77,7 +78,7 @@ final class ObjectFileSCollectionOps[T](private val self: SCollection[T]) extend def saveAsObjectFile( path: String, numShards: Int = AvroIO.WriteParam.DefaultNumShards, - suffix: String = ".obj.avro", + suffix: String = AvroIO.WriteParam.DefaultSuffixObjectFile, codec: CodecFactory = AvroIO.WriteParam.DefaultCodec, metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, shardNameTemplate: String = AvroIO.WriteParam.DefaultShardNameTemplate, @@ -96,7 +97,7 @@ final class ObjectFileSCollectionOps[T](private val self: SCollection[T]) extend shardNameTemplate, tempDirectory ) - self.write(ObjectFileIO(path))(param) + self.write(ObjectFileIO[T](path))(param) } } @@ -151,7 +152,7 @@ final class TypedAvroSCollectionOps[T <: HasAvroAnnotation](private val self: SC filenamePolicySupplier: FilenamePolicySupplier = AvroIO.WriteParam.DefaultFilenamePolicySupplier, prefix: String = AvroIO.WriteParam.DefaultPrefix - )(implicit ct: ClassTag[T], tt: TypeTag[T], coder: Coder[T]): ClosedTap[T] = { + )(implicit tt: TypeTag[T], coder: Coder[T]): ClosedTap[T] = { val param = AvroIO.WriteParam( numShards, suffix, @@ -177,7 +178,7 @@ final class ProtobufSCollectionOps[T <: Message](private val self: SCollection[T def saveAsProtobufFile( path: String, numShards: Int = AvroIO.WriteParam.DefaultNumShards, - suffix: String = ".protobuf.avro", + suffix: String = AvroIO.WriteParam.DefaultSuffixProtobuf, codec: CodecFactory = AvroIO.WriteParam.DefaultCodec, metadata: Map[String, AnyRef] = AvroIO.WriteParam.DefaultMetadata, shardNameTemplate: String = AvroIO.WriteParam.DefaultShardNameTemplate, @@ -185,7 +186,7 @@ final class ProtobufSCollectionOps[T <: Message](private val self: SCollection[T filenamePolicySupplier: FilenamePolicySupplier = AvroIO.WriteParam.DefaultFilenamePolicySupplier, prefix: String = AvroIO.WriteParam.DefaultPrefix - )(implicit ct: ClassTag[T], coder: Coder[T]): ClosedTap[T] = { + )(implicit ct: ClassTag[T]): ClosedTap[T] = { val param = ProtobufIO.WriteParam( numShards, 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 6276d19811..945703c6d7 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 @@ -19,9 +19,10 @@ package com.spotify.scio.avro import com.google.protobuf.Message import com.spotify.scio._ +import com.spotify.scio.avro.io.AvroFileStorage import com.spotify.scio.avro.types.AvroType.HasAvroAnnotation import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} -import com.spotify.scio.io.{FileStorage, Tap, Taps} +import com.spotify.scio.io.{Tap, Taps} import com.spotify.scio.values._ import com.twitter.chill.Externalizer import org.apache.avro.Schema @@ -41,7 +42,7 @@ final case class GenericRecordTap( private lazy val s = Externalizer(schema) override def value: Iterator[GenericRecord] = - FileStorage(path, params.suffix).avroFile[GenericRecord](s.get) + AvroFileStorage(path, params.suffix).avroFile[GenericRecord](s.get) override def open(sc: ScioContext): SCollection[GenericRecord] = sc.read(GenericRecordIO(path, s.get))(params) @@ -53,7 +54,7 @@ final case class SpecificRecordTap[T <: SpecificRecord: ClassTag: Coder]( params: AvroIO.ReadParam ) extends Tap[T] { override def value: Iterator[T] = - FileStorage(path, params.suffix).avroFile[T]() + AvroFileStorage(path, params.suffix).avroFile[T]() override def open(sc: ScioContext): SCollection[T] = sc.read(SpecificRecordIO[T](path))(params) @@ -69,7 +70,7 @@ final case class GenericRecordParseTap[T: Coder]( params: AvroIO.ReadParam ) extends Tap[T] { override def value: Iterator[T] = - FileStorage(path, params.suffix) + AvroFileStorage(path, params.suffix) // Read Avro GenericRecords, with the writer specified schema .avroFile[GenericRecord](schema = null) .map(parseFn) @@ -88,7 +89,7 @@ final case class ObjectFileTap[T: Coder]( ) extends Tap[T] { override def value: Iterator[T] = { val elemCoder = CoderMaterializer.beamWithDefault(Coder[T]) - FileStorage(path, params.suffix) + AvroFileStorage(path, params.suffix) .avroFile[GenericRecord](AvroBytesUtil.schema) .map(r => AvroBytesUtil.decode(elemCoder, r)) } diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/AvroBytesUtil.scala b/scio-avro/src/main/scala/com/spotify/scio/coders/AvroBytesUtil.scala similarity index 99% rename from scio-core/src/main/scala/com/spotify/scio/coders/AvroBytesUtil.scala rename to scio-avro/src/main/scala/com/spotify/scio/coders/AvroBytesUtil.scala index a5d32c6aa7..ebca52ca9f 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/AvroBytesUtil.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/coders/AvroBytesUtil.scala @@ -17,13 +17,12 @@ package com.spotify.scio.coders -import java.nio.ByteBuffer - -import org.apache.avro.{Schema => ASchema} import org.apache.avro.generic.{GenericData, GenericRecord} +import org.apache.avro.{Schema => ASchema} import org.apache.beam.sdk.coders.{Coder => BCoder} import org.apache.beam.sdk.util.CoderUtils +import java.nio.ByteBuffer import scala.jdk.CollectionConverters._ private[scio] object AvroBytesUtil { diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/AvroCoders.scala b/scio-avro/src/main/scala/com/spotify/scio/coders/avro/AvroCoders.scala similarity index 99% rename from scio-core/src/main/scala/com/spotify/scio/coders/instances/AvroCoders.scala rename to scio-avro/src/main/scala/com/spotify/scio/coders/avro/AvroCoders.scala index 4ba9b93c9b..1f65cf9194 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/AvroCoders.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/coders/avro/AvroCoders.scala @@ -15,7 +15,7 @@ * under the License. */ -package com.spotify.scio.coders.instances +package com.spotify.scio.coders.avro import com.spotify.scio.coders.Coder import com.spotify.scio.util.ScioUtil diff --git a/scio-core/src/main/scala/com/spotify/scio/util/ProtobufUtil.scala b/scio-avro/src/main/scala/com/spotify/scio/protobuf/util/ProtobufUtil.scala similarity index 92% rename from scio-core/src/main/scala/com/spotify/scio/util/ProtobufUtil.scala rename to scio-avro/src/main/scala/com/spotify/scio/protobuf/util/ProtobufUtil.scala index 74a07fa14a..ecebe9ff8a 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/ProtobufUtil.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/protobuf/util/ProtobufUtil.scala @@ -15,9 +15,10 @@ * under the License. */ -package com.spotify.scio.util +package com.spotify.scio.protobuf.util import com.google.protobuf.Message +import com.spotify.scio.avro._ import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord @@ -32,8 +33,7 @@ object ProtobufUtil { * * `implicit val avroMessageCoder: Coder[GenericRecord] = ProtobufUtil.AvroMessageCoder` */ - lazy val AvroMessageCoder: Coder[GenericRecord] = - Coder.avroGenericRecordCoder(AvroBytesUtil.schema) + lazy val AvroMessageCoder: Coder[GenericRecord] = avroGenericRecordCoder(AvroBytesUtil.schema) /** The Avro [[Schema]] corresponding to an Avro-encoded Protobuf [[Message]]. */ lazy val AvroMessageSchema: Schema = AvroBytesUtil.schema 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 6728c377c0..ad2838adab 100644 --- a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala +++ b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala @@ -712,6 +712,24 @@ class ScioContext private[scio] ( ): SCollection[String] = this.read(TextIO(path))(TextIO.ReadParam(compression, emptyMatchTreatment, suffix)) + /** + * Get an SCollection of `Array[Byte]` from a binary file. + * + * @param reader + * An instance of `BinaryFileReader` for the specific binary format used by the input file. + * @group input + */ + def binaryFile( + path: String, + reader: BinaryIO.BinaryFileReader, + compression: beam.Compression = BinaryIO.ReadParam.DefaultCompression, + emptyMatchTreatment: beam.fs.EmptyMatchTreatment = TextIO.ReadParam.DefaultEmptyMatchTreatment, + suffix: String = BinaryIO.ReadParam.DefaultSuffix + ): SCollection[Array[Byte]] = + this.read(BinaryIO(path))( + BinaryIO.ReadParam(reader, compression, emptyMatchTreatment, suffix) + ) + /** * Get an SCollection with a custom input transform. The transform should have a unique name. * @group input diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/Coder.scala b/scio-core/src/main/scala/com/spotify/scio/coders/Coder.scala index 96b8f3e0cd..1cfb37a258 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/Coder.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/Coder.scala @@ -217,7 +217,6 @@ sealed trait CoderGrammar { object Coder extends CoderGrammar with TupleCoders - with AvroCoders with ProtobufCoders with AlgebirdCoders with GuavaCoders 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 7d5c7f22d8..381442176a 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 @@ -17,16 +17,20 @@ package com.spotify.scio.io +import com.google.common.io.CountingInputStream + import java.io.{BufferedInputStream, InputStream, OutputStream} -import java.nio.channels.{Channels, WritableByteChannel} +import java.nio.channels.{Channels, ReadableByteChannel, WritableByteChannel} import com.spotify.scio.ScioContext import com.spotify.scio.io.BinaryIO.BytesSink -import com.spotify.scio.util.ScioUtil -import com.spotify.scio.util.FilenamePolicySupplier +import com.spotify.scio.util.{FilenamePolicySupplier, Functions, ScioUtil} import com.spotify.scio.values.SCollection +import org.apache.beam.sdk.coders.ByteArrayCoder +import org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment import org.apache.beam.sdk.io._ import org.apache.beam.sdk.io.fs.MatchResult.Metadata -import org.apache.beam.sdk.io.fs.ResourceId +import org.apache.beam.sdk.io.fs.{EmptyMatchTreatment, ResourceId} +import org.apache.beam.sdk.options.PipelineOptions import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider import org.apache.beam.sdk.transforms.SerializableFunctions import org.apache.beam.sdk.util.MimeTypes @@ -42,14 +46,36 @@ import scala.util.Try * a path to write to. */ final case class BinaryIO(path: String) extends ScioIO[Array[Byte]] { - override type ReadP = Nothing + override type ReadP = BinaryIO.ReadParam override type WriteP = BinaryIO.WriteParam + // write options are insufficient to derive a reader for the output files override val tapT: TapT.Aux[Array[Byte], Nothing] = EmptyTapOf[Array[Byte]] override def testId: String = s"BinaryIO($path)" - override protected def read(sc: ScioContext, params: ReadP): SCollection[Array[Byte]] = - throw new UnsupportedOperationException("BinaryIO is write-only") + override protected def read(sc: ScioContext, params: ReadP): SCollection[Array[Byte]] = { + val filePattern = ScioUtil.filePattern(path, params.suffix) + val desiredBundleSizeBytes = 64 * 1024 * 1024L // 64 mb + val coder = ByteArrayCoder.of() + val srcFn = Functions.serializableFn { path: String => + new BinaryIO.BinarySource(path, params.emptyMatchTreatment, params.reader) + } + + sc.withName("Create filepattern") + .parallelize(List(filePattern)) + .applyTransform("Match All", FileIO.matchAll()) + .applyTransform( + "Read Matches", + FileIO + .readMatches() + .withCompression(params.compression) + .withDirectoryTreatment(DirectoryTreatment.PROHIBIT) + ) + .applyTransform( + "Read all via FileBasedSource", + new ReadAllViaFileBasedSource[Array[Byte]](desiredBundleSizeBytes, srcFn, coder) + ) + } private def binaryOut( path: String, @@ -109,7 +135,7 @@ final case class BinaryIO(path: String) extends ScioIO[Array[Byte]] { EmptyTap } - override def tap(params: Nothing): Tap[Nothing] = EmptyTap + override def tap(read: BinaryIO.ReadParam): Tap[Nothing] = EmptyTap } object BinaryIO { @@ -206,4 +232,114 @@ object BinaryIO { } } } + + object ReadParam { + val DefaultSuffix: String = ".bin" + val DefaultCompression: Compression = Compression.UNCOMPRESSED + val DefaultEmptyMatchTreatment: EmptyMatchTreatment = EmptyMatchTreatment.DISALLOW + } + + final case class ReadParam private ( + reader: BinaryFileReader, + compression: Compression = ReadParam.DefaultCompression, + emptyMatchTreatment: EmptyMatchTreatment = ReadParam.DefaultEmptyMatchTreatment, + suffix: String = ReadParam.DefaultSuffix + ) + + /** Trait for reading binary file formats */ + trait BinaryFileReader { + + /** + * State constructed during `start` and maintained through reading, potentially updated during + * each record read + */ + type State + + /** Read any header and construct the initial read state. */ + def start(is: InputStream): State + + /** + * @return + * The updated read state, record bytes. Return null for record byte array when no record is + * read. + */ + def readRecord(state: State, is: InputStream): (State, Array[Byte]) + + /** Read any footer and perform any required validation after the last record is read. */ + def end(state: State, is: InputStream): Unit = () + } + + final private[scio] class BinarySingleFileSource( + binaryFileReader: BinaryFileReader, + metadata: Metadata, + start: Long, + end: Long + ) extends FileBasedSource[Array[Byte]](metadata, Long.MaxValue, start, end) { + override def isSplittable: Boolean = false + + override def createForSubrangeOfFile( + fileMetadata: Metadata, + start: Long, + end: Long + ): FileBasedSource[Array[Byte]] = + throw new NotImplementedError() + + override def createSingleFileReader( + options: PipelineOptions + ): FileBasedSource.FileBasedReader[Array[Byte]] = + new FileBasedSource.FileBasedReader[Array[Byte]](this) { + var is: CountingInputStream = _ + var state: binaryFileReader.State = _ + var startOfRecord: Long = _ + var current: Option[Array[Byte]] = None + + // exception matches method contract + override def getCurrentOffset: Long = current.map(_ => startOfRecord).get + + // exception matches method contract + override def getCurrent: Array[Byte] = current.get + + override def startReading(channel: ReadableByteChannel): Unit = { + is = new CountingInputStream(Channels.newInputStream(channel)) + val newState = binaryFileReader.start(is) + state = newState + } + + override def readNextRecord(): Boolean = { + startOfRecord = is.getCount + 1 + val (newState, record) = binaryFileReader.readRecord(state, is) + state = newState + current = Option(record) + current match { + case Some(_) => true + case None => + binaryFileReader.end(state, is) + false + } + } + } + } + + final private[scio] class BinarySource( + path: String, + emptyMatchTreatment: EmptyMatchTreatment, + binaryFileReader: BinaryFileReader + ) extends FileBasedSource[Array[Byte]]( + StaticValueProvider.of(path), + emptyMatchTreatment, + Long.MaxValue + ) { + override def isSplittable: Boolean = false + + override def createForSubrangeOfFile( + fileMetadata: Metadata, + start: Long, + end: Long + ): FileBasedSource[Array[Byte]] = + new BinarySingleFileSource(binaryFileReader, fileMetadata, start, end) + + override def createSingleFileReader( + options: PipelineOptions + ): FileBasedSource.FileBasedReader[Array[Byte]] = throw new NotImplementedError() + } } 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 4a26729bfa..892c9d7c32 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 @@ -18,10 +18,6 @@ package com.spotify.scio.io import com.spotify.scio.util.ScioUtil -import org.apache.avro.Schema -import org.apache.avro.file.{DataFileReader, SeekableInput} -import org.apache.avro.generic.GenericDatumReader -import org.apache.avro.specific.SpecificDatumReader import org.apache.beam.sdk.io.FileSystems import org.apache.beam.sdk.io.fs.EmptyMatchTreatment import org.apache.beam.sdk.io.fs.MatchResult.Metadata @@ -29,13 +25,11 @@ 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.channels.Channels 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 private[scio] object FileStorage { @@ -43,55 +37,24 @@ 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] = + def listFiles(path: String, suffix: String): Seq[Metadata] = FileSystems .`match`(ScioUtil.filePattern(path, suffix), EmptyMatchTreatment.DISALLOW) .metadata() .iterator .asScala .toSeq +} - private def getObjectInputStream(meta: Metadata): InputStream = - Channels.newInputStream(FileSystems.open(meta.resourceId())) - - private def getAvroSeekableInput(meta: Metadata): SeekableInput = - new SeekableInput { - require(meta.isReadSeekEfficient) - private val in = { - val channel = FileSystems.open(meta.resourceId()).asInstanceOf[SeekableByteChannel] - // metadata is lazy loaded on GCS FS and only triggered upon first read - channel.read(ByteBuffer.allocate(1)) - // reset position - channel.position(0) - } - override def read(b: Array[Byte], off: Int, len: Int): Int = - in.read(ByteBuffer.wrap(b, off, len)) - override def tell(): Long = in.position() - override def length(): Long = in.size() - override def seek(p: Long): Unit = { - in.position(p) - () - } - override def close(): Unit = in.close() - } +final private[scio] class FileStorage(path: String, suffix: String) { - def avroFile[T](schema: Schema): Iterator[T] = - avroFile(new GenericDatumReader[T](schema)) + import FileStorage._ - def avroFile[T: ClassTag](): Iterator[T] = - avroFile(new SpecificDatumReader[T](ScioUtil.classOf[T])) + private def listFiles: Seq[Metadata] = FileStorage.listFiles(path, suffix) - def avroFile[T](reader: GenericDatumReader[T]): Iterator[T] = - listFiles - .map(m => DataFileReader.openReader(getAvroSeekableInput(m), reader)) - .map(_.iterator().asScala) - .reduce(_ ++ _) + private def getObjectInputStream(meta: Metadata): InputStream = + Channels.newInputStream(FileSystems.open(meta.resourceId())) def textFile: Iterator[String] = { val factory = new CompressorStreamFactory() 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 7437c41981..ab55e61316 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 @@ -18,15 +18,14 @@ package com.spotify.scio.io import java.util.UUID -import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} +import com.spotify.scio.coders.{Coder, CoderMaterializer} import com.spotify.scio.util.ScioUtil import com.spotify.scio.values.SCollection import com.spotify.scio.{ScioContext, ScioResult} -import org.apache.avro.generic.GenericRecord -import org.apache.beam.sdk.coders.{Coder => BCoder} -import org.apache.beam.sdk.extensions.avro.io.AvroIO -import org.apache.beam.sdk.transforms.DoFn -import org.apache.beam.sdk.transforms.DoFn.{Element, OutputReceiver, ProcessElement} +import org.apache.beam.sdk.coders.{ByteArrayCoder, Coder => BCoder} +import org.apache.beam.sdk.util.CoderUtils + +import java.io.{EOFException, InputStream} /** * Placeholder to an external data set that can either be load into memory as an iterator or opened @@ -88,12 +87,31 @@ final private[scio] class MaterializeTap[T: Coder] private (path: String, coder: extends Tap[T] { override def value: Iterator[T] = { - val storage = FileStorage(path, MaterializeTap.Suffix) - + val storage = FileStorage(path, BinaryIO.ReadParam.DefaultSuffix) if (storage.isDone()) { - storage - .avroFile[GenericRecord](AvroBytesUtil.schema) - .map(AvroBytesUtil.decode(coder, _)) + val filePattern = ScioUtil.filePattern(path, BinaryIO.ReadParam.DefaultSuffix) + BinaryIO + .openInputStreamsFor(filePattern) + .flatMap { is => + new Iterator[T] { + private val reader = MaterializeTap.MaterializeReader + private val ignoredState = reader.start(is) + private var rec: Option[Array[Byte]] = None + read() + + def read(): Unit = { + val (_, optRecord) = reader.readRecord(ignoredState, is) + rec = Option(optRecord).flatMap(x => if (x.isEmpty) None else Some(x)) + } + + override def hasNext: Boolean = rec.isDefined + override def next(): T = { + val ret = rec.map(arr => CoderUtils.decodeFromByteArray(coder, arr)).get + read() + ret + } + } + } } else { throw new RuntimeException( "Tap failed to materialize to filesystem. Did you " + @@ -102,31 +120,30 @@ final private[scio] class MaterializeTap[T: Coder] private (path: String, coder: } } - private def dofn = - new DoFn[GenericRecord, T] { - @ProcessElement - private[scio] def processElement( - @Element element: GenericRecord, - out: OutputReceiver[T] - ): Unit = - out.output(AvroBytesUtil.decode(coder, element)) - } - override def open(sc: ScioContext): SCollection[T] = sc.requireNotClosed { - val filePattern = ScioUtil.filePattern(path, MaterializeTap.Suffix) - val read = AvroIO - .readGenericRecords(AvroBytesUtil.schema) - .from(filePattern) - sc.applyTransform(read).parDo(dofn) + sc.binaryFile(path, reader = MaterializeTap.MaterializeReader) + .map(ar => CoderUtils.decodeFromByteArray[T](coder, ar)) } } 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])) + + case object MaterializeReader extends BinaryIO.BinaryFileReader { + private val c: BCoder[Array[Byte]] = ByteArrayCoder.of() + override type State = Unit + override def start(is: InputStream): State = () + override def readRecord(state: State, is: InputStream): (State, Array[Byte]) = { + val out = + try { + c.decode(is) + } catch { + case _: EOFException => null + } + (state, out) + } + } } final case class ClosedTap[T] private (private[scio] val underlying: Tap[T]) { 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 bfeecbfc92..b7638c0700 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 @@ -17,24 +17,15 @@ package com.spotify.scio.io.dynamic.syntax -import com.google.protobuf.Message import com.spotify.scio.io.{ClosedTap, EmptyTap, TextIO} -import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} -import com.spotify.scio.util.{Functions, ProtobufUtil, ScioUtil} +import com.spotify.scio.util.{Functions, ScioUtil} import com.spotify.scio.values.SCollection -import org.apache.avro.Schema -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.{Compression, FileIO} import org.apache.beam.sdk.{io => beam} -import org.apache.beam.sdk.extensions.avro.io.{AvroIO => BAvroIO} -import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import scala.util.chaining._ -import java.util.{HashMap => JHashMap} object DynamicSCollectionOps { private[scio] def writeDynamic[A]( @@ -57,104 +48,6 @@ object DynamicSCollectionOps { } } -final class DynamicSpecificRecordSCollectionOps[T <: SpecificRecord]( - private val self: SCollection[T] -) extends AnyVal { - import DynamicSCollectionOps.writeDynamic - - /** Save this SCollection as Avro files specified by the destination function. */ - def saveAsDynamicAvroFile( - path: String, - numShards: Int = 0, - suffix: String = ".avro", - codec: CodecFactory = CodecFactory.deflateCodec(6), - metadata: Map[String, AnyRef] = Map.empty, - tempDirectory: String = null, - prefix: String = null - )( - destinationFn: T => String - )(implicit ct: ClassTag[T]): ClosedTap[Nothing] = { - if (self.context.isTest) { - throw new NotImplementedError( - "Avro file with dynamic destinations cannot be used in a test context" - ) - } else { - val cls = ct.runtimeClass.asInstanceOf[Class[T]] - val nm = new JHashMap[String, AnyRef]() - nm.putAll(metadata.asJava) - val sink = BAvroIO - .sink(cls) - .withCodec(codec) - .withMetadata(nm) - val write = - writeDynamic( - path = path, - destinationFn = destinationFn, - numShards = numShards, - prefix = prefix, - suffix = suffix, - tempDirectory = tempDirectory - ).via(sink) - - self.applyInternal(write) - } - - ClosedTap[Nothing](EmptyTap) - } -} - -/** - * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with dynamic destinations - * methods. - */ -final class DynamicGenericRecordSCollectionOps[T <: GenericRecord](private val self: SCollection[T]) - extends AnyVal { - import DynamicSCollectionOps.writeDynamic - - /** Save this SCollection as Avro files specified by the destination function. */ - def saveAsDynamicAvroFile( - path: String, - schema: Schema, - numShards: Int = 0, - suffix: String = ".avro", - codec: CodecFactory = CodecFactory.deflateCodec(6), - metadata: Map[String, AnyRef] = Map.empty, - tempDirectory: String = null, - prefix: String = null - )( - destinationFn: T => String - ): ClosedTap[Nothing] = { - if (self.context.isTest) { - throw new NotImplementedError( - "Avro file with dynamic destinations cannot be used in a test context" - ) - } else { - val nm = new JHashMap[String, AnyRef]() - nm.putAll(metadata.asJava) - val sink = BAvroIO - .sinkViaGenericRecords( - schema, - (element: T, _: Schema) => element - ) - .withCodec(codec) - .withMetadata(nm) - val write = - writeDynamic( - path = path, - destinationFn = destinationFn, - numShards = numShards, - prefix = prefix, - suffix = suffix, - tempDirectory = tempDirectory - ).via(sink) - - self.applyInternal(write) - } - - ClosedTap[Nothing](EmptyTap) - } -} - /** * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with dynamic destinations * methods. @@ -204,68 +97,7 @@ final class DynamicSCollectionOps[T](private val self: SCollection[T]) extends A } } -final class DynamicProtobufSCollectionOps[T <: Message](private val self: SCollection[T]) - extends AnyVal { - import DynamicSCollectionOps.writeDynamic - - def saveAsDynamicProtobufFile( - path: String, - numShards: Int = 0, - suffix: String = ".protobuf", - codec: CodecFactory = CodecFactory.deflateCodec(6), - metadata: Map[String, AnyRef] = Map.empty, - 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) - val avroSchema = AvroBytesUtil.schema - val nm = new JHashMap[String, AnyRef]() - nm.putAll((metadata ++ ProtobufUtil.schemaMetadataOf(ct)).asJava) - - if (self.context.isTest) { - throw new NotImplementedError( - "Protobuf file with dynamic destinations cannot be used in a test context" - ) - } else { - val sink = BAvroIO - .sinkViaGenericRecords( - avroSchema, - (element: T, _: Schema) => AvroBytesUtil.encode(elemCoder, element) - ) - .withCodec(codec) - .withMetadata(nm) - val write = writeDynamic( - path = path, - destinationFn = destinationFn, - numShards = numShards, - prefix = prefix, - suffix = suffix, - tempDirectory = tempDirectory - ).via(sink) - - self.applyInternal(write) - } - - ClosedTap[Nothing](EmptyTap) - } -} - trait SCollectionSyntax { - implicit def dynamicSpecificRecordSCollectionOps[T <: SpecificRecord]( - sc: SCollection[T] - ): DynamicSpecificRecordSCollectionOps[T] = - new DynamicSpecificRecordSCollectionOps(sc) - - implicit def dynamicGenericRecordSCollectionOps[T <: GenericRecord]( - sc: SCollection[T] - ): DynamicGenericRecordSCollectionOps[T] = - new DynamicGenericRecordSCollectionOps(sc) - implicit def dynamicSCollectionOps[T](sc: SCollection[T]): DynamicSCollectionOps[T] = new DynamicSCollectionOps(sc) - - implicit def dynamicProtobufSCollectionOps[T <: Message]( - sc: SCollection[T] - ): DynamicProtobufSCollectionOps[T] = new DynamicProtobufSCollectionOps(sc) } diff --git a/scio-core/src/main/scala/com/spotify/scio/schemas/Schema.scala b/scio-core/src/main/scala/com/spotify/scio/schemas/Schema.scala index f0b03af355..0d33875e1f 100644 --- a/scio-core/src/main/scala/com/spotify/scio/schemas/Schema.scala +++ b/scio-core/src/main/scala/com/spotify/scio/schemas/Schema.scala @@ -18,10 +18,8 @@ package com.spotify.scio.schemas import java.util.{List => jList, Map => jMap} - import com.spotify.scio.{FeatureFlag, IsJavaBean, MacroSettings} import com.spotify.scio.schemas.instances.{ - AvroInstances, JavaInstances, JodaInstances, LowPrioritySchemaDerivation, @@ -40,7 +38,7 @@ import org.apache.beam.sdk.values.TupleTag import scala.collection.{mutable, SortedSet} -object Schema extends JodaInstances with AvroInstances with LowPrioritySchemaDerivation { +object Schema extends JodaInstances with LowPrioritySchemaDerivation { @inline final def apply[T](implicit c: Schema[T]): Schema[T] = c implicit val jByteSchema: Type[java.lang.Byte] = JavaInstances.jByteSchema diff --git a/scio-core/src/main/scala/com/spotify/scio/schemas/instances/AllInstances.scala b/scio-core/src/main/scala/com/spotify/scio/schemas/instances/AllInstances.scala index 0ec98b0987..ba83fc497b 100644 --- a/scio-core/src/main/scala/com/spotify/scio/schemas/instances/AllInstances.scala +++ b/scio-core/src/main/scala/com/spotify/scio/schemas/instances/AllInstances.scala @@ -21,5 +21,4 @@ trait AllInstances extends ScalaInstances with JavaInstances with JodaInstances - with AvroInstances with LowPrioritySchemaDerivation 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 9fb9938488..bf1d46bf09 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 @@ -21,7 +21,7 @@ import java.io.PrintStream import java.lang.{Boolean => JBoolean, Double => JDouble, Iterable => JIterable} import java.util.concurrent.ThreadLocalRandom import com.spotify.scio.ScioContext -import com.spotify.scio.coders.{AvroBytesUtil, BeamCoders, Coder, CoderMaterializer} +import com.spotify.scio.coders.{BeamCoders, Coder, CoderMaterializer} import com.spotify.scio.estimators.{ ApproxDistinctCounter, ApproximateUniqueCounter, @@ -35,19 +35,17 @@ import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.util._ import com.spotify.scio.util.random.{BernoulliSampler, PoissonSampler} import com.twitter.algebird.{Aggregator, Monoid, MonoidAggregator, Semigroup} -import org.apache.avro.file.CodecFactory -import org.apache.beam.sdk.coders.{Coder => BCoder} +import org.apache.beam.sdk.coders.{ByteArrayCoder, Coder => BCoder} import org.apache.beam.sdk.schemas.SchemaCoder import org.apache.beam.sdk.io.Compression import org.apache.beam.sdk.io.FileIO.ReadMatches.DirectoryTreatment import org.apache.beam.sdk.transforms.DoFn.{Element, OutputReceiver, ProcessElement, Timestamp} import org.apache.beam.sdk.transforms._ import org.apache.beam.sdk.transforms.windowing._ -import org.apache.beam.sdk.util.SerializableUtils +import org.apache.beam.sdk.util.{CoderUtils, SerializableUtils} import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode import org.apache.beam.sdk.values._ import org.apache.beam.sdk.{io => beam} -import org.apache.beam.sdk.extensions.avro.io.{AvroIO => BAvroIO} import org.joda.time.{Duration, Instant} import org.slf4j.LoggerFactory @@ -1553,18 +1551,17 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] { saveAsInMemoryTap } else { val elemCoder = CoderMaterializer.beam(context, coder) - val schema = AvroBytesUtil.schema - val avroCoder = Coder.avroGenericRecordCoder(schema) - val write = BAvroIO - .writeGenericRecords(schema) - .to(ScioUtil.pathWithPrefix(path, "part")) - .withSuffix(".obj.avro") - .withCodec(CodecFactory.deflateCodec(6)) - .withMetadata(Map.empty[String, AnyRef].asJava) - + val arrCoder = ByteArrayCoder.of() this - .map(c => AvroBytesUtil.encode(elemCoder, c))(avroCoder) - .applyInternal(write) + .map { e => + CoderUtils.encodeToByteArray( + arrCoder, + CoderUtils.encodeToByteArray(elemCoder, e), + // encode record length + BCoder.Context.NESTED + ) + } + .saveAsBinaryFile(path) ClosedTap(MaterializeTap[T](path, context)) } 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..25934777dd 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 @@ -24,9 +24,7 @@ package com.spotify.scio.examples.extra import com.spotify.scio._ -import com.spotify.scio.coders.Coder import com.spotify.scio.avro._ -import com.spotify.scio.avro.Account import com.spotify.scio.avro.types.AvroType import com.spotify.scio.io.ClosedTap import org.apache.avro.Schema @@ -106,7 +104,7 @@ object AvroExample { private def genericOut(sc: ScioContext, args: Args): ClosedTap[GenericRecord] = { // Avro generic record encoding is more efficient with an explicit schema - implicit def genericCoder = Coder.avroGenericRecordCoder(schema) + implicit def genericCoder = avroGenericRecordCoder(schema) sc.parallelize(1 to 100) .map[GenericRecord] { i => val r = new GenericData.Record(schema) diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BinaryInOut.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BinaryInOut.scala new file mode 100644 index 0000000000..bd8cf93554 --- /dev/null +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BinaryInOut.scala @@ -0,0 +1,115 @@ +/* + * Copyright 2023 Spotify AB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.spotify.scio.examples.extra + +import com.spotify.scio.ContextAndArgs +import java.nio.ByteBuffer +import com.spotify.scio.io.BinaryIO.BinaryFileReader +import java.io.InputStream + +// Example: Binary Input and Output +// Usage: +// `sbt "runMain com.spotify.scio.examples.extra.BinaryInOut +// --project=[PROJECT] --runner=DataflowRunner --region=[REGION NAME] --output=[OUTPUT]"` + +object BinaryInOut { + def main(cmdlineArgs: Array[String]): Unit = { + val (sc, args) = ContextAndArgs(cmdlineArgs) + + val in = (1 to 10) + .map(i => 0 to i) + .map(_.toArray.map(_.toByte)) + + def intToPaddedArray(i: Int): Array[Byte] = ByteBuffer.allocate(4).putInt(i).array() + + sc.parallelize(in) + .saveAsBinaryFile( + args("output"), + // Write each file with a static header ... + header = Array[Byte](1, 2, 3), + // a record prefix containing an integer record length ... + framePrefix = arr => intToPaddedArray(arr.length), + // and a single byte suffix. + frameSuffix = _ => Array[Byte](0) + ) + sc.run().waitUntilDone() + + val (sc2, _) = ContextAndArgs(cmdlineArgs) + sc2 + // Read back the just-written data, parsing the written files with `MyBinaryFileReader`. + .binaryFile(args("output"), MyBinaryFileReader) + .debug() + sc2.run().waitUntilDone() + } + + // This BinaryFileReader subclass must match the behavior of the write. + case object MyBinaryFileReader extends BinaryFileReader { + // This example has no state, but a record header could for example contain a number of + // expected records in the entire file or some block, and the state could then be used track + // the number of records read and to determine when the reader would need to switch modes from + // reading records to reading block metadata or the file footer. + override type State = Unit + + private def fail(msg: String) = throw new IllegalStateException(msg) + + override def start(is: InputStream): State = { + // Read the expected magic number from the first bytes of the file, and fail if it is not + // found. + val b = new Array[Byte](3) + val readBytes = is.read(b) + if (readBytes != b.length) fail("Failed to read header") + + val magicNumberOk = b(0) == 1 && b(1) == 2 && b(2) == 3 + if (!magicNumberOk) fail("Failed to find correct magic number") + () + } + + override def readRecord(state: State, is: InputStream): (State, Array[Byte]) = { + // Read the number of expected bytes for a record, corresponding to the `framePrefix` + // argument. + val sizeBuf = new Array[Byte](4) + val sizeBytesRead = is.read(sizeBuf) + // If the entire file has been consumed so return `null` to indicate that the read is + // complete. If a record count was maintained in `state`, then once all records were read a + // similar value should be returned. + if (sizeBytesRead == -1) (state, null) + else { + if (sizeBytesRead != 4) fail(s"Failed to read record size $sizeBytesRead") + else { + val size = ByteBuffer.wrap(sizeBuf).getInt + if (size < 0) fail(s"Bad record size $size") + val elementBytes = new Array[Byte](size) + val bytesRead = is.read(elementBytes) + // Ensure all expected bytes are read. + if (bytesRead != size) fail("Failed to read expected record bytes") + else { + val suffix = is.read() + // Ensure `frameSuffix` is read. + if (suffix != 0) fail(s"Failed to read expected record suffix $suffix") + // Return the record and state. If a record count was maintained in `state` it would + // be incremented here. + else (state, elementBytes) + } + } + } + } + + // There is no footer to read and no validation which needs to occur, so return Unit + override def end(state: State, is: InputStream): Unit = () + } +} 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..2321171a33 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 @@ -22,7 +22,6 @@ package com.spotify.scio.examples.extra import com.spotify.scio._ -import com.spotify.scio.coders.Coder import com.spotify.scio.avro._ import com.spotify.scio.examples.common.ExampleData @@ -48,7 +47,7 @@ object MagnolifyAvroWriteExample { import MagnolifyAvroExample._ val (sc, args) = ContextAndArgs(cmdlineArgs) - implicit def genericCoder = Coder.avroGenericRecordCoder(wordCountType.schema) + implicit def genericCoder = avroGenericRecordCoder(wordCountType.schema) sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) .countByValue diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/ParquetExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/ParquetExample.scala index 55264817be..5825a12c53 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/ParquetExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/ParquetExample.scala @@ -24,10 +24,10 @@ package com.spotify.scio.examples.extra import com.google.protobuf.ByteString import com.spotify.scio._ +import com.spotify.scio.avro._ import com.spotify.scio.parquet.avro._ import com.spotify.scio.parquet.types._ import com.spotify.scio.parquet.tensorflow._ -import com.spotify.scio.avro.{Account, AccountStatus} import com.spotify.scio.coders.Coder import com.spotify.scio.io.ClosedTap import com.spotify.scio.parquet.ParquetConfiguration @@ -120,7 +120,7 @@ object ParquetExample { private def avroGenericIn(sc: ScioContext, args: Args): ClosedTap[String] = { val schema = Account.getClassSchema - implicit val genericRecordCoder: Coder[GenericRecord] = Coder.avroGenericRecordCoder(schema) + implicit val genericRecordCoder: Coder[GenericRecord] = avroGenericRecordCoder(schema) val parquetIn = sc.parquetAvroFile[GenericRecord](args("input"), schema) diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/SortMergeBucketExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/SortMergeBucketExample.scala index ed758c4401..26765e03bd 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/SortMergeBucketExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/SortMergeBucketExample.scala @@ -27,7 +27,7 @@ package com.spotify.scio.examples.extra import com.spotify.scio.{ContextAndArgs, ScioContext} -import com.spotify.scio.avro.Account +import com.spotify.scio.avro._ import com.spotify.scio.coders.Coder import org.apache.avro.Schema import org.apache.avro.file.CodecFactory @@ -70,7 +70,7 @@ object SortMergeBucketWriteExample { import com.spotify.scio.smb._ implicit val coder: Coder[GenericRecord] = - Coder.avroGenericRecordCoder(SortMergeBucketExample.UserDataSchema) + avroGenericRecordCoder(SortMergeBucketExample.UserDataSchema) def pipeline(cmdLineArgs: Array[String]): ScioContext = { val (sc, args) = ContextAndArgs(cmdLineArgs) @@ -126,7 +126,7 @@ object SortMergeBucketJoinExample { import com.spotify.scio.smb._ implicit val coder: Coder[GenericRecord] = - Coder.avroGenericRecordCoder(SortMergeBucketExample.UserDataSchema) + avroGenericRecordCoder(SortMergeBucketExample.UserDataSchema) case class UserAccountData(userId: String, age: Int, balance: Double) { override def toString: String = s"$userId\t$age\t$balance" diff --git a/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigquery/TypedBigQueryIT.scala b/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigquery/TypedBigQueryIT.scala index be36504ed5..b0e6dd9dbb 100644 --- a/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigquery/TypedBigQueryIT.scala +++ b/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigquery/TypedBigQueryIT.scala @@ -19,9 +19,9 @@ package com.spotify.scio.bigquery import com.google.protobuf.ByteString import com.spotify.scio._ +import com.spotify.scio.avro._ import com.spotify.scio.bigquery.BigQueryTypedTable.Format import com.spotify.scio.bigquery.client.BigQuery -import com.spotify.scio.coders.Coder import com.spotify.scio.testing._ import magnolify.scalacheck.auto._ import org.apache.avro.{LogicalTypes, Schema} @@ -109,7 +109,7 @@ class TypedBigQueryIT extends PipelineSpec with BeforeAndAfterAll { it should "convert to avro format" in { val sc = ScioContext(options) - implicit val coder = Coder.avroGenericRecordCoder(Record.avroSchema) + implicit val coder = avroGenericRecordCoder(Record.avroSchema) sc.typedBigQuery[Record](tableRowTable) .map(Record.toAvro) .map(Record.fromAvro) should containInAnyOrder( @@ -128,7 +128,7 @@ class TypedBigQueryIT extends PipelineSpec with BeforeAndAfterAll { it should "read GenericRecord recors" in { val sc = ScioContext(options) - implicit val coder = Coder.avroGenericRecordCoder(Record.avroSchema) + implicit val coder = avroGenericRecordCoder(Record.avroSchema) sc .bigQueryTable(tableRowTable, Format.GenericRecord) .map(Record.fromAvro) should containInAnyOrder(records) @@ -137,7 +137,7 @@ class TypedBigQueryIT extends PipelineSpec with BeforeAndAfterAll { it should "write GenericRecord records" in { val sc = ScioContext(options) - implicit val coder = Coder.avroGenericRecordCoder(Record.avroSchema) + implicit val coder = avroGenericRecordCoder(Record.avroSchema) val schema = BigQueryUtil.parseSchema(""" |{ @@ -188,7 +188,7 @@ class TypedBigQueryIT extends PipelineSpec with BeforeAndAfterAll { new Schema.Field("datetime", Schema.create(Schema.Type.STRING), "", "") ).asJava ) - implicit val coder = Coder.avroGenericRecordCoder(schema) + implicit val coder = avroGenericRecordCoder(schema) val ltRecords: Seq[GenericRecord] = Seq( new GenericRecordBuilder(schema) 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 209d9ff3f8..97dca8aca2 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 @@ -23,6 +23,7 @@ import com.spotify.scio.coders.{Coder, CoderMaterializer} import com.spotify.scio.testing.TestDataManager import com.spotify.scio.util.{Functions, JMapWrapper, ScioUtil} import com.spotify.scio.values.SCollection +import com.spotify.scio.avro._ import com.spotify.scio.io._ import com.spotify.scio.pubsub.coders._ import org.apache.avro.specific.SpecificRecord @@ -209,7 +210,7 @@ final private case class AvroPubsubIOWithoutAttributes[T <: SpecificRecord: Clas private[this] val cls = ScioUtil.classOf[T] override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = { - val coder = CoderMaterializer.beam(sc, Coder.avroSpecificRecordCoder[T]) + val coder = CoderMaterializer.beam(sc, avroSpecificRecordCoder[T]) val t = setup(beam.PubsubIO.readAvros(cls), params) sc.applyTransform(t).setCoder(coder) } 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..946a7520e8 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 @@ -65,8 +65,7 @@ class GroupByBenchmark { } val source = "src/test/resources/events-10000-0.avro" - implicit val coderGenericRecord: Coder[GenericRecord] = - Coder.avroGenericRecordCoder(avroSchema) + implicit val coderGenericRecord: Coder[GenericRecord] = avroGenericRecordCoder(avroSchema) val charCoder: BCoder[Char] = CoderMaterializer.beamWithDefault(Coder[Char]) val doubleCoder: BCoder[Double] = CoderMaterializer.beamWithDefault(Coder[Double]) diff --git a/scio-neo4j/src/it/scala/com/spotify/scio/neo4j/Neo4jIOIT.scala b/scio-neo4j/src/it/scala/com/spotify/scio/neo4j/Neo4jIOIT.scala index 6bf7b7a038..5197723022 100644 --- a/scio-neo4j/src/it/scala/com/spotify/scio/neo4j/Neo4jIOIT.scala +++ b/scio-neo4j/src/it/scala/com/spotify/scio/neo4j/Neo4jIOIT.scala @@ -2,13 +2,12 @@ package com.spotify.scio.neo4j import com.dimafeng.testcontainers.{ForAllTestContainer, Neo4jContainer} import com.spotify.scio.testing.PipelineSpec -import org.apache.beam.sdk.options.PipelineOptionsFactory +import org.apache.beam.sdk.options.{PipelineOptions, PipelineOptionsFactory} import org.neo4j.driver.{AuthTokens, Driver, GraphDatabase} import org.scalatest.concurrent.Eventually import org.testcontainers.utility.DockerImageName import scala.jdk.CollectionConverters._ -import org.apache.beam.sdk.options.PipelineOptions object Neo4jIOIT { 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 80d400d6d4..93c3efe2f7 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 @@ -19,7 +19,6 @@ package com.spotify.scio.parquet.avro 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, ScioIOTest, TapSpec, TextIO} import com.spotify.scio.parquet.ParquetConfiguration @@ -188,7 +187,7 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { implicit val coder = { GenericData.get().addLogicalTypeConversion(new TimeConversions.TimestampConversion) GenericData.get().addLogicalTypeConversion(new Conversions.DecimalConversion) - Coder.avroGenericRecordCoder(TestLogicalTypes.SCHEMA$) + avroGenericRecordCoder(TestLogicalTypes.SCHEMA$) } val sc1 = ScioContext() @@ -279,7 +278,7 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { 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) + implicit val coder = avroGenericRecordCoder(AvroUtils.schema) sc1 .parallelize(genericRecords) .saveAsParquetAvroFile(dir.getAbsolutePath, numShards = 1, schema = AvroUtils.schema) @@ -303,7 +302,7 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { val genericRecords = (0 until 10).map(AvroUtils.newGenericRecord) val options = PipelineOptionsFactory.fromArgs("--streaming=true").create() val sc1 = ScioContext(options) - implicit val coder = Coder.avroGenericRecordCoder(AvroUtils.schema) + implicit val coder = avroGenericRecordCoder(AvroUtils.schema) sc1 .parallelize(genericRecords) // Explicit optional arguments `Duration.Zero` and `WindowOptions()` as a workaround for the @@ -364,7 +363,7 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { 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) + implicit val coder = avroGenericRecordCoder(AvroUtils.schema) sc.parallelize(genericRecords) .saveAsParquetAvroFile( dir.getAbsolutePath, @@ -393,7 +392,7 @@ class ParquetAvroIOTest extends ScioIOSpec with TapSpec with BeforeAndAfterAll { 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) + implicit val coder = avroGenericRecordCoder(AvroUtils.schema) an[NotImplementedError] should be thrownBy { val sc = ScioContext() 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 cadaa1ff92..1302d08147 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 @@ -17,8 +17,7 @@ package com.spotify.scio.parquet.read import com.spotify.scio.ScioContext -import com.spotify.scio.avro.Account -import com.spotify.scio.coders.Coder +import com.spotify.scio.avro._ import com.spotify.scio.parquet.ParquetConfiguration import com.spotify.scio.parquet.avro._ import com.spotify.scio.parquet.types._ @@ -184,7 +183,7 @@ class ParquetReadFnTest extends PipelineSpec with BeforeAndAfterAll { new GenericRecordBuilder(projection).set("id", i).build() } - implicit val coder = Coder.avroGenericRecordCoder(projection) + implicit val coder = avroGenericRecordCoder(projection) val sc = ScioContext() sc .parallelize(listFiles(s"${testSingleDir.getAbsolutePath}/avro")) @@ -201,7 +200,7 @@ class ParquetReadFnTest extends PipelineSpec with BeforeAndAfterAll { it should "work with a projection and projectionFn" in { val projection = Projection[Account](_.getId) - implicit val coder = Coder.avroGenericRecordCoder(projection) + implicit val coder = avroGenericRecordCoder(projection) val sc = ScioContext() sc .parallelize(listFiles(s"${testSingleDir.getAbsolutePath}/avro")) @@ -219,7 +218,7 @@ class ParquetReadFnTest extends PipelineSpec with BeforeAndAfterAll { it should "work with a projection and projectionFn on files with multiple row groups" in { val projection = Projection[Account](_.getId) - implicit val coder = Coder.avroGenericRecordCoder(projection) + implicit val coder = avroGenericRecordCoder(projection) val sc = ScioContext() sc .parallelize(listFiles(s"${testMultiDir.getAbsolutePath}/avro")) 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 b5e55269f6..01bb28fa10 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 @@ -53,7 +53,7 @@ class SortMergeBucketParityIT extends AnyFlatSpec with Matchers { private val keyFn: GenericRecord => Integer = _.get("key").toString.toInt - implicit private val coder: Coder[GenericRecord] = Coder.avroGenericRecordCoder(schema) + implicit private val coder: Coder[GenericRecord] = avroGenericRecordCoder(schema) "sortMergeCoGroup" should "have parity with a 2-way CoGroup" in withNumSources(2) { inputs => compareResults( diff --git a/scio-test/src/test/scala/com/spotify/scio/coders/AvroCoderTest.scala b/scio-test/src/test/scala/com/spotify/scio/avro/AvroCoderTest.scala similarity index 92% rename from scio-test/src/test/scala/com/spotify/scio/coders/AvroCoderTest.scala rename to scio-test/src/test/scala/com/spotify/scio/avro/AvroCoderTest.scala index f4408fc47b..77ca3b94a1 100644 --- a/scio-test/src/test/scala/com/spotify/scio/coders/AvroCoderTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/avro/AvroCoderTest.scala @@ -15,13 +15,14 @@ * under the License. */ -package com.spotify.scio.coders +package com.spotify.scio.avro -import org.scalatest.flatspec.AnyFlatSpec +import com.spotify.scio.coders.{Avro, Coder, FixedSpecificDataExample} import com.spotify.scio.testing.CoderAssertions._ import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificRecord import org.scalactic.Equality +import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers final class AvroCoderTest extends AnyFlatSpec with Matchers { @@ -48,7 +49,8 @@ final class AvroCoderTest extends AnyFlatSpec with Matchers { val schema = Avro.user.getSchema val record: GenericRecord = Avro.user - implicit val c: Coder[GenericRecord] = Coder.avroGenericRecordCoder(schema) + implicit val c: Coder[GenericRecord] = avroGenericRecordCoder + schema implicit val eq: Equality[GenericRecord] = (a: GenericRecord, b: Any) => a.toString === b.toString 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 a17621a7e2..2d58c3098c 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 @@ -22,7 +22,6 @@ import java.nio.file.Files import com.spotify.scio.{CoreSysProps, ScioContext} import com.spotify.scio.avro.AvroUtils.schema import com.spotify.scio.avro._ -import com.spotify.scio.coders.Coder import com.spotify.scio.proto.Track.TrackPB import com.spotify.scio.testing._ import com.spotify.scio.util.ScioUtil @@ -30,6 +29,8 @@ import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.values.{SCollection, WindowOptions} import org.apache.avro.file.CodecFactory import org.apache.avro.generic.GenericRecord +import org.apache.beam.sdk.coders.ByteArrayCoder +import org.apache.beam.sdk.util.CoderUtils import org.apache.beam.sdk.values.PCollection.IsBounded import org.apache.commons.io.FileUtils import org.joda.time.{Duration, Instant} @@ -329,7 +330,7 @@ class ScioIOTest extends ScioIOSpec { it should "work with GenericRecord" in { import AvroUtils.schema - implicit val coder = Coder.avroGenericRecordCoder(schema) + implicit val 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)) @@ -343,7 +344,7 @@ class ScioIOTest extends ScioIOSpec { } it should "work with GenericRecord and a parseFn" in { - implicit val coder = Coder.avroGenericRecordCoder(schema) + implicit val coder = avroGenericRecordCoder(schema) val xs = (1 to 100).map(AvroUtils.newGenericRecord) // No test for saveAsAvroFile because parseFn is only for input testJobTest(xs)(AvroIO(_))( @@ -499,11 +500,43 @@ class ScioIOTest extends ScioIOSpec { } "BinaryIO" should "work" in { - val xs = (1 to 100).map(i => ByteBuffer.allocate(4).putInt(i).array) - testJobTestOutput(xs)(BinaryIO(_))(_.saveAsBinaryFile(_)) + val xs = List((1 to 100).toArray.map(_.toByte)) + testJobTest(xs)(BinaryIO(_))(_.binaryFile(_, MaterializeTap.MaterializeReader))( + _.saveAsBinaryFile(_) + ) + } + + it should "round-trip records" in { + import org.apache.beam.sdk.coders.{Coder => BCoder} + val bac = ByteArrayCoder.of() + val in = (1 to 10) + .map(i => 0 to i) + .map(_.toArray.map(_.toByte)) + + val tmpDir = Files.createTempDirectory("binary-io-") + val sc = ScioContext() + val xs = in.map(CoderUtils.encodeToByteArray(bac, _, BCoder.Context.NESTED)) + sc.parallelize(xs).saveAsBinaryFile(tmpDir.toString) + sc.run() + + import com.google.protobuf.ByteString + val sc2 = ScioContext() + val x = sc2 + .binaryFile(tmpDir.toString, MaterializeTap.MaterializeReader) + .map(ByteString.copyFrom) + val expected = sc2.parallelize(in).map(ByteString.copyFrom) + x.intersection(expected).count.tap { cnt => + if (cnt != in.size) throw new IllegalStateException(s"Expected ${in.size}, got $cnt") + } + x.subtract(expected).count.tap { cnt => + if (cnt != 0) throw new IllegalStateException(s"Expected 0, got $cnt") + } + sc2.run() + + FileUtils.deleteDirectory(tmpDir.toFile) } - "BinaryIO" should "output files to $prefix/part-*" in { + it should "output files to $prefix/part-*" in { val tmpDir = Files.createTempDirectory("binary-io-") val sc = ScioContext() 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 2cbde8960e..d78967ab0c 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 @@ -72,7 +72,7 @@ trait TapSpec extends PipelineSpec { class TapTest extends TapSpec { val schema: Schema = newGenericRecord(1).getSchema - implicit def coder: Coder[GenericRecord] = Coder.avroGenericRecordCoder(schema) + implicit def coder: Coder[GenericRecord] = avroGenericRecordCoder(schema) private def makeRecords(sc: ScioContext) = sc.parallelize(Seq(1, 2, 3)) @@ -111,7 +111,7 @@ class TapTest extends TapSpec { it should "support saveAsAvroFile with reflect record" in withTempDir { dir => import com.spotify.scio.coders.AvroBytesUtil - implicit val coder = Coder.avroGenericRecordCoder(AvroBytesUtil.schema) + implicit val coder = avroGenericRecordCoder(AvroBytesUtil.schema) val tap = runWithFileFuture { _.parallelize(Seq("a", "b", "c")) 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 ae95a183f4..a664275142 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 @@ -21,7 +21,7 @@ import java.nio.file.Files 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.avro.dynamic._ import com.spotify.scio.io.TapSpec import com.spotify.scio.proto.SimpleV2.SimplePB import com.spotify.scio.testing.PipelineSpec @@ -124,7 +124,7 @@ class DynamicFileTest extends PipelineSpec with TapSpec { it should "support generic Avro files" in withTempDir { dir => val sc1 = ScioContext() - implicit val coder = Coder.avroGenericRecordCoder(schema) + implicit val coder = avroGenericRecordCoder(schema) sc1 .parallelize(1 to 10) .map(newGenericRecord) @@ -206,8 +206,8 @@ class DynamicFileTest extends PipelineSpec with TapSpec { verifyOutput(dir, "even", "odd") val sc2 = ScioContext() - val even = sc2.protobufFile[SimplePB](s"$dir/even/*.protobuf") - val odd = sc2.protobufFile[SimplePB](s"$dir/odd/*.protobuf") + val even = sc2.protobufFile[SimplePB](s"$dir/even/*.protobuf.avro") + val odd = sc2.protobufFile[SimplePB](s"$dir/odd/*.protobuf.avro") val (expectedEven, expectedOdd) = (1 to 10).partition(_ % 2 == 0) even should containInAnyOrder(expectedEven.map(mkProto)) odd should containInAnyOrder(expectedOdd.map(mkProto)) 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..eef7504925 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 @@ -71,7 +71,7 @@ object GenericAvroFileJob { object GenericParseFnAvroFileJob { - implicit val coder: Coder[GenericRecord] = Coder.avroGenericRecordCoder(AvroUtils.schema) + implicit val coder: Coder[GenericRecord] = avroGenericRecordCoder(AvroUtils.schema) // A class with some fields from the Avro Record case class PartialFieldsAvro(intField: Int) @@ -366,7 +366,7 @@ class JobTestTest extends PipelineSpec { } def testGenericAvroFileJob(xs: Seq[GenericRecord]): Unit = { - implicit val coder = Coder.avroGenericRecordCoder + implicit val coder = avroGenericRecordCoder JobTest[GenericAvroFileJob.type] .args("--input=in.avro", "--output=out.avro") .input(AvroIO[GenericRecord]("in.avro"), (1 to 3).map(newGenericRecord)) @@ -389,7 +389,7 @@ class JobTestTest extends PipelineSpec { def testGenericParseAvroFileJob(xs: Seq[GenericRecord]): Unit = { import GenericParseFnAvroFileJob.PartialFieldsAvro - implicit val coder: Coder[GenericRecord] = Coder.avroGenericRecordCoder + implicit val coder: Coder[GenericRecord] = avroGenericRecordCoder JobTest[GenericParseFnAvroFileJob.type] .args("--input=in.avro", "--output=out.avro") .input(AvroIO[PartialFieldsAvro]("in.avro"), (1 to 3).map(PartialFieldsAvro)) 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 8adbc9e607..6ec6f2fa2c 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 @@ -20,11 +20,11 @@ package com.spotify.scio.util import java.io.File import java.nio.channels.Channels import java.nio.file.Files - import com.spotify.scio.ScioContext import com.spotify.scio.avro._ import com.spotify.scio.coders.Coder import com.spotify.scio.proto.Track.TrackPB +import com.spotify.scio.protobuf.util.ProtobufUtil import org.apache.avro.file.DataFileStream import org.apache.avro.generic.{GenericDatumReader, GenericRecord} import org.apache.beam.sdk.io.{FileSystems, LocalResources} diff --git a/scio-test/src/test/scala/com/spotify/scio/values/SCollectionTest.scala b/scio-test/src/test/scala/com/spotify/scio/values/SCollectionTest.scala index 89cc4ca5c0..10f1dfeae3 100644 --- a/scio-test/src/test/scala/com/spotify/scio/values/SCollectionTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/values/SCollectionTest.scala @@ -44,10 +44,11 @@ import com.spotify.scio.schemas.Schema import org.apache.beam.sdk.coders.{NullableCoder, StringUtf8Coder} import java.nio.charset.StandardCharsets +import java.util.concurrent.ConcurrentLinkedQueue object SCollectionTest { // used to check local side effect in tap() - val elements: mutable.Buffer[Any] = mutable.Buffer.empty + val elements = new ConcurrentLinkedQueue[Any]() } class SCollectionTest extends PipelineSpec { @@ -875,7 +876,7 @@ class SCollectionTest extends PipelineSpec { val input = Seq(1, 2, 3) runWithContext { sc => val original = sc.parallelize(input) - val tapped = original.tap(elements += _) + val tapped = original.tap(elements.add) // tap should not modify internal coder val originalCoder = original.internal.getCoder diff --git a/site/src/main/paradox/io/Binary.md b/site/src/main/paradox/io/Binary.md index bdf3e0b4fd..a0f7d14724 100644 --- a/site/src/main/paradox/io/Binary.md +++ b/site/src/main/paradox/io/Binary.md @@ -2,7 +2,21 @@ ## Read Binary files -See @ref:[read as binary](ReadFiles.md#read-as-binary) for reading an entire file as a binary record. +See @ref:[read as binary](ReadFiles.md#read-as-binary) for reading an entire file as a single binary record. + +Binary reads are supported via the @scaladoc[binaryFile](com.spotify.scio.ScioContext#binaryFile(path:String,reader:com.spotify.scio.io.BinaryIO.BinaryFileReader,compression:org.apache.beam.sdk.io.Compression,emptyMatchTreatment:org.apache.beam.sdk.io.fs.EmptyMatchTreatment,suffix:String):com.spotify.scio.values.SCollection[Array[Byte]]), with a @scaladoc[BinaryFileReader](com.spotify.scio.io.BinaryIO.BinaryFileReader) instance provided that can parse the underlying binary file format. + +```scala mdoc:compile-only +import com.spotify.scio.ScioContext +import com.spotify.scio.io.BinaryIO.BinaryFileReader + +val sc: ScioContext = ??? +val myBinaryFileReader: BinaryFileReader = ??? +sc.binaryFile("gs://", myBinaryFileReader) +``` + +The complexity of the reader is determined by the complexity of the input format. +See @extref[BinaryInOut](example:BinaryInOut) for a fully-worked example. ## Write Binary files @@ -15,18 +29,21 @@ val byteArrays: SCollection[Array[Byte]] = ??? byteArrays.saveAsBinaryFile("gs://") ``` -A static `header` and `footer` argument are provided, along with the framing parameters `framePrefix` and `frameSuffix`: +A static `header` and `footer` argument are provided, along with the framing parameters `framePrefix` and `frameSuffix`. +In this example, we record a magic number in the header along with the number of records in the file and a magic number in the footer. ```scala mdoc:compile-only import com.spotify.scio.values.SCollection import java.nio.ByteBuffer +def intToPaddedArray(i: Int) = ByteBuffer.allocate(4).putInt(i).array() + val byteArrays: SCollection[Array[Byte]] = ??? byteArrays.saveAsBinaryFile( "gs://", header = Array(1, 2, 3), footer = Array(4, 5, 6), - framePrefix = arr => ByteBuffer.allocate(4).putInt(arr.length).array(), + framePrefix = arr => intToPaddedArray(arr.length), frameSuffix = _ => Array(0) ) ```