From 417a68583385dc9210c9a9d66bb5a24748258144 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Mon, 12 Feb 2024 13:03:27 +0100 Subject: [PATCH 1/8] Update base version to 0.15 --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index dd69361e31..1e04403495 100644 --- a/build.sbt +++ b/build.sbt @@ -147,7 +147,7 @@ val scalatestplusVersion = s"$scalatestVersion.0" val NothingFilter: explicitdeps.ModuleFilter = { _ => false } // project -ThisBuild / tlBaseVersion := "0.14" +ThisBuild / tlBaseVersion := "0.15" ThisBuild / tlSonatypeUseLegacyHost := true ThisBuild / organization := "com.spotify" ThisBuild / organizationName := "Spotify AB" From bcd7d5e0eb9545d714383f23a97ee419a50db515 Mon Sep 17 00:00:00 2001 From: kellen Date: Wed, 10 Apr 2024 09:50:02 -0400 Subject: [PATCH 2/8] Magnolify API (#5286) --- build.sbt | 9 +- .../com/spotify/scio/avro/AvroTypedIO.scala | 45 ++++- .../com/spotify/scio/avro/ObjectFileIO.scala | 6 +- .../com/spotify/scio/avro/ProtobufIO.scala | 80 ++++++-- .../scio/avro/syntax/SCollectionSyntax.scala | 96 +++++++++- .../scio/avro/syntax/ScioContextSyntax.scala | 26 ++- .../scala/com/spotify/scio/avro/taps.scala | 4 +- .../spotify/scio/avro/types/AvroType.scala | 11 ++ .../avro/types/ConverterProviderTest.scala | 1 - .../examples/extra/MagnolifyAvroExample.scala | 19 +- .../extra/MagnolifyBigtableExample.scala | 32 ++-- .../extra/MagnolifyDatastoreExample.scala | 18 +- .../extra/MagnolifyTensorFlowExample.scala | 14 +- .../MagnolifyTypedBigQueryTornadoes.scala | 72 ++++++++ ...gnolifyTypedStorageBigQueryTornadoes.scala | 79 ++++++++ .../extra/MagnolifyAvroExampleTest.scala | 14 +- .../extra/MagnolifyBigtableExampleTest.scala | 58 ++++++ .../extra/MagnolifyDatastoreExampleTest.scala | 17 +- .../MagnolifyTensorFlowExampleTest.scala | 44 ++--- .../MagnolifyTypedBigQueryTornadoesTest.scala | 46 +++++ ...ifyTypedStorageBigQueryTornadoesTest.scala | 51 ++++++ .../spotify/scio/bigquery/BigQueryIO.scala | 130 ++++++++++++- .../dynamic/syntax/SCollectionSyntax.scala | 44 ++++- .../com/spotify/scio/bigquery/package.scala | 2 + .../bigquery/syntax/MagnolifySyntax.scala | 148 +++++++++++++++ .../com/spotify/scio/bigquery/taps.scala | 10 + .../scio/bigquery/types/BigQueryType.scala | 14 ++ .../spotify/scio/bigtable/BigTableIO.scala | 172 ++++++++++++++---- .../bigtable/syntax/SCollectionSyntax.scala | 54 +++++- .../bigtable/syntax/ScioContextSyntax.scala | 121 +++++++++--- .../spotify/scio/datastore/DatastoreIO.scala | 94 ++++++++-- .../datastore/syntax/SCollectionSyntax.scala | 25 ++- .../datastore/syntax/ScioContextSyntax.scala | 25 ++- .../scio/bigquery/BigQueryIOTest.scala | 54 ++++++ .../scio/bigtable/BigtableIOTest.scala | 24 +++ .../scio/datastore/DatastoreIOTest.scala | 42 ++++- .../spotify/scio/tensorflow/TFRecordIO.scala | 57 +++++- .../tensorflow/syntax/SCollectionSyntax.scala | 38 ++++ .../tensorflow/syntax/ScioContextSyntax.scala | 18 +- .../scio/tensorflow/TFExampleIOTest.scala | 9 +- .../scio/testing/SCollectionMatchers.scala | 2 +- .../com/spotify/scio/testing/ScioIOSpec.scala | 18 +- .../com/spotify/scio/avro/AvroIOTest.scala | 23 ++- 43 files changed, 1602 insertions(+), 264 deletions(-) create mode 100644 scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoes.scala create mode 100644 scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoes.scala create mode 100644 scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExampleTest.scala create mode 100644 scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoesTest.scala create mode 100644 scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoesTest.scala create mode 100644 scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/MagnolifySyntax.scala diff --git a/build.sbt b/build.sbt index eb9890422f..c5246e65e2 100644 --- a/build.sbt +++ b/build.sbt @@ -750,6 +750,9 @@ lazy val `scio-avro` = project // compile "com.esotericsoftware" % "kryo-shaded" % kryoVersion, "com.google.protobuf" % "protobuf-java" % protobufVersion, + "com.spotify" %% "magnolify-avro" % magnolifyVersion, + "com.spotify" %% "magnolify-protobuf" % magnolifyVersion, + "com.spotify" %% "magnolify-shared" % magnolifyVersion, "com.twitter" %% "chill" % chillVersion, "com.twitter" % "chill-java" % chillVersion, "me.lyh" %% "protobuf-generic" % protobufGenericVersion, @@ -809,6 +812,10 @@ lazy val `scio-google-cloud-platform` = project "com.google.http-client" % "google-http-client" % googleHttpClientVersion, "com.google.http-client" % "google-http-client-gson" % googleHttpClientVersion, "com.google.protobuf" % "protobuf-java" % protobufVersion, + "com.spotify" %% "magnolify-bigquery" % magnolifyVersion, + "com.spotify" %% "magnolify-bigtable" % magnolifyVersion, + "com.spotify" %% "magnolify-datastore" % magnolifyVersion, + "com.spotify" %% "magnolify-shared" % magnolifyVersion, "com.twitter" %% "chill" % chillVersion, "com.twitter" % "chill-java" % chillVersion, "commons-io" % "commons-io" % commonsIoVersion, @@ -1123,6 +1130,7 @@ lazy val `scio-tensorflow` = project "org.apache.beam" % "beam-sdks-java-core" % beamVersion, "org.apache.beam" % "beam-vendor-guava-32_1_2-jre" % beamVendorVersion, "org.apache.commons" % "commons-compress" % commonsCompressVersion, + "com.spotify" %% "magnolify-tensorflow" % magnolifyVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.tensorflow" % "ndarray" % ndArrayVersion, "org.tensorflow" % "tensorflow-core-api" % tensorFlowVersion, @@ -1227,7 +1235,6 @@ lazy val `scio-examples` = project "com.spotify" %% "magnolify-avro" % magnolifyVersion, "com.spotify" %% "magnolify-bigtable" % magnolifyVersion, "com.spotify" %% "magnolify-datastore" % magnolifyVersion, - "com.spotify" %% "magnolify-shared" % magnolifyVersion, "com.spotify" %% "magnolify-tensorflow" % magnolifyVersion, "com.twitter" %% "algebird-core" % algebirdVersion, "joda-time" % "joda-time" % jodaTimeVersion, diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroTypedIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroTypedIO.scala index d82ca8df4c..bfaed96d9b 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/AvroTypedIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/AvroTypedIO.scala @@ -21,6 +21,7 @@ import com.spotify.scio.avro.types.AvroType.HasAvroAnnotation import com.spotify.scio.coders.Coder import com.spotify.scio.io.{ScioIO, Tap, TapOf, TapT} import com.spotify.scio.values.SCollection +import magnolify.avro.{AvroType => AvroMagnolifyType} import org.apache.avro.generic.GenericRecord import scala.reflect.runtime.universe._ @@ -38,13 +39,18 @@ final case class AvroTypedIO[T <: HasAvroAnnotation: TypeTag: Coder](path: Strin private lazy val underlying: GenericRecordIO = GenericRecordIO(path, schema) override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = - sc.read(underlying)(params).map(avroT.fromGenericRecord) + sc.transform(_.read(underlying)(params).map(avroT.fromGenericRecord)) override protected def write(data: SCollection[T], params: WriteP): Tap[T] = { val datumFactory = Option(params.datumFactory).getOrElse(GenericRecordDatumFactory) implicit val coder: Coder[GenericRecord] = avroCoder(datumFactory, schema) - data.map(avroT.toGenericRecord).write(underlying)(params) - tap(AvroIO.ReadParam(params)) + underlying + .writeWithContext( + data.transform(_.map(avroT.toGenericRecord)), + params + ) + .underlying + .map(avroT.fromGenericRecord) } override def tap(read: ReadP): Tap[T] = @@ -61,6 +67,35 @@ object AvroTypedIO { @deprecated("Use AvroTypedIO instead", "0.14.0") object AvroTyped { type AvroIO[T <: HasAvroAnnotation] = AvroTypedIO[T] - def AvroIO[T <: HasAvroAnnotation: TypeTag: Coder](path: String): AvroIO[T] = - AvroTypedIO[T](path) + def AvroIO[T <: HasAvroAnnotation: TypeTag: Coder](path: String): AvroIO[T] = AvroTypedIO[T](path) +} + +final case class AvroMagnolifyTypedIO[T: AvroMagnolifyType: Coder](path: String) extends ScioIO[T] { + override type ReadP = AvroMagnolifyTypedIO.ReadParam + override type WriteP = AvroMagnolifyTypedIO.WriteParam + override val tapT: TapT.Aux[T, T] = TapOf[T] + + override def testId: String = s"AvroIO($path)" + + private lazy val avroT: AvroMagnolifyType[T] = implicitly + private lazy val schema = avroT.schema + private lazy val underlying: GenericRecordIO = GenericRecordIO(path, schema) + + override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = + sc.transform(_.read(underlying)(params).map(avroT.from)) + + override protected def write(data: SCollection[T], params: WriteP): Tap[T] = { + val datumFactory = Option(params.datumFactory).getOrElse(GenericRecordDatumFactory) + implicit val coder: Coder[GenericRecord] = avroCoder(datumFactory, schema) + underlying.writeWithContext(data.transform(_.map(avroT.to)), params).underlying.map(avroT.from) + } + + override def tap(read: ReadP): Tap[T] = underlying.tap(read).map(avroT.from) +} + +object AvroMagnolifyTypedIO { + type ReadParam = GenericRecordIO.ReadParam + val ReadParam = GenericRecordIO.ReadParam + type WriteParam = GenericRecordIO.WriteParam + val WriteParam = GenericRecordIO.WriteParam } diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/ObjectFileIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/ObjectFileIO.scala index 3671b207ac..6f285427e7 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/ObjectFileIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/ObjectFileIO.scala @@ -39,7 +39,11 @@ final case class ObjectFileIO[T: Coder](path: String) extends ScioIO[T] { */ override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = { val objectCoder = CoderMaterializer.beamWithDefault(Coder[T]) - sc.read(underlying)(params).map(record => AvroBytesUtil.decode(objectCoder, record)) + sc.transform { self => + self + .read(underlying)(params) + .map(record => AvroBytesUtil.decode(objectCoder, record)) + } } /** diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/ProtobufIO.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/ProtobufIO.scala index e2dae68683..2853fcdf78 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/ProtobufIO.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/ProtobufIO.scala @@ -18,17 +18,35 @@ package com.spotify.scio.avro import com.google.protobuf.Message import com.spotify.scio.ScioContext -import com.spotify.scio.io.{ScioIO, Tap, TapOf, TapT} +import com.spotify.scio.coders.Coder +import com.spotify.scio.io.{ScioIO, Tap, TapOf, TapT, TestIO} import com.spotify.scio.protobuf.util.ProtobufUtil import com.spotify.scio.values.SCollection +import magnolify.protobuf.ProtobufType import scala.reflect.ClassTag -final case class ProtobufIO[T <: Message: ClassTag](path: String) extends ScioIO[T] { - override type ReadP = ProtobufIO.ReadParam - override type WriteP = ProtobufIO.WriteParam - override val tapT: TapT.Aux[T, T] = TapOf[T] +sealed trait ProtobufIO[T] extends ScioIO[T] { + final override val tapT: TapT.Aux[T, T] = TapOf[T] +} + +object ProtobufIO { + final def apply[T](path: String): ProtobufIO[T] = + new ProtobufIO[T] with TestIO[T] { + override def testId: String = s"ProtobufIO($path)" + } +} + +object ProtobufObjectFileIO { + type ReadParam = GenericRecordIO.ReadParam + val ReadParam = GenericRecordIO.ReadParam + type WriteParam = GenericRecordIO.WriteParam + val WriteParam = GenericRecordIO.WriteParam +} +final case class ProtobufObjectFileIO[T <: Message: ClassTag](path: String) extends ProtobufIO[T] { + override type ReadP = ProtobufObjectFileIO.ReadParam + override type WriteP = ProtobufObjectFileIO.WriteParam override def testId: String = s"ProtobufIO($path)" private lazy val underlying: ObjectFileIO[T] = ObjectFileIO(path) @@ -53,13 +71,51 @@ final case class ProtobufIO[T <: Message: ClassTag](path: String) extends ScioIO data.write(underlying)(params.copy(metadata = metadata)).underlying } - override def tap(read: ReadP): Tap[T] = - ProtobufFileTap(path, read) + override def tap(read: ReadP): Tap[T] = ProtobufFileTap(path, read) } -object ProtobufIO { - type ReadParam = GenericRecordIO.ReadParam - val ReadParam = GenericRecordIO.ReadParam - type WriteParam = GenericRecordIO.WriteParam - val WriteParam = GenericRecordIO.WriteParam +final case class ProtobufTypedObjectFileIO[T: Coder, U <: Message: ClassTag]( + path: String +)(implicit pt: ProtobufType[T, U]) + extends ProtobufIO[T] { + override type ReadP = ProtobufTypedObjectFileIO.ReadParam + override type WriteP = ProtobufTypedObjectFileIO.WriteParam + override def testId: String = s"ProtobufIO($path)" + + private lazy val underlying: ObjectFileIO[U] = ObjectFileIO(path) + + /** + * Get an SCollection for a Protobuf file. + * + * Protobuf messages are serialized into `Array[Byte]` and stored in Avro files to leverage Avro's + * block file format. + */ + override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = + sc.transform(_.read(underlying)(params).map(pt.from)) + + /** + * Save this SCollection as a Protobuf file. + * + * Protobuf messages are serialized into `Array[Byte]` and stored in Avro files to leverage Avro's + * block file format. + */ + override protected def write(data: SCollection[T], params: WriteP): Tap[T] = { + val metadata = params.metadata ++ ProtobufUtil.schemaMetadataOf[U] + underlying + .writeWithContext( + data.transform(_.map(pt.to)), + params.copy(metadata = metadata) + ) + .underlying + .map(pt.from) + } + + override def tap(read: ReadP): Tap[T] = ProtobufFileTap[U](path, read).map(pt.from) +} + +object ProtobufTypedObjectFileIO { + type ReadParam = ProtobufObjectFileIO.ReadParam + val ReadParam = ProtobufObjectFileIO.ReadParam + type WriteParam = ProtobufObjectFileIO.WriteParam + val WriteParam = ProtobufObjectFileIO.WriteParam } 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 129613ef0f..a8ceb0d581 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 @@ -24,6 +24,8 @@ import com.spotify.scio.coders.Coder import com.spotify.scio.io.ClosedTap import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.values._ +import magnolify.avro.{AvroType => AvroMagnolifyType} +import magnolify.protobuf.ProtobufType import org.apache.avro.Schema import org.apache.avro.file.CodecFactory import org.apache.avro.specific.SpecificRecord @@ -183,17 +185,17 @@ final class ProtobufSCollectionOps[T <: Message](private val self: SCollection[T */ def saveAsProtobufFile( path: String, - numShards: Int = ProtobufIO.WriteParam.DefaultNumShards, - suffix: String = ProtobufIO.WriteParam.DefaultSuffixProtobuf, - codec: CodecFactory = ProtobufIO.WriteParam.DefaultCodec, - metadata: Map[String, AnyRef] = ProtobufIO.WriteParam.DefaultMetadata, - shardNameTemplate: String = ProtobufIO.WriteParam.DefaultShardNameTemplate, - tempDirectory: String = ProtobufIO.WriteParam.DefaultTempDirectory, + numShards: Int = ProtobufObjectFileIO.WriteParam.DefaultNumShards, + suffix: String = ProtobufObjectFileIO.WriteParam.DefaultSuffixProtobuf, + codec: CodecFactory = ProtobufObjectFileIO.WriteParam.DefaultCodec, + metadata: Map[String, AnyRef] = ProtobufObjectFileIO.WriteParam.DefaultMetadata, + shardNameTemplate: String = ProtobufObjectFileIO.WriteParam.DefaultShardNameTemplate, + tempDirectory: String = ProtobufObjectFileIO.WriteParam.DefaultTempDirectory, filenamePolicySupplier: FilenamePolicySupplier = - ProtobufIO.WriteParam.DefaultFilenamePolicySupplier, - prefix: String = ProtobufIO.WriteParam.DefaultPrefix + ProtobufObjectFileIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = ProtobufObjectFileIO.WriteParam.DefaultPrefix )(implicit ct: ClassTag[T]): ClosedTap[T] = { - val param = ProtobufIO.WriteParam[GenericRecord]( + val param = ProtobufObjectFileIO.WriteParam[GenericRecord]( numShards, suffix, codec, @@ -203,7 +205,73 @@ final class ProtobufSCollectionOps[T <: Message](private val self: SCollection[T shardNameTemplate, tempDirectory ) - self.write(ProtobufIO[T](path))(param) + self.write(ProtobufObjectFileIO[T](path))(param) + } +} + +final class TypedMagnolifyProtobufSCollectionOps[T](private val self: SCollection[T]) + extends AnyVal { + + /** + * Save this SCollection as a Protobuf file. + * + * Protobuf messages are serialized into `Array[Byte]` and stored in Avro files to leverage Avro's + * block file format. + */ + def saveAsProtobufFile[U <: Message: ClassTag]( + path: String, + numShards: Int = ProtobufTypedObjectFileIO.WriteParam.DefaultNumShards, + suffix: String = ProtobufTypedObjectFileIO.WriteParam.DefaultSuffixProtobuf, + codec: CodecFactory = ProtobufTypedObjectFileIO.WriteParam.DefaultCodec, + metadata: Map[String, AnyRef] = ProtobufTypedObjectFileIO.WriteParam.DefaultMetadata, + shardNameTemplate: String = ProtobufTypedObjectFileIO.WriteParam.DefaultShardNameTemplate, + tempDirectory: String = ProtobufTypedObjectFileIO.WriteParam.DefaultTempDirectory, + filenamePolicySupplier: FilenamePolicySupplier = + ProtobufTypedObjectFileIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = ProtobufTypedObjectFileIO.WriteParam.DefaultPrefix + )(implicit pt: ProtobufType[T, U]): ClosedTap[T] = { + implicit val tCoder: Coder[T] = self.coder + val param = ProtobufTypedObjectFileIO.WriteParam[GenericRecord]( + numShards, + suffix, + codec, + metadata, + filenamePolicySupplier, + prefix, + shardNameTemplate, + tempDirectory + ) + self.write(ProtobufTypedObjectFileIO[T, U](path))(param) + } +} + +final class TypedMagnolifyAvroSCollectionOps[T](private val self: SCollection[T]) { + + def saveAsAvroFile( + path: String, + numShards: Int = AvroTypedIO.WriteParam.DefaultNumShards, + suffix: String = AvroTypedIO.WriteParam.DefaultSuffix, + codec: CodecFactory = AvroTypedIO.WriteParam.DefaultCodec, + metadata: Map[String, AnyRef] = AvroTypedIO.WriteParam.DefaultMetadata, + shardNameTemplate: String = AvroTypedIO.WriteParam.DefaultShardNameTemplate, + tempDirectory: String = AvroTypedIO.WriteParam.DefaultTempDirectory, + filenamePolicySupplier: FilenamePolicySupplier = + AvroTypedIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = AvroTypedIO.WriteParam.DefaultPrefix, + datumFactory: AvroDatumFactory[GenericRecord] = AvroTypedIO.WriteParam.DefaultDatumFactory + )(implicit coder: Coder[T], at: AvroMagnolifyType[T]): ClosedTap[T] = { + val param = AvroMagnolifyTypedIO.WriteParam( + numShards, + suffix, + codec, + metadata, + filenamePolicySupplier, + prefix, + shardNameTemplate, + tempDirectory, + datumFactory + ) + self.write(AvroMagnolifyTypedIO[T](path))(param) } } @@ -228,4 +296,12 @@ trait SCollectionSyntax { implicit def avroProtobufSCollectionOps[T <: Message]( c: SCollection[T] ): ProtobufSCollectionOps[T] = new ProtobufSCollectionOps[T](c) + + implicit def typedAvroProtobufSCollectionOps[T]( + c: SCollection[T] + ): TypedMagnolifyProtobufSCollectionOps[T] = new TypedMagnolifyProtobufSCollectionOps[T](c) + + implicit def typedMagnolifyAvroSCollectionOps[T]( + c: SCollection[T] + ): TypedMagnolifyAvroSCollectionOps[T] = new TypedMagnolifyAvroSCollectionOps(c) } 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 c79a068ddc..bacbfda8aa 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 @@ -24,6 +24,8 @@ import com.spotify.scio.avro._ import com.spotify.scio.avro.types.AvroType.HasAvroAnnotation import com.spotify.scio.coders.Coder import com.spotify.scio.values._ +import magnolify.protobuf.ProtobufType +import magnolify.avro.{AvroType => AvroMagnolifyType} import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificRecord @@ -167,6 +169,16 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { ): SCollection[T] = self.read(AvroTypedIO[T](path))(AvroTypedIO.ReadParam(suffix)) + /** + * Read avro data from `path` as `GenericRecord` and convert to `T` via the implicitly-available + * `magnolify.avro.AvroType[T]` + */ + def typedAvroFileMagnolify[T: AvroMagnolifyType: Coder]( + path: String, + suffix: String = AvroMagnolifyTypedIO.ReadParam.DefaultSuffix + ): SCollection[T] = + self.read(AvroMagnolifyTypedIO[T](path))(AvroMagnolifyTypedIO.ReadParam(suffix)) + /** * Get an SCollection for a Protobuf file. * @@ -175,9 +187,19 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { */ def protobufFile[T <: Message: ClassTag]( path: String, - suffix: String = ProtobufIO.ReadParam.DefaultSuffix + suffix: String = ProtobufObjectFileIO.ReadParam.DefaultSuffix ): SCollection[T] = - self.read(ProtobufIO[T](path))(ProtobufIO.ReadParam(suffix)) + self.read(ProtobufObjectFileIO[T](path))(ProtobufObjectFileIO.ReadParam(suffix)) + + /** + * Read back protobuf messages serialized to `Array[Byte]` and stored in Avro files then map them + * automatically to type `T` via the implicit [[magnolify.protobuf.ProtobufType]] + */ + def typedProtobufFile[T: Coder, U <: Message: ClassTag]( + path: String, + suffix: String = ProtobufObjectFileIO.ReadParam.DefaultSuffix + )(implicit pt: ProtobufType[T, U]): SCollection[T] = + self.read(ProtobufTypedObjectFileIO[T, U](path))(ProtobufObjectFileIO.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 c8d80feed6..8a9714eb5d 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 @@ -79,7 +79,7 @@ object ObjectFileTap { } object ProtobufFileTap { - def apply[T <: Message: ClassTag](path: String, params: ProtobufIO.ReadParam): Tap[T] = + def apply[T <: Message: ClassTag](path: String, params: ProtobufObjectFileIO.ReadParam): Tap[T] = ObjectFileTap(path, params)(Coder.protoMessageCoder[T]) } @@ -99,7 +99,7 @@ final case class AvroTaps(self: Taps) { /** Get a `Future[Tap[T]]` of a Protobuf file. */ def protobufFile[T <: Message: ClassTag]( path: String, - params: ProtobufIO.ReadParam = ProtobufIO.ReadParam() + params: ProtobufObjectFileIO.ReadParam = ProtobufObjectFileIO.ReadParam() ): Future[Tap[T]] = self.mkTap( s"Protobuf: $path", diff --git a/scio-avro/src/main/scala/com/spotify/scio/avro/types/AvroType.scala b/scio-avro/src/main/scala/com/spotify/scio/avro/types/AvroType.scala index b05895f80a..2ce6b640a5 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/avro/types/AvroType.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/avro/types/AvroType.scala @@ -51,6 +51,7 @@ import scala.reflect.runtime.universe._ * @groupname Ungrouped * Other Members */ +@deprecated("Use magnolify API instead.", "0.15.0") object AvroType { /** @@ -87,6 +88,7 @@ object AvroType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class fromSchema(schema: String) extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.schemaImpl } @@ -131,6 +133,7 @@ object AvroType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class fromPath(folderGlob: String) extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.pathImpl } @@ -161,6 +164,7 @@ object AvroType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class fromSchemaFile(schemaFile: String) extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.schemaFileImpl } @@ -188,6 +192,7 @@ object AvroType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class toSchema extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.toSchemaImpl } @@ -218,9 +223,11 @@ object AvroType { * Trait for case classes with generated companion objects. * @group trait */ + @deprecated("Use magnolify API instead.", "0.15.0") trait HasAvroAnnotation /** Generate [[org.apache.avro.Schema Schema]] for a case class. */ + @deprecated("Use magnolify API instead.", "0.15.0") def schemaOf[T: TypeTag]: Schema = SchemaProvider.schemaOf[T] /** @@ -228,6 +235,7 @@ object AvroType { * the given case class `T`. * @group converters */ + @deprecated("Use magnolify API instead.", "0.15.0") def fromGenericRecord[T]: GenericRecord => T = macro ConverterProvider.fromGenericRecordImpl[T] @@ -236,10 +244,12 @@ object AvroType { * [[org.apache.avro.generic.GenericRecord GenericRecord]]. * @group converters */ + @deprecated("Use magnolify API instead.", "0.15.0") def toGenericRecord[T]: T => GenericRecord = macro ConverterProvider.toGenericRecordImpl[T] /** Create a new AvroType instance. */ + @deprecated("Use magnolify API instead.", "0.15.0") def apply[T: TypeTag]: AvroType[T] = new AvroType[T] } @@ -248,6 +258,7 @@ object AvroType { * * This decouples generated fields and methods from macro expansion to keep core macro free. */ +@deprecated("Use magnolify API instead.", "0.15.0") class AvroType[T: TypeTag] extends Serializable { private val instance = runtimeMirror(getClass.getClassLoader) .reflectModule(typeOf[T].typeSymbol.companion.asModule) 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 e8c7b98778..cc7b81036a 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 @@ -18,7 +18,6 @@ package com.spotify.scio.avro.types import java.nio.file.Files - import com.spotify.scio._ import com.spotify.scio.avro._ import org.apache.commons.io.FileUtils 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 f4cb43ee7a..b7120890a5 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 @@ -23,16 +23,9 @@ package com.spotify.scio.examples.extra import com.spotify.scio._ import com.spotify.scio.avro._ -import com.spotify.scio.coders.Coder import com.spotify.scio.examples.common.ExampleData -import com.spotify.scio.examples.extra.MagnolifyAvroExample.wordCountType -import org.apache.avro.generic.GenericRecord object MagnolifyAvroExample { - // limit import scope to avoid polluting namespace - import magnolify.avro._ - - val wordCountType: AvroType[WordCount] = AvroType[WordCount] case class WordCount(word: String, count: Long) } @@ -47,9 +40,6 @@ object MagnolifyAvroExample { // --output=gs://[BUCKET]/[PATH]/wordcount-avro"` object MagnolifyAvroWriteExample { - implicit val genericCoder: Coder[GenericRecord] = - avroGenericRecordCoder(wordCountType.schema) - def main(cmdlineArgs: Array[String]): Unit = { import MagnolifyAvroExample._ @@ -57,8 +47,9 @@ object MagnolifyAvroWriteExample { sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) .countByValue - .map(t => wordCountType(WordCount.tupled(t))) - .saveAsAvroFile(args("output"), schema = wordCountType.schema) + .map { case (word, count) => WordCount(word, count) } + // uses implicitly-derived magnolify.avro.AvroType[WordCount] to save to avro + .saveAsAvroFile(args("output")) sc.run() () } @@ -78,8 +69,8 @@ object MagnolifyAvroReadExample { import MagnolifyAvroExample._ val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.avroFile(args("input"), wordCountType.schema) - .map(e => wordCountType(e)) + // uses implicitly-derived magnolify.avro.AvroType[WordCount] to read from avro + sc.typedAvroFileMagnolify[WordCount](args("input")) .map(wc => wc.word + ": " + wc.count) .saveAsTextFile(args("output")) sc.run() diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala index fd52360ea7..7e8e974183 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala @@ -34,11 +34,8 @@ import magnolify.bigtable._ import scala.collection.compat._ object MagnolifyBigtableExample { - // Define case class representation of TensorFlow `Example` case class WordCount(cnt: Long) - // `BigtableType` provides mapping between case classes and `Seq[Mutation]`/`Row` - // for writing/reading. - val WordCountType: BigtableType[WordCount] = BigtableType[WordCount] + val ColumnFamily = "counts" } // ## Magnolify Bigtable Write Example @@ -65,14 +62,10 @@ object MagnolifyBigtableWriteExample { sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) .countByValue - // Convert case class to `Seq[Mutation]` and lift it into a key-value pair - // for saving to Bigtable table. - .map { case (word, count) => - val mutations = - WordCountType(WordCount(count), columnFamily = "counts").iterator.to(Iterable) - ByteString.copyFromUtf8(word) -> mutations - } - .saveAsBigtable(btProjectId, btInstanceId, btTableId) + .mapValues(cnt => WordCount(cnt)) + // `keyFn` converts word to a ByteString, while the value is converted via an + // implicitly derived BigtableType[WordCount] + .saveAsBigtable(btProjectId, btInstanceId, btTableId, ColumnFamily, ByteString.copyFromUtf8 _) sc.run() () @@ -99,12 +92,15 @@ object MagnolifyBigtableReadExample { val btInstanceId = args("bigtableInstanceId") val btTableId = args("bigtableTableId") - sc.bigtable(btProjectId, btInstanceId, btTableId) - .map { row => - // Convert Bigtable `Row` to the case class and lift it into a key-value pair. - row.getKey.toStringUtf8 -> WordCountType(row, columnFamily = "counts").cnt - } - .saveAsTextFile(args("output")) + // Internally converts Bigtable `Row` to `(String, WordCount)` via implicit + // BigtableType[WordCount] and the provided `keyFn` + sc.typedBigtable[String, WordCount]( + btProjectId, + btInstanceId, + btTableId, + ColumnFamily, + (bs: ByteString) => bs.toStringUtf8 + ).saveAsTextFile(args("output")) sc.run() () diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExample.scala index 6e7037daf5..2564bb6156 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExample.scala @@ -22,19 +22,14 @@ // convert between case classes and Datastore `Entity` types. package com.spotify.scio.examples.extra -import com.google.datastore.v1.client.DatastoreHelper.makeKey import com.google.datastore.v1.Query import com.spotify.scio._ import com.spotify.scio.datastore._ import com.spotify.scio.examples.common.ExampleData -import magnolify.datastore._ object MagnolifyDatastoreExample { - val kind = "magnolify" // Define case class representation of Datastore entities case class WordCount(word: String, count: Long) - // `DatastoreType` provides mapping between case classes and Datatore entities - val wordCountType: EntityType[WordCount] = EntityType[WordCount] } // ## Magnolify Datastore Write Example @@ -54,14 +49,7 @@ object MagnolifyDatastoreWriteExample { sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) .countByValue - .map { t => - // Convert case class to `Entity.Builder` - wordCountType - .to(WordCount.tupled(t)) - // Set entity key - .setKey(makeKey(kind, t._1)) - .build() - } + .map { case (word, count) => WordCount(word, count) } .saveAsDatastore(args("output")) sc.run() () @@ -82,9 +70,7 @@ object MagnolifyDatastoreReadExample { import MagnolifyDatastoreExample._ val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.datastore(args("input"), Query.getDefaultInstance) - // Convert `Entity` to case class - .map(e => wordCountType(e)) + sc.typedDatastore[WordCount](args("input"), Query.getDefaultInstance) .map(wc => wc.word + ": " + wc.count) .saveAsTextFile(args("output")) sc.run() diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExample.scala index f52421c46e..e7ad1de029 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExample.scala @@ -35,8 +35,6 @@ object MagnolifyTensorFlowExample { // `Example` type doesn't support `String` natively, derive one from `ByteString` implicit val efString: ExampleField.Primitive[String] = ExampleField.from[ByteString](_.toStringUtf8)(ByteString.copyFromUtf8) - // `TensorFlowType` provides mapping between case classes and TensorFlow `Example` - val wordCountType: ExampleType[WordCount] = ExampleType[WordCount] } // ## Magnolify Tensorflow Write Example @@ -56,8 +54,8 @@ object MagnolifyTensorFlowWriteExample { sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) .countByValue - // Convert case class to `Example` and then serialize as `Array[Byte]` - .map(t => wordCountType(WordCount.tupled(t)).toByteArray) + .map { case (word, count) => WordCount(word, count) } + // converts WordCount to Example with the implicitly-derived ExampleType[WordCount] .saveAsTfRecordFile(args("output")) sc.run() () @@ -78,12 +76,8 @@ object MagnolifyTensorFlowReadExample { import MagnolifyTensorFlowExample._ val (sc, args) = ContextAndArgs(cmdlineArgs) - sc.tfRecordFile(args("input")) - .map { b => - // Deserialize `Array[Byte]` as `Example` and then convert to case class - wordCountType(Example.parseFrom(b)) - } - .map(wc => wc.word + ": " + wc.count) + // reads TF Examples and converts to WordCount via the implicitly-derived ExampleType[WordCount] + sc.typedTfRecordFile[WordCount](args("input")) .saveAsTextFile(args("output")) sc.run() () diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoes.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoes.scala new file mode 100644 index 0000000000..6649a6b75c --- /dev/null +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoes.scala @@ -0,0 +1,72 @@ +/* + * 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. + */ + +// Example: Read and write using typed BigQuery API with case classes +// Usage: + +// `sbt "runMain com.spotify.scio.examples.extra.TypedBigQueryTornadoes +// --project=[PROJECT] --runner=DataflowRunner --region=[REGION NAME] +// --output=[PROJECT]:[DATASET].[TABLE]"` +package com.spotify.scio.examples.extra + +import com.spotify.scio.bigquery._ +import com.spotify.scio.{ContextAndArgs, ScioContext} + +object MagnolifyTypedBigQueryTornadoes { + val query: String = "SELECT tornado, month FROM [bigquery-public-data:samples.gsod]" + case class Row(tornado: Option[Boolean], month: Long) + case class Result(month: Long, tornado_count: Long) + + def pipeline(cmdlineArgs: Array[String]): ScioContext = { + val (sc, args) = ContextAndArgs(cmdlineArgs) + + val resultTap = sc + // Get input from BigQuery and convert elements from `TableRow` to `Row` with the + // implicitly-available `TableRowType[Row]` + .typedBigQuerySelect[Row](Query(query)) + .flatMap(r => if (r.tornado.getOrElse(false)) Seq(r.month) else Nil) + .countByValue + .map(kv => Result(kv._1, kv._2)) + // Save output to BigQuery, convert elements from `Result` to `TableRow` with the + // implicitly-available `TableRowType[Result]` + .saveAsBigQueryTable( + Table.Spec(args("output")), + writeDisposition = WRITE_TRUNCATE, + createDisposition = CREATE_IF_NEEDED + ) + + // Access the loaded tables + resultTap + .output(BigQueryIO.SuccessfulTableLoads) + .map(_.getTableSpec) + .debug(prefix = "Loaded table: ") + + // Access the failed records + resultTap + .output(BigQueryIO.FailedInserts) + .count + .debug(prefix = "Failed inserts: ") + + sc + } + + def main(cmdlineArgs: Array[String]): Unit = { + val sc = pipeline(cmdlineArgs) + sc.run().waitUntilDone() + () + } +} diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoes.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoes.scala new file mode 100644 index 0000000000..16a435f004 --- /dev/null +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoes.scala @@ -0,0 +1,79 @@ +/* + * 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. + */ + +// Example: Read using typed BigQuery Storage API with annotated case classes +// Usage: + +// `sbt "runMain com.spotify.scio.examples.extra.TypedStorageBigQueryTornadoes +// --project=[PROJECT] --runner=DataflowRunner --region=[REGION NAME] +// --output=[PROJECT]:[DATASET].[TABLE]"` +package com.spotify.scio.examples.extra + +import com.spotify.scio.bigquery._ +import com.spotify.scio.{ContextAndArgs, ScioContext} +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method + +object MagnolifyTypedStorageBigQueryTornadoes { + val table: String = "bigquery-public-data:samples.gsod" + case class Row(month: Long, tornado: Option[Boolean]) + case class Result(month: Long, tornado_count: Long) + + def pipeline(cmdlineArgs: Array[String]): ScioContext = { + val (sc, args) = ContextAndArgs(cmdlineArgs) + + val resultTap = sc + // Get input from BigQuery and convert elements from `TableRow` to `Row` with the + // implicitly-available `TableRowType[Row]` + .typedBigQueryStorageMagnolify[Row]( + Table.Spec(table), + selectedFields = List("tornado", "month"), + rowRestriction = "tornado = true" + ) + .map(_.month) + .countByValue + .map(kv => Result(kv._1, kv._2)) + // Save output to BigQuery, convert elements from `Result` to `TableRow` with the + // implicitly-available `TableRowType[Result]` + .saveAsBigQueryTable( + Table.Spec(args("output")), + method = Method.STORAGE_WRITE_API, + writeDisposition = WRITE_TRUNCATE, + createDisposition = CREATE_IF_NEEDED, + successfulInsertsPropagation = true + ) + + // Access the inserted records + resultTap + .output(BigQueryIO.SuccessfulStorageApiInserts) + .count + .debug(prefix = "Successful inserts: ") + + // Access the failed records + resultTap + .output(BigQueryIO.FailedStorageApiInserts) + .count + .debug(prefix = "Failed inserts: ") + + sc + } + + def main(cmdlineArgs: Array[String]): Unit = { + val sc = pipeline(cmdlineArgs) + sc.run().waitUntilDone() + () + } +} diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyAvroExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyAvroExampleTest.scala index 478d8bed46..7566495777 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyAvroExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyAvroExampleTest.scala @@ -20,35 +20,27 @@ package com.spotify.scio.examples.extra import com.spotify.scio.avro.AvroIO import com.spotify.scio.io._ import com.spotify.scio.testing._ -import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder} class MagnolifyAvroExampleTest extends PipelineSpec { import MagnolifyAvroExample._ val textIn: Seq[String] = Seq("a b c d e", "a b a b") val wordCount: Seq[(String, Long)] = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) - val records: Seq[GenericRecord] = wordCount.map { kv => - new GenericRecordBuilder(wordCountType.schema) - .set("word", kv._1) - .set("count", kv._2) - .build() - } + val records: Seq[WordCount] = wordCount.map { case (word, count) => WordCount(word, count) } val textOut: Seq[String] = wordCount.map(kv => kv._1 + ": " + kv._2) "MagnolifyAvroWriteExample" should "work" in { - import MagnolifyAvroWriteExample.genericCoder JobTest[com.spotify.scio.examples.extra.MagnolifyAvroWriteExample.type] .args("--input=in.txt", "--output=wc.avro") .input(TextIO("in.txt"), textIn) - .output(AvroIO[GenericRecord]("wc.avro"))(coll => coll should containInAnyOrder(records)) + .output(AvroIO[WordCount]("wc.avro"))(coll => coll should containInAnyOrder(records)) .run() } "MagnolifyAvroReadExample" should "work" in { - import MagnolifyAvroWriteExample.genericCoder JobTest[com.spotify.scio.examples.extra.MagnolifyAvroReadExample.type] .args("--input=wc.avro", "--output=out.txt") - .input(AvroIO[GenericRecord]("wc.avro"), records) + .input(AvroIO[WordCount]("wc.avro"), records) .output(TextIO("out.txt"))(coll => coll should containInAnyOrder(textOut)) .run() } diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExampleTest.scala new file mode 100644 index 0000000000..eb6c13b653 --- /dev/null +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExampleTest.scala @@ -0,0 +1,58 @@ +/* + * Copyright 2024 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.bigtable.BigtableIO +import com.spotify.scio.io._ +import com.spotify.scio.testing._ + +class MagnolifyBigtableExampleTest extends PipelineSpec { + import MagnolifyBigtableExample._ + + val project = "my-project" + val instance = "my-instance" + val table = "my-table" + val bigtableOptions: Seq[String] = Seq( + s"--bigtableProjectId=$project", + s"--bigtableInstanceId=$instance", + s"--bigtableTableId=$table" + ) + + val textIn: Seq[String] = Seq("a b c d e", "a b a b") + val wordCount: Seq[(String, Long)] = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) + val expected: Seq[(String, WordCount)] = wordCount.map { case (k, v) => (k, WordCount(v)) } + val expectedText: Seq[String] = expected.map(_.toString) + + "MagnolifyBigtableWriteExample" should "work" in { + JobTest[MagnolifyBigtableWriteExample.type] + .args(bigtableOptions :+ "--input=in.txt": _*) + .input(TextIO("in.txt"), textIn) + .output(BigtableIO[(String, WordCount)](project, instance, table))(coll => + coll should containInAnyOrder(expected) + ) + .run() + } + + "MagnolifyBigtableReadExample" should "work" in { + JobTest[MagnolifyBigtableReadExample.type] + .args(bigtableOptions :+ "--output=out.txt": _*) + .input(BigtableIO[(String, WordCount)](project, instance, table), expected) + .output(TextIO("out.txt"))(coll => coll should containInAnyOrder(expectedText)) + .run() + } +} diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExampleTest.scala index ab8514b90b..50cdfc84d0 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyDatastoreExampleTest.scala @@ -17,37 +17,30 @@ package com.spotify.scio.examples.extra -import com.google.datastore.v1.client.DatastoreHelper.{makeKey, makeValue} -import com.google.datastore.v1.Entity import com.spotify.scio.io._ import com.spotify.scio.datastore._ import com.spotify.scio.testing._ class MagnolifyDatastoreExampleTest extends PipelineSpec { + import MagnolifyDatastoreExample._ + val textIn: Seq[String] = Seq("a b c d e", "a b a b") val wordCount: Seq[(String, Long)] = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) - val entities: Seq[Entity] = wordCount.map { kv => - Entity - .newBuilder() - .setKey(makeKey(MagnolifyDatastoreExample.kind, kv._1)) - .putProperties("word", makeValue(kv._1).build()) - .putProperties("count", makeValue(kv._2).build()) - .build() - } + val entities: Seq[WordCount] = wordCount.map { case (word, count) => WordCount(word, count) } val textOut: Seq[String] = wordCount.map(kv => kv._1 + ": " + kv._2) "MagnolifyDatastoreWriteExample" should "work" in { JobTest[com.spotify.scio.examples.extra.MagnolifyDatastoreWriteExample.type] .args("--input=in.txt", "--output=project") .input(TextIO("in.txt"), textIn) - .output(DatastoreIO("project"))(coll => coll should containInAnyOrder(entities)) + .output(DatastoreIO[WordCount]("project"))(_ should containInAnyOrder(entities)) .run() } "MagnolifyDatastoreReadExample" should "work" in { JobTest[com.spotify.scio.examples.extra.MagnolifyDatastoreReadExample.type] .args("--input=project", "--output=out.txt") - .input(DatastoreIO("project"), entities) + .input(DatastoreIO[WordCount]("project"), entities) .output(TextIO("out.txt"))(coll => coll should containInAnyOrder(textOut)) .run() } diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExampleTest.scala index 916d764463..94aed60d6b 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTensorFlowExampleTest.scala @@ -17,54 +17,32 @@ package com.spotify.scio.examples.extra -import com.google.protobuf.ByteString import com.spotify.scio.io._ -import com.spotify.scio.tensorflow.TFRecordIO +import com.spotify.scio.tensorflow.TFExampleTypedIO import com.spotify.scio.testing._ -import org.tensorflow.proto.example._ class MagnolifyTensorFlowExampleTest extends PipelineSpec { + import MagnolifyTensorFlowExample._ + val textIn: Seq[String] = Seq("a b c d e", "a b a b") - val wordCount: Seq[(String, Long)] = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) - val examples: Seq[Example] = wordCount.map { kv => - Example - .newBuilder() - .setFeatures( - Features - .newBuilder() - .putFeature( - "word", - Feature - .newBuilder() - .setBytesList(BytesList.newBuilder().addValue(ByteString.copyFromUtf8(kv._1))) - .build() - ) - .putFeature( - "count", - Feature - .newBuilder() - .setInt64List(Int64List.newBuilder().addValue(kv._2)) - .build() - ) - ) - .build() - } - val textOut: Seq[String] = wordCount.map(kv => kv._1 + ": " + kv._2) + val wordCount: Seq[WordCount] = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) + .map { case (word, count) => WordCount(word, count) } + val textOut: Seq[String] = wordCount.map(_.toString()) "MagnolifyTensorFlowWriteExample" should "work" in { - JobTest[com.spotify.scio.examples.extra.MagnolifyTensorFlowWriteExample.type] + JobTest[MagnolifyTensorFlowWriteExample.type] .args("--input=in.txt", "--output=wc.tfrecords") .input(TextIO("in.txt"), textIn) - .output(TFRecordIO("wc.tfrecords")) { - _.map(Example.parseFrom) should containInAnyOrder(examples) + .output(TFExampleTypedIO[WordCount]("wc.tfrecords")) { + _ should containInAnyOrder(wordCount) } .run() } "MagnolifyTensorFlowReadExample" should "work" in { - JobTest[com.spotify.scio.examples.extra.MagnolifyTensorFlowReadExample.type] + JobTest[MagnolifyTensorFlowReadExample.type] .args("--input=wc.tfrecords", "--output=out.txt") - .input(TFRecordIO("wc.tfrecords"), examples.map(_.toByteArray)) + .input(TFExampleTypedIO[WordCount]("wc.tfrecords"), wordCount) .output(TextIO("out.txt"))(coll => coll should containInAnyOrder(textOut)) .run() } diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoesTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoesTest.scala new file mode 100644 index 0000000000..abb58bcb83 --- /dev/null +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedBigQueryTornadoesTest.scala @@ -0,0 +1,46 @@ +/* + * 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.examples.extra + +import com.spotify.scio.bigquery._ +import com.spotify.scio.testing._ + +class MagnolifyTypedBigQueryTornadoesTest extends PipelineSpec { + import MagnolifyTypedBigQueryTornadoes.{Result, Row} + + val inData: Seq[Row] = Seq( + Row(Some(true), 1), + Row(Some(false), 1), + Row(Some(false), 2), + Row(Some(true), 3), + Row(Some(true), 4), + Row(Some(true), 4) + ) + + val expected: Seq[Result] = Seq(Result(1, 1), Result(3, 1), Result(4, 2)) + + "MagnolifyTypedBigQueryTornadoes" should "work" in { + JobTest[com.spotify.scio.examples.extra.MagnolifyTypedBigQueryTornadoes.type] + .args("--output=dataset.table") + .input(BigQueryIO(MagnolifyTypedBigQueryTornadoes.query), inData) + .output(BigQueryIO[Result]("dataset.table")) { coll => + coll should containInAnyOrder(expected) + } + .run() + } +} diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoesTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoesTest.scala new file mode 100644 index 0000000000..9482c88215 --- /dev/null +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/MagnolifyTypedStorageBigQueryTornadoesTest.scala @@ -0,0 +1,51 @@ +/* + * 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.examples.extra + +import com.spotify.scio.bigquery._ +import com.spotify.scio.testing._ + +final class MagnolifyTypedStorageBigQueryTornadoesTest extends PipelineSpec { + import MagnolifyTypedStorageBigQueryTornadoes.{Result, Row} + + val inData: Seq[Row] = Seq( + Row(1, Some(true)), + Row(3, Some(true)), + Row(4, Some(true)), + Row(4, Some(true)) + ) + + val expected: Seq[Result] = Seq(Result(1, 1), Result(3, 1), Result(4, 2)) + + "MagnolifyStorageTypedBigQueryTornadoes" should "work" in { + JobTest[com.spotify.scio.examples.extra.MagnolifyTypedStorageBigQueryTornadoes.type] + .args("--output=dataset.table") + .input( + BigQueryIO( + MagnolifyTypedStorageBigQueryTornadoes.table, + List("tornado", "month"), + Some("tornado = true") + ), + inData + ) + .output(BigQueryIO[Result]("dataset.table")) { coll => + coll should containInAnyOrder(expected) + } + .run() + } +} 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 3cc3f6ab03..4ce3af2947 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 @@ -26,6 +26,7 @@ import com.spotify.scio.io._ import com.spotify.scio.util.{FilenamePolicySupplier, Functions, ScioUtil} import com.spotify.scio.values.{SCollection, SideOutput, SideOutputCollections} import com.twitter.chill.ClosureCleaner +import magnolify.bigquery.TableRowType import org.apache.avro.generic.GenericRecord import org.apache.beam.sdk.extensions.gcp.options.GcpOptions import org.apache.beam.sdk.io.Compression @@ -280,7 +281,7 @@ final case class BigQueryTypedSelect[T: Coder]( } override protected def write(data: SCollection[T], params: WriteP): Tap[T] = - throw new UnsupportedOperationException("BigQuerySelect is read-only") + throw new UnsupportedOperationException("BigQueryTypedSelect is read-only") override def tap(params: ReadP): Tap[T] = { val tableReference = BigQuery @@ -402,7 +403,7 @@ object BigQueryTypedTable { * Creates a new instance of [[BigQueryTypedTable]] based on the supplied [[Format]]. * * NOTE: LogicalType support when using `Format.GenericRecord` has some caveats: Reading: Bigquery - * types DATE, TIME, DATIME will be read as STRING. Writing: Supports LogicalTypes only for DATE + * types DATE, TIME, DATEIME will be read as STRING. Writing: Supports LogicalTypes only for DATE * and TIME. DATETIME is not yet supported. https://issuetracker.google.com/issues/140681683 */ def apply[F: Coder](table: Table, format: Format[F]): BigQueryTypedTable[F] = @@ -567,7 +568,7 @@ final case class BigQueryStorageSelect(sqlQuery: Query) extends BigQueryIO[Table sc.read(underlying)(BigQueryTypedSelect.ReadParam()) override protected def write(data: SCollection[TableRow], params: WriteP): Tap[TableRow] = - throw new UnsupportedOperationException("BigQuerySelect is read-only") + throw new UnsupportedOperationException("BigQueryStorageSelect is read-only") override def tap(params: ReadP): Tap[TableRow] = underlying.tap(BigQueryTypedSelect.ReadParam()) } @@ -916,3 +917,126 @@ object BigQueryTyped { } } } + +// SELECT + +object BigQueryMagnolifyTypedSelectIO { + type ReadParam = BigQueryTypedSelect.ReadParam + val ReadParam = BigQueryTypedSelect.ReadParam +} + +final case class BigQueryMagnolifyTypedSelectIO[T: TableRowType: Coder]( + query: Query +) extends BigQueryIO[T] { + override type ReadP = BigQuerySelect.ReadParam + override type WriteP = Nothing // ReadOnly + + private lazy val tableRowType: TableRowType[T] = implicitly + private[this] lazy val underlying = + BigQueryTypedSelect(beam.BigQueryIO.readTableRows(), query, identity)(coders.tableRowCoder) + + override def testId: String = s"BigQueryIO(${query.underlying})" + + override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = + sc.transform(_.read(underlying)(params).map(row => tableRowType(row))) + + override protected def write(data: SCollection[T], params: WriteP): Tap[T] = + throw new UnsupportedOperationException("MagnolifyBigQuerySelect is read-only") + + override def tap(params: ReadP): Tap[T] = underlying.tap(params).map(row => tableRowType(row)) +} + +// TABLE + +final case class BigQueryMagnolifyTypedTable[T: TableRowType: Coder]( + table: Table +) extends BigQueryIO[T] + with WriteResultIO[T] { + override type ReadP = Unit + override type WriteP = BigQueryTypedTable.WriteParam[T] + + override def testId: String = s"BigQueryIO(${table.spec})" + + private val tableRowType: TableRowType[T] = implicitly + private val readFn = Functions.serializableFn[SchemaAndRecord, T](sar => + tableRowType(BigQueryUtils.convertGenericRecordToTableRow(sar.getRecord, sar.getTableSchema)) + ) + private val writeFn = Functions.serializableFn[T, TableRow](t => tableRowType(t)) + + private lazy val underlying = { + BigQueryTypedTable( + beam.BigQueryIO.read(readFn), + beam.BigQueryIO.write().withFormatFunction(writeFn), + table, + (gr, ts) => tableRowType(BigQueryUtils.convertGenericRecordToTableRow(gr, ts)) + ) + } + + override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = + sc.read(underlying) + + override protected def writeWithResult( + data: SCollection[T], + params: WriteP + ): (Tap[T], SideOutputCollections) = { + val outputs = data + .write(underlying)(params) + .outputs + .get + + (tap(()), outputs) + } + + override def tap(read: ReadP): Tap[T] = + BigQueryTableRowTypedTap[T](table, tableRowType.apply) +} + +// STORAGE + +final case class BigQueryMagnolifyTypedStorage[T: TableRowType: Coder]( + table: Table, + selectedFields: List[String], + rowRestriction: Option[String] +) extends BigQueryIO[T] { + override type ReadP = Unit + override type WriteP = Nothing // ReadOnly + + override def testId: String = + s"BigQueryIO(${table.spec}, List(${selectedFields.mkString(",")}), $rowRestriction)" + + private lazy val tableRowType: TableRowType[T] = implicitly + private lazy val underlying = BigQueryStorage(table, selectedFields, rowRestriction) + + override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = + sc.transform(_.read(underlying).map(tr => tableRowType(tr))) + + override protected def write(data: SCollection[T], params: WriteP): Tap[T] = + throw new UnsupportedOperationException("MagnolifyBigQueryStorage is read-only") + + override def tap(read: ReadP): Tap[T] = + underlying.tap(read).map(tr => tableRowType(tr)) +} + +object BigQueryMagnolifyTypedStorage { + val ReadParam = BigQueryStorage.ReadParam +} + +final case class BigQueryMagnolifyTypedStorageSelect[T: TableRowType: Coder](sqlQuery: Query) + extends BigQueryIO[T] { + override type ReadP = Unit + override type WriteP = Nothing // ReadOnly + + private[this] lazy val underlying = BigQueryStorageSelect(sqlQuery) + private lazy val tableRowType: TableRowType[T] = implicitly + + override def testId: String = s"BigQueryIO(${sqlQuery.underlying})" + + override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = + sc.transform(_.read(underlying).map(tr => tableRowType(tr))) + + override protected def write(data: SCollection[T], params: WriteP): Tap[T] = + throw new UnsupportedOperationException("MagnolifyBigQueryStorageSelect is read-only") + + override def tap(params: ReadP): Tap[T] = + underlying.tap(params).map(tr => tableRowType(tr)) +} diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/dynamic/syntax/SCollectionSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/dynamic/syntax/SCollectionSyntax.scala index b17542f267..de32a6191a 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/dynamic/syntax/SCollectionSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/dynamic/syntax/SCollectionSyntax.scala @@ -25,6 +25,7 @@ import com.spotify.scio.bigquery.{TableRow, Writes} import com.spotify.scio.io.{ClosedTap, EmptyTap} import com.spotify.scio.util.Functions import com.spotify.scio.values.SCollection +import magnolify.bigquery.TableRowType import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.{ CreateDisposition, Method, @@ -37,12 +38,37 @@ import org.apache.beam.sdk.values.ValueInSingleWindow import scala.reflect.runtime.universe._ import scala.util.chaining._ +object DynamicWriteParam extends Writes.WriteParamDefaults + /** * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with dynamic destinations * methods. */ final class DynamicBigQueryOps[T](private val self: SCollection[T]) extends AnyVal { + /** + * Save this SCollection to dynamic BigQuery tables specified by `tableFn`, converting elements of + * type `T` to `TableRow` via the implicitly-available `TableRowType[T]` + */ + def saveAsBigQuery( + writeDisposition: WriteDisposition = DynamicWriteParam.DefaultWriteDisposition, + createDisposition: CreateDisposition = DynamicWriteParam.DefaultCreateDisposition, + extendedErrorInfo: Boolean = DynamicWriteParam.DefaultExtendedErrorInfo + )( + tableFn: ValueInSingleWindow[T] => TableDestination + )(implicit tableRowType: TableRowType[T]): ClosedTap[Nothing] = { + val destinations = DynamicDestinationsUtil.tableFn(tableFn, tableRowType.schema) + + new DynamicBigQueryOps(self).saveAsBigQuery( + destinations, + tableRowType.to, + writeDisposition, + createDisposition, + false, + extendedErrorInfo + ) + } + /** * Save this SCollection to dynamic BigQuery tables using the table and schema specified by the * [[org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinations DynamicDestinations]]. @@ -52,8 +78,8 @@ final class DynamicBigQueryOps[T](private val self: SCollection[T]) extends AnyV formatFn: T => TableRow, writeDisposition: WriteDisposition, createDisposition: CreateDisposition, - successfulInsertsPropagation: Boolean = false, - extendedErrorInfo: Boolean = false + successfulInsertsPropagation: Boolean, + extendedErrorInfo: Boolean ): ClosedTap[Nothing] = { if (self.context.isTest) { throw new NotImplementedError( @@ -91,15 +117,16 @@ final class DynamicTableRowBigQueryOps[T <: TableRow](private val self: SCollect */ def saveAsBigQuery( schema: TableSchema, - writeDisposition: WriteDisposition = null, - createDisposition: CreateDisposition = null, - extendedErrorInfo: Boolean = false + writeDisposition: WriteDisposition = DynamicWriteParam.DefaultWriteDisposition, + createDisposition: CreateDisposition = DynamicWriteParam.DefaultCreateDisposition, + extendedErrorInfo: Boolean = DynamicWriteParam.DefaultExtendedErrorInfo )(tableFn: ValueInSingleWindow[T] => TableDestination): ClosedTap[Nothing] = new DynamicBigQueryOps(self).saveAsBigQuery( DynamicDestinationsUtil.tableFn(tableFn, schema), identity, writeDisposition, createDisposition, + false, extendedErrorInfo ) } @@ -117,9 +144,9 @@ final class DynamicTypedBigQueryOps[T <: HasAnnotation](private val self: SColle * [[com.spotify.scio.bigquery.types.BigQueryType BigQueryType]]. */ def saveAsTypedBigQuery( - writeDisposition: WriteDisposition = null, - createDisposition: CreateDisposition = null, - extendedErrorInfo: Boolean = false + writeDisposition: WriteDisposition = DynamicWriteParam.DefaultWriteDisposition, + createDisposition: CreateDisposition = DynamicWriteParam.DefaultCreateDisposition, + extendedErrorInfo: Boolean = DynamicWriteParam.DefaultExtendedErrorInfo )( tableFn: ValueInSingleWindow[T] => TableDestination )(implicit tt: TypeTag[T]): ClosedTap[Nothing] = { @@ -131,6 +158,7 @@ final class DynamicTypedBigQueryOps[T <: HasAnnotation](private val self: SColle bqt.toTableRow, writeDisposition, createDisposition, + false, extendedErrorInfo ) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/package.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/package.scala index 357b2f7b51..62a377f5b1 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/package.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/package.scala @@ -21,6 +21,7 @@ import com.google.api.services.bigquery.model.{TableRow => GTableRow} import com.spotify.scio.bigquery.instances.CoderInstances import com.spotify.scio.bigquery.syntax.{ FileStorageSyntax, + MagnolifySyntax, SCollectionSyntax, ScioContextSyntax, TableReferenceSyntax, @@ -51,6 +52,7 @@ package object bigquery with TableRowSyntax with TableReferenceSyntax with FileStorageSyntax + with MagnolifySyntax with CoderInstances { /** Alias for BigQuery `CreateDisposition`. */ diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/MagnolifySyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/MagnolifySyntax.scala new file mode 100644 index 0000000000..ef6cca096d --- /dev/null +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/syntax/MagnolifySyntax.scala @@ -0,0 +1,148 @@ +/* + * Copyright 2024 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.bigquery.syntax + +import com.spotify.scio.ScioContext +import com.spotify.scio.bigquery.{ + BigQueryMagnolifyTypedSelectIO, + BigQueryMagnolifyTypedStorage, + BigQueryMagnolifyTypedStorageSelect, + BigQueryMagnolifyTypedTable, + BigQueryTypedTable, + Clustering, + Query, + Sharding, + Table, + TimePartitioning +} +import com.spotify.scio.coders.Coder +import com.spotify.scio.io.ClosedTap +import com.spotify.scio.values.SCollection +import magnolify.bigquery.TableRowType +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.{ + CreateDisposition, + Method, + WriteDisposition +} +import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy +import org.joda.time.Duration + +final class MagnolifyBigQueryScioContextOps(private val self: ScioContext) extends AnyVal { + + /** + * Get an SCollection for a BigQuery SELECT query. Both + * [[https://cloud.google.com/bigquery/docs/reference/legacy-sql Legacy SQL]] and + * [[https://cloud.google.com/bigquery/docs/reference/standard-sql/ Standard SQL]] dialects are + * supported. By default the query dialect will be automatically detected. To override this + * behavior, start the query string with `#legacysql` or `#standardsql`. + */ + def typedBigQuerySelect[T: TableRowType: Coder]( + sqlQuery: Query, + flattenResults: Boolean = BigQueryMagnolifyTypedSelectIO.ReadParam.DefaultFlattenResults + ): SCollection[T] = + self.read(BigQueryMagnolifyTypedSelectIO(sqlQuery))( + BigQueryMagnolifyTypedSelectIO.ReadParam(flattenResults) + ) + + /** Get an SCollection for a BigQuery table. */ + def typedBigQueryTable[T: TableRowType: Coder](table: Table): SCollection[T] = + self.read(BigQueryMagnolifyTypedTable(table)) + + /** + * Get an SCollection for a BigQuery table using the storage API. + * + * @param selectedFields + * names of the fields in the table that should be read. If empty, all fields will be read. If + * the specified field is a nested field, all the sub-fields in the field will be selected. + * Fields will always appear in the generated class in the same order as they appear in the + * table, regardless of the order specified in selectedFields. + * @param rowRestriction + * SQL text filtering statement, similar ti a WHERE clause in a query. Currently, we support + * combinations of predicates that are a comparison between a column and a constant value in SQL + * statement. Aggregates are not supported. For example: + * + * {{{ + * "a > DATE '2014-09-27' AND (b > 5 AND c LIKE 'date')" + * }}} + */ + def typedBigQueryStorageMagnolify[T: TableRowType: Coder]( + table: Table, + selectedFields: List[String] = BigQueryMagnolifyTypedStorage.ReadParam.DefaultSelectFields, + rowRestriction: String = null + ): SCollection[T] = + self.read(BigQueryMagnolifyTypedStorage(table, selectedFields, Option(rowRestriction))) + + /** + * Get an SCollection for a BigQuery SELECT query using the storage API. + * + * @param query + * SQL query + */ + def typedBigQueryStorageMagnolify[T: TableRowType: Coder](query: Query): SCollection[T] = + self.read(BigQueryMagnolifyTypedStorageSelect(query)) + +} + +final class MagnolifyBigQuerySCollectionOps[T](private val self: SCollection[T]) { + + def saveAsBigQueryTable( + table: Table, + timePartitioning: TimePartitioning = BigQueryTypedTable.WriteParam.DefaultTimePartitioning, + writeDisposition: WriteDisposition = BigQueryTypedTable.WriteParam.DefaultWriteDisposition, + createDisposition: CreateDisposition = BigQueryTypedTable.WriteParam.DefaultCreateDisposition, + clustering: Clustering = BigQueryTypedTable.WriteParam.DefaultClustering, + method: Method = BigQueryTypedTable.WriteParam.DefaultMethod, + triggeringFrequency: Duration = BigQueryTypedTable.WriteParam.DefaultTriggeringFrequency, + sharding: Sharding = BigQueryTypedTable.WriteParam.DefaultSharding, + failedInsertRetryPolicy: InsertRetryPolicy = + BigQueryTypedTable.WriteParam.DefaultFailedInsertRetryPolicy, + successfulInsertsPropagation: Boolean = + BigQueryTypedTable.WriteParam.DefaultSuccessfulInsertsPropagation, + extendedErrorInfo: Boolean = BigQueryTypedTable.WriteParam.DefaultExtendedErrorInfo, + configOverride: BigQueryTypedTable.WriteParam.ConfigOverride[T] = + BigQueryTypedTable.WriteParam.DefaultConfigOverride + )(implicit coder: Coder[T], tableRowType: TableRowType[T]): ClosedTap[T] = { + val param = BigQueryTypedTable.WriteParam[T]( + method, + tableRowType.schema, + writeDisposition, + createDisposition, + tableRowType.description, + timePartitioning, + clustering, + triggeringFrequency, + sharding, + failedInsertRetryPolicy, + successfulInsertsPropagation, + extendedErrorInfo, + configOverride + ) + self.write(BigQueryMagnolifyTypedTable[T](table))(param) + } + +} + +trait MagnolifySyntax { + implicit def magnolifyBigQueryScioContextOps(sc: ScioContext): MagnolifyBigQueryScioContextOps = + new MagnolifyBigQueryScioContextOps(sc) + + implicit def magnolifyBigQuerySCollectionOps[T]( + scoll: SCollection[T] + ): MagnolifyBigQuerySCollectionOps[T] = + new MagnolifyBigQuerySCollectionOps(scoll) +} 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 8c83dd24dd..1100fca56a 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 @@ -42,6 +42,16 @@ final case class TableRowJsonTap(path: String, params: TableRowJsonIO.ReadParam) sc.read(TableRowJsonIO(path))(params) } +final case class BigQueryTableRowTypedTap[T: Coder](table: Table, fn: TableRow => T) + extends Tap[T] { + lazy val client: BigQuery = BigQuery.defaultInstance() + + override def value: Iterator[T] = client.tables.rows(table).map(fn) + + override def open(sc: ScioContext): SCollection[T] = + sc.read(BigQueryTypedTable(table, Format.TableRow)(tableRowCoder)).map(fn) +} + final case class BigQueryTypedTap[T: Coder](table: Table, fn: (GenericRecord, TableSchema) => T) extends Tap[T] { lazy val client: BigQuery = BigQuery.defaultInstance() diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/types/BigQueryType.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/types/BigQueryType.scala index 0992e70b47..e82b10842c 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/types/BigQueryType.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigquery/types/BigQueryType.scala @@ -53,6 +53,7 @@ import scala.util.Try * @groupname Ungrouped * Other Members */ +@deprecated("Use magnolify API instead.", "0.15.0") object BigQueryType { /** @@ -194,6 +195,7 @@ object BigQueryType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class fromTable(tableSpec: String, args: String*) extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.tableImpl } @@ -225,6 +227,7 @@ object BigQueryType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class fromSchema(schema: String) extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.schemaImpl } @@ -275,6 +278,7 @@ object BigQueryType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class fromStorage( tableSpec: String, args: List[Any] = Nil, @@ -322,6 +326,7 @@ object BigQueryType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class fromQuery(query: String, args: Any*) extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.queryImpl } @@ -342,34 +347,40 @@ object BigQueryType { @compileTimeOnly( "enable macro paradise (2.12) or -Ymacro-annotations (2.13) to expand macro annotations" ) + @deprecated("Use magnolify API instead.", "0.15.0") class toTable extends StaticAnnotation { def macroTransform(annottees: Any*): Any = macro TypeProvider.toTableImpl } /** Generate [[org.apache.avro.Schema Schema]] for a case class. */ + @deprecated("Use magnolify API instead.", "0.15.0") def avroSchemaOf[T: TypeTag]: Schema = SchemaProvider.avroSchemaOf[T] /** * Generate [[com.google.api.services.bigquery.model.TableSchema TableSchema]] for a case class. */ + @deprecated("Use magnolify API instead.", "0.15.0") def schemaOf[T: TypeTag]: TableSchema = SchemaProvider.schemaOf[T] /** * Generate a converter function from Avro [[GenericRecord]] to the given case class `T`. * @group converters */ + @deprecated("Use magnolify API instead.", "0.15.0") def fromAvro[T]: GenericRecord => T = macro ConverterProvider.fromAvroImpl[T] /** * Generate a converter function from the given case class `T` to [[GenericRecord]]. * @group converters */ + @deprecated("Use magnolify API instead.", "0.15.0") def toAvro[T]: T => GenericRecord = macro ConverterProvider.toAvroImpl[T] /** * Generate a converter function from [[TableRow]] to the given case class `T`. * @group converters */ + @deprecated("Use magnolify API instead.", "0.15.0") def fromTableRow[T]: TableRow => T = macro ConverterProvider.fromTableRowImpl[T] @@ -377,9 +388,11 @@ object BigQueryType { * Generate a converter function from the given case class `T` to [[TableRow]]. * @group converters */ + @deprecated("Use magnolify API instead.", "0.15.0") def toTableRow[T]: T => TableRow = macro ConverterProvider.toTableRowImpl[T] /** Create a new BigQueryType instance. */ + @deprecated("Use magnolify API instead.", "0.15.0") @inline final def apply[T: TypeTag]: BigQueryType[T] = new BigQueryType[T] } @@ -388,6 +401,7 @@ object BigQueryType { * * This decouples generated fields and methods from macro expansion to keep core macro free. */ +@deprecated("Use magnolify API instead.", "0.15.0") class BigQueryType[T: TypeTag] { private[this] val bases = typeOf[T].companion.baseClasses 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 1cabe73a7f..fcbb6c1236 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 @@ -25,6 +25,7 @@ import com.spotify.scio.coders.{Coder, CoderMaterializer} import com.spotify.scio.io.{EmptyTap, EmptyTapOf, ScioIO, Tap, TapT, TestIO} import com.spotify.scio.util.Functions import com.spotify.scio.values.SCollection +import magnolify.bigtable.BigtableType import org.apache.beam.sdk.io.gcp.{bigtable => beam} import org.apache.beam.sdk.io.range.ByteKeyRange import org.apache.beam.sdk.values.KV @@ -56,17 +57,13 @@ final case class BigtableRead(bigtableOptions: BigtableOptions, tableId: String) override protected def read(sc: ScioContext, params: ReadP): SCollection[Row] = { val coder = CoderMaterializer.beam(sc, Coder.protoMessageCoder[Row]) - val opts = bigtableOptions // defeat closure - val read = beam.BigtableIO - .read() - .withProjectId(bigtableOptions.getProjectId) - .withInstanceId(bigtableOptions.getInstanceId) - .withTableId(tableId) - .withBigtableOptionsConfigurator(Functions.serializableFn(_ => opts.toBuilder)) - .withMaxBufferElementCount(params.maxBufferElementCount.map(Int.box).orNull) - .pipe(r => if (params.keyRanges.isEmpty) r else r.withKeyRanges(params.keyRanges.asJava)) - .pipe(r => Option(params.rowFilter).fold(r)(r.withRowFilter)): @nowarn("cat=deprecation") - + val read = BigtableRead.read( + bigtableOptions, + tableId, + params.maxBufferElementCount, + params.keyRanges, + params.rowFilter + ) sc.applyTransform(read).setCoder(coder) } @@ -97,14 +94,119 @@ object BigtableRead { maxBufferElementCount: Option[Int] = ReadParam.DefaultMaxBufferElementCount ) - final def apply(projectId: String, instanceId: String, tableId: String): BigtableRead = { - val bigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) - .build - BigtableRead(bigtableOptions, tableId) + private[scio] def read( + bigtableOptions: BigtableOptions, + tableId: String, + maxBufferElementCount: Option[Int], + keyRanges: Seq[ByteKeyRange], + rowFilter: RowFilter + ): beam.BigtableIO.Read = { + val opts = bigtableOptions // defeat closure + beam.BigtableIO + .read() + .withProjectId(bigtableOptions.getProjectId) + .withInstanceId(bigtableOptions.getInstanceId) + .withTableId(tableId) + .withBigtableOptionsConfigurator(Functions.serializableFn(_ => opts.toBuilder)) + .withMaxBufferElementCount(maxBufferElementCount.map(Int.box).orNull) + .pipe(r => if (keyRanges.isEmpty) r else r.withKeyRanges(keyRanges.asJava)) + .pipe(r => Option(rowFilter).fold(r)(r.withRowFilter)): @nowarn("cat=deprecation") + } +} + +final case class BigtableTypedIO[K: Coder, T: BigtableType: Coder]( + bigtableOptions: BigtableOptions, + tableId: String +) extends BigtableIO[(K, T)] { + override type ReadP = BigtableTypedIO.ReadParam[K] + override type WriteP = BigtableTypedIO.WriteParam[K] + + override def testId: String = + s"BigtableIO(${bigtableOptions.getProjectId}\t${bigtableOptions.getInstanceId}\t$tableId)" + + override protected def read( + sc: ScioContext, + params: ReadP + ): SCollection[(K, T)] = { + val coder = CoderMaterializer.beam(sc, Coder.protoMessageCoder[Row]) + val read = BigtableRead.read( + bigtableOptions, + tableId, + params.maxBufferElementCount, + params.keyRanges, + params.rowFilter + ) + + val bigtableType: BigtableType[T] = implicitly + val cf = params.columnFamily + val keyFn = params.keyFn + sc.transform( + _.applyTransform(read) + .setCoder(coder) + .map(row => keyFn(row.getKey) -> bigtableType(row, cf)) + ) } + + override protected def write( + data: SCollection[(K, T)], + params: WriteP + ): Tap[Nothing] = { + val bigtableType: BigtableType[T] = implicitly + val btParams = params.numOfShards match { + case None => BigtableWrite.Default + case Some(numShards) => + BigtableWrite.Bulk( + numShards, + Option(params.flushInterval).getOrElse(BigtableWrite.Bulk.DefaultFlushInterval) + ) + } + val cf = params.columnFamily + val ts = params.timestamp + val keyFn = params.keyFn + data.transform_("Bigtable write") { coll => + coll + .map { case (key, t) => + val mutations = Iterable(bigtableType.apply(t, cf, ts)).asJava + .asInstanceOf[java.lang.Iterable[Mutation]] + KV.of(keyFn(key), mutations) + } + .applyInternal(BigtableWrite.sink(tableId, bigtableOptions, btParams)) + } + EmptyTap + } + + override def tap(params: ReadP): Tap[Nothing] = + throw new NotImplementedError("Bigtable tap not implemented") +} + +object BigtableTypedIO { + object ReadParam { + val DefaultKeyRanges: Seq[ByteKeyRange] = Seq.empty[ByteKeyRange] + val DefaultRowFilter: RowFilter = null + val DefaultMaxBufferElementCount: Option[Int] = None + } + + final case class ReadParam[K] private ( + columnFamily: String, + keyFn: ByteString => K, + keyRanges: Seq[ByteKeyRange] = ReadParam.DefaultKeyRanges, + rowFilter: RowFilter = ReadParam.DefaultRowFilter, + maxBufferElementCount: Option[Int] = ReadParam.DefaultMaxBufferElementCount + ) + + object WriteParam { + val DefaultTimestamp: Long = 0L + val DefaultNumOfShards: Option[Int] = None + val DefaultFlushInterval: Duration = null + } + + final case class WriteParam[K] private ( + columnFamily: String, + keyFn: K => ByteString, + timestamp: Long = WriteParam.DefaultTimestamp, + numOfShards: Option[Int] = WriteParam.DefaultNumOfShards, + flushInterval: Duration = WriteParam.DefaultFlushInterval + ) } final case class BigtableWrite[T <: Mutation](bigtableOptions: BigtableOptions, tableId: String) @@ -127,27 +229,12 @@ final case class BigtableWrite[T <: Mutation](bigtableOptions: BigtableOptions, data: SCollection[(ByteString, Iterable[T])], params: WriteP ): Tap[Nothing] = { - val sink = - params match { - case BigtableWrite.Default => - val opts = bigtableOptions // defeat closure - beam.BigtableIO - .write() - .withProjectId(bigtableOptions.getProjectId) - .withInstanceId(bigtableOptions.getInstanceId) - .withTableId(tableId) - .withBigtableOptionsConfigurator( - Functions.serializableFn(_ => opts.toBuilder) - ): @nowarn("cat=deprecation") - case BigtableWrite.Bulk(numOfShards, flushInterval) => - new BigtableBulkWriter(tableId, bigtableOptions, numOfShards, flushInterval) - } data.transform_("Bigtable write") { coll => coll .map { case (key, value) => KV.of(key, value.asJava.asInstanceOf[java.lang.Iterable[Mutation]]) } - .applyInternal(sink) + .applyInternal(BigtableWrite.sink(tableId, bigtableOptions, params)) } EmptyTap } @@ -181,4 +268,21 @@ object BigtableWrite { .build BigtableWrite[T](bigtableOptions, tableId) } + + private[scio] def sink(tableId: String, bigtableOptions: BigtableOptions, params: WriteParam) = { + params match { + case BigtableWrite.Default => + val opts = bigtableOptions // defeat closure + beam.BigtableIO + .write() + .withProjectId(bigtableOptions.getProjectId) + .withInstanceId(bigtableOptions.getInstanceId) + .withTableId(tableId) + .withBigtableOptionsConfigurator( + Functions.serializableFn(_ => opts.toBuilder) + ): @nowarn("cat=deprecation") + case BigtableWrite.Bulk(numOfShards, flushInterval) => + new BigtableBulkWriter(tableId, bigtableOptions, numOfShards, flushInterval) + } + } } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala index f918df27bb..c5d395ab67 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala @@ -23,8 +23,9 @@ import com.google.protobuf.ByteString import com.spotify.scio.io.ClosedTap import com.spotify.scio.values.SCollection import org.joda.time.Duration - -import com.spotify.scio.bigtable.BigtableWrite +import com.spotify.scio.bigtable.{BigtableTypedIO, BigtableWrite} +import com.spotify.scio.coders.Coder +import magnolify.bigtable.BigtableType /** * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with Bigtable methods. @@ -56,8 +57,57 @@ final class SCollectionMutationOps[T <: Mutation]( ) } +final class BigtableTypedOps[K: Coder, T: BigtableType: Coder]( + private val self: SCollection[(K, T)] +) { + private def btOpts(projectId: String, instanceId: String): BigtableOptions = + BigtableOptions.builder().setProjectId(projectId).setInstanceId(instanceId).build + + def saveAsBigtable( + projectId: String, + instanceId: String, + tableId: String, + columnFamily: String, + keyFn: K => ByteString + ): ClosedTap[Nothing] = { + val params = BigtableTypedIO.WriteParam[K](columnFamily, keyFn) + self.write(BigtableTypedIO[K, T](btOpts(projectId, instanceId), tableId))(params) + } + + def saveAsBigtable( + projectId: String, + instanceId: String, + tableId: String, + columnFamily: String, + keyFn: K => ByteString, + timestamp: Long + ): ClosedTap[Nothing] = { + val params = BigtableTypedIO.WriteParam[K](columnFamily, keyFn, timestamp) + self.write(BigtableTypedIO[K, T](btOpts(projectId, instanceId), tableId))(params) + } + + def saveAsBigtable( + bigtableOptions: BigtableOptions, + tableId: String, + columnFamily: String, + keyFn: K => ByteString, + timestamp: Long = BigtableTypedIO.WriteParam.DefaultTimestamp, + numOfShards: Int, + flushInterval: Duration = BigtableTypedIO.WriteParam.DefaultFlushInterval + ): ClosedTap[Nothing] = { + val params = + BigtableTypedIO + .WriteParam[K](columnFamily, keyFn, timestamp, Some(numOfShards), flushInterval) + self.write(BigtableTypedIO[K, T](bigtableOptions, tableId))(params) + } +} + trait SCollectionSyntax { implicit def bigtableMutationOps[T <: Mutation]( sc: SCollection[(ByteString, Iterable[T])] ): SCollectionMutationOps[T] = new SCollectionMutationOps[T](sc) + + implicit def bigtableTypedOps[K: Coder, T: BigtableType: Coder]( + sc: SCollection[(K, T)] + ): BigtableTypedOps[K, T] = new BigtableTypedOps[K, T](sc) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala index cd21ee3f90..c6517bfec2 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala @@ -20,11 +20,12 @@ package com.spotify.scio.bigtable.syntax import com.google.bigtable.admin.v2.GcRule import com.google.bigtable.v2._ import com.google.cloud.bigtable.config.BigtableOptions +import com.google.protobuf.ByteString import com.spotify.scio.ScioContext -import com.spotify.scio.bigtable.BigtableRead -import com.spotify.scio.bigtable.BigtableUtil -import com.spotify.scio.bigtable.TableAdmin +import com.spotify.scio.bigtable.{BigtableRead, BigtableTypedIO, BigtableUtil, TableAdmin} +import com.spotify.scio.coders.Coder import com.spotify.scio.values.SCollection +import magnolify.bigtable.BigtableType import org.apache.beam.sdk.io.range.ByteKeyRange import org.joda.time.Duration @@ -39,6 +40,73 @@ object ScioContextOps { final class ScioContextOps(private val self: ScioContext) extends AnyVal { import ScioContextOps._ + private def btOpts(projectId: String, instanceId: String): BigtableOptions = + BigtableOptions.builder().setProjectId(projectId).setInstanceId(instanceId).build + + def typedBigtable[K: Coder, T: BigtableType: Coder]( + projectId: String, + instanceId: String, + tableId: String, + columnFamily: String, + keyFn: ByteString => K + ): SCollection[(K, T)] = + typedBigtable(btOpts(projectId, instanceId), tableId, columnFamily, keyFn) + + def typedBigtable[K: Coder, T: BigtableType: Coder]( + projectId: String, + instanceId: String, + tableId: String, + columnFamily: String, + keyFn: ByteString => K, + keyRanges: Seq[ByteKeyRange] + ): SCollection[(K, T)] = + typedBigtable(btOpts(projectId, instanceId), tableId, columnFamily, keyFn, keyRanges) + + def typedBigtable[K: Coder, T: BigtableType: Coder]( + projectId: String, + instanceId: String, + tableId: String, + columnFamily: String, + keyFn: ByteString => K, + keyRanges: Seq[ByteKeyRange], + rowFilter: RowFilter + ): SCollection[(K, T)] = + typedBigtable(btOpts(projectId, instanceId), tableId, columnFamily, keyFn, keyRanges, rowFilter) + + def typedBigtable[K: Coder, T: BigtableType: Coder]( + projectId: String, + instanceId: String, + tableId: String, + columnFamily: String, + keyFn: ByteString => K, + keyRanges: Seq[ByteKeyRange], + rowFilter: RowFilter, + maxBufferElementCount: Option[Int] + ): SCollection[(K, T)] = + typedBigtable( + btOpts(projectId, instanceId), + tableId, + columnFamily, + keyFn, + keyRanges, + rowFilter, + maxBufferElementCount + ) + + def typedBigtable[K: Coder, T: BigtableType: Coder]( + bigtableOptions: BigtableOptions, + tableId: String, + columnFamily: String, + keyFn: ByteString => K, + keyRanges: Seq[ByteKeyRange] = BigtableRead.ReadParam.DefaultKeyRanges, + rowFilter: RowFilter = BigtableRead.ReadParam.DefaultRowFilter, + maxBufferElementCount: Option[Int] = BigtableRead.ReadParam.DefaultMaxBufferElementCount + ): SCollection[(K, T)] = { + val params = + BigtableTypedIO.ReadParam(columnFamily, keyFn, keyRanges, rowFilter, maxBufferElementCount) + self.read(BigtableTypedIO[K, T](bigtableOptions, tableId))(params) + } + /** Get an SCollection for a Bigtable table. */ def bigtable( projectId: String, @@ -47,7 +115,7 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { keyRange: ByteKeyRange, rowFilter: RowFilter ): SCollection[Row] = - bigtable(projectId, instanceId, tableId, Seq(keyRange), rowFilter) + bigtable(btOpts(projectId, instanceId), tableId, Seq(keyRange), rowFilter) /** Get an SCollection for a Bigtable table. */ def bigtable( @@ -58,20 +126,32 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { rowFilter: RowFilter, maxBufferElementCount: Option[Int] ): SCollection[Row] = - bigtable(projectId, instanceId, tableId, Seq(keyRange), rowFilter, maxBufferElementCount) + bigtable( + btOpts(projectId, instanceId), + tableId, + Seq(keyRange), + rowFilter, + maxBufferElementCount + ) + + /** Get an SCollection for a Bigtable table. */ + def bigtable( + projectId: String, + instanceId: String, + tableId: String + ): SCollection[Row] = + bigtable(btOpts(projectId, instanceId), tableId) /** Get an SCollection for a Bigtable table. */ def bigtable( projectId: String, instanceId: String, tableId: String, - keyRanges: Seq[ByteKeyRange] = BigtableRead.ReadParam.DefaultKeyRanges, - rowFilter: RowFilter = BigtableRead.ReadParam.DefaultRowFilter, - maxBufferElementCount: Option[Int] = BigtableRead.ReadParam.DefaultMaxBufferElementCount - ): SCollection[Row] = { - val parameters = BigtableRead.ReadParam(keyRanges, rowFilter, maxBufferElementCount) - self.read(BigtableRead(projectId, instanceId, tableId))(parameters) - } + keyRanges: Seq[ByteKeyRange], + rowFilter: RowFilter, + maxBufferElementCount: Option[Int] + ): SCollection[Row] = + bigtable(btOpts(projectId, instanceId), tableId, keyRanges, rowFilter, maxBufferElementCount) /** Get an SCollection for a Bigtable table. */ def bigtable( @@ -96,20 +176,9 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { def bigtable( bigtableOptions: BigtableOptions, tableId: String, - keyRanges: Seq[ByteKeyRange], - rowFilter: RowFilter - ): SCollection[Row] = { - val parameters = BigtableRead.ReadParam(keyRanges, rowFilter) - self.read(BigtableRead(bigtableOptions, tableId))(parameters) - } - - /** Get an SCollection for a Bigtable table. */ - def bigtable( - bigtableOptions: BigtableOptions, - tableId: String, - keyRanges: Seq[ByteKeyRange], - rowFilter: RowFilter, - maxBufferElementCount: Option[Int] + keyRanges: Seq[ByteKeyRange] = BigtableRead.ReadParam.DefaultKeyRanges, + rowFilter: RowFilter = BigtableRead.ReadParam.DefaultRowFilter, + maxBufferElementCount: Option[Int] = BigtableRead.ReadParam.DefaultMaxBufferElementCount ): SCollection[Row] = { val parameters = BigtableRead.ReadParam(keyRanges, rowFilter, maxBufferElementCount) self.read(BigtableRead(bigtableOptions, tableId))(parameters) 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 2977d464d8..a37b7da1d6 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 @@ -19,30 +19,69 @@ package com.spotify.scio.datastore import com.spotify.scio.ScioContext import com.spotify.scio.values.SCollection -import com.spotify.scio.io.{EmptyTap, EmptyTapOf, ScioIO, Tap, TapT} +import com.spotify.scio.io.{EmptyTap, EmptyTapOf, ScioIO, Tap, TapT, TestIO} import com.google.datastore.v1.{Entity, Query} import com.spotify.scio.coders.{Coder, CoderMaterializer} +import com.spotify.scio.datastore.DatastoreTypedIO.{ReadParam, WriteParam} +import magnolify.datastore.EntityType import org.apache.beam.sdk.io.gcp.datastore.{DatastoreIO => BDatastoreIO, DatastoreV1 => BDatastore} -final case class DatastoreIO(projectId: String) extends ScioIO[Entity] { - override type ReadP = DatastoreIO.ReadParam - override type WriteP = DatastoreIO.WriteParam +sealed trait DatastoreIO[T] extends ScioIO[T] { + final override val tapT: TapT.Aux[T, Nothing] = EmptyTapOf[T] +} - override val tapT: TapT.Aux[Entity, Nothing] = EmptyTapOf[Entity] +object DatastoreIO { + final def apply[T](projectId: String): DatastoreIO[T] = + new DatastoreIO[T] with TestIO[T] { + override def testId: String = s"DatastoreIO($projectId)" + } +} - override protected def read(sc: ScioContext, params: ReadP): SCollection[Entity] = { - val coder = CoderMaterializer.beam(sc, Coder.protoMessageCoder[Entity]) - val read = BDatastoreIO - .v1() - .read() - .withProjectId(projectId) - .withNamespace(params.namespace) - .withQuery(params.query) - sc.applyTransform( - Option(params.configOverride).map(_(read)).getOrElse(read) - ).setCoder(coder) +final case class DatastoreTypedIO[T: EntityType: Coder](projectId: String) extends DatastoreIO[T] { + override type ReadP = DatastoreTypedIO.ReadParam + override type WriteP = DatastoreTypedIO.WriteParam + override def testId: String = s"DatastoreIO($projectId)" + + override protected def read(sc: ScioContext, params: ReadParam): SCollection[T] = { + val entityType: EntityType[T] = implicitly + sc.transform { ctx => + DatastoreEntityIO + .read(ctx, projectId, params.namespace, params.query, params.configOverride) + .map(e => entityType(e)) + } + } + + override protected def write(data: SCollection[T], params: WriteParam): Tap[Nothing] = { + val entityType: EntityType[T] = implicitly + val write = BDatastoreIO.v1.write.withProjectId(projectId) + data.transform_ { scoll => + scoll + .map(t => entityType(t)) + .applyInternal( + Option(params.configOverride).map(_(write)).getOrElse(write) + ) + } + EmptyTap } + override def tap(read: ReadParam): Tap[Nothing] = EmptyTap +} + +object DatastoreTypedIO { + type ReadParam = DatastoreEntityIO.ReadParam + val ReadParam = DatastoreEntityIO.ReadParam + type WriteParam = DatastoreEntityIO.WriteParam + val WriteParam = DatastoreEntityIO.WriteParam +} + +final case class DatastoreEntityIO(projectId: String) extends DatastoreIO[Entity] { + override type ReadP = DatastoreEntityIO.ReadParam + override type WriteP = DatastoreEntityIO.WriteParam + override def testId: String = s"DatastoreIO($projectId)" + + override protected def read(sc: ScioContext, params: ReadP): SCollection[Entity] = + DatastoreEntityIO.read(sc, projectId, params.namespace, params.query, params.configOverride) + override protected def write(data: SCollection[Entity], params: WriteP): Tap[Nothing] = { val write = BDatastoreIO.v1.write.withProjectId(projectId) data.applyInternal( @@ -51,10 +90,10 @@ final case class DatastoreIO(projectId: String) extends ScioIO[Entity] { EmptyTap } - override def tap(read: DatastoreIO.ReadParam): Tap[Nothing] = EmptyTap + override def tap(read: DatastoreEntityIO.ReadParam): Tap[Nothing] = EmptyTap } -object DatastoreIO { +object DatastoreEntityIO { object ReadParam { val DefaultNamespace: String = null @@ -74,4 +113,23 @@ object DatastoreIO { final case class WriteParam private ( configOverride: BDatastore.Write => BDatastore.Write = WriteParam.DefaultConfigOverride ) + + private[scio] def read( + sc: ScioContext, + projectId: String, + namespace: String, + query: Query, + configOverride: BDatastore.Read => BDatastore.Read + ): SCollection[Entity] = { + val coder = CoderMaterializer.beam(sc, Coder.protoMessageCoder[Entity]) + val read = BDatastoreIO + .v1() + .read() + .withProjectId(projectId) + .withNamespace(namespace) + .withQuery(query) + sc.applyTransform( + Option(configOverride).map(_(read)).getOrElse(read) + ).setCoder(coder) + } } 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 8ed685b3bf..bfc420cba9 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 @@ -19,10 +19,11 @@ import com.google.datastore.v1.Entity package com.spotify.scio.datastore.syntax import com.spotify.scio.values.SCollection -import com.spotify.scio.datastore.DatastoreIO +import com.spotify.scio.datastore.{DatastoreEntityIO, DatastoreTypedIO} import com.spotify.scio.io.ClosedTap import com.google.datastore.v1.Entity -import com.spotify.scio.datastore.DatastoreIO.WriteParam +import com.spotify.scio.coders.Coder +import magnolify.datastore.EntityType import org.apache.beam.sdk.io.gcp.datastore.{DatastoreV1 => BDatastore} final class SCollectionEntityOps[T <: Entity](private val coll: SCollection[T]) extends AnyVal { @@ -33,13 +34,29 @@ final class SCollectionEntityOps[T <: Entity](private val coll: SCollection[T]) */ def saveAsDatastore( projectId: String, - configOverride: BDatastore.Write => BDatastore.Write = WriteParam.DefaultConfigOverride + configOverride: BDatastore.Write => BDatastore.Write = + DatastoreEntityIO.WriteParam.DefaultConfigOverride ): ClosedTap[Nothing] = - coll.covary_[Entity].write(DatastoreIO(projectId))(WriteParam(configOverride)) + coll + .covary_[Entity] + .write(DatastoreEntityIO(projectId))(DatastoreEntityIO.WriteParam(configOverride)) +} + +final class TypedEntitySCollectionOps[T: EntityType: Coder](private val coll: SCollection[T]) { + def saveAsDatastore( + projectId: String, + configOverride: BDatastore.Write => BDatastore.Write = + DatastoreTypedIO.WriteParam.DefaultConfigOverride + ): ClosedTap[Nothing] = + coll.write(DatastoreTypedIO(projectId))(DatastoreTypedIO.WriteParam(configOverride)) } trait SCollectionSyntax { implicit def datastoreEntitySCollectionOps[T <: Entity]( coll: SCollection[T] ): SCollectionEntityOps[T] = new SCollectionEntityOps(coll) + + implicit def typedDatastoreEntitySCollectionOps[T: EntityType: Coder]( + coll: SCollection[T] + ): TypedEntitySCollectionOps[T] = new TypedEntitySCollectionOps(coll) } 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 bb50a920a1..83c82c7165 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 @@ -19,9 +19,10 @@ package com.spotify.scio.datastore.syntax import com.spotify.scio.ScioContext import com.spotify.scio.values.SCollection -import com.spotify.scio.datastore.DatastoreIO +import com.spotify.scio.datastore.{DatastoreEntityIO, DatastoreTypedIO} import com.google.datastore.v1.{Entity, Query} -import com.spotify.scio.datastore.DatastoreIO.ReadParam +import com.spotify.scio.coders.Coder +import magnolify.datastore.EntityType import org.apache.beam.sdk.io.gcp.datastore.{DatastoreV1 => BDatastore} final class ScioContextOps(private val sc: ScioContext) extends AnyVal { @@ -33,10 +34,24 @@ final class ScioContextOps(private val sc: ScioContext) extends AnyVal { def datastore( projectId: String, query: Query, - namespace: String = ReadParam.DefaultNamespace, - configOverride: BDatastore.Read => BDatastore.Read = ReadParam.DefaultConfigOverride + namespace: String = DatastoreEntityIO.ReadParam.DefaultNamespace, + configOverride: BDatastore.Read => BDatastore.Read = + DatastoreEntityIO.ReadParam.DefaultConfigOverride ): SCollection[Entity] = - sc.read(DatastoreIO(projectId))(ReadParam(query, namespace, configOverride)) + sc.read(DatastoreEntityIO(projectId))( + DatastoreEntityIO.ReadParam(query, namespace, configOverride) + ) + + def typedDatastore[T: EntityType: Coder]( + projectId: String, + query: Query, + namespace: String = DatastoreTypedIO.ReadParam.DefaultNamespace, + configOverride: BDatastore.Read => BDatastore.Read = + DatastoreTypedIO.ReadParam.DefaultConfigOverride + ): SCollection[T] = + sc.read(DatastoreTypedIO(projectId))( + DatastoreTypedIO.ReadParam(query, namespace, configOverride) + ) } trait ScioContextSyntax { diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigquery/BigQueryIOTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigquery/BigQueryIOTest.scala index 45fbe21966..30a1575509 100644 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigquery/BigQueryIOTest.scala +++ b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigquery/BigQueryIOTest.scala @@ -38,6 +38,8 @@ object BigQueryIOTest { @BigQueryType.toTable case class BQRecord(i: Int, s: String, r: List[String]) + case class MagnolifyRecord(i: Int, s: String) + // BQ Write transform display id data for tableDescription private val TableDescriptionId = DisplayData.Identifier.of( DisplayData.Path.root(), @@ -191,6 +193,58 @@ final class BigQueryIOTest extends ScioIOSpec { testJobTest(xs)(TableRowJsonIO(_))(_.tableRowJsonFile(_))(_.saveAsTableRowJsonFile(_)) } + "MagnolifyBigQuerySelect" should "work" in { + // unsafe implicits must be explicitly imported for TableRowType[MagnolifyRecord] to be derived + import magnolify.bigquery.unsafe._ + val xs = (1 to 100).map(x => MagnolifyRecord(x, x.toString)) + testJobTest(xs, in = "select * from x", out = "project:dataset.out_table") { + BigQueryIO(_) + } { (coll, s) => + coll.typedBigQuerySelect[MagnolifyRecord](Query(s)) + } { (coll, s) => + coll.saveAsBigQueryTable(Table.Spec(s)) + } + } + + "MagnolifyBigQueryTable" should "work" in { + // unsafe implicits must be explicitly imported for TableRowType[MagnolifyRecord] to be derived + import magnolify.bigquery.unsafe._ + val xs = (1 to 100).map(x => MagnolifyRecord(x, x.toString)) + testJobTest(xs, in = "project:dataset.in_table", out = "project:dataset.out_table") { + BigQueryIO(_) + } { (coll, s) => + coll.typedBigQueryTable[MagnolifyRecord](Table.Spec(s)) + } { (coll, s) => + coll.saveAsBigQueryTable(Table.Spec(s)) + } + } + + "MagnolifyBigQueryStorage" should "work with Table" in { + // unsafe implicits must be explicitly imported for TableRowType[MagnolifyRecord] to be derived + import magnolify.bigquery.unsafe._ + val xs = (1 to 100).map(x => MagnolifyRecord(x, x.toString)) + testJobTest(xs, in = "project:dataset.in_table", out = "project:dataset.out_table")( + BigQueryIO(_, List(), None), + Some(BigQueryIO(_)) + ) { (coll, s) => + coll.typedBigQueryStorageMagnolify[MagnolifyRecord](Table.Spec(s)) + } { (coll, s) => + coll.saveAsBigQueryTable(Table.Spec(s)) + } + } + + it should "work with Query" in { + // unsafe implicits must be explicitly imported for TableRowType[MagnolifyRecord] to be derived + import magnolify.bigquery.unsafe._ + val xs = (1 to 100).map(x => MagnolifyRecord(x, x.toString)) + testJobTest(xs, in = "select x, y from z", out = "project:dataset.out_table") { + BigQueryIO(_) + } { (coll, s) => + coll.typedBigQueryStorageMagnolify[MagnolifyRecord](Query(s)) + } { (coll, s) => + coll.saveAsBigQueryTable(Table.Spec(s)) + } + } } object JobWithDuplicateInput { diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala index eb47617917..95564b34eb 100644 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala +++ b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala @@ -22,9 +22,13 @@ import com.google.bigtable.v2.{Mutation, Row} import com.google.protobuf.ByteString import com.spotify.scio.testing._ +// must be defined outside the test class or test job will hang +case class Foo(i: Int, s: String) + class BigtableIOTest extends ScioIOSpec { val projectId = "project" val instanceId = "instance" + val columnFamily = "columnFamily" "BigtableIO" should "work with input" in { val xs = (1 to 100).map { x => @@ -46,4 +50,24 @@ class BigtableIOTest extends ScioIOSpec { _.saveAsBigtable(projectId, instanceId, _) ) } + + it should "work with typed input" in { + val xs = (1 to 100).map(x => x.toString -> Foo(x, x.toString)) + testJobTestInput(xs)(BigtableIO[(String, Foo)](projectId, instanceId, _))( + _.typedBigtable[String, Foo]( + projectId, + instanceId, + _, + columnFamily, + (bs: ByteString) => bs.toStringUtf8 + ) + ) + } + + it should "work with typed output" in { + val xs = (1 to 100).map(x => (x.toString, Foo(x, x.toString))) + testJobTestOutput(xs)(BigtableIO(projectId, instanceId, _))( + _.saveAsBigtable(projectId, instanceId, _, columnFamily, ByteString.copyFromUtf8 _) + ) + } } diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/datastore/DatastoreIOTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/datastore/DatastoreIOTest.scala index 9461a9c793..785035d2ec 100644 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/datastore/DatastoreIOTest.scala +++ b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/datastore/DatastoreIOTest.scala @@ -34,7 +34,20 @@ object DatastoreJob { } } +object TypedDatastoreJob { + case class MyEntity(int: Long) + + def main(cmdlineArgs: Array[String]): Unit = { + val (sc, args) = ContextAndArgs(cmdlineArgs) + sc.datastore(args("input"), null, null) + .saveAsDatastore(args("output")) + sc.run() + () + } +} + class DatastoreIOTest extends PipelineSpec with ScioIOSpec { + import TypedDatastoreJob.MyEntity "DatastoreIO" should "work" in { val xs = (1L to 100L).map { x => @@ -57,7 +70,7 @@ class DatastoreIOTest extends PipelineSpec with ScioIOSpec { JobTest[DatastoreJob.type] .args("--input=store.in", "--output=store.out") .input(DatastoreIO("store.in"), (1L to 3L).map(newEntity)) - .output(DatastoreIO("store.out"))(coll => coll should containInAnyOrder(xs)) + .output(DatastoreIO[Entity]("store.out"))(coll => coll should containInAnyOrder(xs)) .run() it should "pass correct DatastoreJob" in { @@ -73,4 +86,31 @@ class DatastoreIOTest extends PipelineSpec with ScioIOSpec { } } + it should "work with typed data" in { + val xs = (1L to 100L).map(x => MyEntity(x)) + testJobTest(xs)(DatastoreIO(_))(_.typedDatastore[MyEntity](_, null))(_.saveAsDatastore(_)) + } + + def testTypedDatastore(xs: Seq[MyEntity]): Unit = { + val in = (1L to 3L).map(MyEntity) + JobTest[TypedDatastoreJob.type] + .args("--input=store.in", "--output=store.out") + .input(DatastoreIO[MyEntity]("store.in"), in) + .output(DatastoreIO[MyEntity]("store.out"))(coll => coll should containInAnyOrder(xs)) + .run() + } + + it should "pass correct TypedDatastoreJob" in { + testTypedDatastore((1L to 3L).map(MyEntity)) + } + + it should "fail incorrect TypedDatastoreJob" in { + an[AssertionError] should be thrownBy { + testTypedDatastore((1L to 2L).map(MyEntity)) + } + an[AssertionError] should be thrownBy { + testTypedDatastore((1L to 4L).map(MyEntity)) + } + } + } 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 7d157c756e..da1e313052 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 @@ -31,7 +31,9 @@ import org.apache.beam.sdk.io.{ import org.apache.beam.sdk.{io => beam} import org.tensorflow.proto.example.{Example, SequenceExample} import com.spotify.scio.io.TapT +import com.spotify.scio.tensorflow.TFExampleIO.ReadParam import com.spotify.scio.util.FilenamePolicySupplier +import magnolify.tensorflow.ExampleType import org.apache.beam.sdk.io.fs.ResourceId import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider import org.apache.beam.sdk.transforms.SerializableFunctions @@ -94,6 +96,59 @@ object TFRecordIO { ) } +object TFExampleTypedIO { + type ReadParam = TFRecordIO.ReadParam + val ReadParam = TFRecordIO.ReadParam + type WriteParam = TFRecordIO.WriteParam + val WriteParam = TFRecordIO.WriteParam +} + +final case class TFExampleTypedIO[T: ExampleType: Coder](path: String) extends ScioIO[T] { + override type WriteP = TFExampleTypedIO.WriteParam + override type ReadP = TFExampleTypedIO.ReadParam + override val tapT: TapT.Aux[T, T] = TapOf[T] + + override def testId: String = s"TFExampleTypedIO($path)" + + override protected def write(data: SCollection[T], params: WriteP): Tap[T] = { + val exampleType: ExampleType[T] = implicitly + data.transform_ { scoll => + scoll + .map(t => exampleType(t).toByteArray) + .applyInternal( + TFRecordMethods.tfWrite( + path, + params.suffix, + params.numShards, + params.compression, + params.filenamePolicySupplier, + params.prefix, + params.shardNameTemplate, + ScioUtil.isWindowed(data), + ScioUtil.tempDirOrDefault(params.tempDirectory, data.context) + ) + ) + } + tap(TFExampleIO.ReadParam(params)) + } + + override def tap(params: ReadP): Tap[T] = { + val exampleType: ExampleType[T] = implicitly + TFRecordMethods + .tap(path, params) + .map(bytes => exampleType(Example.parseFrom(bytes))) + } + + override protected def read(sc: ScioContext, params: ReadParam): SCollection[T] = { + val exampleType: ExampleType[T] = implicitly + sc.transform { ctx => + TFRecordMethods + .read(ctx, path, params) + .map(bytes => exampleType(Example.parseFrom(bytes))) + } + } +} + final case class TFExampleIO(path: String) extends ScioIO[Example] { override type ReadP = TFExampleIO.ReadParam override type WriteP = TFExampleIO.WriteParam @@ -162,7 +217,7 @@ private object TFRecordMethods { ) } - private def tfWrite( + private[scio] def tfWrite( path: String, suffix: String, numShards: Int, 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 91d4bcdcfc..9d5a565db8 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 @@ -25,12 +25,14 @@ import com.spotify.scio.io.ClosedTap import com.spotify.scio.tensorflow.{ SavedBundlePredictDoFn, TFExampleIO, + TFExampleTypedIO, TFRecordIO, TFSequenceExampleIO } import com.spotify.scio.util.FilenamePolicySupplier import com.spotify.scio.values.SCollection import com.spotify.zoltar.tf.TensorFlowModel +import magnolify.tensorflow.ExampleType /** * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with TensorFlow methods. @@ -143,6 +145,38 @@ object PredictSCollectionOps { val DefaultFetchOps: Option[Seq[String]] = None } +final class TypedExampleSCollectionOps[T](private val self: SCollection[T]) { + + /** + * Converts this collection of `T` into Tensorflow [[org.tensorflow.proto.example.Example]]s with + * the provided [[magnolify.tensorflow.ExampleType]], then saves these as a TensorFlow TFRecord + * file. + */ + def saveAsTfRecordFile( + path: String, + suffix: String = TFExampleIO.WriteParam.DefaultSuffix, + compression: Compression = TFExampleIO.WriteParam.DefaultCompression, + numShards: Int = TFExampleIO.WriteParam.DefaultNumShards, + shardNameTemplate: String = TFExampleIO.WriteParam.DefaultShardNameTemplate, + tempDirectory: String = TFExampleIO.WriteParam.DefaultTempDirectory, + filenamePolicySupplier: FilenamePolicySupplier = + TFExampleIO.WriteParam.DefaultFilenamePolicySupplier, + prefix: String = TFExampleIO.WriteParam.DefaultPrefix + )(implicit exampleType: ExampleType[T]): ClosedTap[T] = { + implicit val tCoder: Coder[T] = self.coder + val param = TFExampleTypedIO.WriteParam( + suffix, + compression, + numShards, + filenamePolicySupplier, + prefix, + shardNameTemplate, + tempDirectory + ) + self.write(TFExampleTypedIO(path))(param) + } +} + final class ExampleSCollectionOps[T <: Example](private val self: SCollection[T]) extends AnyVal { /** @@ -320,4 +354,8 @@ trait SCollectionSyntax { implicit def tensorFlowSequenceExampleSCollectionOps[T <: SequenceExample]( s: SCollection[T] ): SequenceExampleSCollectionOps[T] = new SequenceExampleSCollectionOps(s) + + implicit def tensorFlowTypedExampleSCollectionOps[T]( + s: SCollection[T] + ): TypedExampleSCollectionOps[T] = new TypedExampleSCollectionOps(s) } diff --git a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/ScioContextSyntax.scala b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/ScioContextSyntax.scala index 7ecedb47ce..b997440cd8 100644 --- a/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/ScioContextSyntax.scala +++ b/scio-tensorflow/src/main/scala/com/spotify/scio/tensorflow/syntax/ScioContextSyntax.scala @@ -18,16 +18,30 @@ package com.spotify.scio.tensorflow.syntax import java.nio.file.Files - import com.spotify.scio.ScioContext -import com.spotify.scio.tensorflow.{TFExampleIO, TFRecordIO, TFSequenceExampleIO} +import com.spotify.scio.coders.Coder +import com.spotify.scio.tensorflow.{TFExampleIO, TFExampleTypedIO, TFRecordIO, TFSequenceExampleIO} import com.spotify.scio.values.{DistCache, SCollection} +import magnolify.tensorflow.ExampleType import org.apache.beam.sdk.io.Compression import org.tensorflow.proto.example.{Example, SequenceExample} import org.tensorflow.metadata.v0._ final class ScioContextOps(private val self: ScioContext) extends AnyVal { + /** + * Get an SCollection for a TensorFlow TFRecord file. Input must be Records are read back as + * Tensorflow [[org.tensorflow.proto.example.Example]]s then mapped to the user type `T` with the + * implicit [[magnolify.tensorflow.ExampleType]] + * + * @group input + */ + def typedTfRecordFile[T: ExampleType: Coder]( + path: String, + compression: Compression = Compression.AUTO + ): SCollection[T] = + self.read(TFExampleTypedIO(path))(TFExampleTypedIO.ReadParam(compression)) + /** * Get an SCollection for a TensorFlow TFRecord file. Note that TFRecord files are not splittable. * The recommended record encoding is [[org.tensorflow.proto.example.Example]] protocol buffers 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 743972c5a8..f04cab6d3a 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 @@ -39,6 +39,13 @@ class TFExampleIOTest extends ScioIOSpec { testTap(xs)(_.saveAsTfRecordFile(_))(".tfrecords") testJobTest(xs)(TFExampleIO(_))(_.tfRecordExampleFile(_))(_.saveAsTfRecordFile(_)) } + + it should "work with typed records" in { + val xs = (1 to 100).map(x => Record(x, x.toString)) + implicit val exampleType: ExampleType[Record] = recordT + testTap(xs)(_.saveAsTfRecordFile(_))(".tfrecords") + testJobTest(xs)(TFExampleTypedIO(_))(_.typedTfRecordFile(_))(_.saveAsTfRecordFile(_)) + } } class TFExampleIOFileNamePolicyTest extends FileNamePolicySpec[Example] { @@ -65,7 +72,7 @@ class TFExampleIOFileNamePolicyTest extends FileNamePolicySpec[Example] { _.map(x => recordT(Record(x, x.toString))).saveAsTfRecordFile( "nonsense", shardNameTemplate = "SSS-of-NNN", - filenamePolicySupplier = testFilenamePolicySupplier + filenamePolicySupplier = testFilenamePolicySupplier(_, _) ) ) } diff --git a/scio-test/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala b/scio-test/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala index f7d2c91a22..a492bcb3f5 100644 --- a/scio-test/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala +++ b/scio-test/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala @@ -330,7 +330,7 @@ trait SCollectionMatchers extends EqInstances { ): Matcher[T] = matcher.matcher(_.inEarlyGlobalWindowPanes) - /** Assert that the SCollection in question contains the provided elements. */ + /** Assert that the SCollection in question contains exactly the provided elements. */ def containInAnyOrder[T: Coder: Eq]( value: Iterable[T] ): IterableMatcher[SCollection[T], T] = 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 183c8fd0d4..657cc884d3 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 @@ -149,30 +149,34 @@ trait ScioIOSpec extends PipelineSpec { } def testJobTest[T: Coder](xs: Seq[T], in: String = "in", out: String = "out")( - ioFn: String => ScioIO[T] + ioFn: String => ScioIO[T], + optOutIOFn: Option[String => ScioIO[T]] = None )( readFn: (ScioContext, String) => SCollection[T] )( writeFn: (SCollection[T], String) => ClosedTap[_] ): Unit = { + val inIO = ioFn(in) + val outIO = optOutIOFn.map(outIoFn => outIoFn(out)).getOrElse(ioFn(out)) + val testJob = (sc: ScioContext) => writeFn(readFn(sc, in), out) JobTest(testJob) - .input(ioFn(in), xs) - .output(ioFn(out))(_ should containInAnyOrder(xs)) + .input(inIO, xs) + .output(outIO)(_ should containInAnyOrder(xs)) .run() the[IllegalArgumentException] thrownBy { JobTest(testJob) .input(CustomIO[T](in), xs) - .output(ioFn(out))(_ should containInAnyOrder(xs)) + .output(outIO)(_ should containInAnyOrder(xs)) .run() - } should have message s"requirement failed: Missing test input: ${ioFn(in).testId}, available: [CustomIO($in)]" + } should have message s"requirement failed: Missing test input: ${inIO.testId}, available: [CustomIO($in)]" the[IllegalArgumentException] thrownBy { JobTest(testJob) - .input(ioFn(in), xs) + .input(inIO, xs) .output(CustomIO[T](out))(_ should containInAnyOrder(xs)) .run() - } should have message s"requirement failed: Missing test output: ${ioFn(out).testId}, available: [CustomIO($out)]" + } should have message s"requirement failed: Missing test output: ${outIO.testId}, available: [CustomIO($out)]" } } diff --git a/scio-test/src/test/scala/com/spotify/scio/avro/AvroIOTest.scala b/scio-test/src/test/scala/com/spotify/scio/avro/AvroIOTest.scala index dc7e535152..57efe11582 100644 --- a/scio-test/src/test/scala/com/spotify/scio/avro/AvroIOTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/avro/AvroIOTest.scala @@ -34,6 +34,9 @@ import java.io.File object AvroIOTest { @AvroType.toSchema case class AvroRecord(i: Int, s: String, r: List[String]) + case class Track(trackId: String) + + case class Record(i: Int, s: String, r: List[String]) } class AvroIOFileNamePolicyTest extends FileNamePolicySpec[TestRecord] { @@ -59,7 +62,7 @@ class AvroIOFileNamePolicyTest extends FileNamePolicySpec[TestRecord] { _.map(AvroUtils.newSpecificRecord).saveAsAvroFile( "nonsense", shardNameTemplate = "SSS-of-NNN", - filenamePolicySupplier = testFilenamePolicySupplier + filenamePolicySupplier = testFilenamePolicySupplier(_, _) ) ) } @@ -89,7 +92,7 @@ class ObjectIOFileNamePolicyTest extends FileNamePolicySpec[AvroIOTest.AvroRecor _.map(x => AvroRecord(x, x.toString, (1 to x).map(_.toString).toList)).saveAsObjectFile( "nonsense", shardNameTemplate = "SSS-of-NNN", - filenamePolicySupplier = testFilenamePolicySupplier + filenamePolicySupplier = testFilenamePolicySupplier(_, _) ) ) } @@ -117,7 +120,7 @@ class ProtobufIOFileNamePolicyTest extends FileNamePolicySpec[TrackPB] { _.map(x => TrackPB.newBuilder().setTrackId(x.toString).build()).saveAsProtobufFile( "nonsense", shardNameTemplate = "SSS-of-NNN", - filenamePolicySupplier = testFilenamePolicySupplier + filenamePolicySupplier = testFilenamePolicySupplier(_, _) ) ) } @@ -193,6 +196,12 @@ class AvroIOTest extends ScioIOSpec { testJobTest(xs)(io)(_.typedAvroFile[AvroRecord](_))(_.saveAsTypedAvroFile(_)) } + it should "work with typed Avro with magnolify AvroType" in { + val xs = (1 to 100).map(x => Record(x, x.toString, (1 to x).map(_.toString).toList)) + testTap(xs)(_.saveAsAvroFile(_))(".avro") + testJobTest(xs)(AvroIO[Record])(_.typedAvroFileMagnolify[Record](_))(_.saveAsAvroFile(_)) + } + "ObjectFileIO" should "work" in { val xs = (1 to 100).map(x => AvroRecord(x, x.toString, (1 to x).map(_.toString).toList)) testTap(xs)(_.saveAsObjectFile(_))(".obj.avro") @@ -209,4 +218,12 @@ class AvroIOTest extends ScioIOSpec { testJobTest(xs)(ProtobufIO(_))(_.protobufFile[TrackPB](_))(_.saveAsProtobufFile(_)) } + "TypedProtobufIO" should "work" in { + val xs = (1 to 100).map(x => Track(x.toString)) + val suffix = ".protobuf.avro" + testTap(xs)(_.saveAsProtobufFile[TrackPB](_))(suffix) + testJobTest(xs)(ProtobufIO(_))(_.typedProtobufFile[Track, TrackPB](_))( + _.saveAsProtobufFile[TrackPB](_) + ) + } } From ed034fccefce9a5ae34c91d09e19d6670639f5cc Mon Sep 17 00:00:00 2001 From: Claire McGinty Date: Thu, 16 Jan 2025 15:27:35 -0500 Subject: [PATCH 3/8] Drop Java 8 Targeting (#5203) --- README.md | 3 ++- build.sbt | 2 +- site/src/main/paradox/FAQ.md | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 14f32f499a..c523b0dc5f 100644 --- a/README.md +++ b/README.md @@ -31,7 +31,8 @@ Scio is a Scala API for [Apache Beam](http://beam.incubator.apache.org/) and [Go # Quick Start -Download and install the [Java Development Kit (JDK)](https://adoptopenjdk.net/index.html) version 8. +Download and install the Java Development Kit (JDK) version 11 or higher, +eg. [adoptium](https://adoptium.net/index.html) or [corretto](https://aws.amazon.com/corretto/). Install [sbt](https://www.scala-sbt.org/1.x/docs/Setup.html). diff --git a/build.sbt b/build.sbt index f689f396fa..1040422e1a 100644 --- a/build.sbt +++ b/build.sbt @@ -211,7 +211,7 @@ val scala212 = "2.12.20" val scalaDefault = scala213 // compiler settings -ThisBuild / tlJdkRelease := Some(8) +ThisBuild / tlJdkRelease := Some(11) ThisBuild / tlFatalWarnings := false ThisBuild / scalaVersion := scalaDefault ThisBuild / crossScalaVersions := Seq(scalaDefault, scala212) diff --git a/site/src/main/paradox/FAQ.md b/site/src/main/paradox/FAQ.md index 2fe9536556..740e1f43d4 100644 --- a/site/src/main/paradox/FAQ.md +++ b/site/src/main/paradox/FAQ.md @@ -484,7 +484,7 @@ If you encounter an SBT error with message "Initial heap size set to a larger va You might get an error message like `java.io.IOException: Unable to create parent directories of /Applications/IntelliJ IDEA CE.app/Contents/bin/.bigquery/012345abcdef.schema.json`. This usually happens to people who run IntelliJ IDEA with its bundled JVM. There are two solutions. -- Install JDK from [java.com](https://www.java.com/) and switch to it by following the "All platforms: switch between installed runtimes" section in this [page](https://intellij-support.jetbrains.com/hc/en-us/articles/206544879-Selecting-the-JDK-version-the-IDE-will-run-under). +- Install JDK from a vendor like [adoptium](https://adoptium.net/index.html) or [corretto](https://aws.amazon.com/corretto/) and switch to it by following the "All platforms: switch between installed runtimes" section in this [page](https://intellij-support.jetbrains.com/hc/en-us/articles/206544879-Selecting-the-JDK-version-the-IDE-will-run-under). - Override the bigquery `.cache` directory as a JVM compiler parameter. On the bottom right of the IntelliJ window, click the icon that looks like a clock, and then "Configure...". Then, edit the JVM parameters to include the line `-Dbigquery.cache.directory=/.bigquery`. Then, restart the compile server by clicking on the clock icon -> Stop, and then Start. #### How to make IntelliJ IDEA work with type safe BigQuery classes? From 8dc44523186b0b0d185e70865562942bab5598e4 Mon Sep 17 00:00:00 2001 From: kellen Date: Tue, 21 Jan 2025 15:52:14 -0500 Subject: [PATCH 4/8] Remove developers list (#5548) --- build.sbt | 62 ------------------------------------------------------- 1 file changed, 62 deletions(-) diff --git a/build.sbt b/build.sbt index 1040422e1a..0e8340f216 100644 --- a/build.sbt +++ b/build.sbt @@ -142,68 +142,6 @@ ThisBuild / organizationName := "Spotify AB" ThisBuild / startYear := Some(2016) ThisBuild / licenses := Seq(License.Apache2) ThisBuild / sonatypeCredentialHost := Sonatype.sonatypeLegacy -ThisBuild / developers := List( - Developer( - id = "sinisa_lyh", - name = "Neville Li", - email = "neville.lyh@gmail.com", - url = url("https://twitter.com/sinisa_lyh") - ), - Developer( - id = "ravwojdyla", - name = "Rafal Wojdyla", - email = "ravwojdyla@gmail.com", - url = url("https://twitter.com/ravwojdyla") - ), - Developer( - id = "andrewsmartin", - name = "Andrew Martin", - email = "andrewsmartin.mg@gmail.com", - url = url("https://twitter.com/andrew_martin92") - ), - Developer( - id = "fallonfofallon", - name = "Fallon Chen", - email = "fallon@spotify.com", - url = url("https://twitter.com/fallonfofallon") - ), - Developer( - id = "regadas", - name = "Filipe Regadas", - email = "filiperegadas@gmail.com", - url = url("https://twitter.com/regadas") - ), - Developer( - id = "jto", - name = "Julien Tournay", - email = "julient@spotify.com", - url = url("https://twitter.com/skaalf") - ), - Developer( - id = "clairemcginty", - name = "Claire McGinty", - email = "clairem@spotify.com", - url = url("http://github.com/clairemcginty") - ), - Developer( - id = "syodage", - name = "Shameera Rathnayaka", - email = "shameerayodage@gmail.com", - url = url("http://github.com/syodage") - ), - Developer( - id = "kellen", - name = "Kellen Dye", - email = "dye.kellen@gmail.com", - url = url("http://github.com/kellen") - ), - Developer( - id = "farzad-sedghi", - name = "farzad sedghi", - email = "farzadsedghi2@gmail.com", - url = url("http://github.com/farzad-sedghi") - ) -) // scala versions val scala213 = "2.13.15" From 8412f4e6d2855fa3730123b383d412370f6fd308 Mon Sep 17 00:00:00 2001 From: kellen Date: Wed, 29 Jan 2025 19:36:52 -0500 Subject: [PATCH 5/8] Remove stale comments, make some previously-deprecated methods private (#5584) --- build.sbt | 8 ++------ .../scio/transforms/BaseAsyncBatchLookupDoFn.java | 11 ----------- .../com/spotify/scio/transforms/BaseAsyncDoFn.java | 8 -------- .../spotify/scio/transforms/BaseAsyncLookupDoFn.java | 11 ----------- .../com/spotify/scio/transforms/FileDownloadDoFn.java | 8 -------- .../src/main/scala/com/spotify/scio/ScioContext.scala | 1 - .../spotify/scio/coders/instances/ScalaCoders.scala | 1 - .../com/spotify/scio/testing/TestDataManager.scala | 1 - .../coders/instances/kryo/JodaSerializerTest.scala | 1 - .../com/spotify/scio/extra/sparkey/package.scala | 9 +++------ .../scala/com/spotify/scio/extra/PropertySpec.scala | 1 - .../spotify/scio/testing/SCollectionMatchers.scala | 2 -- 12 files changed, 5 insertions(+), 57 deletions(-) diff --git a/build.sbt b/build.sbt index f0dd1fe37d..bbae3033ec 100644 --- a/build.sbt +++ b/build.sbt @@ -842,10 +842,7 @@ lazy val `scio-test-core` = project "org.apache.beam" % "beam-runners-direct-java" % beamVersion % Runtime, // test "org.slf4j" % "slf4j-simple" % slf4jVersion % Test - ), - // only releases after 0.14.4 - tlMimaPreviousVersions := tlMimaPreviousVersions.value - .filter(v => VersionNumber(v).numbers.last >= 4) + ) ) lazy val `scio-test-google-cloud-platform` = project @@ -1299,8 +1296,7 @@ lazy val `scio-snowflake` = project "joda-time" % "joda-time" % jodaTimeVersion, "org.apache.beam" % "beam-sdks-java-core" % beamVersion, "org.apache.beam" % "beam-sdks-java-io-snowflake" % beamVersion - ), - tlMimaPreviousVersions := Set.empty // TODO: remove once released + ) ) val tensorFlowMetadataSourcesDir = diff --git a/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncBatchLookupDoFn.java b/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncBatchLookupDoFn.java index 9ac82678c2..9f0c6e5fc8 100644 --- a/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncBatchLookupDoFn.java +++ b/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncBatchLookupDoFn.java @@ -160,17 +160,6 @@ public void startBundle(StartBundleContext context) { semaphore.release(maxPendingRequests); } - // kept for binary compatibility. Must not be used - // TODO: remove in 0.15.0 - @Deprecated - public void processElement( - Input input, - Instant timestamp, - OutputReceiver> out, - BoundedWindow window) { - processElement(input, timestamp, window, null, out); - } - @ProcessElement public void processElement( @Element Input input, diff --git a/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncDoFn.java b/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncDoFn.java index d7a6c247f0..38c390ca45 100644 --- a/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncDoFn.java +++ b/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncDoFn.java @@ -73,14 +73,6 @@ public void finishBundle(FinishBundleContext context) { flush(r -> context.output(r.getValue(), r.getTimestamp(), r.getWindow())); } - // kept for binary compatibility. Must not be used - // TODO: remove in 0.15.0 - @Deprecated - public void processElement( - Input input, Instant timestamp, OutputReceiver out, BoundedWindow window) { - processElement(input, timestamp, window, null, out); - } - @ProcessElement public void processElement( @Element Input element, diff --git a/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncLookupDoFn.java b/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncLookupDoFn.java index 483d2a7304..007e3be00d 100644 --- a/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncLookupDoFn.java +++ b/scio-core/src/main/java/com/spotify/scio/transforms/BaseAsyncLookupDoFn.java @@ -161,17 +161,6 @@ public void startBundle(StartBundleContext context) { semaphore.release(maxPendingRequests); } - // kept for binary compatibility. Must not be used - // TODO: remove in 0.15.0 - @Deprecated - public void processElement( - Input input, - Instant timestamp, - OutputReceiver> out, - BoundedWindow window) { - processElement(input, timestamp, window, null, out); - } - @SuppressWarnings("unchecked") @ProcessElement public void processElement( diff --git a/scio-core/src/main/java/com/spotify/scio/transforms/FileDownloadDoFn.java b/scio-core/src/main/java/com/spotify/scio/transforms/FileDownloadDoFn.java index 94a899f819..a51d047a39 100644 --- a/scio-core/src/main/java/com/spotify/scio/transforms/FileDownloadDoFn.java +++ b/scio-core/src/main/java/com/spotify/scio/transforms/FileDownloadDoFn.java @@ -79,14 +79,6 @@ public void startBundle(StartBundleContext context) { this.batch.clear(); } - // kept for binary compatibility. Must not be used - // TODO: remove in 0.15.0 - @Deprecated - public void processElement( - URI element, Instant timestamp, OutputReceiver out, BoundedWindow window) { - processElement(element, timestamp, window, null, out); - } - @ProcessElement public void processElement( @DoFn.Element URI element, 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 ce3e169b6a..4b20b1e04d 100644 --- a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala +++ b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala @@ -485,7 +485,6 @@ class ScioContext private[scio] ( } private[scio] def prepare(): Unit = { - // TODO: make sure this works for other PipelineOptions RunnerContext.prepareOptions(options, artifacts) ScioContext.validateOptions(options) } diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/ScalaCoders.scala b/scio-core/src/main/scala/com/spotify/scio/coders/instances/ScalaCoders.scala index 84150bcada..db00e79bab 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/ScalaCoders.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/instances/ScalaCoders.scala @@ -449,7 +449,6 @@ trait ScalaCoders extends CoderGrammar with CoderDerivation { implicit def seqCoder[T: Coder]: Coder[Seq[T]] = transform(Coder[T])(bc => beam(new SeqCoder[T](bc))) - // TODO: proper chunking implementation implicit def iterableCoder[T: Coder]: Coder[Iterable[T]] = transform(Coder[T])(bc => beam(new IterableCoder[T](bc))) diff --git a/scio-core/src/main/scala/com/spotify/scio/testing/TestDataManager.scala b/scio-core/src/main/scala/com/spotify/scio/testing/TestDataManager.scala index b4504873a3..eb92cbae6d 100644 --- a/scio-core/src/main/scala/com/spotify/scio/testing/TestDataManager.scala +++ b/scio-core/src/main/scala/com/spotify/scio/testing/TestDataManager.scala @@ -87,7 +87,6 @@ private[scio] class TestOutput(val m: Map[String, SCollection[_] => Any]) { java.util.concurrent.ConcurrentHashMap.newKeySet[String]().asScala def apply[T](io: ScioIO[T]): SCollection[T] => Any = apply(io.testId) def apply[T](key: String): SCollection[T] => Any = { - // TODO: support Materialize outputs, maybe Materialized[T]? require( m.contains(key), s"Missing test output: $key, available: ${m.keys.mkString("[", ", ", "]")}" diff --git a/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JodaSerializerTest.scala b/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JodaSerializerTest.scala index 13525a27b7..acd6783f53 100644 --- a/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JodaSerializerTest.scala +++ b/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JodaSerializerTest.scala @@ -27,7 +27,6 @@ import scala.jdk.CollectionConverters._ import scala.util.Try class JodaSerializerTest extends AnyFlatSpec with Checkers { - // TODO: remove this once https://github.com/scalatest/scalatest/issues/1090 is addressed implicit override val generatorDrivenConfig: PropertyCheckConfiguration = PropertyCheckConfiguration(minSuccessful = 100) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/sparkey/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/sparkey/package.scala index 7396a74871..f14b27dcd5 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/sparkey/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/sparkey/package.scala @@ -218,7 +218,6 @@ package object sparkey extends SparkeyReaderInstances with SparkeyCoders { ClosedTap[Nothing](EmptyTap) } - // TODO for 0.15: make private, remove deprecation /** * Write the key-value pairs of this SCollection as a Sparkey file to a specific location. * @@ -233,8 +232,7 @@ package object sparkey extends SparkeyReaderInstances with SparkeyCoders { * @return * A singleton SCollection containing the [[SparkeyUri]] of the saved files. */ - @deprecated("Use saveAsSparkey instead", since = "0.14.0") - def asSparkey( + private[scio] def asSparkey( path: String = null, maxMemoryUsage: Long = -1, numShards: Short = SparkeyIO.DefaultNumShards, @@ -261,15 +259,14 @@ package object sparkey extends SparkeyReaderInstances with SparkeyCoders { ) } - // TODO for 0.15: make private, remove deprecation /** * Write the key-value pairs of this SCollection as a Sparkey file to a temporary location. * * @return * A singleton SCollection containing the [[SparkeyUri]] of the saved files. */ - @deprecated("Use saveAsSparkey instead", since = "0.14.0") - def asSparkey(implicit w: SparkeyWritable[K, V]): SCollection[SparkeyUri] = this.asSparkey() + private[scio] def asSparkey(implicit w: SparkeyWritable[K, V]): SCollection[SparkeyUri] = + this.asSparkey() /** * Convert this SCollection to a SideInput, mapping key-value pairs of each window to a diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/PropertySpec.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/PropertySpec.scala index c32e32e9cc..54ac6154a3 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/PropertySpec.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/PropertySpec.scala @@ -22,7 +22,6 @@ import org.scalatest.matchers.should.Matchers import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks trait PropertySpec extends AnyPropSpec with ScalaCheckDrivenPropertyChecks with Matchers { - // TODO: remove this once https://github.com/scalatest/scalatest/issues/1090 is addressed implicit override val generatorDrivenConfig: PropertyCheckConfiguration = PropertyCheckConfiguration(minSuccessful = 100) } diff --git a/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala b/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala index 637eaeb925..f6e062de72 100644 --- a/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala +++ b/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala @@ -443,8 +443,6 @@ trait SCollectionMatchers extends EqInstances { } } - // TODO: investigate why multi-map doesn't work - /** Assert that the SCollection in question satisfies the provided function. */ def satisfy[T: Coder: Eq]( predicate: Iterable[T] => Boolean From 5c959f12db971c862d0937243e82994cbb4a259e Mon Sep 17 00:00:00 2001 From: Kellen Dye Date: Wed, 29 Jan 2025 19:51:51 -0500 Subject: [PATCH 6/8] Fix deps --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index f0dd1fe37d..9229300ff1 100644 --- a/build.sbt +++ b/build.sbt @@ -1437,11 +1437,11 @@ lazy val `scio-examples` = project "com.softwaremill.magnolia1_2" %% "magnolia" % magnoliaVersion, "com.spotify" %% "magnolify-avro" % magnolifyVersion, "com.spotify" %% "magnolify-bigtable" % magnolifyVersion, + "com.spotify" %% "magnolify-bigquery" % magnolifyVersion, "com.spotify" %% "magnolify-datastore" % magnolifyVersion, "com.spotify" %% "magnolify-guava" % magnolifyVersion, "com.spotify" %% "magnolify-neo4j" % magnolifyVersion, "com.spotify" %% "magnolify-parquet" % magnolifyVersion, - "com.spotify" %% "magnolify-shared" % magnolifyVersion, "com.spotify" %% "magnolify-tensorflow" % magnolifyVersion, "com.twitter" %% "algebird-core" % algebirdVersion, "joda-time" % "joda-time" % jodaTimeVersion, From c23e862efaabaf38d479a598ebaff2cfce99a3c8 Mon Sep 17 00:00:00 2001 From: Kellen Dye Date: Mon, 10 Feb 2025 15:58:22 -0500 Subject: [PATCH 7/8] Update packages --- build.sbt | 17 ++--------------- .../main/scala/com/spotify/scio/avro/taps.scala | 2 +- .../spotify/scio/coders/AvroKryoRegistrar.scala | 2 +- .../coders/instances/kryo/AvroSerializer.scala | 2 +- .../scio/cassandra/DataTypeExternalizer.scala | 2 +- .../instances/kryo/JTraversableSerializer.scala | 2 +- .../instances/kryo/JTraversableSerializer.scala | 2 +- .../spotify/scio/coders/CoderDerivation.scala | 2 +- .../spotify/scio/coders/KryoAtomicCoder.scala | 6 +++--- .../instances/kryo/ByteStringSerializer.scala | 2 +- .../coders/instances/kryo/CoderSerializer.scala | 2 +- .../coders/instances/kryo/GrpcSerializer.scala | 2 +- .../coders/instances/kryo/JPathSerializer.scala | 2 +- .../coders/instances/kryo/KVSerializer.scala | 2 +- .../instances/kryo/ThrowableSerializer.scala | 2 +- .../scala/com/spotify/scio/schemas/Schema.scala | 2 +- .../scio/transforms/ParallelismDoFns.scala | 2 +- .../scio/transforms/WithResourceDoFns.scala | 2 +- .../syntax/SCollectionSafeSyntax.scala | 2 +- .../scio/util/FilenamePolicySupplier.scala | 2 +- .../scala/com/spotify/scio/util/Functions.scala | 2 +- .../scio/util/FunctionsWithSideInput.scala | 2 +- .../scio/util/FunctionsWithSideOutput.scala | 2 +- .../scio/util/FunctionsWithWindowedValue.scala | 2 +- .../com/spotify/scio/values/SCollection.scala | 2 +- .../scio/values/SCollectionWithSideInput.scala | 2 +- .../scio/coders/KryoAtomicCoderTest.scala | 2 +- .../kryo/ByteStringSerializerTest.scala | 2 +- .../kryo/JTraversableSerializerTest.scala | 2 +- .../com/spotify/scio/bigquery/BigQueryIO.scala | 2 +- .../scala/com/spotify/scio/bigquery/taps.scala | 2 +- .../spotify/scio/coders/GcpKryoRegistrar.scala | 2 +- .../coders/instances/kryo/GcpSerializer.scala | 2 +- .../spotify/scio/grpc/SCollectionSyntax.scala | 2 +- .../com/spotify/scio/jmh/CoderBenchmark.scala | 2 +- .../scio/parquet/avro/ParquetAvroIO.scala | 2 +- .../spotify/scio/parquet/read/ParquetRead.scala | 2 +- .../scio/testing/SCollectionMatchers.scala | 4 ++-- .../scio/testing/SCollectionMatchersTest.scala | 2 +- site/src/main/paradox/FAQ.md | 10 +++++----- site/src/main/paradox/internals/Kryo.md | 4 ++-- 41 files changed, 50 insertions(+), 63 deletions(-) diff --git a/build.sbt b/build.sbt index 3956919d59..00ffbe890d 100644 --- a/build.sbt +++ b/build.sbt @@ -77,12 +77,12 @@ val perfmarkVersion = "0.27.0" val algebirdVersion = "0.13.10" val annoy4sVersion = "0.10.0" val annoyVersion = "0.2.6" +val asmVersion = "4.16" val breezeVersion = "2.1.0" val caffeineVersion = "2.9.3" val cassandraDriverVersion = "3.11.5" val cassandraVersion = "3.11.17" val catsVersion = "2.13.0" -val chillVersion = "0.10.0" val circeVersion = "0.14.10" val commonsTextVersion = "1.10.0" val elasticsearch7Version = "7.17.21" @@ -753,11 +753,7 @@ lazy val `scio-core` = project "com.google.http-client" % "google-http-client-gson" % gcpBom.key.value, "com.google.protobuf" % "protobuf-java" % gcpBom.key.value, "com.softwaremill.magnolia1_2" %% "magnolia" % magnoliaVersion, - "com.twitter" % "chill-java" % chillVersion, - "com.twitter" % "chill-protobuf" % chillVersion, "com.twitter" %% "algebird-core" % algebirdVersion, - "com.twitter" %% "chill" % chillVersion, - "com.twitter" %% "chill-algebird" % chillVersion, "commons-io" % "commons-io" % commonsIoVersion, "io.grpc" % "grpc-api" % gcpBom.key.value, "joda-time" % "joda-time" % jodaTimeVersion, @@ -767,6 +763,7 @@ lazy val `scio-core` = project "org.apache.commons" % "commons-compress" % commonsCompressVersion, "org.apache.commons" % "commons-lang3" % commonsLang3Version, "org.apache.commons" % "commons-math3" % commonsMath3Version, + "org.apache.xbean" % "xbean-asm7-shaded" % asmVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.typelevel" %% "algebra" % algebraVersion, // provided @@ -782,7 +779,6 @@ lazy val `scio-core` = project "com.spotify.sparkey" % "sparkey" % sparkeyVersion % Test, "com.spotify" % "annoy" % annoyVersion % Test, "com.spotify" %% "magnolify-guava" % magnolifyVersion % Test, - "com.twitter" %% "chill" % chillVersion % Test, "commons-io" % "commons-io" % commonsIoVersion % Test, "junit" % "junit" % junitVersion % Test, "org.apache.beam" % "beam-runners-direct-java" % beamVersion % Test, @@ -828,7 +824,6 @@ lazy val `scio-test-core` = project "com.google.http-client" % "google-http-client-gson" % gcpBom.key.value, // TODO should we have this here ? "com.lihaoyi" %% "fansi" % fansiVersion, "com.lihaoyi" %% "pprint" % pprintVersion, - "com.twitter" %% "chill" % chillVersion, "commons-io" % "commons-io" % commonsIoVersion, "joda-time" % "joda-time" % jodaTimeVersion, "org.apache.avro" % "avro" % avroVersion, // TODO should we have this here ? @@ -926,8 +921,6 @@ lazy val `scio-avro` = project "com.spotify" %% "magnolify-avro" % magnolifyVersion, "com.spotify" %% "magnolify-protobuf" % magnolifyVersion, "com.spotify" %% "magnolify-shared" % magnolifyVersion, - "com.twitter" %% "chill" % chillVersion, - "com.twitter" % "chill-java" % chillVersion, "me.lyh" %% "protobuf-generic" % protobufGenericVersion, "org.apache.beam" % "beam-sdks-java-core" % beamVersion, "org.apache.beam" % "beam-sdks-java-extensions-avro" % beamVersion, @@ -999,8 +992,6 @@ lazy val `scio-google-cloud-platform` = project "com.spotify" %% "magnolify-bigtable" % magnolifyVersion, "com.spotify" %% "magnolify-datastore" % magnolifyVersion, "com.spotify" %% "magnolify-shared" % magnolifyVersion, - "com.twitter" %% "chill" % chillVersion, - "com.twitter" % "chill-java" % chillVersion, "commons-io" % "commons-io" % commonsIoVersion, "io.grpc" % "grpc-api" % gcpBom.key.value, "io.grpc" % "grpc-auth" % gcpBom.key.value, @@ -1042,8 +1033,6 @@ lazy val `scio-cassandra3` = project "com.esotericsoftware" % "kryo-shaded" % kryoVersion, "com.google.guava" % "guava" % guavaVersion, "com.google.protobuf" % "protobuf-java" % gcpBom.key.value, - "com.twitter" % "chill-java" % chillVersion, - "com.twitter" %% "chill" % chillVersion, "org.apache.cassandra" % "cassandra-all" % cassandraVersion, "org.apache.hadoop" % "hadoop-common" % hadoopVersion, "org.apache.hadoop" % "hadoop-mapreduce-client-core" % hadoopVersion, @@ -1177,7 +1166,6 @@ lazy val `scio-grpc` = project // compile "com.google.guava" % "failureaccess" % failureAccessVersion, "com.google.guava" % "guava" % guavaVersion, - "com.twitter" %% "chill" % chillVersion, "io.grpc" % "grpc-api" % gcpBom.key.value, "io.grpc" % "grpc-stub" % gcpBom.key.value, "org.apache.beam" % "beam-sdks-java-core" % beamVersion, @@ -1255,7 +1243,6 @@ lazy val `scio-parquet` = project "com.google.cloud.bigdataoss" % "util-hadoop" % s"hadoop2-$bigdataossVersion", "com.google.protobuf" % "protobuf-java" % gcpBom.key.value, "com.spotify" %% "magnolify-parquet" % magnolifyVersion, - "com.twitter" %% "chill" % chillVersion, "me.lyh" %% "parquet-avro" % parquetExtraVersion, "org.apache.avro" % "avro" % avroVersion, "org.apache.avro" % "avro-compiler" % avroVersion, 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 ad86792780..c5f4a7a4d2 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 @@ -25,7 +25,7 @@ import com.spotify.scio.coders.{AvroBytesUtil, Coder, CoderMaterializer} import com.spotify.scio.io.{Tap, Taps} import com.spotify.scio.util.ScioUtil import com.spotify.scio.values._ -import com.twitter.chill.Externalizer +import com.spotify.scio.vendor.chill.Externalizer import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.{SpecificData, SpecificRecord} diff --git a/scio-avro/src/main/scala/com/spotify/scio/coders/AvroKryoRegistrar.scala b/scio-avro/src/main/scala/com/spotify/scio/coders/AvroKryoRegistrar.scala index 51c092be3c..e18a8eda9c 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/coders/AvroKryoRegistrar.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/coders/AvroKryoRegistrar.scala @@ -17,7 +17,7 @@ package com.spotify.scio.coders import com.spotify.scio.coders.instances.kryo.{GenericAvroSerializer, SpecificAvroSerializer} -import com.twitter.chill._ +import com.spotify.scio.vendor.chill._ import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificRecord diff --git a/scio-avro/src/main/scala/com/spotify/scio/coders/instances/kryo/AvroSerializer.scala b/scio-avro/src/main/scala/com/spotify/scio/coders/instances/kryo/AvroSerializer.scala index af8b71179e..6fe6c71f17 100644 --- a/scio-avro/src/main/scala/com/spotify/scio/coders/instances/kryo/AvroSerializer.scala +++ b/scio-avro/src/main/scala/com/spotify/scio/coders/instances/kryo/AvroSerializer.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificRecord diff --git a/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/DataTypeExternalizer.scala b/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/DataTypeExternalizer.scala index 556c461899..d9144eaa85 100644 --- a/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/DataTypeExternalizer.scala +++ b/scio-cassandra/cassandra3/src/main/scala/com/spotify/scio/cassandra/DataTypeExternalizer.scala @@ -22,7 +22,7 @@ import java.util.{ArrayList => JArrayList, Collection => JCollection} import com.datastax.driver.core.DataType import com.google.common.collect.{ImmutableList, ImmutableSet} -import com.twitter.chill._ +import com.spotify.scio.vendor.chill._ import scala.jdk.CollectionConverters._ diff --git a/scio-core/src/main/scala-2.12/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala b/scio-core/src/main/scala-2.12/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala index da9a083290..cc984984e5 100644 --- a/scio-core/src/main/scala-2.12/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala +++ b/scio-core/src/main/scala-2.12/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, InputChunked, Output, OutputChunked} -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer import scala.jdk.CollectionConverters._ import scala.collection.generic.CanBuildFrom diff --git a/scio-core/src/main/scala-2.13/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala b/scio-core/src/main/scala-2.13/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala index d89bbc7281..fd842280f4 100644 --- a/scio-core/src/main/scala-2.13/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala +++ b/scio-core/src/main/scala-2.13/com/spotify/scio/coders/instances/kryo/JTraversableSerializer.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, InputChunked, Output, OutputChunked} -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer import scala.jdk.CollectionConverters._ import scala.collection.mutable diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/CoderDerivation.scala b/scio-core/src/main/scala/com/spotify/scio/coders/CoderDerivation.scala index 31904d6689..2cdb75b7a5 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/CoderDerivation.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/CoderDerivation.scala @@ -18,7 +18,7 @@ package com.spotify.scio.coders import com.spotify.scio.MagnoliaMacros -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import magnolia1._ import scala.reflect.ClassTag diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/KryoAtomicCoder.scala b/scio-core/src/main/scala/com/spotify/scio/coders/KryoAtomicCoder.scala index ae267aa48d..3181092339 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/KryoAtomicCoder.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/KryoAtomicCoder.scala @@ -27,9 +27,9 @@ import com.spotify.scio.options.ScioOptions import java.io.{EOFException, InputStream, OutputStream} import java.nio.file.Path import java.util.concurrent.atomic.AtomicInteger -import com.twitter.chill._ -import com.twitter.chill.algebird.AlgebirdRegistrar -import com.twitter.chill.protobuf.ProtobufSerializer +import com.spotify.scio.vendor.chill._ +import com.spotify.scio.vendor.chill.algebird.AlgebirdRegistrar +import com.spotify.scio.vendor.chill.protobuf.ProtobufSerializer import org.apache.beam.sdk.coders.Coder.NonDeterministicException import org.apache.beam.sdk.coders.{AtomicCoder, CoderException => BCoderException, InstantCoder} import org.apache.beam.sdk.options.{PipelineOptions, PipelineOptionsFactory} diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializer.scala b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializer.scala index 508deda61d..cd88ee1221 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializer.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializer.scala @@ -20,7 +20,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} import com.google.protobuf.ByteString -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer private[coders] class ByteStringSerializer extends KSerializer[ByteString] { override def read(kryo: Kryo, input: Input, tpe: Class[ByteString]): ByteString = { diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/CoderSerializer.scala b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/CoderSerializer.scala index 2a0278afe0..ea649b15ab 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/CoderSerializer.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/CoderSerializer.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer import org.apache.beam.sdk.coders.{Coder => BCoder} import org.apache.beam.sdk.util.CoderUtils diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/GrpcSerializer.scala b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/GrpcSerializer.scala index 70e6367398..49d6fa14e1 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/GrpcSerializer.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/GrpcSerializer.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer import io.grpc.Status private[coders] class StatusSerializer extends KSerializer[Status] { diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/JPathSerializer.scala b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/JPathSerializer.scala index 570d2a7798..155a4daa2e 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/JPathSerializer.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/JPathSerializer.scala @@ -21,7 +21,7 @@ import java.nio.file.{Path, Paths} import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer private[coders] class JPathSerializer extends KSerializer[Path] { override def read(kryo: Kryo, input: Input, tpe: Class[Path]): Path = diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/KVSerializer.scala b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/KVSerializer.scala index 3fb8e41e91..e4429283d0 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/KVSerializer.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/KVSerializer.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer import org.apache.beam.sdk.values.KV private[coders] class KVSerializer[K, V] extends KSerializer[KV[K, V]] { diff --git a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ThrowableSerializer.scala b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ThrowableSerializer.scala index 7e6e35ee7b..9f348a7a72 100644 --- a/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ThrowableSerializer.scala +++ b/scio-core/src/main/scala/com/spotify/scio/coders/instances/kryo/ThrowableSerializer.scala @@ -18,7 +18,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.KryoException import com.google.api.gax.rpc.{ApiException, ApiExceptionFactory, StatusCode} -import com.twitter.chill.{Input, KSerializer, Kryo, Output} +import com.spotify.scio.vendor.chill.{Input, KSerializer, Kryo, Output} import io.grpc.{Status, StatusException, StatusRuntimeException} import java.io.{InputStream, ObjectInputStream, ObjectOutputStream, OutputStream} 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 0a8f69cfc6..42a757ba45 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 @@ -30,7 +30,7 @@ import org.apache.beam.sdk.schemas.Schema.FieldType import org.apache.beam.sdk.schemas.{Schema => BSchema, SchemaProvider} import org.apache.beam.sdk.transforms.SerializableFunction import org.apache.beam.sdk.values.{Row, TypeDescriptor} -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag diff --git a/scio-core/src/main/scala/com/spotify/scio/transforms/ParallelismDoFns.scala b/scio-core/src/main/scala/com/spotify/scio/transforms/ParallelismDoFns.scala index 2e202c27a3..0b6f5271e8 100644 --- a/scio-core/src/main/scala/com/spotify/scio/transforms/ParallelismDoFns.scala +++ b/scio-core/src/main/scala/com/spotify/scio/transforms/ParallelismDoFns.scala @@ -17,7 +17,7 @@ package com.spotify.scio.transforms import com.spotify.scio.util.ParallelLimitedFn -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import org.apache.beam.sdk.transforms.DoFn import scala.collection.compat._ diff --git a/scio-core/src/main/scala/com/spotify/scio/transforms/WithResourceDoFns.scala b/scio-core/src/main/scala/com/spotify/scio/transforms/WithResourceDoFns.scala index 45dcdeead4..0ea4b50e4e 100644 --- a/scio-core/src/main/scala/com/spotify/scio/transforms/WithResourceDoFns.scala +++ b/scio-core/src/main/scala/com/spotify/scio/transforms/WithResourceDoFns.scala @@ -17,7 +17,7 @@ package com.spotify.scio.transforms import com.spotify.scio.transforms.DoFnWithResource.ResourceType -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import org.apache.beam.sdk.transforms.DoFn.{Element, OutputReceiver, ProcessElement} import scala.collection.compat._ diff --git a/scio-core/src/main/scala/com/spotify/scio/transforms/syntax/SCollectionSafeSyntax.scala b/scio-core/src/main/scala/com/spotify/scio/transforms/syntax/SCollectionSafeSyntax.scala index 591c6c4bc7..cf3de5e6ea 100644 --- a/scio-core/src/main/scala/com/spotify/scio/transforms/syntax/SCollectionSafeSyntax.scala +++ b/scio-core/src/main/scala/com/spotify/scio/transforms/syntax/SCollectionSafeSyntax.scala @@ -19,7 +19,7 @@ package com.spotify.scio.transforms.syntax import com.spotify.scio.values.SCollection import com.spotify.scio.coders.{Coder, CoderMaterializer} import com.spotify.scio.util.NamedDoFn -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import org.apache.beam.sdk.transforms.DoFn.{Element, MultiOutputReceiver, ProcessElement} import org.apache.beam.sdk.transforms.ParDo import org.apache.beam.sdk.values.{TupleTag, TupleTagList} 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 d66e1f3ec2..bf21895ac3 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 @@ -16,7 +16,7 @@ package com.spotify.scio.util -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import org.apache.beam.sdk.io.FileBasedSink import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions diff --git a/scio-core/src/main/scala/com/spotify/scio/util/Functions.scala b/scio-core/src/main/scala/com/spotify/scio/util/Functions.scala index b44b103f7b..10cdeedd61 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/Functions.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/Functions.scala @@ -22,7 +22,7 @@ import java.util.{ArrayList => JArrayList, List => JList} import com.spotify.scio.ScioContext import com.spotify.scio.options.ScioOptions import com.spotify.scio.coders.{Coder, CoderMaterializer} -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import com.twitter.algebird.{Monoid, Semigroup} import org.apache.beam.sdk.coders.{Coder => BCoder, CoderRegistry} import org.apache.beam.sdk.options.PipelineOptionsFactory diff --git a/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideInput.scala b/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideInput.scala index 358eba379e..0b63b4921e 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideInput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideInput.scala @@ -21,7 +21,7 @@ import com.spotify.scio.values.SideInputContext import org.apache.beam.sdk.transforms.DoFn import org.apache.beam.sdk.transforms.DoFn.ProcessElement import org.apache.beam.sdk.transforms.windowing.BoundedWindow -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import scala.collection.compat._ diff --git a/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideOutput.scala b/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideOutput.scala index c025dd1ab2..6aa1cde530 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideOutput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithSideOutput.scala @@ -20,7 +20,7 @@ package com.spotify.scio.util import org.apache.beam.sdk.transforms.DoFn import com.spotify.scio.values.SideOutputContext import org.apache.beam.sdk.transforms.DoFn.ProcessElement -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import scala.collection.compat._ diff --git a/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithWindowedValue.scala b/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithWindowedValue.scala index 9948cf9c78..c2d9971872 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithWindowedValue.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/FunctionsWithWindowedValue.scala @@ -21,7 +21,7 @@ import com.spotify.scio.values.WindowedValue import org.apache.beam.sdk.transforms.DoFn import org.apache.beam.sdk.transforms.DoFn.{Element, OutputReceiver, ProcessElement, Timestamp} import org.apache.beam.sdk.transforms.windowing.{BoundedWindow, PaneInfo} -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import org.joda.time.Instant import scala.collection.compat._ 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 07126b2686..9b8f8c3b90 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 @@ -53,7 +53,7 @@ import scala.collection.compat._ import scala.collection.immutable.TreeMap import scala.reflect.ClassTag import scala.util.Try -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import org.typelevel.scalaccompat.annotation.{nowarn, unused} /** Convenience functions for creating SCollections. */ diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala index da6ca3cb6d..1748ff7176 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala @@ -30,7 +30,7 @@ import org.apache.beam.sdk.values.{PCollection, TupleTag, TupleTagList} import scala.jdk.CollectionConverters._ import scala.collection.compat._ import scala.util.Try -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner /** * An enhanced SCollection that provides access to one or more [[SideInput]] s for some transforms. diff --git a/scio-core/src/test/scala/com/spotify/scio/coders/KryoAtomicCoderTest.scala b/scio-core/src/test/scala/com/spotify/scio/coders/KryoAtomicCoderTest.scala index 208415c595..67ae6f130a 100644 --- a/scio-core/src/test/scala/com/spotify/scio/coders/KryoAtomicCoderTest.scala +++ b/scio-core/src/test/scala/com/spotify/scio/coders/KryoAtomicCoderTest.scala @@ -24,7 +24,7 @@ import com.spotify.scio.avro.AvroUtils._ import com.spotify.scio.avro.TestRecord import com.spotify.scio.testing.CoderAssertions._ import com.spotify.scio.testing.PipelineSpec -import com.twitter.chill.{java => _, _} +import com.spotify.scio.vendor.chill.{java => _, _} import org.apache.avro.generic.GenericRecord import org.apache.beam.sdk.Pipeline.PipelineExecutionException import org.apache.beam.sdk.coders.{Coder => BCoder} diff --git a/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializerTest.scala b/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializerTest.scala index e1c9d49052..a66b6c4c7e 100644 --- a/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializerTest.scala +++ b/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/ByteStringSerializerTest.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.io.{Input, Output} import com.google.protobuf.ByteString -import com.twitter.chill.{Kryo, KryoSerializer} +import com.spotify.scio.vendor.chill.{Kryo, KryoSerializer} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers diff --git a/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JTraversableSerializerTest.scala b/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JTraversableSerializerTest.scala index 503be57f45..e1e2c9632b 100644 --- a/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JTraversableSerializerTest.scala +++ b/scio-core/src/test/scala/com/spotify/scio/coders/instances/kryo/JTraversableSerializerTest.scala @@ -18,7 +18,7 @@ package com.spotify.scio.coders.instances.kryo import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.{Kryo, KryoSerializer} +import com.spotify.scio.vendor.chill.{Kryo, KryoSerializer} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers 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 4ce3af2947..036f806897 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,7 +25,7 @@ import com.spotify.scio.coders._ import com.spotify.scio.io._ import com.spotify.scio.util.{FilenamePolicySupplier, Functions, ScioUtil} import com.spotify.scio.values.{SCollection, SideOutput, SideOutputCollections} -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import magnolify.bigquery.TableRowType import org.apache.avro.generic.GenericRecord import org.apache.beam.sdk.extensions.gcp.options.GcpOptions 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 63f9608fa3..a56b7c66ac 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 @@ -31,7 +31,7 @@ import scala.jdk.CollectionConverters._ import scala.concurrent.Future import scala.reflect.runtime.universe._ import com.spotify.scio.bigquery.BigQueryTypedTable.Format -import com.twitter.chill.Externalizer +import com.spotify.scio.vendor.chill.Externalizer /** Tap for BigQuery TableRow JSON files. */ final case class TableRowJsonTap(path: String, params: TableRowJsonIO.ReadParam) diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala index 094fd8ec3c..70e85e3fad 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala @@ -19,7 +19,7 @@ package com.spotify.scio.coders import com.google.cloud.bigtable.data.v2.models.MutateRowsException import com.spotify.scio.bigquery.TableRow import com.spotify.scio.coders.instances.kryo.{CoderSerializer, MutateRowsExceptionSerializer} -import com.twitter.chill._ +import com.spotify.scio.vendor.chill._ import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder @KryoRegistrar diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala index f4c4b92a7c..9e90ac7722 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala @@ -18,7 +18,7 @@ package com.spotify.scio.coders.instances.kryo import com.google.api.gax.rpc.{ApiException, StatusCode} import com.google.cloud.bigtable.data.v2.models.MutateRowsException -import com.twitter.chill._ +import com.spotify.scio.vendor.chill._ private[coders] class MutateRowsExceptionSerializer extends KSerializer[MutateRowsException] { override def write(kryo: Kryo, output: Output, e: MutateRowsException): Unit = { diff --git a/scio-grpc/src/main/scala/com/spotify/scio/grpc/SCollectionSyntax.scala b/scio-grpc/src/main/scala/com/spotify/scio/grpc/SCollectionSyntax.scala index 3752af62d5..57857cd440 100644 --- a/scio-grpc/src/main/scala/com/spotify/scio/grpc/SCollectionSyntax.scala +++ b/scio-grpc/src/main/scala/com/spotify/scio/grpc/SCollectionSyntax.scala @@ -24,7 +24,7 @@ import com.spotify.scio.transforms.JavaAsyncConverters._ import com.spotify.scio.util.Functions import com.spotify.scio.util.TupleFunctions.kvToTuple import com.spotify.scio.values.SCollection -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import io.grpc.Channel import io.grpc.stub.{AbstractFutureStub, AbstractStub, StreamObserver} import org.apache.commons.lang3.tuple.Pair diff --git a/scio-jmh/src/test/scala/com/spotify/scio/jmh/CoderBenchmark.scala b/scio-jmh/src/test/scala/com/spotify/scio/jmh/CoderBenchmark.scala index f963939756..09aabae10c 100644 --- a/scio-jmh/src/test/scala/com/spotify/scio/jmh/CoderBenchmark.scala +++ b/scio-jmh/src/test/scala/com/spotify/scio/jmh/CoderBenchmark.scala @@ -24,7 +24,7 @@ import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Kryo, Serializer} import com.spotify.scio.coders._ import com.spotify.scio.schemas._ -import com.twitter.chill.IKryoRegistrar +import com.spotify.scio.vendor.chill.IKryoRegistrar import org.apache.beam.sdk.coders.{ AtomicCoder, ByteArrayCoder, 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 2f31724918..c97aa18c47 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 @@ -26,7 +26,7 @@ import com.spotify.scio.parquet.{GcsConnectorUtil, ParquetConfiguration} import com.spotify.scio.testing.TestDataManager import com.spotify.scio.util.{FilenamePolicySupplier, Functions, ScioUtil} import com.spotify.scio.values.SCollection -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import org.apache.avro.Schema import org.apache.avro.reflect.ReflectData import org.apache.avro.specific.SpecificRecord diff --git a/scio-parquet/src/main/scala/com/spotify/scio/parquet/read/ParquetRead.scala b/scio-parquet/src/main/scala/com/spotify/scio/parquet/read/ParquetRead.scala index 947b832321..9a090f34f2 100644 --- a/scio-parquet/src/main/scala/com/spotify/scio/parquet/read/ParquetRead.scala +++ b/scio-parquet/src/main/scala/com/spotify/scio/parquet/read/ParquetRead.scala @@ -18,7 +18,7 @@ package com.spotify.scio.parquet.read import com.spotify.scio.parquet.ParquetConfiguration import com.spotify.scio.util.{Functions, ScioUtil} -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import magnolify.parquet.ParquetType import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord diff --git a/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala b/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala index f6e062de72..ef53f84ff5 100644 --- a/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala +++ b/scio-test/core/src/main/scala/com/spotify/scio/testing/SCollectionMatchers.scala @@ -21,7 +21,7 @@ import java.lang.{Iterable => JIterable} import java.util.{Map => JMap} import com.spotify.scio.coders.Coder import com.spotify.scio.values.SCollection -import com.twitter.chill.Externalizer +import com.spotify.scio.vendor.chill.Externalizer import org.apache.beam.sdk.testing.PAssert import org.apache.beam.sdk.testing.PAssert.{IterableAssert, SingletonAssert} import org.apache.beam.sdk.transforms.SerializableFunction @@ -33,7 +33,7 @@ import org.hamcrest.Matchers import org.hamcrest.MatcherAssert.assertThat import scala.jdk.CollectionConverters._ -import com.twitter.chill.ClosureCleaner +import com.spotify.scio.vendor.chill.ClosureCleaner import cats.kernel.Eq import org.apache.beam.sdk.testing.SerializableMatchers import com.spotify.scio.coders.CoderMaterializer diff --git a/scio-test/core/src/test/scala/com/spotify/scio/testing/SCollectionMatchersTest.scala b/scio-test/core/src/test/scala/com/spotify/scio/testing/SCollectionMatchersTest.scala index 9c7c10c8be..2e3006c076 100644 --- a/scio-test/core/src/test/scala/com/spotify/scio/testing/SCollectionMatchersTest.scala +++ b/scio-test/core/src/test/scala/com/spotify/scio/testing/SCollectionMatchersTest.scala @@ -35,7 +35,7 @@ import java.io.ObjectInputStream import java.io.IOException import java.io.NotSerializableException import cats.kernel.Eq -import com.twitter.chill.Externalizer +import com.spotify.scio.vendor.chill.Externalizer import com.esotericsoftware.kryo.KryoSerializable import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} diff --git a/site/src/main/paradox/FAQ.md b/site/src/main/paradox/FAQ.md index 740e1f43d4..4ec3c2aac5 100644 --- a/site/src/main/paradox/FAQ.md +++ b/site/src/main/paradox/FAQ.md @@ -409,7 +409,7 @@ Define a registrar class that extends `IKryoRegistrar` and annotate it with `@Kr ```scala mdoc:reset:invisible trait UserRecord trait AccountRecord -import com.twitter.chill.KSerializer +import com.spotify.scio.vendor.chill.KSerializer import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} @@ -424,10 +424,10 @@ class AccountRecordSerializer extends KSerializer[AccountRecord] { ``` ```scala mdoc:silent -import com.twitter.chill._ +import com.spotify.scio.vendor.chill._ import com.esotericsoftware.kryo.Kryo import com.spotify.scio.coders.KryoRegistrar -import com.twitter.chill.IKryoRegistrar +import com.spotify.scio.vendor.chill.IKryoRegistrar @KryoRegistrar class MyKryoRegistrar extends IKryoRegistrar { @@ -449,10 +449,10 @@ trait MyRecord2 ``` ```scala mdoc:silent -import com.twitter.chill._ +import com.spotify.scio.vendor.chill._ import com.esotericsoftware.kryo.Kryo import com.spotify.scio.coders.KryoRegistrar -import com.twitter.chill.IKryoRegistrar +import com.spotify.scio.vendor.chill.IKryoRegistrar @KryoRegistrar class MyKryoRegistrar extends IKryoRegistrar { diff --git a/site/src/main/paradox/internals/Kryo.md b/site/src/main/paradox/internals/Kryo.md index 5cfd4a3a7d..895fc8e2d3 100644 --- a/site/src/main/paradox/internals/Kryo.md +++ b/site/src/main/paradox/internals/Kryo.md @@ -9,7 +9,7 @@ Add the following class. You can rename it, but its name has to end in `KryoRegi ```scala import com.spotify.scio.coders.KryoRegistrar -import com.twitter.chill._ +import com.spotify.scio.vendor.chill._ @KryoRegistrar class MyKryoRegistrar extends IKryoRegistrar { @@ -46,7 +46,7 @@ You can add the following class to your test folder; it will enforce registratio ```scala import com.esotericsoftware.kryo.Kryo import com.spotify.scio.coders.KryoRegistrar -import com.twitter.chill.IKryoRegistrar +import com.spotify.scio.vendor.chill.IKryoRegistrar /** Makes sure we don't forget to register encoders, enabled only in tests not to crash production. */ @KryoRegistrar From 448d0acce17b3e48fd797d757e5328c6a045a596 Mon Sep 17 00:00:00 2001 From: Kellen Dye Date: Mon, 10 Feb 2025 16:02:37 -0500 Subject: [PATCH 8/8] Vendor chill --- .../com/spotify/scio/vendor/chill/Base64.java | 2603 +++++++++++++++++ .../scio/vendor/chill/ClassRegistrar.java | 53 + .../scio/vendor/chill/IKryoRegistrar.java | 29 + .../scio/vendor/chill/KryoInstantiator.java | 112 + .../spotify/scio/vendor/chill/KryoPool.java | 135 + .../chill/ReflectingDefaultRegistrar.java | 62 + .../vendor/chill/ReflectingRegistrar.java | 75 + .../scio/vendor/chill/ResourcePool.java | 51 + .../spotify/scio/vendor/chill/SerDeState.java | 97 + .../vendor/chill/SingleDefaultRegistrar.java | 36 + .../scio/vendor/chill/SingleRegistrar.java | 35 + .../scio/vendor/chill/config/Config.java | 64 + .../chill/config/ConfigurationException.java | 31 + .../chill/config/ConfiguredInstantiator.java | 198 ++ .../vendor/chill/config/JavaMapConfig.java | 49 + .../chill/config/ReflectingInstantiator.java | 302 ++ .../chill/java/ArraysAsListSerializer.java | 113 + .../vendor/chill/java/BitSetSerializer.java | 117 + .../java/InetSocketAddressSerializer.java | 45 + .../vendor/chill/java/IterableRegistrar.java | 54 + .../java/IterableRegistrarSerializer.java | 44 + .../chill/java/Java8ClosureRegistrar.java | 36 + .../vendor/chill/java/LocaleSerializer.java | 32 + .../vendor/chill/java/PackageRegistrar.java | 45 + .../chill/java/PriorityQueueSerializer.java | 79 + .../vendor/chill/java/RegexSerializer.java | 42 + .../java/SimpleDateFormatSerializer.java | 32 + .../vendor/chill/java/SqlDateSerializer.java | 42 + .../vendor/chill/java/SqlTimeSerializer.java | 42 + .../chill/java/TimestampSerializer.java | 45 + .../scio/vendor/chill/java/URISerializer.java | 42 + .../vendor/chill/java/UUIDSerializer.java | 43 + .../UnmodifiableCollectionSerializer.java | 54 + .../UnmodifiableJavaCollectionSerializer.java | 76 + .../java/UnmodifiableListSerializer.java | 57 + .../chill/java/UnmodifiableMapSerializer.java | 53 + .../chill/java/UnmodifiableSetSerializer.java | 55 + .../java/UnmodifiableSortedMapSerializer.java | 54 + .../java/UnmodifiableSortedSetSerializer.java | 54 + .../chill/protobuf/ProtobufSerializer.java | 101 + .../chill/AllScalaRegistrarCompat.scala | 32 + .../chill/ClassManifestSerializer.scala | 27 + .../scio/vendor/chill/RichKryoCompat.scala | 48 + .../scio/vendor/chill/Traversable.scala | 49 + .../vendor/chill/WrappedArraySerializer.scala | 41 + .../chill/AllScalaRegistrarCompat.scala | 59 + .../scio/vendor/chill/RichKryoCompat.scala | 48 + .../scio/vendor/chill/Traversable.scala | 49 + .../vendor/chill/WrappedArraySerializer.scala | 40 + .../vendor/chill/BigDecimalSerializer.scala | 30 + .../scio/vendor/chill/BitSetSerializer.scala | 53 + .../vendor/chill/ClassTagSerializer.scala | 29 + .../vendor/chill/CleaningSerializer.scala | 27 + .../scio/vendor/chill/ClosureCleaner.scala | 368 +++ .../scio/vendor/chill/EitherSerializer.scala | 33 + .../vendor/chill/EnumerationSerializer.scala | 51 + .../scio/vendor/chill/Externalizer.scala | 210 ++ .../chill/JavaIterableWrapperSerializer.scala | 63 + .../spotify/scio/vendor/chill/KryoBase.scala | 188 ++ .../scio/vendor/chill/KryoSerializer.scala | 36 + .../vendor/chill/ManifestSerializer.scala | 73 + .../scio/vendor/chill/MeatLocker.scala | 41 + .../scio/vendor/chill/ObjectSerializer.scala | 44 + .../scio/vendor/chill/RegexSerializer.scala | 27 + .../spotify/scio/vendor/chill/RichKryo.scala | 91 + .../vendor/chill/ScalaKryoInstantiator.scala | 302 ++ .../vendor/chill/SingletonSerializer.scala | 25 + .../scio/vendor/chill/SomeSerializer.scala | 25 + .../vendor/chill/SortedMapSerializer.scala | 54 + .../vendor/chill/SortedSetSerializer.scala | 52 + .../scio/vendor/chill/StreamSerializer.scala | 25 + .../scio/vendor/chill/TupleSerializers.scala | 1331 +++++++++ .../chill/VolatileByteRefSerializer.scala | 27 + .../chill/algebird/AlgebirdRegistrar.scala | 62 + .../chill/algebird/AlgebirdSerializers.scala | 108 + .../ReflectingInstantiatorBuilder.scala | 56 + .../chill/config/ScalaAnyRefMapConfig.scala | 42 + .../vendor/chill/config/ScalaMapConfig.scala | 37 + .../spotify/scio/vendor/chill/package.scala | 46 + .../vendor/chill/java/TestCollections.java | 91 + .../scio/vendor/chill/java/TestLists.java | 64 + .../src/test/protobuf/test_messages.proto | 16 + .../chill/RegistrationIdsSpecData.scala | 174 ++ .../vendor/chill/SerializedExamplesData.scala | 225 ++ .../chill/RegistrationIdsSpecData.scala | 177 ++ .../vendor/chill/SerializedExamplesData.scala | 241 ++ .../scio/vendor/chill/BaseProperties.scala | 71 + .../vendor/chill/ClosureCleanerSpec.scala | 233 ++ .../scio/vendor/chill/ExternalizerSpec.scala | 85 + .../vendor/chill/FunctionSerialization.scala | 62 + .../spotify/scio/vendor/chill/KryoSpec.scala | 363 +++ .../vendor/chill/RegistrationIdsSpec.scala | 83 + ...SerializedExamplesOfStandardDataSpec.scala | 173 ++ .../chill/StandardDataRegistrationsSpec.scala | 214 ++ .../algebird/AlgebirdSerializersSpec.scala | 94 + .../config/ConfiguredInstantiatorTest.scala | 64 + .../config/ReflectingInstantiatorTest.scala | 121 + .../vendor/chill/java/ArraysAsListTest.scala | 71 + .../scio/vendor/chill/java/BitSetTest.scala | 122 + .../scio/vendor/chill/java/LocaleTest.scala | 46 + .../chill/java/OldBitSetSerializer.java | 56 + .../vendor/chill/java/PriorityQueueTest.scala | 74 + .../vendor/chill/java/SerDeStateTest.scala | 40 + .../java/UnmodifiableCollectionsTest.scala | 77 + .../vendor/chill/protobuf/ProtobufTest.scala | 71 + 105 files changed, 12346 insertions(+) create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/Base64.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/ClassRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/IKryoRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoInstantiator.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoPool.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingDefaultRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/ResourcePool.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/SerDeState.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleDefaultRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/config/Config.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfigurationException.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfiguredInstantiator.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/config/JavaMapConfig.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ReflectingInstantiator.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/ArraysAsListSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/BitSetSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/InetSocketAddressSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrarSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/Java8ClosureRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/LocaleSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PackageRegistrar.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PriorityQueueSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/RegexSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SimpleDateFormatSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlDateSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlTimeSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/TimestampSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/URISerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UUIDSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UnmodifiableCollectionSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UnmodifiableJavaCollectionSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UnmodifiableListSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UnmodifiableMapSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UnmodifiableSetSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UnmodifiableSortedMapSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/java/UnmodifiableSortedSetSerializer.java create mode 100644 scio-core/src/main/java/com/spotify/scio/vendor/chill/protobuf/ProtobufSerializer.java create mode 100644 scio-core/src/main/scala-2.12/com/spotify/scio/vendor/chill/AllScalaRegistrarCompat.scala create mode 100644 scio-core/src/main/scala-2.12/com/spotify/scio/vendor/chill/ClassManifestSerializer.scala create mode 100644 scio-core/src/main/scala-2.12/com/spotify/scio/vendor/chill/RichKryoCompat.scala create mode 100644 scio-core/src/main/scala-2.12/com/spotify/scio/vendor/chill/Traversable.scala create mode 100644 scio-core/src/main/scala-2.12/com/spotify/scio/vendor/chill/WrappedArraySerializer.scala create mode 100644 scio-core/src/main/scala-2.13/com/spotify/scio/vendor/chill/AllScalaRegistrarCompat.scala create mode 100644 scio-core/src/main/scala-2.13/com/spotify/scio/vendor/chill/RichKryoCompat.scala create mode 100644 scio-core/src/main/scala-2.13/com/spotify/scio/vendor/chill/Traversable.scala create mode 100644 scio-core/src/main/scala-2.13/com/spotify/scio/vendor/chill/WrappedArraySerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/BigDecimalSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/BitSetSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/ClassTagSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/CleaningSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/ClosureCleaner.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/EitherSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/EnumerationSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/Externalizer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/JavaIterableWrapperSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/KryoBase.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/KryoSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/ManifestSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/MeatLocker.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/ObjectSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/RegexSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/RichKryo.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/ScalaKryoInstantiator.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/SingletonSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/SomeSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/SortedMapSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/SortedSetSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/StreamSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/TupleSerializers.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/VolatileByteRefSerializer.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/algebird/AlgebirdRegistrar.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/algebird/AlgebirdSerializers.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/config/ReflectingInstantiatorBuilder.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/config/ScalaAnyRefMapConfig.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/config/ScalaMapConfig.scala create mode 100644 scio-core/src/main/scala/com/spotify/scio/vendor/chill/package.scala create mode 100644 scio-core/src/test/java/com/spotify/scio/vendor/chill/java/TestCollections.java create mode 100644 scio-core/src/test/java/com/spotify/scio/vendor/chill/java/TestLists.java create mode 100644 scio-core/src/test/protobuf/test_messages.proto create mode 100644 scio-core/src/test/scala-2.12/com/spotify/scio/vendor/chill/RegistrationIdsSpecData.scala create mode 100644 scio-core/src/test/scala-2.12/com/spotify/scio/vendor/chill/SerializedExamplesData.scala create mode 100644 scio-core/src/test/scala-2.13/com/spotify/scio/vendor/chill/RegistrationIdsSpecData.scala create mode 100644 scio-core/src/test/scala-2.13/com/spotify/scio/vendor/chill/SerializedExamplesData.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/BaseProperties.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/ClosureCleanerSpec.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/ExternalizerSpec.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/FunctionSerialization.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/KryoSpec.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/RegistrationIdsSpec.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/SerializedExamplesOfStandardDataSpec.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/StandardDataRegistrationsSpec.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/algebird/AlgebirdSerializersSpec.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/config/ConfiguredInstantiatorTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/config/ReflectingInstantiatorTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/java/ArraysAsListTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/java/BitSetTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/java/LocaleTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/java/OldBitSetSerializer.java create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/java/PriorityQueueTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/java/SerDeStateTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/java/UnmodifiableCollectionsTest.scala create mode 100644 scio-core/src/test/scala/com/spotify/scio/vendor/chill/protobuf/ProtobufTest.scala diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/Base64.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/Base64.java new file mode 100644 index 0000000000..067c738f70 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/Base64.java @@ -0,0 +1,2603 @@ +/* +Copyright 2012 Twitter, Inc. + +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.vendor.chill; +/** + * Encodes and decodes to and from Base64 notation. + * + *

Homepage: http://iharder.net/base64. + * + *

Example: String encoded = Base64.encode( myByteArray );
+ * byte[] myByteArray = Base64.decode( encoded ); + * + *

The options parameter, which appears in a few places, is used to pass several + * pieces of information to the encoder. In the "higher level" methods such as encodeBytes( bytes, + * options ) the options parameter can be used to indicate such things as first gzipping the bytes + * before encoding them, not inserting linefeeds, and encoding using the URL-safe and Ordered + * dialects. + * + *

Note, according to RFC3548, Section 2.1, + * implementations should not add line feeds unless explicitly told to do so. I've got Base64 set to + * this behavior now, although earlier versions broke lines by default. + * + *

The constants defined in Base64 can be OR-ed together to combine options, so you might make a + * call like this: + * String encoded = Base64.encodeBytes( mybytes, Base64.GZIP | Base64.DO_BREAK_LINES ); + * + *

to compress the data before encoding it and then making the output have newline characters. + * + *

Also... String encoded = Base64.encodeBytes( crazyString.getBytes() ); + * + *

Change Log: + * + *

    + *
  • v2.3.7 - Fixed subtle bug when base 64 input stream contained the value 01111111, which is + * an invalid base 64 character but should not throw an ArrayIndexOutOfBoundsException either. + * Led to discovery of mishandling (or potential for better handling) of other bad input + * characters. You should now get an IOException if you try decoding something that has bad + * characters in it. + *
  • v2.3.6 - Fixed bug when breaking lines and the final byte of the encoded string ended in + * the last column; the buffer was not properly shrunk and contained an extra (null) byte that + * made it into the string. + *
  • v2.3.5 - Fixed bug in {@link #encodeFromFile} where estimated buffer size was wrong for + * files of size 31, 34, and 37 bytes. + *
  • v2.3.4 - Fixed bug when working with gzipped streams whereby flushing the + * Base64.OutputStream closed the Base64 encoding (by padding with equals signs) too soon. + * Also added an option to suppress the automatic decoding of gzipped streams. Also added + * experimental support for specifying a class loader when using the {@link + * #decodeToObject(java.lang.String, int, java.lang.ClassLoader)} method. + *
  • v2.3.3 - Changed default char encoding to US-ASCII which reduces the internal Java + * footprint with its CharEncoders and so forth. Fixed some javadocs that were inconsistent. + * Removed imports and specified things like java.io.IOException explicitly inline. + *
  • v2.3.2 - Reduced memory footprint! Finally refined the "guessing" of how big the final + * encoded data will be so that the code doesn't have to create two output arrays: an + * oversized initial one and then a final, exact-sized one. Big win when using the {@link + * #encodeBytesToBytes(byte[])} family of methods (and not using the gzip options which uses a + * different mechanism with streams and stuff). + *
  • v2.3.1 - Added {@link #encodeBytesToBytes(byte[], int, int, int)} and some similar helper + * methods to be more efficient with memory by not returning a String but just a byte array. + *
  • v2.3 - This is not a drop-in replacement! This is two years of comments + * and bug fixes queued up and finally executed. Thanks to everyone who sent me stuff, and I'm + * sorry I wasn't able to distribute your fixes to everyone else. Much bad coding was cleaned + * up including throwing exceptions where necessary instead of returning null values or + * something similar. Here are some changes that may affect you: + *
      + *
    • Does not break lines, by default. This is to keep in compliance with RFC3548. + *
    • Throws exceptions instead of returning null values. Because some operations + * (especially those that may permit the GZIP option) use IO streams, there is a + * possiblity of an java.io.IOException being thrown. After some discussion and thought, + * I've changed the behavior of the methods to throw java.io.IOExceptions rather than + * return null if ever there's an error. I think this is more appropriate, though it + * will require some changes to your code. Sorry, it should have been done this way to + * begin with. + *
    • Removed all references to System.out, System.err, and the like. Shame on me. + * All I can say is sorry they were ever there. + *
    • Throws NullPointerExceptions and IllegalArgumentExceptions as needed such as + * when passed arrays are null or offsets are invalid. + *
    • Cleaned up as much javadoc as I could to avoid any javadoc warnings. This was + * especially annoying before for people who were thorough in their own projects and + * then had gobs of javadoc warnings on this file. + *
    + *
  • v2.2.1 - Fixed bug using URL_SAFE and ORDERED encodings. Fixed bug when using very small + * files (~< 40 bytes). + *
  • v2.2 - Added some helper methods for encoding/decoding directly from one file to the next. + * Also added a main() method to support command line encoding/decoding from one file to the + * next. Also added these Base64 dialects: + *
      + *
    1. The default is RFC3548 format. + *
    2. Calling Base64.setFormat(Base64.BASE64_FORMAT.URLSAFE_FORMAT) generates URL and file + * name friendly format as described in Section 4 of RFC3548. + * http://www.faqs.org/rfcs/rfc3548.html + *
    3. Calling Base64.setFormat(Base64.BASE64_FORMAT.ORDERED_FORMAT) generates URL and file + * name friendly format that preserves lexical ordering as described in + * http://www.faqs.org/qa/rfcc-1940.html + *
    + * Special thanks to Jim Kellerman at http://www.powerset.com/ for contributing the new + * Base64 dialects. + *
  • v2.1 - Cleaned up javadoc comments and unused variables and methods. Added some convenience + * methods for reading and writing to and from files. + *
  • v2.0.2 - Now specifies UTF-8 encoding in places where the code fails on systems with other + * encodings (like EBCDIC). + *
  • v2.0.1 - Fixed an error when decoding a single byte, that is, when the encoded data was a + * single byte. + *
  • v2.0 - I got rid of methods that used booleans to set options. Now everything is more + * consolidated and cleaner. The code now detects when data that's being decoded is + * gzip-compressed and will decompress it automatically. Generally things are cleaner. You'll + * probably have to change some method calls that you were making to support the new options + * format (ints that you "OR" together). + *
  • v1.5.1 - Fixed bug when decompressing and decoding to a byte[] using + * decode( String s, boolean gzipCompressed ). Added the ability to "suspend" encoding + * in the Output Stream so you can turn on and off the encoding if you need to embed base64 + * data in an otherwise "normal" stream (like an XML file). + *
  • v1.5 - Output stream pases on flush() command but doesn't do anything itself. This helps + * when using GZIP streams. Added the ability to GZip-compress objects before encoding them. + *
  • v1.4 - Added helper methods to read/write files. + *
  • v1.3.6 - Fixed OutputStream.flush() so that 'position' is reset. + *
  • v1.3.5 - Added flag to turn on and off line breaks. Fixed bug in input stream where last + * buffer being read, if not completely full, was not returned. + *
  • v1.3.4 - Fixed when "improperly padded stream" error was thrown at the wrong time. + *
  • v1.3.3 - Fixed I/O streams which were totally messed up. + *
+ * + *

I am placing this code in the Public Domain. Do with it as you will. This software comes with + * no guarantees or warranties but with plenty of well-wishing instead! Please visit http://iharder.net/base64 periodically to check for updates + * or to contribute improvements. + * + * @author Robert Harder + * @author rob@iharder.net + * @version 2.3.7 + */ +public class Base64 { + + /* ******** P U B L I C F I E L D S ******** */ + + /** No options specified. Value is zero. */ + public static final int NO_OPTIONS = 0; + + /** Specify encoding in first bit. Value is one. */ + public static final int ENCODE = 1; + + /** Specify decoding in first bit. Value is zero. */ + public static final int DECODE = 0; + + /** Specify that data should be gzip-compressed in second bit. Value is two. */ + public static final int GZIP = 2; + + /** Specify that gzipped data should not be automatically gunzipped. */ + public static final int DONT_GUNZIP = 4; + + /** Do break lines when encoding. Value is 8. */ + public static final int DO_BREAK_LINES = 8; + + /** + * Encode using Base64-like encoding that is URL- and Filename-safe as described in Section 4 of + * RFC3548: http://www.faqs.org/rfcs/rfc3548.html. It is + * important to note that data encoded this way is not officially valid Base64, or at the + * very least should not be called Base64 without also specifying that is was encoded using the + * URL- and Filename-safe dialect. + */ + public static final int URL_SAFE = 16; + + /** + * Encode using the special "ordered" dialect of Base64 described here: http://www.faqs.org/qa/rfcc-1940.html. + */ + public static final int ORDERED = 32; + + /* ******** P R I V A T E F I E L D S ******** */ + + /** Maximum line length (76) of Base64 output. */ + private static final int MAX_LINE_LENGTH = 76; + + /** The equals sign (=) as a byte. */ + private static final byte EQUALS_SIGN = (byte) '='; + + /** The new line character (\n) as a byte. */ + private static final byte NEW_LINE = (byte) '\n'; + + /** Preferred encoding. */ + private static final String PREFERRED_ENCODING = "US-ASCII"; + + private static final byte WHITE_SPACE_ENC = -5; // Indicates white space in encoding + private static final byte EQUALS_SIGN_ENC = -1; // Indicates equals sign in encoding + + /* ******** S T A N D A R D B A S E 6 4 A L P H A B E T ******** */ + + /** The 64 valid Base64 values. */ + /* Host platform me be something funny like EBCDIC, so we hardcode these values. */ + private static final byte[] _STANDARD_ALPHABET = { + (byte) 'A', (byte) 'B', (byte) 'C', (byte) 'D', (byte) 'E', (byte) 'F', (byte) 'G', (byte) 'H', + (byte) 'I', (byte) 'J', (byte) 'K', (byte) 'L', (byte) 'M', (byte) 'N', (byte) 'O', (byte) 'P', + (byte) 'Q', (byte) 'R', (byte) 'S', (byte) 'T', (byte) 'U', (byte) 'V', (byte) 'W', (byte) 'X', + (byte) 'Y', (byte) 'Z', (byte) 'a', (byte) 'b', (byte) 'c', (byte) 'd', (byte) 'e', (byte) 'f', + (byte) 'g', (byte) 'h', (byte) 'i', (byte) 'j', (byte) 'k', (byte) 'l', (byte) 'm', (byte) 'n', + (byte) 'o', (byte) 'p', (byte) 'q', (byte) 'r', (byte) 's', (byte) 't', (byte) 'u', (byte) 'v', + (byte) 'w', (byte) 'x', (byte) 'y', (byte) 'z', (byte) '0', (byte) '1', (byte) '2', (byte) '3', + (byte) '4', (byte) '5', (byte) '6', (byte) '7', (byte) '8', (byte) '9', (byte) '+', (byte) '/' + }; + + /** + * Translates a Base64 value to either its 6-bit reconstruction value or a negative number + * indicating some other meaning. + */ + private static final byte[] _STANDARD_DECODABET = { + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 0 - 8 + -5, + -5, // Whitespace: Tab and Linefeed + -9, + -9, // Decimal 11 - 12 + -5, // Whitespace: Carriage Return + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 14 - 26 + -9, + -9, + -9, + -9, + -9, // Decimal 27 - 31 + -5, // Whitespace: Space + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 33 - 42 + 62, // Plus sign at decimal 43 + -9, + -9, + -9, // Decimal 44 - 46 + 63, // Slash at decimal 47 + 52, + 53, + 54, + 55, + 56, + 57, + 58, + 59, + 60, + 61, // Numbers zero through nine + -9, + -9, + -9, // Decimal 58 - 60 + -1, // Equals sign at decimal 61 + -9, + -9, + -9, // Decimal 62 - 64 + 0, + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, // Letters 'A' through 'N' + 14, + 15, + 16, + 17, + 18, + 19, + 20, + 21, + 22, + 23, + 24, + 25, // Letters 'O' through 'Z' + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 91 - 96 + 26, + 27, + 28, + 29, + 30, + 31, + 32, + 33, + 34, + 35, + 36, + 37, + 38, // Letters 'a' through 'm' + 39, + 40, + 41, + 42, + 43, + 44, + 45, + 46, + 47, + 48, + 49, + 50, + 51, // Letters 'n' through 'z' + -9, + -9, + -9, + -9, + -9 // Decimal 123 - 127 + , + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 128 - 139 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 140 - 152 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 153 - 165 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 166 - 178 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 179 - 191 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 192 - 204 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 205 - 217 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 218 - 230 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 231 - 243 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9 // Decimal 244 - 255 + }; + + /* ******** U R L S A F E B A S E 6 4 A L P H A B E T ******** */ + + /** + * Used in the URL- and Filename-safe dialect described in Section 4 of RFC3548: http://www.faqs.org/rfcs/rfc3548.html. Notice + * that the last two bytes become "hyphen" and "underscore" instead of "plus" and "slash." + */ + private static final byte[] _URL_SAFE_ALPHABET = { + (byte) 'A', (byte) 'B', (byte) 'C', (byte) 'D', (byte) 'E', (byte) 'F', (byte) 'G', (byte) 'H', + (byte) 'I', (byte) 'J', (byte) 'K', (byte) 'L', (byte) 'M', (byte) 'N', (byte) 'O', (byte) 'P', + (byte) 'Q', (byte) 'R', (byte) 'S', (byte) 'T', (byte) 'U', (byte) 'V', (byte) 'W', (byte) 'X', + (byte) 'Y', (byte) 'Z', (byte) 'a', (byte) 'b', (byte) 'c', (byte) 'd', (byte) 'e', (byte) 'f', + (byte) 'g', (byte) 'h', (byte) 'i', (byte) 'j', (byte) 'k', (byte) 'l', (byte) 'm', (byte) 'n', + (byte) 'o', (byte) 'p', (byte) 'q', (byte) 'r', (byte) 's', (byte) 't', (byte) 'u', (byte) 'v', + (byte) 'w', (byte) 'x', (byte) 'y', (byte) 'z', (byte) '0', (byte) '1', (byte) '2', (byte) '3', + (byte) '4', (byte) '5', (byte) '6', (byte) '7', (byte) '8', (byte) '9', (byte) '-', (byte) '_' + }; + + /** Used in decoding URL- and Filename-safe dialects of Base64. */ + private static final byte[] _URL_SAFE_DECODABET = { + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 0 - 8 + -5, + -5, // Whitespace: Tab and Linefeed + -9, + -9, // Decimal 11 - 12 + -5, // Whitespace: Carriage Return + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 14 - 26 + -9, + -9, + -9, + -9, + -9, // Decimal 27 - 31 + -5, // Whitespace: Space + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 33 - 42 + -9, // Plus sign at decimal 43 + -9, // Decimal 44 + 62, // Minus sign at decimal 45 + -9, // Decimal 46 + -9, // Slash at decimal 47 + 52, + 53, + 54, + 55, + 56, + 57, + 58, + 59, + 60, + 61, // Numbers zero through nine + -9, + -9, + -9, // Decimal 58 - 60 + -1, // Equals sign at decimal 61 + -9, + -9, + -9, // Decimal 62 - 64 + 0, + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, // Letters 'A' through 'N' + 14, + 15, + 16, + 17, + 18, + 19, + 20, + 21, + 22, + 23, + 24, + 25, // Letters 'O' through 'Z' + -9, + -9, + -9, + -9, // Decimal 91 - 94 + 63, // Underscore at decimal 95 + -9, // Decimal 96 + 26, + 27, + 28, + 29, + 30, + 31, + 32, + 33, + 34, + 35, + 36, + 37, + 38, // Letters 'a' through 'm' + 39, + 40, + 41, + 42, + 43, + 44, + 45, + 46, + 47, + 48, + 49, + 50, + 51, // Letters 'n' through 'z' + -9, + -9, + -9, + -9, + -9 // Decimal 123 - 127 + , + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 128 - 139 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 140 - 152 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 153 - 165 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 166 - 178 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 179 - 191 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 192 - 204 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 205 - 217 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 218 - 230 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 231 - 243 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9 // Decimal 244 - 255 + }; + + /* ******** O R D E R E D B A S E 6 4 A L P H A B E T ******** */ + + /** + * I don't get the point of this technique, but someone requested it, and it is described here: http://www.faqs.org/qa/rfcc-1940.html. + */ + private static final byte[] _ORDERED_ALPHABET = { + (byte) '-', (byte) '0', (byte) '1', (byte) '2', (byte) '3', (byte) '4', (byte) '5', (byte) '6', + (byte) '7', (byte) '8', (byte) '9', (byte) 'A', (byte) 'B', (byte) 'C', (byte) 'D', (byte) 'E', + (byte) 'F', (byte) 'G', (byte) 'H', (byte) 'I', (byte) 'J', (byte) 'K', (byte) 'L', (byte) 'M', + (byte) 'N', (byte) 'O', (byte) 'P', (byte) 'Q', (byte) 'R', (byte) 'S', (byte) 'T', (byte) 'U', + (byte) 'V', (byte) 'W', (byte) 'X', (byte) 'Y', (byte) 'Z', (byte) '_', (byte) 'a', (byte) 'b', + (byte) 'c', (byte) 'd', (byte) 'e', (byte) 'f', (byte) 'g', (byte) 'h', (byte) 'i', (byte) 'j', + (byte) 'k', (byte) 'l', (byte) 'm', (byte) 'n', (byte) 'o', (byte) 'p', (byte) 'q', (byte) 'r', + (byte) 's', (byte) 't', (byte) 'u', (byte) 'v', (byte) 'w', (byte) 'x', (byte) 'y', (byte) 'z' + }; + + /** Used in decoding the "ordered" dialect of Base64. */ + private static final byte[] _ORDERED_DECODABET = { + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 0 - 8 + -5, + -5, // Whitespace: Tab and Linefeed + -9, + -9, // Decimal 11 - 12 + -5, // Whitespace: Carriage Return + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 14 - 26 + -9, + -9, + -9, + -9, + -9, // Decimal 27 - 31 + -5, // Whitespace: Space + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 33 - 42 + -9, // Plus sign at decimal 43 + -9, // Decimal 44 + 0, // Minus sign at decimal 45 + -9, // Decimal 46 + -9, // Slash at decimal 47 + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, // Numbers zero through nine + -9, + -9, + -9, // Decimal 58 - 60 + -1, // Equals sign at decimal 61 + -9, + -9, + -9, // Decimal 62 - 64 + 11, + 12, + 13, + 14, + 15, + 16, + 17, + 18, + 19, + 20, + 21, + 22, + 23, // Letters 'A' through 'M' + 24, + 25, + 26, + 27, + 28, + 29, + 30, + 31, + 32, + 33, + 34, + 35, + 36, // Letters 'N' through 'Z' + -9, + -9, + -9, + -9, // Decimal 91 - 94 + 37, // Underscore at decimal 95 + -9, // Decimal 96 + 38, + 39, + 40, + 41, + 42, + 43, + 44, + 45, + 46, + 47, + 48, + 49, + 50, // Letters 'a' through 'm' + 51, + 52, + 53, + 54, + 55, + 56, + 57, + 58, + 59, + 60, + 61, + 62, + 63, // Letters 'n' through 'z' + -9, + -9, + -9, + -9, + -9 // Decimal 123 - 127 + , + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 128 - 139 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 140 - 152 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 153 - 165 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 166 - 178 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 179 - 191 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 192 - 204 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 205 - 217 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 218 - 230 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, // Decimal 231 - 243 + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9, + -9 // Decimal 244 - 255 + }; + + /* ******** D E T E R M I N E W H I C H A L H A B E T ******** */ + + /** + * Returns one of the _SOMETHING_ALPHABET byte arrays depending on the options specified. It's + * possible, though silly, to specify ORDERED and URLSAFE in which case one of them will be + * picked, though there is no guarantee as to which one will be picked. + */ + private static final byte[] getAlphabet(int options) { + if ((options & URL_SAFE) == URL_SAFE) { + return _URL_SAFE_ALPHABET; + } else if ((options & ORDERED) == ORDERED) { + return _ORDERED_ALPHABET; + } else { + return _STANDARD_ALPHABET; + } + } // end getAlphabet + + /** + * Returns one of the _SOMETHING_DECODABET byte arrays depending on the options specified. It's + * possible, though silly, to specify ORDERED and URL_SAFE in which case one of them will be + * picked, though there is no guarantee as to which one will be picked. + */ + private static final byte[] getDecodabet(int options) { + if ((options & URL_SAFE) == URL_SAFE) { + return _URL_SAFE_DECODABET; + } else if ((options & ORDERED) == ORDERED) { + return _ORDERED_DECODABET; + } else { + return _STANDARD_DECODABET; + } + } // end getAlphabet + + /** Defeats instantiation. */ + private Base64() {} + + /* ******** E N C O D I N G M E T H O D S ******** */ + + /** + * Encodes up to the first three bytes of array threeBytes and returns a four-byte + * array in Base64 notation. The actual number of significant bytes in your array is given by + * numSigBytes. The array threeBytes needs only be as big as + * numSigBytes. Code can reuse a byte array by passing a four-byte array as + * b4. + * + * @param b4 A reusable byte array to reduce array instantiation + * @param threeBytes the array to convert + * @param numSigBytes the number of significant bytes in your array + * @return four byte array in Base64 notation. + * @since 1.5.1 + */ + private static byte[] encode3to4(byte[] b4, byte[] threeBytes, int numSigBytes, int options) { + encode3to4(threeBytes, 0, numSigBytes, b4, 0, options); + return b4; + } // end encode3to4 + + /** + * Encodes up to three bytes of the array source and writes the resulting four Base64 + * bytes to destination. The source and destination arrays can be manipulated anywhere + * along their length by specifying srcOffset and destOffset. This method + * does not check to make sure your arrays are large enough to accommodate srcOffset + + * 3 for the source array or destOffset + 4 for the destination + * array. The actual number of significant bytes in your array is given by numSigBytes. + * + *

This is the lowest level of the encoding methods with all possible parameters. + * + * @param source the array to convert + * @param srcOffset the index where conversion begins + * @param numSigBytes the number of significant bytes in your array + * @param destination the array to hold the conversion + * @param destOffset the index where output will be put + * @return the destination array + * @since 1.3 + */ + private static byte[] encode3to4( + byte[] source, + int srcOffset, + int numSigBytes, + byte[] destination, + int destOffset, + int options) { + + byte[] ALPHABET = getAlphabet(options); + + // 1 2 3 + // 01234567890123456789012345678901 Bit position + // --------000000001111111122222222 Array position from threeBytes + // --------| || || || | Six bit groups to index ALPHABET + // >>18 >>12 >> 6 >> 0 Right shift necessary + // 0x3f 0x3f 0x3f Additional AND + + // Create buffer with zero-padding if there are only one or two + // significant bytes passed in the array. + // We have to shift left 24 in order to flush out the 1's that appear + // when Java treats a value as negative that is cast from a byte to an int. + int inBuff = + (numSigBytes > 0 ? ((source[srcOffset] << 24) >>> 8) : 0) + | (numSigBytes > 1 ? ((source[srcOffset + 1] << 24) >>> 16) : 0) + | (numSigBytes > 2 ? ((source[srcOffset + 2] << 24) >>> 24) : 0); + + switch (numSigBytes) { + case 3: + destination[destOffset] = ALPHABET[(inBuff >>> 18)]; + destination[destOffset + 1] = ALPHABET[(inBuff >>> 12) & 0x3f]; + destination[destOffset + 2] = ALPHABET[(inBuff >>> 6) & 0x3f]; + destination[destOffset + 3] = ALPHABET[(inBuff) & 0x3f]; + return destination; + + case 2: + destination[destOffset] = ALPHABET[(inBuff >>> 18)]; + destination[destOffset + 1] = ALPHABET[(inBuff >>> 12) & 0x3f]; + destination[destOffset + 2] = ALPHABET[(inBuff >>> 6) & 0x3f]; + destination[destOffset + 3] = EQUALS_SIGN; + return destination; + + case 1: + destination[destOffset] = ALPHABET[(inBuff >>> 18)]; + destination[destOffset + 1] = ALPHABET[(inBuff >>> 12) & 0x3f]; + destination[destOffset + 2] = EQUALS_SIGN; + destination[destOffset + 3] = EQUALS_SIGN; + return destination; + + default: + return destination; + } // end switch + } // end encode3to4 + + /** + * Performs Base64 encoding on the raw ByteBuffer, writing it to the encoded + * ByteBuffer. This is an experimental feature. Currently it does not pass along any + * options (such as {@link #DO_BREAK_LINES} or {@link #GZIP}. + * + * @param raw input buffer + * @param encoded output buffer + * @since 2.3 + */ + public static void encode(java.nio.ByteBuffer raw, java.nio.ByteBuffer encoded) { + byte[] raw3 = new byte[3]; + byte[] enc4 = new byte[4]; + + while (raw.hasRemaining()) { + int rem = Math.min(3, raw.remaining()); + raw.get(raw3, 0, rem); + Base64.encode3to4(enc4, raw3, rem, Base64.NO_OPTIONS); + encoded.put(enc4); + } // end input remaining + } + + /** + * Performs Base64 encoding on the raw ByteBuffer, writing it to the encoded + * CharBuffer. This is an experimental feature. Currently it does not pass along any + * options (such as {@link #DO_BREAK_LINES} or {@link #GZIP}. + * + * @param raw input buffer + * @param encoded output buffer + * @since 2.3 + */ + public static void encode(java.nio.ByteBuffer raw, java.nio.CharBuffer encoded) { + byte[] raw3 = new byte[3]; + byte[] enc4 = new byte[4]; + + while (raw.hasRemaining()) { + int rem = Math.min(3, raw.remaining()); + raw.get(raw3, 0, rem); + Base64.encode3to4(enc4, raw3, rem, Base64.NO_OPTIONS); + for (int i = 0; i < 4; i++) { + encoded.put((char) (enc4[i] & 0xFF)); + } + } // end input remaining + } + + /** + * Serializes an object and returns the Base64-encoded version of that serialized object. + * + *

As of v 2.3, if the object cannot be serialized or there is another error, the method will + * throw an java.io.IOException. This is new to v2.3! In earlier versions, it just returned + * a null value, but in retrospect that's a pretty poor way to handle it. The object is not + * GZip-compressed before being encoded. + * + * @param serializableObject The object to encode + * @return The Base64-encoded object + * @throws java.io.IOException if there is an error + * @throws NullPointerException if serializedObject is null + * @since 1.4 + */ + public static String encodeObject(java.io.Serializable serializableObject) + throws java.io.IOException { + return encodeObject(serializableObject, NO_OPTIONS); + } // end encodeObject + + /** + * Serializes an object and returns the Base64-encoded version of that serialized object. + * + *

As of v 2.3, if the object cannot be serialized or there is another error, the method will + * throw an java.io.IOException. This is new to v2.3! In earlier versions, it just returned + * a null value, but in retrospect that's a pretty poor way to handle it. The object is not + * GZip-compressed before being encoded. + * + *

Example options: + * + *

+   *   GZIP: gzip-compresses object before encoding it.
+   *   DO_BREAK_LINES: break lines at 76 characters
+   * 
+ * + *

Example: encodeObject( myObj, Base64.GZIP ) or + * + *

Example: encodeObject( myObj, Base64.GZIP | Base64.DO_BREAK_LINES ) + * + * @param serializableObject The object to encode + * @param options Specified options + * @return The Base64-encoded object + * @see Base64#GZIP + * @see Base64#DO_BREAK_LINES + * @throws java.io.IOException if there is an error + * @since 2.0 + */ + public static String encodeObject(java.io.Serializable serializableObject, int options) + throws java.io.IOException { + + if (serializableObject == null) { + throw new NullPointerException("Cannot serialize a null object."); + } // end if: null + + // Streams + java.io.ByteArrayOutputStream baos = null; + java.io.OutputStream b64os = null; + java.util.zip.GZIPOutputStream gzos = null; + java.io.ObjectOutputStream oos = null; + + try { + // ObjectOutputStream -> (GZIP) -> Base64 -> ByteArrayOutputStream + baos = new java.io.ByteArrayOutputStream(); + b64os = new Base64.OutputStream(baos, ENCODE | options); + if ((options & GZIP) != 0) { + // Gzip + gzos = new java.util.zip.GZIPOutputStream(b64os); + oos = new java.io.ObjectOutputStream(gzos); + } else { + // Not gzipped + oos = new java.io.ObjectOutputStream(b64os); + } + oos.writeObject(serializableObject); + } // end try + catch (java.io.IOException e) { + // Catch it and then throw it immediately so that + // the finally{} block is called for cleanup. + throw e; + } // end catch + finally { + try { + oos.close(); + } catch (Exception e) { + } + try { + gzos.close(); + } catch (Exception e) { + } + try { + b64os.close(); + } catch (Exception e) { + } + try { + baos.close(); + } catch (Exception e) { + } + } // end finally + + // Return value according to relevant encoding. + try { + return new String(baos.toByteArray(), PREFERRED_ENCODING); + } // end try + catch (java.io.UnsupportedEncodingException uue) { + // Fall back to some Java default + return new String(baos.toByteArray()); + } // end catch + } // end encode + + /** + * Encodes a byte array into Base64 notation. Does not GZip-compress data. + * + * @param source The data to convert + * @return The data in Base64-encoded form + * @throws NullPointerException if source array is null + * @since 1.4 + */ + public static String encodeBytes(byte[] source) { + // Since we're not going to have the GZIP encoding turned on, + // we're not going to have an java.io.IOException thrown, so + // we should not force the user to have to catch it. + String encoded = null; + try { + encoded = encodeBytes(source, 0, source.length, NO_OPTIONS); + } catch (java.io.IOException ex) { + assert false : ex.getMessage(); + } // end catch + assert encoded != null; + return encoded; + } // end encodeBytes + + /** + * Encodes a byte array into Base64 notation. + * + *

Example options: + * + *

+   *   GZIP: gzip-compresses object before encoding it.
+   *   DO_BREAK_LINES: break lines at 76 characters
+   *     Note: Technically, this makes your encoding non-compliant.
+   * 
+ * + *

Example: encodeBytes( myData, Base64.GZIP ) or + * + *

Example: encodeBytes( myData, Base64.GZIP | Base64.DO_BREAK_LINES ) + * + *

As of v 2.3, if there is an error with the GZIP stream, the method will throw an + * java.io.IOException. This is new to v2.3! In earlier versions, it just returned a null + * value, but in retrospect that's a pretty poor way to handle it. + * + * @param source The data to convert + * @param options Specified options + * @return The Base64-encoded data as a String + * @see Base64#GZIP + * @see Base64#DO_BREAK_LINES + * @throws java.io.IOException if there is an error + * @throws NullPointerException if source array is null + * @since 2.0 + */ + public static String encodeBytes(byte[] source, int options) throws java.io.IOException { + return encodeBytes(source, 0, source.length, options); + } // end encodeBytes + + /** + * Encodes a byte array into Base64 notation. Does not GZip-compress data. + * + *

As of v 2.3, if there is an error, the method will throw an java.io.IOException. This is + * new to v2.3! In earlier versions, it just returned a null value, but in retrospect that's a + * pretty poor way to handle it. + * + * @param source The data to convert + * @param off Offset in array where conversion should begin + * @param len Length of data to convert + * @return The Base64-encoded data as a String + * @throws NullPointerException if source array is null + * @throws IllegalArgumentException if source array, offset, or length are invalid + * @since 1.4 + */ + public static String encodeBytes(byte[] source, int off, int len) { + // Since we're not going to have the GZIP encoding turned on, + // we're not going to have an java.io.IOException thrown, so + // we should not force the user to have to catch it. + String encoded = null; + try { + encoded = encodeBytes(source, off, len, NO_OPTIONS); + } catch (java.io.IOException ex) { + assert false : ex.getMessage(); + } // end catch + assert encoded != null; + return encoded; + } // end encodeBytes + + /** + * Encodes a byte array into Base64 notation. + * + *

Example options: + * + *

+   *   GZIP: gzip-compresses object before encoding it.
+   *   DO_BREAK_LINES: break lines at 76 characters
+   *     Note: Technically, this makes your encoding non-compliant.
+   * 
+ * + *

Example: encodeBytes( myData, Base64.GZIP ) or + * + *

Example: encodeBytes( myData, Base64.GZIP | Base64.DO_BREAK_LINES ) + * + *

As of v 2.3, if there is an error with the GZIP stream, the method will throw an + * java.io.IOException. This is new to v2.3! In earlier versions, it just returned a null + * value, but in retrospect that's a pretty poor way to handle it. + * + * @param source The data to convert + * @param off Offset in array where conversion should begin + * @param len Length of data to convert + * @param options Specified options + * @return The Base64-encoded data as a String + * @see Base64#GZIP + * @see Base64#DO_BREAK_LINES + * @throws java.io.IOException if there is an error + * @throws NullPointerException if source array is null + * @throws IllegalArgumentException if source array, offset, or length are invalid + * @since 2.0 + */ + public static String encodeBytes(byte[] source, int off, int len, int options) + throws java.io.IOException { + byte[] encoded = encodeBytesToBytes(source, off, len, options); + + // Return value according to relevant encoding. + try { + return new String(encoded, PREFERRED_ENCODING); + } // end try + catch (java.io.UnsupportedEncodingException uue) { + return new String(encoded); + } // end catch + } // end encodeBytes + + /** + * Similar to {@link #encodeBytes(byte[])} but returns a byte array instead of instantiating a + * String. This is more efficient if you're working with I/O streams and have large data sets to + * encode. + * + * @param source The data to convert + * @return The Base64-encoded data as a byte[] (of ASCII characters) + * @throws NullPointerException if source array is null + * @since 2.3.1 + */ + public static byte[] encodeBytesToBytes(byte[] source) { + byte[] encoded = null; + try { + encoded = encodeBytesToBytes(source, 0, source.length, Base64.NO_OPTIONS); + } catch (java.io.IOException ex) { + assert false + : "IOExceptions only come from GZipping, which is turned off: " + ex.getMessage(); + } + return encoded; + } + + /** + * Similar to {@link #encodeBytes(byte[], int, int, int)} but returns a byte array instead of + * instantiating a String. This is more efficient if you're working with I/O streams and have + * large data sets to encode. + * + * @param source The data to convert + * @param off Offset in array where conversion should begin + * @param len Length of data to convert + * @param options Specified options + * @return The Base64-encoded data as a String + * @see Base64#GZIP + * @see Base64#DO_BREAK_LINES + * @throws java.io.IOException if there is an error + * @throws NullPointerException if source array is null + * @throws IllegalArgumentException if source array, offset, or length are invalid + * @since 2.3.1 + */ + public static byte[] encodeBytesToBytes(byte[] source, int off, int len, int options) + throws java.io.IOException { + + if (source == null) { + throw new NullPointerException("Cannot serialize a null array."); + } // end if: null + + if (off < 0) { + throw new IllegalArgumentException("Cannot have negative offset: " + off); + } // end if: off < 0 + + if (len < 0) { + throw new IllegalArgumentException("Cannot have length offset: " + len); + } // end if: len < 0 + + if (off + len > source.length) { + throw new IllegalArgumentException( + String.format( + "Cannot have offset of %d and length of %d with array of length %d", + off, len, source.length)); + } // end if: off < 0 + + // Compress? + if ((options & GZIP) != 0) { + java.io.ByteArrayOutputStream baos = null; + java.util.zip.GZIPOutputStream gzos = null; + Base64.OutputStream b64os = null; + + try { + // GZip -> Base64 -> ByteArray + baos = new java.io.ByteArrayOutputStream(); + b64os = new Base64.OutputStream(baos, ENCODE | options); + gzos = new java.util.zip.GZIPOutputStream(b64os); + + gzos.write(source, off, len); + gzos.close(); + } // end try + catch (java.io.IOException e) { + // Catch it and then throw it immediately so that + // the finally{} block is called for cleanup. + throw e; + } // end catch + finally { + try { + gzos.close(); + } catch (Exception e) { + } + try { + b64os.close(); + } catch (Exception e) { + } + try { + baos.close(); + } catch (Exception e) { + } + } // end finally + + return baos.toByteArray(); + } // end if: compress + + // Else, don't compress. Better not to use streams at all then. + else { + boolean breakLines = (options & DO_BREAK_LINES) != 0; + + // int len43 = len * 4 / 3; + // byte[] outBuff = new byte[ ( len43 ) // Main 4:3 + // + ( (len % 3) > 0 ? 4 : 0 ) // Account for padding + // + (breakLines ? ( len43 / MAX_LINE_LENGTH ) : 0) ]; // New lines + // Try to determine more precisely how big the array needs to be. + // If we get it right, we don't have to do an array copy, and + // we save a bunch of memory. + int encLen = (len / 3) * 4 + (len % 3 > 0 ? 4 : 0); // Bytes needed for actual encoding + if (breakLines) { + encLen += encLen / MAX_LINE_LENGTH; // Plus extra newline characters + } + byte[] outBuff = new byte[encLen]; + + int d = 0; + int e = 0; + int len2 = len - 2; + int lineLength = 0; + for (; d < len2; d += 3, e += 4) { + encode3to4(source, d + off, 3, outBuff, e, options); + + lineLength += 4; + if (breakLines && lineLength >= MAX_LINE_LENGTH) { + outBuff[e + 4] = NEW_LINE; + e++; + lineLength = 0; + } // end if: end of line + } // en dfor: each piece of array + + if (d < len) { + encode3to4(source, d + off, len - d, outBuff, e, options); + e += 4; + } // end if: some padding needed + + // Only resize array if we didn't guess it right. + if (e <= outBuff.length - 1) { + // If breaking lines and the last byte falls right at + // the line length (76 bytes per line), there will be + // one extra byte, and the array will need to be resized. + // Not too bad of an estimate on array size, I'd say. + byte[] finalOut = new byte[e]; + System.arraycopy(outBuff, 0, finalOut, 0, e); + // System.err.println("Having to resize array from " + outBuff.length + " to " + e ); + return finalOut; + } else { + // System.err.println("No need to resize array."); + return outBuff; + } + } // end else: don't compress + } // end encodeBytesToBytes + + /* ******** D E C O D I N G M E T H O D S ******** */ + + /** + * Decodes four bytes from array source and writes the resulting bytes (up to three of + * them) to destination. The source and destination arrays can be manipulated anywhere + * along their length by specifying srcOffset and destOffset. This method + * does not check to make sure your arrays are large enough to accomodate srcOffset + 4 + * for the source array or destOffset + 3 for the destination + * array. This method returns the actual number of bytes that were converted from the Base64 + * encoding. + * + *

This is the lowest level of the decoding methods with all possible parameters. + * + * @param source the array to convert + * @param srcOffset the index where conversion begins + * @param destination the array to hold the conversion + * @param destOffset the index where output will be put + * @param options alphabet type is pulled from this (standard, url-safe, ordered) + * @return the number of decoded bytes converted + * @throws NullPointerException if source or destination arrays are null + * @throws IllegalArgumentException if srcOffset or destOffset are invalid or there is not enough + * room in the array. + * @since 1.3 + */ + private static int decode4to3( + byte[] source, int srcOffset, byte[] destination, int destOffset, int options) { + + // Lots of error checking and exception throwing + if (source == null) { + throw new NullPointerException("Source array was null."); + } // end if + if (destination == null) { + throw new NullPointerException("Destination array was null."); + } // end if + if (srcOffset < 0 || srcOffset + 3 >= source.length) { + throw new IllegalArgumentException( + String.format( + "Source array with length %d cannot have offset of %d and still process four bytes.", + source.length, srcOffset)); + } // end if + if (destOffset < 0 || destOffset + 2 >= destination.length) { + throw new IllegalArgumentException( + String.format( + "Destination array with length %d cannot have offset of %d and still store three bytes.", + destination.length, destOffset)); + } // end if + + byte[] DECODABET = getDecodabet(options); + + // Example: Dk== + if (source[srcOffset + 2] == EQUALS_SIGN) { + // Two ways to do the same thing. Don't know which way I like best. + // int outBuff = ( ( DECODABET[ source[ srcOffset ] ] << 24 ) >>> 6 ) + // | ( ( DECODABET[ source[ srcOffset + 1] ] << 24 ) >>> 12 ); + int outBuff = + ((DECODABET[source[srcOffset]] & 0xFF) << 18) + | ((DECODABET[source[srcOffset + 1]] & 0xFF) << 12); + + destination[destOffset] = (byte) (outBuff >>> 16); + return 1; + } + + // Example: DkL= + else if (source[srcOffset + 3] == EQUALS_SIGN) { + // Two ways to do the same thing. Don't know which way I like best. + // int outBuff = ( ( DECODABET[ source[ srcOffset ] ] << 24 ) >>> 6 ) + // | ( ( DECODABET[ source[ srcOffset + 1 ] ] << 24 ) >>> 12 ) + // | ( ( DECODABET[ source[ srcOffset + 2 ] ] << 24 ) >>> 18 ); + int outBuff = + ((DECODABET[source[srcOffset]] & 0xFF) << 18) + | ((DECODABET[source[srcOffset + 1]] & 0xFF) << 12) + | ((DECODABET[source[srcOffset + 2]] & 0xFF) << 6); + + destination[destOffset] = (byte) (outBuff >>> 16); + destination[destOffset + 1] = (byte) (outBuff >>> 8); + return 2; + } + + // Example: DkLE + else { + // Two ways to do the same thing. Don't know which way I like best. + // int outBuff = ( ( DECODABET[ source[ srcOffset ] ] << 24 ) >>> 6 ) + // | ( ( DECODABET[ source[ srcOffset + 1 ] ] << 24 ) >>> 12 ) + // | ( ( DECODABET[ source[ srcOffset + 2 ] ] << 24 ) >>> 18 ) + // | ( ( DECODABET[ source[ srcOffset + 3 ] ] << 24 ) >>> 24 ); + int outBuff = + ((DECODABET[source[srcOffset]] & 0xFF) << 18) + | ((DECODABET[source[srcOffset + 1]] & 0xFF) << 12) + | ((DECODABET[source[srcOffset + 2]] & 0xFF) << 6) + | ((DECODABET[source[srcOffset + 3]] & 0xFF)); + + destination[destOffset] = (byte) (outBuff >> 16); + destination[destOffset + 1] = (byte) (outBuff >> 8); + destination[destOffset + 2] = (byte) (outBuff); + + return 3; + } + } // end decodeToBytes + + /** + * Low-level access to decoding ASCII characters in the form of a byte array. Ignores + * GUNZIP option, if it's set. This is not generally a recommended method, although it is + * used internally as part of the decoding process. Special case: if len = 0, an empty array is + * returned. Still, if you need more speed and reduced memory footprint (and aren't gzipping), + * consider this method. + * + * @param source The Base64 encoded data + * @return decoded data + * @since 2.3.1 + */ + public static byte[] decode(byte[] source) throws java.io.IOException { + byte[] decoded = null; + // try { + decoded = decode(source, 0, source.length, Base64.NO_OPTIONS); + // } catch( java.io.IOException ex ) { + // assert false : "IOExceptions only come from GZipping, which is turned off: " + + // ex.getMessage(); + // } + return decoded; + } + + /** + * Low-level access to decoding ASCII characters in the form of a byte array. Ignores + * GUNZIP option, if it's set. This is not generally a recommended method, although it is + * used internally as part of the decoding process. Special case: if len = 0, an empty array is + * returned. Still, if you need more speed and reduced memory footprint (and aren't gzipping), + * consider this method. + * + * @param source The Base64 encoded data + * @param off The offset of where to begin decoding + * @param len The length of characters to decode + * @param options Can specify options such as alphabet type to use + * @return decoded data + * @throws java.io.IOException If bogus characters exist in source data + * @since 1.3 + */ + public static byte[] decode(byte[] source, int off, int len, int options) + throws java.io.IOException { + + // Lots of error checking and exception throwing + if (source == null) { + throw new NullPointerException("Cannot decode null source array."); + } // end if + if (off < 0 || off + len > source.length) { + throw new IllegalArgumentException( + String.format( + "Source array with length %d cannot have offset of %d and process %d bytes.", + source.length, off, len)); + } // end if + + if (len == 0) { + return new byte[0]; + } else if (len < 4) { + throw new IllegalArgumentException( + "Base64-encoded string must have at least four characters, but length specified was " + + len); + } // end if + + byte[] DECODABET = getDecodabet(options); + + int len34 = len * 3 / 4; // Estimate on array size + byte[] outBuff = new byte[len34]; // Upper limit on size of output + int outBuffPosn = 0; // Keep track of where we're writing + + byte[] b4 = new byte[4]; // Four byte buffer from source, eliminating white space + int b4Posn = 0; // Keep track of four byte input buffer + int i = 0; // Source array counter + byte sbiDecode = 0; // Special value from DECODABET + + for (i = off; i < off + len; i++) { // Loop through source + + sbiDecode = DECODABET[source[i] & 0xFF]; + + // White space, Equals sign, or legit Base64 character + // Note the values such as -5 and -9 in the + // DECODABETs at the top of the file. + if (sbiDecode >= WHITE_SPACE_ENC) { + if (sbiDecode >= EQUALS_SIGN_ENC) { + b4[b4Posn++] = source[i]; // Save non-whitespace + if (b4Posn > 3) { // Time to decode? + outBuffPosn += decode4to3(b4, 0, outBuff, outBuffPosn, options); + b4Posn = 0; + + // If that was the equals sign, break out of 'for' loop + if (source[i] == EQUALS_SIGN) { + break; + } // end if: equals sign + } // end if: quartet built + } // end if: equals sign or better + } // end if: white space, equals sign or better + else { + // There's a bad input character in the Base64 stream. + throw new java.io.IOException( + String.format( + "Bad Base64 input character decimal %d in array position %d", + ((int) source[i]) & 0xFF, i)); + } // end else: + } // each input character + + byte[] out = new byte[outBuffPosn]; + System.arraycopy(outBuff, 0, out, 0, outBuffPosn); + return out; + } // end decode + + /** + * Decodes data from Base64 notation, automatically detecting gzip-compressed data and + * decompressing it. + * + * @param s the string to decode + * @return the decoded data + * @throws java.io.IOException If there is a problem + * @since 1.4 + */ + public static byte[] decode(String s) throws java.io.IOException { + return decode(s, NO_OPTIONS); + } + + /** + * Decodes data from Base64 notation, automatically detecting gzip-compressed data and + * decompressing it. + * + * @param s the string to decode + * @param options encode options such as URL_SAFE + * @return the decoded data + * @throws java.io.IOException if there is an error + * @throws NullPointerException if s is null + * @since 1.4 + */ + public static byte[] decode(String s, int options) throws java.io.IOException { + + if (s == null) { + throw new NullPointerException("Input string was null."); + } // end if + + byte[] bytes; + try { + bytes = s.getBytes(PREFERRED_ENCODING); + } // end try + catch (java.io.UnsupportedEncodingException uee) { + bytes = s.getBytes(); + } // end catch + // + + // Decode + bytes = decode(bytes, 0, bytes.length, options); + + // Check to see if it's gzip-compressed + // GZIP Magic Two-Byte Number: 0x8b1f (35615) + boolean dontGunzip = (options & DONT_GUNZIP) != 0; + if ((bytes != null) && (bytes.length >= 4) && (!dontGunzip)) { + + int head = ((int) bytes[0] & 0xff) | ((bytes[1] << 8) & 0xff00); + if (java.util.zip.GZIPInputStream.GZIP_MAGIC == head) { + java.io.ByteArrayInputStream bais = null; + java.util.zip.GZIPInputStream gzis = null; + java.io.ByteArrayOutputStream baos = null; + byte[] buffer = new byte[2048]; + int length = 0; + + try { + baos = new java.io.ByteArrayOutputStream(); + bais = new java.io.ByteArrayInputStream(bytes); + gzis = new java.util.zip.GZIPInputStream(bais); + + while ((length = gzis.read(buffer)) >= 0) { + baos.write(buffer, 0, length); + } // end while: reading input + + // No error? Get new bytes. + bytes = baos.toByteArray(); + + } // end try + catch (java.io.IOException e) { + e.printStackTrace(); + // Just return originally-decoded bytes + } // end catch + finally { + try { + baos.close(); + } catch (Exception e) { + } + try { + gzis.close(); + } catch (Exception e) { + } + try { + bais.close(); + } catch (Exception e) { + } + } // end finally + } // end if: gzipped + } // end if: bytes.length >= 2 + + return bytes; + } // end decode + + /** + * Attempts to decode Base64 data and deserialize a Java Object within. Returns null + * if there was an error. + * + * @param encodedObject The Base64 data to decode + * @return The decoded and deserialized object + * @throws NullPointerException if encodedObject is null + * @throws java.io.IOException if there is a general error + * @throws ClassNotFoundException if the decoded object is of a class that cannot be found by the + * JVM + * @since 1.5 + */ + public static Object decodeToObject(String encodedObject) + throws java.io.IOException, java.lang.ClassNotFoundException { + return decodeToObject(encodedObject, NO_OPTIONS, null); + } + + /** + * Attempts to decode Base64 data and deserialize a Java Object within. Returns null + * if there was an error. If loader is not null, it will be the class loader used + * when deserializing. + * + * @param encodedObject The Base64 data to decode + * @param options Various parameters related to decoding + * @param loader Optional class loader to use in deserializing classes. + * @return The decoded and deserialized object + * @throws NullPointerException if encodedObject is null + * @throws java.io.IOException if there is a general error + * @throws ClassNotFoundException if the decoded object is of a class that cannot be found by the + * JVM + * @since 2.3.4 + */ + public static Object decodeToObject(String encodedObject, int options, final ClassLoader loader) + throws java.io.IOException, java.lang.ClassNotFoundException { + + // Decode and gunzip if necessary + byte[] objBytes = decode(encodedObject, options); + + java.io.ByteArrayInputStream bais = null; + java.io.ObjectInputStream ois = null; + Object obj = null; + + try { + bais = new java.io.ByteArrayInputStream(objBytes); + + // If no custom class loader is provided, use Java's builtin OIS. + if (loader == null) { + ois = new java.io.ObjectInputStream(bais); + } // end if: no loader provided + + // Else make a customized object input stream that uses + // the provided class loader. + else { + ois = + new java.io.ObjectInputStream(bais) { + @Override + public Class resolveClass(java.io.ObjectStreamClass streamClass) + throws java.io.IOException, ClassNotFoundException { + Class c = Class.forName(streamClass.getName(), false, loader); + if (c == null) { + return super.resolveClass(streamClass); + } else { + return c; // Class loader knows of this class. + } // end else: not null + } // end resolveClass + }; // end ois + } // end else: no custom class loader + + obj = ois.readObject(); + } // end try + catch (java.io.IOException e) { + throw e; // Catch and throw in order to execute finally{} + } // end catch + catch (java.lang.ClassNotFoundException e) { + throw e; // Catch and throw in order to execute finally{} + } // end catch + finally { + try { + bais.close(); + } catch (Exception e) { + } + try { + ois.close(); + } catch (Exception e) { + } + } // end finally + + return obj; + } // end decodeObject + + /** + * Convenience method for encoding data to a file. + * + *

As of v 2.3, if there is a error, the method will throw an java.io.IOException. This is + * new to v2.3! In earlier versions, it just returned false, but in retrospect that's a pretty + * poor way to handle it. + * + * @param dataToEncode byte array of data to encode in base64 form + * @param filename Filename for saving encoded data + * @throws java.io.IOException if there is an error + * @throws NullPointerException if dataToEncode is null + * @since 2.1 + */ + public static void encodeToFile(byte[] dataToEncode, String filename) throws java.io.IOException { + + if (dataToEncode == null) { + throw new NullPointerException("Data to encode was null."); + } // end iff + + Base64.OutputStream bos = null; + try { + bos = new Base64.OutputStream(new java.io.FileOutputStream(filename), Base64.ENCODE); + bos.write(dataToEncode); + } // end try + catch (java.io.IOException e) { + throw e; // Catch and throw to execute finally{} block + } // end catch: java.io.IOException + finally { + try { + bos.close(); + } catch (Exception e) { + } + } // end finally + } // end encodeToFile + + /** + * Convenience method for decoding data to a file. + * + *

As of v 2.3, if there is a error, the method will throw an java.io.IOException. This is + * new to v2.3! In earlier versions, it just returned false, but in retrospect that's a pretty + * poor way to handle it. + * + * @param dataToDecode Base64-encoded data as a string + * @param filename Filename for saving decoded data + * @throws java.io.IOException if there is an error + * @since 2.1 + */ + public static void decodeToFile(String dataToDecode, String filename) throws java.io.IOException { + + Base64.OutputStream bos = null; + try { + bos = new Base64.OutputStream(new java.io.FileOutputStream(filename), Base64.DECODE); + bos.write(dataToDecode.getBytes(PREFERRED_ENCODING)); + } // end try + catch (java.io.IOException e) { + throw e; // Catch and throw to execute finally{} block + } // end catch: java.io.IOException + finally { + try { + bos.close(); + } catch (Exception e) { + } + } // end finally + } // end decodeToFile + + /** + * Convenience method for reading a base64-encoded file and decoding it. + * + *

As of v 2.3, if there is a error, the method will throw an java.io.IOException. This is + * new to v2.3! In earlier versions, it just returned false, but in retrospect that's a pretty + * poor way to handle it. + * + * @param filename Filename for reading encoded data + * @return decoded byte array + * @throws java.io.IOException if there is an error + * @since 2.1 + */ + public static byte[] decodeFromFile(String filename) throws java.io.IOException { + + byte[] decodedData = null; + Base64.InputStream bis = null; + try { + // Set up some useful variables + java.io.File file = new java.io.File(filename); + byte[] buffer = null; + int length = 0; + int numBytes = 0; + + // Check for size of file + // See: https://stackoverflow.com/questions/3038392/do-java-arrays-have-a-maximum-size + if (file.length() > Integer.MAX_VALUE - 8) { + throw new java.io.IOException( + "File is too big for this convenience method (" + file.length() + " bytes)."); + } // end if: file too big for int index + buffer = new byte[(int) file.length()]; + + // Open a stream + bis = + new Base64.InputStream( + new java.io.BufferedInputStream(new java.io.FileInputStream(file)), Base64.DECODE); + + // Read until done + while ((numBytes = bis.read(buffer, length, 4096)) >= 0) { + length += numBytes; + } // end while + + // Save in a variable to return + decodedData = new byte[length]; + System.arraycopy(buffer, 0, decodedData, 0, length); + + } // end try + catch (java.io.IOException e) { + throw e; // Catch and release to execute finally{} + } // end catch: java.io.IOException + finally { + try { + bis.close(); + } catch (Exception e) { + } + } // end finally + + return decodedData; + } // end decodeFromFile + + /** + * Convenience method for reading a binary file and base64-encoding it. + * + *

As of v 2.3, if there is a error, the method will throw an java.io.IOException. This is + * new to v2.3! In earlier versions, it just returned false, but in retrospect that's a pretty + * poor way to handle it. + * + * @param filename Filename for reading binary data + * @return base64-encoded string + * @throws java.io.IOException if there is an error + * @since 2.1 + */ + public static String encodeFromFile(String filename) throws java.io.IOException { + + String encodedData = null; + Base64.InputStream bis = null; + try { + // Set up some useful variables + java.io.File file = new java.io.File(filename); + byte[] buffer = + new byte + [Math.max( + (int) (file.length() * 1.4 + 1), + 40)]; // Need max() for math on small files (v2.2.1); Need +1 for a few corner + // cases (v2.3.5) + int length = 0; + int numBytes = 0; + + // Open a stream + bis = + new Base64.InputStream( + new java.io.BufferedInputStream(new java.io.FileInputStream(file)), Base64.ENCODE); + + // Read until done + while ((numBytes = bis.read(buffer, length, 4096)) >= 0) { + length += numBytes; + } // end while + + // Save in a variable to return + encodedData = new String(buffer, 0, length, Base64.PREFERRED_ENCODING); + + } // end try + catch (java.io.IOException e) { + throw e; // Catch and release to execute finally{} + } // end catch: java.io.IOException + finally { + try { + bis.close(); + } catch (Exception e) { + } + } // end finally + + return encodedData; + } // end encodeFromFile + + /** + * Reads infile and encodes it to outfile. + * + * @param infile Input file + * @param outfile Output file + * @throws java.io.IOException if there is an error + * @since 2.2 + */ + public static void encodeFileToFile(String infile, String outfile) throws java.io.IOException { + + String encoded = Base64.encodeFromFile(infile); + java.io.OutputStream out = null; + try { + out = new java.io.BufferedOutputStream(new java.io.FileOutputStream(outfile)); + out.write(encoded.getBytes("US-ASCII")); // Strict, 7-bit output. + } // end try + catch (java.io.IOException e) { + throw e; // Catch and release to execute finally{} + } // end catch + finally { + try { + out.close(); + } catch (Exception ex) { + } + } // end finally + } // end encodeFileToFile + + /** + * Reads infile and decodes it to outfile. + * + * @param infile Input file + * @param outfile Output file + * @throws java.io.IOException if there is an error + * @since 2.2 + */ + public static void decodeFileToFile(String infile, String outfile) throws java.io.IOException { + + byte[] decoded = Base64.decodeFromFile(infile); + java.io.OutputStream out = null; + try { + out = new java.io.BufferedOutputStream(new java.io.FileOutputStream(outfile)); + out.write(decoded); + } // end try + catch (java.io.IOException e) { + throw e; // Catch and release to execute finally{} + } // end catch + finally { + try { + out.close(); + } catch (Exception ex) { + } + } // end finally + } // end decodeFileToFile + + /* ******** I N N E R C L A S S I N P U T S T R E A M ******** */ + + /** + * A {@link Base64.InputStream} will read data from another java.io.InputStream, + * given in the constructor, and encode/decode to/from Base64 notation on the fly. + * + * @see Base64 + * @since 1.3 + */ + public static class InputStream extends java.io.FilterInputStream { + + private boolean encode; // Encoding or decoding + private int position; // Current position in the buffer + private byte[] buffer; // Small buffer holding converted data + private int bufferLength; // Length of buffer (3 or 4) + private int numSigBytes; // Number of meaningful bytes in the buffer + private int lineLength; + private boolean breakLines; // Break lines at less than 80 characters + private int options; // Record options used to create the stream. + private byte[] decodabet; // Local copies to avoid extra method calls + + /** + * Constructs a {@link Base64.InputStream} in DECODE mode. + * + * @param in the java.io.InputStream from which to read data. + * @since 1.3 + */ + public InputStream(java.io.InputStream in) { + this(in, DECODE); + } // end constructor + + /** + * Constructs a {@link Base64.InputStream} in either ENCODE or DECODE mode. + * + *

Valid options: + * + *

+     *   ENCODE or DECODE: Encode or Decode as data is read.
+     *   DO_BREAK_LINES: break lines at 76 characters
+     *     (only meaningful when encoding)
+     * 
+ * + *

Example: new Base64.InputStream( in, Base64.DECODE ) + * + * @param in the java.io.InputStream from which to read data. + * @param options Specified options + * @see Base64#ENCODE + * @see Base64#DECODE + * @see Base64#DO_BREAK_LINES + * @since 2.0 + */ + public InputStream(java.io.InputStream in, int options) { + + super(in); + this.options = options; // Record for later + this.breakLines = (options & DO_BREAK_LINES) > 0; + this.encode = (options & ENCODE) > 0; + this.bufferLength = encode ? 4 : 3; + this.buffer = new byte[bufferLength]; + this.position = -1; + this.lineLength = 0; + this.decodabet = getDecodabet(options); + } // end constructor + + /** + * Reads enough of the input stream to convert to/from Base64 and returns the next byte. + * + * @return next byte + * @since 1.3 + */ + @Override + public int read() throws java.io.IOException { + + // Do we need to get data? + if (position < 0) { + if (encode) { + byte[] b3 = new byte[3]; + int numBinaryBytes = 0; + for (int i = 0; i < 3; i++) { + int b = in.read(); + + // If end of stream, b is -1. + if (b >= 0) { + b3[i] = (byte) b; + numBinaryBytes++; + } else { + break; // out of for loop + } // end else: end of stream + } // end for: each needed input byte + + if (numBinaryBytes > 0) { + encode3to4(b3, 0, numBinaryBytes, buffer, 0, options); + position = 0; + numSigBytes = 4; + } // end if: got data + else { + return -1; // Must be end of stream + } // end else + } // end if: encoding + + // Else decoding + else { + byte[] b4 = new byte[4]; + int i = 0; + for (i = 0; i < 4; i++) { + // Read four "meaningful" bytes: + int b = 0; + do { + b = in.read(); + } while (b >= 0 && decodabet[b & 0x7f] <= WHITE_SPACE_ENC); + + if (b < 0) { + break; // Reads a -1 if end of stream + } // end if: end of stream + + b4[i] = (byte) b; + } // end for: each needed input byte + + if (i == 4) { + numSigBytes = decode4to3(b4, 0, buffer, 0, options); + position = 0; + } // end if: got four characters + else if (i == 0) { + return -1; + } // end else if: also padded correctly + else { + // Must have broken out from above. + throw new java.io.IOException("Improperly padded Base64 input."); + } // end + } // end else: decode + } // end else: get data + + // Got data? + if (position >= 0) { + // End of relevant data? + if ( + /*!encode &&*/ position >= numSigBytes) { + return -1; + } // end if: got data + + if (encode && breakLines && lineLength >= MAX_LINE_LENGTH) { + lineLength = 0; + return '\n'; + } // end if + else { + lineLength++; // This isn't important when decoding + // but throwing an extra "if" seems + // just as wasteful. + + int b = buffer[position++]; + + if (position >= bufferLength) { + position = -1; + } // end if: end + + return b & 0xFF; // This is how you "cast" a byte that's + // intended to be unsigned. + } // end else + } // end if: position >= 0 + + // Else error + else { + throw new java.io.IOException("Error in Base64 code reading stream."); + } // end else + } // end read + + /** + * Calls {@link #read()} repeatedly until the end of stream is reached or len bytes + * are read. Returns number of bytes read into array or -1 if end of stream is encountered. + * + * @param dest array to hold values + * @param off offset for array + * @param len max number of bytes to read into array + * @return bytes read into array or -1 if end of stream is encountered. + * @since 1.3 + */ + @Override + public int read(byte[] dest, int off, int len) throws java.io.IOException { + int i; + int b; + for (i = 0; i < len; i++) { + b = read(); + + if (b >= 0) { + dest[off + i] = (byte) b; + } else if (i == 0) { + return -1; + } else { + break; // Out of 'for' loop + } // Out of 'for' loop + } // end for: each byte read + return i; + } // end read + } // end inner class InputStream + + /* ******** I N N E R C L A S S O U T P U T S T R E A M ******** */ + + /** + * A {@link Base64.OutputStream} will write data to another java.io.OutputStream, + * given in the constructor, and encode/decode to/from Base64 notation on the fly. + * + * @see Base64 + * @since 1.3 + */ + public static class OutputStream extends java.io.FilterOutputStream { + + private boolean encode; + private int position; + private byte[] buffer; + private int bufferLength; + private int lineLength; + private boolean breakLines; + private byte[] b4; // Scratch used in a few places + private boolean suspendEncoding; + private int options; // Record for later + private byte[] decodabet; // Local copies to avoid extra method calls + + /** + * Constructs a {@link Base64.OutputStream} in ENCODE mode. + * + * @param out the java.io.OutputStream to which data will be written. + * @since 1.3 + */ + public OutputStream(java.io.OutputStream out) { + this(out, ENCODE); + } // end constructor + + /** + * Constructs a {@link Base64.OutputStream} in either ENCODE or DECODE mode. + * + *

Valid options: + * + *

+     *   ENCODE or DECODE: Encode or Decode as data is read.
+     *   DO_BREAK_LINES: don't break lines at 76 characters
+     *     (only meaningful when encoding)
+     * 
+ * + *

Example: new Base64.OutputStream( out, Base64.ENCODE ) + * + * @param out the java.io.OutputStream to which data will be written. + * @param options Specified options. + * @see Base64#ENCODE + * @see Base64#DECODE + * @see Base64#DO_BREAK_LINES + * @since 1.3 + */ + public OutputStream(java.io.OutputStream out, int options) { + super(out); + this.breakLines = (options & DO_BREAK_LINES) != 0; + this.encode = (options & ENCODE) != 0; + this.bufferLength = encode ? 3 : 4; + this.buffer = new byte[bufferLength]; + this.position = 0; + this.lineLength = 0; + this.suspendEncoding = false; + this.b4 = new byte[4]; + this.options = options; + this.decodabet = getDecodabet(options); + } // end constructor + + /** + * Writes the byte to the output stream after converting to/from Base64 notation. When encoding, + * bytes are buffered three at a time before the output stream actually gets a write() call. + * When decoding, bytes are buffered four at a time. + * + * @param theByte the byte to write + * @since 1.3 + */ + @Override + public void write(int theByte) throws java.io.IOException { + // Encoding suspended? + if (suspendEncoding) { + this.out.write(theByte); + return; + } // end if: supsended + + // Encode? + if (encode) { + buffer[position++] = (byte) theByte; + if (position >= bufferLength) { // Enough to encode. + + this.out.write(encode3to4(b4, buffer, bufferLength, options)); + + lineLength += 4; + if (breakLines && lineLength >= MAX_LINE_LENGTH) { + this.out.write(NEW_LINE); + lineLength = 0; + } // end if: end of line + + position = 0; + } // end if: enough to output + } // end if: encoding + + // Else, Decoding + else { + // Meaningful Base64 character? + if (decodabet[theByte & 0x7f] > WHITE_SPACE_ENC) { + buffer[position++] = (byte) theByte; + if (position >= bufferLength) { // Enough to output. + + int len = Base64.decode4to3(buffer, 0, b4, 0, options); + out.write(b4, 0, len); + position = 0; + } // end if: enough to output + } // end if: meaningful base64 character + else if (decodabet[theByte & 0x7f] != WHITE_SPACE_ENC) { + throw new java.io.IOException("Invalid character in Base64 data."); + } // end else: not white space either + } // end else: decoding + } // end write + + /** + * Calls {@link #write(int)} repeatedly until len bytes are written. + * + * @param theBytes array from which to read bytes + * @param off offset for array + * @param len max number of bytes to read into array + * @since 1.3 + */ + @Override + public void write(byte[] theBytes, int off, int len) throws java.io.IOException { + // Encoding suspended? + if (suspendEncoding) { + this.out.write(theBytes, off, len); + return; + } // end if: supsended + + for (int i = 0; i < len; i++) { + write(theBytes[off + i]); + } // end for: each byte written + } // end write + + /** + * Method added by PHIL. [Thanks, PHIL. -Rob] This pads the buffer without closing the stream. + * + * @throws java.io.IOException if there's an error. + */ + public void flushBase64() throws java.io.IOException { + if (position > 0) { + if (encode) { + out.write(encode3to4(b4, buffer, position, options)); + position = 0; + } // end if: encoding + else { + throw new java.io.IOException("Base64 input not properly padded."); + } // end else: decoding + } // end if: buffer partially full + } // end flush + + /** + * Flushes and closes (I think, in the superclass) the stream. + * + * @since 1.3 + */ + @Override + public void close() throws java.io.IOException { + // 1. Ensure that pending characters are written + flushBase64(); + + // 2. Actually close the stream + // Base class both flushes and closes. + super.close(); + + buffer = null; + out = null; + } // end close + + /** + * Suspends encoding of the stream. May be helpful if you need to embed a piece of + * base64-encoded data in a stream. + * + * @throws java.io.IOException if there's an error flushing + * @since 1.5.1 + */ + public void suspendEncoding() throws java.io.IOException { + flushBase64(); + this.suspendEncoding = true; + } // end suspendEncoding + + /** + * Resumes encoding of the stream. May be helpful if you need to embed a piece of base64-encoded + * data in a stream. + * + * @since 1.5.1 + */ + public void resumeEncoding() { + this.suspendEncoding = false; + } // end resumeEncoding + } // end inner class OutputStream +} // end class Base64 diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/ClassRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ClassRegistrar.java new file mode 100644 index 0000000000..b69829f66e --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ClassRegistrar.java @@ -0,0 +1,53 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; + +/** Register this class to be used with the default serializer for this class */ +public class ClassRegistrar implements IKryoRegistrar { + final Class klass; + + public ClassRegistrar(Class cls) { + klass = cls; + } + + public Class getRegisteredClass() { + return klass; + } + + @Override + public void apply(Kryo k) { + k.register(klass); + } + + @Override + public int hashCode() { + return klass.hashCode(); + } + + @Override + public boolean equals(Object that) { + if (null == that) { + return false; + } else if (that instanceof ClassRegistrar) { + return klass.equals(((ClassRegistrar) that).klass); + } else { + return false; + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/IKryoRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/IKryoRegistrar.java new file mode 100644 index 0000000000..6df97d42fa --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/IKryoRegistrar.java @@ -0,0 +1,29 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; +import java.io.Serializable; + +/** + * A Registrar adds registrations to a given Kryo instance. Examples would be a registrar that + * registers serializers for all objects in a given package. comes from Storm, which took it from + * cascading.kryo + */ +public interface IKryoRegistrar extends Serializable { + void apply(Kryo k); +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoInstantiator.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoInstantiator.java new file mode 100644 index 0000000000..34c1ddece2 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoInstantiator.java @@ -0,0 +1,112 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; +import java.io.Serializable; +import org.objenesis.strategy.InstantiatorStrategy; + +/** + * Class to create a new Kryo instance. Used in initial configuration or pooling of Kryo objects. + * These objects are immutable (and hopefully Kryo serializable) + */ +public class KryoInstantiator implements Serializable { + public Kryo newKryo() { + return new Kryo(); + } + + /** Use this to set a specific classloader */ + public KryoInstantiator setClassLoader(final ClassLoader cl) { + return new KryoInstantiator() { + public Kryo newKryo() { + Kryo k = KryoInstantiator.this.newKryo(); + k.setClassLoader(cl); + return k; + } + }; + } + /** If true, Kryo will error if it sees a class that has not been registered */ + public KryoInstantiator setInstantiatorStrategy(final InstantiatorStrategy inst) { + return new KryoInstantiator() { + public Kryo newKryo() { + Kryo k = KryoInstantiator.this.newKryo(); + k.setInstantiatorStrategy(inst); + return k; + } + }; + } + + /** + * If true, Kryo keeps a map of all the objects it has seen. this can use a ton of memory on + * hadoop, but save serialization costs in some cases + */ + public KryoInstantiator setReferences(final boolean ref) { + return new KryoInstantiator() { + public Kryo newKryo() { + Kryo k = KryoInstantiator.this.newKryo(); + /** + * Kryo 2.17, used in storm, has this method returning void, 2.21 has it returning boolean. + * Try not to call the method if you don't need to. + */ + if (k.getReferences() != ref) { + k.setReferences(ref); + } + return k; + } + }; + } + + /** If true, Kryo will error if it sees a class that has not been registered */ + public KryoInstantiator setRegistrationRequired(final boolean req) { + return new KryoInstantiator() { + public Kryo newKryo() { + Kryo k = KryoInstantiator.this.newKryo(); + /** + * Try to avoid calling this method if you don't need to. We've been burned by binary + * compatibility with Kryo + */ + if (k.isRegistrationRequired() != req) { + k.setRegistrationRequired(req); + } + return k; + } + }; + } + /** + * Use Thread.currentThread().getContextClassLoader() as the ClassLoader where ther newKryo is + * called + */ + public KryoInstantiator setThreadContextClassLoader() { + return new KryoInstantiator() { + public Kryo newKryo() { + Kryo k = KryoInstantiator.this.newKryo(); + k.setClassLoader(Thread.currentThread().getContextClassLoader()); + return k; + } + }; + } + + public KryoInstantiator withRegistrar(final IKryoRegistrar r) { + return new KryoInstantiator() { + public Kryo newKryo() { + Kryo k = KryoInstantiator.this.newKryo(); + r.apply(k); + return k; + } + }; + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoPool.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoPool.java new file mode 100644 index 0000000000..02c6c9cbad --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/KryoPool.java @@ -0,0 +1,135 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +/** Simple ResourcePool to save on Kryo instances, which are expensive to allocate */ +public abstract class KryoPool extends ResourcePool { + + protected KryoPool(int poolSize) { + super(poolSize); + } + + @Override + public void release(SerDeState st) { + st.clear(); + super.release(st); + } + /** Output is created with new Output(outBufferMin, outBufferMax); */ + public static KryoPool withBuffer( + int poolSize, final KryoInstantiator ki, final int outBufferMin, final int outBufferMax) { + return new KryoPool(poolSize) { + protected SerDeState newInstance() { + return new SerDeState(ki.newKryo(), new Input(), new Output(outBufferMin, outBufferMax)); + } + }; + } + + /** + * Output is created with new Output(new ByteArrayOutputStream()) This will automatically resize + * internally + */ + public static KryoPool withByteArrayOutputStream(int poolSize, final KryoInstantiator ki) { + return new KryoPool(poolSize) { + protected SerDeState newInstance() { + return new SerDeState(ki.newKryo(), new Input(), new Output(new ByteArrayOutputStream())) { + /* + * We have to take extra care of the ByteArrayOutputStream + */ + @Override + public void clear() { + super.clear(); + ByteArrayOutputStream byteStream = (ByteArrayOutputStream) output.getOutputStream(); + byteStream.reset(); + } + + @Override + public byte[] outputToBytes() { + output.flush(); + ByteArrayOutputStream byteStream = (ByteArrayOutputStream) output.getOutputStream(); + return byteStream.toByteArray(); + } + + @Override + public void writeOutputTo(OutputStream os) throws IOException { + output.flush(); + ByteArrayOutputStream byteStream = (ByteArrayOutputStream) output.getOutputStream(); + byteStream.writeTo(os); + } + }; + } + }; + } + + public T deepCopy(T obj) { + return (T) fromBytes(toBytesWithoutClass(obj), obj.getClass()); + } + + public Object fromBytes(byte[] ary) { + SerDeState serde = borrow(); + try { + serde.setInput(ary); + return serde.readClassAndObject(); + } finally { + release(serde); + } + } + + public T fromBytes(byte[] ary, Class cls) { + SerDeState serde = borrow(); + try { + serde.setInput(ary); + return serde.readObject(cls); + } finally { + release(serde); + } + } + + public byte[] toBytesWithClass(Object obj) { + SerDeState serde = borrow(); + try { + serde.writeClassAndObject(obj); + return serde.outputToBytes(); + } finally { + release(serde); + } + } + + public byte[] toBytesWithoutClass(Object obj) { + SerDeState serde = borrow(); + try { + serde.writeObject(obj); + return serde.outputToBytes(); + } finally { + release(serde); + } + } + + public boolean hasRegistration(Class obj) { + SerDeState serde = borrow(); + try { + return serde.hasRegistration(obj); + } finally { + release(serde); + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingDefaultRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingDefaultRegistrar.java new file mode 100644 index 0000000000..0f2d581276 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingDefaultRegistrar.java @@ -0,0 +1,62 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; + +/** Set the default serializers for subclasses of the given class */ +public class ReflectingDefaultRegistrar implements IKryoRegistrar { + final Class klass; + // Some serializers handle any class (FieldsSerializer, for instance) + final Class> serializerKlass; + + public ReflectingDefaultRegistrar(Class cls, Class> ser) { + klass = cls; + serializerKlass = ser; + } + + public Class getRegisteredClass() { + return klass; + } + + public Class> getSerializerClass() { + return serializerKlass; + } + + @Override + public void apply(Kryo k) { + k.addDefaultSerializer(klass, serializerKlass); + } + + @Override + public int hashCode() { + return klass.hashCode() ^ serializerKlass.hashCode(); + } + + @Override + public boolean equals(Object that) { + if (null == that) { + return false; + } else if (that instanceof ReflectingDefaultRegistrar) { + return klass.equals(((ReflectingDefaultRegistrar) that).klass) + && serializerKlass.equals(((ReflectingDefaultRegistrar) that).serializerKlass); + } else { + return false; + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingRegistrar.java new file mode 100644 index 0000000000..480773c2e9 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ReflectingRegistrar.java @@ -0,0 +1,75 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.util.Util; + +/** + * Use reflection to instantiate a serializer. Used when serializer classes are written to config + * files + */ +public class ReflectingRegistrar implements IKryoRegistrar { + final Class klass; + // Some serializers handle any class (FieldsSerializer, for instance) + final Class> serializerKlass; + + public Class getRegisteredClass() { + return klass; + } + + public Class> getSerializerClass() { + return serializerKlass; + } + + public ReflectingRegistrar(Class cls, Class> ser) { + klass = cls; + serializerKlass = ser; + } + + @Override + public void apply(Kryo k) { + try { + k.register(klass, serializerKlass.newInstance()); + } catch (Exception ex) { + throw new IllegalArgumentException( + "Unable to create serializer \"" + + serializerKlass.getName() + + "\" for class: " + + Util.className(klass), + ex); + } + } + + @Override + public int hashCode() { + return klass.hashCode() ^ serializerKlass.hashCode(); + } + + @Override + public boolean equals(Object that) { + if (null == that) { + return false; + } else if (that instanceof ReflectingRegistrar) { + return klass.equals(((ReflectingRegistrar) that).klass) + && serializerKlass.equals(((ReflectingRegistrar) that).serializerKlass); + } else { + return false; + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/ResourcePool.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ResourcePool.java new file mode 100644 index 0000000000..18c0c2e039 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/ResourcePool.java @@ -0,0 +1,51 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import java.util.concurrent.ArrayBlockingQueue; + +/** Simple ResourcePool to save on Kryo instances, which are expensive to allocate */ +public abstract class ResourcePool { + private ArrayBlockingQueue pool; + + protected abstract T newInstance(); + + public ResourcePool(int size) { + pool = new ArrayBlockingQueue(size); + } + + public T borrow() { + try { + T res = pool.poll(); + if (null == res) { + return newInstance(); + } else { + return res; + } + } catch (Exception x) { + throw new RuntimeException(x); + } + } + + public void release(T item) { + try { + pool.offer(item); + } catch (Exception x) { + throw new RuntimeException(x); + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/SerDeState.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/SerDeState.java new file mode 100644 index 0000000000..3b7bdc9e4b --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/SerDeState.java @@ -0,0 +1,97 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * This holds a Kryo Instance, Input and Output so that these objects can be pooled and no + * reallocated on each serialization. + */ +public class SerDeState { + protected final Kryo kryo; + protected final Input input; + protected final Output output; + // To reset the Input + static final byte[] EMPTY_BUFFER = new byte[0]; + + protected SerDeState(Kryo k, Input in, Output out) { + kryo = k; + input = in; + output = out; + } + + /** Call this when to reset the state to the initial state */ + public void clear() { + input.setBuffer(EMPTY_BUFFER); + output.clear(); + } + + public void setInput(byte[] in) { + input.setBuffer(in); + } + + public void setInput(byte[] in, int offset, int count) { + input.setBuffer(in, offset, count); + } + + public void setInput(InputStream in) { + input.setInputStream(in); + } + + public int numOfWrittenBytes() { + return (int) output.total(); + } + + public int numOfReadBytes() { + return (int) input.total(); + } + + // Common operations: + public T readObject(Class cls) { + return kryo.readObject(input, cls); + } + + public Object readClassAndObject() { + return kryo.readClassAndObject(input); + } + + public void writeObject(Object o) { + kryo.writeObject(output, o); + } + + public void writeClassAndObject(Object o) { + kryo.writeClassAndObject(output, o); + } + + public byte[] outputToBytes() { + return output.toBytes(); + } + // There for ByteArrayOutputStream cases this can be optimized + public void writeOutputTo(OutputStream os) throws IOException { + os.write(output.getBuffer(), 0, output.position()); + } + + public boolean hasRegistration(Class obj) { + return kryo.getRegistration(obj) != null; + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleDefaultRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleDefaultRegistrar.java new file mode 100644 index 0000000000..01d2badf24 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleDefaultRegistrar.java @@ -0,0 +1,36 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; + +/** Set the default serializers for subclasses of the given class */ +public class SingleDefaultRegistrar implements IKryoRegistrar { + final Class klass; + final Serializer serializer; + + public SingleDefaultRegistrar(Class cls, Serializer ser) { + klass = cls; + serializer = ser; + } + + @Override + public void apply(Kryo k) { + k.addDefaultSerializer(klass, serializer); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleRegistrar.java new file mode 100644 index 0000000000..c2c18fddaa --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/SingleRegistrar.java @@ -0,0 +1,35 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; + +public class SingleRegistrar implements IKryoRegistrar { + final Class klass; + final Serializer serializer; + + public SingleRegistrar(Class cls, Serializer ser) { + klass = cls; + serializer = ser; + } + + @Override + public void apply(Kryo k) { + k.register(klass, serializer); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/Config.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/Config.java new file mode 100644 index 0000000000..6cbc58d9a3 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/Config.java @@ -0,0 +1,64 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.config; + +/** Particular systems subclass this to reuse existing configured Instantiators */ +public abstract class Config { + /** Return null if this key is undefined */ + public abstract String get(String key); + + public abstract void set(String key, String value); + + public String getOrElse(String key, String def) { + String val = get(key); + if (null == val) { + return def; + } else { + return val; + } + } + + public boolean contains(String key) { + return get(key) != null; + } + + public Boolean getBoolean(String key) { + String bval = get(key); + if (null == bval) { + return null; + } else { + return Boolean.valueOf(bval); + } + } + + public boolean getBoolean(String key, boolean defval) { + String bval = get(key); + if (null == bval) { + return defval; + } else { + return Boolean.valueOf(bval).booleanValue(); + } + } + + public void setBoolean(String key, Boolean v) { + if (null == v) { + set(key, null); + } else { + set(key, v.toString()); + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfigurationException.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfigurationException.java new file mode 100644 index 0000000000..1180256c31 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfigurationException.java @@ -0,0 +1,31 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.config; + +public class ConfigurationException extends Exception { + public ConfigurationException(String message) { + super(message); + } + + public ConfigurationException(Exception ex) { + super(ex); + } + + public ConfigurationException(String message, Exception ex) { + super(message, ex); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfiguredInstantiator.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfiguredInstantiator.java new file mode 100644 index 0000000000..d99db57641 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ConfiguredInstantiator.java @@ -0,0 +1,198 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.config; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.Base64; +import com.spotify.scio.vendor.chill.KryoInstantiator; +import java.lang.reflect.InvocationTargetException; + +/** + * This is the standard Config based KryoInstantiator. It delegates to another KryoInstantiator that + * is described a Config object. This is either done via reflection or reflection AND serialization. + * + *

If the KEY is not set, the delegate is the default: new KryoInstantiator() + * + *

In the case of reflection, the class name of the delegate instantiator is given. + * + *

In the case of serialization, we first reflect to create the KryoInstatiator we use to get the + * Kryo we need to deserialize. + */ +public class ConfiguredInstantiator extends KryoInstantiator { + protected final KryoInstantiator delegate; + + /** + * Key we use to configure this class. Format: {@literal(:)} if there is no serialized instantiator, we use the reflected + * instance as the delegate + */ + public static final String KEY = "com.spotify.scio.vendor.chill.config.configuredinstantiator"; + + public ConfiguredInstantiator(Config conf) throws ConfigurationException { + String key = conf.get(KEY); + if (null == key) { + delegate = new KryoInstantiator(); + } else { + String[] parts = fastSplitKey(key); + if (parts == null) { + throw new ConfigurationException("Invalid Config Key: " + conf.get(KEY)); + } + KryoInstantiator reflected = null; + try { + reflected = + reflect( + (Class) + Class.forName(parts[0], true, Thread.currentThread().getContextClassLoader()), + conf); + } catch (ClassNotFoundException x) { + throw new ConfigurationException("Could not find class for: " + parts[0], x); + } + + if (parts.length == 2) { + delegate = fastDeserialize(reflected.newKryo(), parts[1]); + if (null == delegate) { + throw new ConfigurationException("Null delegate from: " + parts[1]); + } + } else { + delegate = reflected; + } + } + } + + /** Calls through to the delegate */ + public Kryo newKryo() { + return delegate.newKryo(); + } + + /** Return the delegated KryoInstantiator */ + public KryoInstantiator getDelegate() { + return delegate; + } + + /** + * In this mode, we are just refecting to another delegated class. This is preferred if you don't + * have any configuration to do at runtime (i.e. you can make a named class that has all the logic + * for your KryoInstantiator). + */ + public static void setReflect(Config conf, Class instClass) { + conf.set(KEY, instClass.getName()); + } + + /** + * This instantiates a KryoInstantiator by: 1) checking if it has a constructor that takes Config + * 2) checking for a no-arg constructor + */ + static KryoInstantiator reflect(Class instClass, Config optConf) + throws ConfigurationException { + try { + try { + return instClass.getConstructor(Config.class).newInstance(optConf); + } catch (NoSuchMethodException ex3) { + return instClass.newInstance(); + } + } catch (InstantiationException x) { + throw new ConfigurationException(x); + } catch (IllegalAccessException x) { + throw new ConfigurationException(x); + } catch (InvocationTargetException x) { + throw new ConfigurationException(x); + } + } + + /** + * Use the default KryoInstantiator to serialize the KryoInstantiator ki same as: + * setSerialized(conf, KryoInstantiator.class, ki) + */ + public static void setSerialized(Config conf, KryoInstantiator ki) throws ConfigurationException { + setSerialized(conf, KryoInstantiator.class, ki); + } + + /** + * If this reflector needs config to be set, that should be done PRIOR to making this call. This + * mode serializes an instance (ki) to be used as the delegate. Only use this mode if reflection + * alone will not work. + */ + public static void setSerialized( + Config conf, Class reflector, KryoInstantiator ki) + throws ConfigurationException { + KryoInstantiator refki = reflect(reflector, conf); + String kistr = serialize(refki.newKryo(), ki); + // Verify, that deserialization works: + KryoInstantiator deser = + deserialize(refki.newKryo(), kistr); // ignore the result, just see if it throws + deser.newKryo(); // just see if we can still create it + conf.set(KEY, reflector.getName() + ":" + kistr); + } + + protected static KryoInstantiator deserialize(Kryo k, String base64Value) + throws ConfigurationException { + try { + return (KryoInstantiator) k.readClassAndObject(new Input(Base64.decode(base64Value))); + } catch (java.io.IOException iox) { + throw new ConfigurationException("could not deserialize: " + base64Value, iox); + } + } + + protected static String serialize(Kryo k, KryoInstantiator ki) { + Output out = new Output(1 << 10, 1 << 19); // 1 MB in config is too much + k.writeClassAndObject(out, ki); + return Base64.encodeBytes(out.toBytes()); + } + + /** + * Simple class to hold the cached copy of the latest kryo instantiator. As well as its + * corresponding base64 encoded data. + */ + private static class CachedKryoInstantiator { + public final KryoInstantiator kryoInstantiator; + public final String base64Value; + + public CachedKryoInstantiator(KryoInstantiator ki, String bv) { + kryoInstantiator = ki; + base64Value = bv; + } + } + + private static CachedKryoInstantiator cachedKryoInstantiator = null; + + private static synchronized KryoInstantiator fastDeserialize(Kryo k, String base64Value) + throws ConfigurationException { + if (cachedKryoInstantiator == null || !cachedKryoInstantiator.base64Value.equals(base64Value)) { + cachedKryoInstantiator = new CachedKryoInstantiator(deserialize(k, base64Value), base64Value); + } + return cachedKryoInstantiator.kryoInstantiator; + } + + /** + * Java's string split is very expensive due to regexes. Implement our own simple version instead. + */ + public static String[] fastSplitKey(String key) { + int i = key.indexOf(':'); + if (-1 == i) { + return new String[] {key}; + } else { + int j = key.indexOf(':', i + 1); + if (-1 != j) { + return null; + } else { + return new String[] {key.substring(0, i), key.substring(i + 1)}; + } + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/JavaMapConfig.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/JavaMapConfig.java new file mode 100644 index 0000000000..1c88f4b33d --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/JavaMapConfig.java @@ -0,0 +1,49 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.config; + +import java.util.Map; + +/** This takes a raw Map and calls toString on the objects before returning them as values */ +public class JavaMapConfig extends Config { + + final Map conf; + + public JavaMapConfig(Map conf) { + this.conf = conf; + } + + public JavaMapConfig() { + this(new java.util.HashMap()); + } + /** Return null if this key is undefined */ + /** Return null if this key is undefined */ + @Override + public String get(String key) { + Object value = conf.get(key); + if (null != value) { + return value.toString(); + } else { + return null; + } + } + + @Override + public void set(String key, String value) { + conf.put(key, value); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ReflectingInstantiator.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ReflectingInstantiator.java new file mode 100644 index 0000000000..16b5cc461c --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/config/ReflectingInstantiator.java @@ -0,0 +1,302 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.config; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.spotify.scio.vendor.chill.ClassRegistrar; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.KryoInstantiator; +import com.spotify.scio.vendor.chill.ReflectingDefaultRegistrar; +import com.spotify.scio.vendor.chill.ReflectingRegistrar; +import java.util.ArrayList; +import java.util.List; +import org.objenesis.strategy.InstantiatorStrategy; +import org.objenesis.strategy.StdInstantiatorStrategy; + +public class ReflectingInstantiator extends KryoInstantiator { + + final boolean regRequired; + final boolean skipMissing; + final Class kryoClass; + final Class instStratClass; + final List registrations; + final List defaultRegistrations; + + public ReflectingInstantiator(Config conf) throws ConfigurationException { + regRequired = conf.getBoolean(REGISTRATION_REQUIRED, REGISTRATION_REQUIRED_DEFAULT); + skipMissing = conf.getBoolean(SKIP_MISSING, SKIP_MISSING_DEFAULT); + + try { + kryoClass = + (Class) + Class.forName( + conf.getOrElse(KRYO_CLASS, KRYO_CLASS_DEFAULT), + true, + Thread.currentThread().getContextClassLoader()); + instStratClass = + (Class) + Class.forName( + conf.getOrElse(INSTANTIATOR_STRATEGY_CLASS, INSTANTIATOR_STRATEGY_CLASS_DEFAULT), + true, + Thread.currentThread().getContextClassLoader()); + + registrations = (List) buildRegistrars(conf.get(REGISTRATIONS), false); + defaultRegistrations = + (List) buildRegistrars(conf.get(DEFAULT_REGISTRATIONS), true); + // Make sure we can make a newKryo, this throws a runtime exception if not. + newKryoWithEx(); + } catch (ClassNotFoundException x) { + throw new ConfigurationException(x); + } catch (InstantiationException x) { + throw new ConfigurationException(x); + } catch (IllegalAccessException x) { + throw new ConfigurationException(x); + } + } + + /** Create an instance using the defaults for non-listed params */ + public ReflectingInstantiator( + Iterable classRegistrations, + Iterable registrations, + Iterable defaults) { + this( + Kryo.class, + StdInstantiatorStrategy.class, + classRegistrations, + registrations, + defaults, + REGISTRATION_REQUIRED_DEFAULT, + SKIP_MISSING_DEFAULT); + } + + public ReflectingInstantiator( + Class kryoClass, + Class stratClass, + Iterable classRegistrations, + Iterable registrations, + Iterable defaults, + boolean regRequired, + boolean skipMissing) { + + this.kryoClass = kryoClass; + instStratClass = stratClass; + this.regRequired = regRequired; + this.skipMissing = skipMissing; + + this.registrations = new ArrayList(); + for (IKryoRegistrar cr : classRegistrations) { + this.registrations.add(cr); + } + for (IKryoRegistrar rr : registrations) { + this.registrations.add(rr); + } + + defaultRegistrations = new ArrayList(); + for (ReflectingDefaultRegistrar rdr : defaults) { + defaultRegistrations.add(rdr); + } + } + + public void set(Config conf) throws ConfigurationException { + conf.setBoolean(REGISTRATION_REQUIRED, regRequired); + conf.setBoolean(SKIP_MISSING, skipMissing); + + conf.set(KRYO_CLASS, kryoClass.getName()); + conf.set(INSTANTIATOR_STRATEGY_CLASS, instStratClass.getName()); + + conf.set(REGISTRATIONS, registrarsToString(registrations)); + conf.set(DEFAULT_REGISTRATIONS, registrarsToString(defaultRegistrations)); + } + + // This one adds expeption annotations that the interface does not have + protected Kryo newKryoWithEx() throws InstantiationException, IllegalAccessException { + Kryo k = kryoClass.newInstance(); + k.setInstantiatorStrategy(instStratClass.newInstance()); + k.setRegistrationRequired(regRequired); + for (IKryoRegistrar kr : registrations) { + kr.apply(k); + } + for (IKryoRegistrar dkr : defaultRegistrations) { + dkr.apply(k); + } + return k; + } + + @Override + public Kryo newKryo() { + try { + return newKryoWithEx(); + } catch (InstantiationException x) { + throw new RuntimeException(x); + } catch (IllegalAccessException x) { + throw new RuntimeException(x); + } + } + + /** All keys are prefixed with this string */ + public static final String prefix = "com.spotify.scio.vendor.chill.config.reflectinginstantiator"; + + /** + * Name of the subclass of kryo to instantiate to start with. If this is empty, we use Kryo.class + */ + public static final String KRYO_CLASS = prefix + ".kryoclass"; + + public static final String KRYO_CLASS_DEFAULT = Kryo.class.getName(); + /** + * Name of the InstatiatorStrategy to use. If this is empty, we use + * org.objenesis.strategy.StdInstantiatorStrategy + */ + public static final String INSTANTIATOR_STRATEGY_CLASS = prefix + ".instantiatorstrategyclass"; + + public static final String INSTANTIATOR_STRATEGY_CLASS_DEFAULT = + StdInstantiatorStrategy.class.getName(); + /** + * KRYO_REGISTRATIONS holds a colon-separated list of classes to register with Kryo. For example, + * the following value: + * + *

"someClass,someSerializer:otherClass:thirdClass,thirdSerializer" + * + *

will direct KryoFactory to register someClass and thirdClass with custom serializers and + * otherClass with Kryo's FieldsSerializer. + */ + public static final String REGISTRATIONS = prefix + ".registrations"; + + /** + * DEFAULT_REGISTRATIONS holds a colon-separated list of classes or interfaces to register with + * Kryo. Default Registrations are searched after basic registrations, and have the ability to + * capture objects that are assignable from the hierarchy's superclass. For example, the following + * value: + * + *

"someClass,someSerializer:someInterface,otherSerializer" + * + *

will configure to serializeobjects that extend from someClass with someSerializer, and + * objects that extend someInterface with otherSerializer. + */ + public static final String DEFAULT_REGISTRATIONS = prefix + ".defaultregistrations"; + + /** + * If SKIP_MISSING is set to false, Kryo will throw an error when Cascading tries to register a + * class or serialization that doesn't exist. + */ + public static final String SKIP_MISSING = prefix + ".skipmissing"; + + public static final boolean SKIP_MISSING_DEFAULT = false; + + /** + * If REGISTRATION_REQUIRED is set to false, Kryo will try to serialize all java objects, not just + * those with custom serializations registered. + */ + public static final String REGISTRATION_REQUIRED = prefix + ".registrationrequired"; + + public static final boolean REGISTRATION_REQUIRED_DEFAULT = false; + + protected List buildRegistrars(String base, boolean isAddDefault) + throws ConfigurationException { + List builder = new ArrayList(); + + if (base == null) return builder; + + for (String s : base.split(":")) { + String[] pair = s.split(","); + try { + switch (pair.length) { + case 1: + if (isAddDefault) { + throw new ConfigurationException( + "default serializers require class and serializer: " + base); + } + builder.add( + new ClassRegistrar( + Class.forName(pair[0], true, Thread.currentThread().getContextClassLoader()))); + break; + case 2: + @SuppressWarnings("unchecked") + Class kls = + Class.forName(pair[0], true, Thread.currentThread().getContextClassLoader()); + Class serializerClass = + (Class) + Class.forName(pair[1], true, Thread.currentThread().getContextClassLoader()); + if (isAddDefault) { + builder.add(new ReflectingDefaultRegistrar(kls, serializerClass)); + } else { + builder.add(new ReflectingRegistrar(kls, serializerClass)); + } + break; + default: + throw new ConfigurationException(base + " is not well-formed."); + } + } catch (ClassNotFoundException e) { + if (skipMissing) { + System.err.println( + "Could not find serialization or class for " + pair[1] + ". Skipping registration."); + } else { + throw new ConfigurationException(e); + } + } + } + return builder; + } + + protected String registrarsToString(Iterable registrars) + throws ConfigurationException { + StringBuilder builder = new StringBuilder(); + boolean isFirst = true; + for (IKryoRegistrar reg : registrars) { + if (!isFirst) builder.append(":"); + isFirst = false; + String part = null; + if (reg instanceof ClassRegistrar) { + ClassRegistrar r = (ClassRegistrar) reg; + part = r.getRegisteredClass().getName(); + } else if (reg instanceof ReflectingRegistrar) { + ReflectingRegistrar r = (ReflectingRegistrar) reg; + part = r.getRegisteredClass().getName() + "," + r.getSerializerClass().getName(); + } else if (reg instanceof ReflectingDefaultRegistrar) { + ReflectingDefaultRegistrar r = (ReflectingDefaultRegistrar) reg; + part = r.getRegisteredClass().getName() + "," + r.getSerializerClass().getName(); + } else { + throw new ConfigurationException( + "Unknown type of reflecting registrar: " + reg.getClass().getName()); + } + builder.append(part); + } + return builder.toString(); + } + + @Override + public int hashCode() { + return kryoClass.hashCode() ^ registrations.hashCode() ^ defaultRegistrations.hashCode(); + } + + @Override + public boolean equals(Object that) { + if (null == that) { + return false; + } else if (that instanceof ReflectingInstantiator) { + ReflectingInstantiator thatri = (ReflectingInstantiator) that; + return (regRequired == thatri.regRequired) + && (skipMissing == thatri.skipMissing) + && kryoClass.equals(thatri.kryoClass) + && instStratClass.equals(thatri.instStratClass) + && registrations.equals(thatri.registrations) + && defaultRegistrations.equals(thatri.defaultRegistrations); + } else { + return false; + } + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/ArraysAsListSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/ArraysAsListSerializer.java new file mode 100644 index 0000000000..cf3b79508a --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/ArraysAsListSerializer.java @@ -0,0 +1,113 @@ +/* + * Copyright 2010 Martin Grotzke + * + * 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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.lang.reflect.Array; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * A kryo {@link Serializer} for lists created via {@link Arrays#asList(Object...)}. + * + *

Note: This serializer does not support cyclic references, so if one of the objects gets set + * the list as attribute this might cause an error during deserialization. + * + * @author Martin Grotzke + */ +public class ArraysAsListSerializer extends Serializer> { + + private static final Map, Class> primitives = + new HashMap, Class>(8, 1.0F); + + static { + primitives.put(byte.class, Byte.class); + primitives.put(short.class, Short.class); + primitives.put(int.class, Integer.class); + primitives.put(long.class, Long.class); + primitives.put(char.class, Character.class); + primitives.put(float.class, Float.class); + primitives.put(double.class, Double.class); + primitives.put(boolean.class, Boolean.class); + } + + public static IKryoRegistrar registrar() { + return new SingleRegistrar(Arrays.asList("").getClass(), new ArraysAsListSerializer()); + } + + private Field _arrayField; + + public ArraysAsListSerializer() { + try { + _arrayField = Class.forName("java.util.Arrays$ArrayList").getDeclaredField("a"); + _arrayField.setAccessible(true); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public List read(final Kryo kryo, final Input input, final Class> type) { + final int length = input.readInt(true); + Class componentType = kryo.readClass(input).getType(); + try { + final Object items = Array.newInstance(getBoxedClass(componentType), length); + for (int i = 0; i < length; i++) { + Array.set(items, i, kryo.readClassAndObject(input)); + } + return Arrays.asList((Object[]) items); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void write(final Kryo kryo, final Output output, final List obj) { + try { + final Object[] array = (Object[]) _arrayField.get(obj); + output.writeInt(array.length, true); + final Class componentType = array.getClass().getComponentType(); + kryo.writeClass(output, componentType); + for (final Object item : array) { + kryo.writeClassAndObject(output, item); + } + } catch (final RuntimeException e) { + // Don't eat and wrap RuntimeExceptions because the ObjectBuffer.write... + // handles SerializationException specifically (resizing the buffer)... + throw e; + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + private static Class getBoxedClass(final Class c) { + if (c.isPrimitive()) { + Class x; + return (x = primitives.get(c)) != null ? x : c; + } + return c; + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/BitSetSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/BitSetSerializer.java new file mode 100644 index 0000000000..0e3b13e7a9 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/BitSetSerializer.java @@ -0,0 +1,117 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoException; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.io.Serializable; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.BitSet; + +public class BitSetSerializer extends Serializer implements Serializable { + + public static IKryoRegistrar registrar() { + return new SingleRegistrar(BitSet.class, new BitSetSerializer()); + } + + private static final Field wordsField; + private static final Constructor bitSetConstructor; + private static final Method recalculateWordsInUseMethod; + + static { + try { + wordsField = BitSet.class.getDeclaredField("words"); + wordsField.setAccessible(true); + } catch (NoSuchFieldException e) { + throw new KryoException("Error while getting field 'words' of bitSet", e); + } + try { + bitSetConstructor = BitSet.class.getDeclaredConstructor(long[].class); + bitSetConstructor.setAccessible(true); + } catch (NoSuchMethodException e) { + throw new KryoException("Unable to get BitSet(long[]) constructor", e); + } + try { + recalculateWordsInUseMethod = BitSet.class.getDeclaredMethod("recalculateWordsInUse"); + recalculateWordsInUseMethod.setAccessible(true); + } catch (NoSuchMethodException e) { + throw new KryoException("Unable to get BitSet.recalculateWordsInUse() method", e); + } + } + + @Override + public void write(Kryo kryo, Output output, BitSet bitSet) { + long words[] = null; + // its sufficent to get only the 'words' field because + // we can recompute the wordsInUse after deserialization + try { + words = (long[]) wordsField.get(bitSet); + } catch (IllegalAccessException e) { + throw new KryoException("Error while accessing field 'words' of bitSet", e); + } + output.writeInt(words.length, true); + + for (int i = 0; i < words.length; i++) { + output.writeLong(words[i]); + } + } + + @Override + public BitSet read(Kryo kryo, Input input, Class bitSetClass) { + int len = input.readInt(true); + long[] target = new long[len]; + + for (int i = 0; i < len; i++) { + target[i] = input.readLong(); + } + + BitSet ret = null; + + try { + ret = bitSetConstructor.newInstance(target); + } catch (InstantiationException e) { + throw new KryoException("Exception thrown while creating new instance BitSetConstructor", e); + } catch (IllegalAccessException e) { + throw new KryoException( + "Exception thrown while creating new instance of BitSetConstructor", e); + } catch (InvocationTargetException e) { + throw new KryoException( + "Exception thrown while creating new instance of BitSetConstructor", e); + } catch (IllegalArgumentException e) { + throw new KryoException( + "Exception thrown while creating new instance of BitSetConstructor", e); + } + try { + recalculateWordsInUseMethod.invoke(ret); + } catch (InvocationTargetException e) { + throw new KryoException("Exception thrown while invoking recalculateWordsInUseMethod", e); + } catch (IllegalAccessException e) { + throw new KryoException("Exception thrown while invoking recalculateWordsInUseMethod", e); + } catch (IllegalArgumentException e) { + throw new KryoException("Exception thrown while invoking recalculateWordsInUseMethod", e); + } + return ret; + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/InetSocketAddressSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/InetSocketAddressSerializer.java new file mode 100644 index 0000000000..405d79cf37 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/InetSocketAddressSerializer.java @@ -0,0 +1,45 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.net.InetSocketAddress; + +public class InetSocketAddressSerializer extends Serializer { + + public static IKryoRegistrar registrar() { + return new SingleRegistrar(InetSocketAddress.class, new InetSocketAddressSerializer()); + } + + @Override + public void write(Kryo kryo, Output output, InetSocketAddress obj) { + output.writeString(obj.getHostName()); + output.writeInt(obj.getPort(), true); + } + + @Override + public InetSocketAddress read(Kryo kryo, Input input, Class klass) { + String host = input.readString(); + int port = input.readInt(true); + return new InetSocketAddress(host, port); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrar.java new file mode 100644 index 0000000000..e1fee2682a --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrar.java @@ -0,0 +1,54 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.DefaultSerializer; +import com.esotericsoftware.kryo.Kryo; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import java.util.Arrays; + +/** + * A Registrar adds registrations to a given Kryo instance. Examples would be a registrar that + * registers serializers for all objects in a given package. comes from Storm, which took it from + * cascading.kryo + */ +@DefaultSerializer(IterableRegistrarSerializer.class) +public class IterableRegistrar implements IKryoRegistrar { + private final Iterable registrarList; + + public IterableRegistrar(IKryoRegistrar... registrars) { + this(Arrays.asList(registrars)); + } + + public IterableRegistrar(Iterable registrars) { + registrarList = registrars; + for (IKryoRegistrar kr : registrarList) { + if (null == kr) throw new IllegalArgumentException("null Registrars not allowed"); + } + } + + @Override + public void apply(Kryo k) { + for (IKryoRegistrar kr : registrarList) { + kr.apply(k); + } + } + + public Iterable getRegistrars() { + return registrarList; + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrarSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrarSerializer.java new file mode 100644 index 0000000000..eae1d990f8 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/IterableRegistrarSerializer.java @@ -0,0 +1,44 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import java.util.ArrayList; + +public class IterableRegistrarSerializer extends Serializer { + public void write(Kryo kryo, Output output, IterableRegistrar obj) { + for (IKryoRegistrar kr : obj.getRegistrars()) { + kryo.writeClassAndObject(output, kr); + output.flush(); + } + kryo.writeClassAndObject(output, null); + } + + public IterableRegistrar read(Kryo kryo, Input input, Class type) { + ArrayList krs = new ArrayList(); + IKryoRegistrar thisKr = (IKryoRegistrar) kryo.readClassAndObject(input); + while (thisKr != null) { + krs.add(thisKr); + thisKr = (IKryoRegistrar) kryo.readClassAndObject(input); + } + return new IterableRegistrar(krs); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/Java8ClosureRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/Java8ClosureRegistrar.java new file mode 100644 index 0000000000..27e020134a --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/Java8ClosureRegistrar.java @@ -0,0 +1,36 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.serializers.ClosureSerializer; +import com.spotify.scio.vendor.chill.IKryoRegistrar; + +/** Enables Java 8 lambda serialization if running on Java 8; no-op otherwise. */ +public class Java8ClosureRegistrar implements IKryoRegistrar { + + @Override + public void apply(Kryo k) { + try { + Class.forName("java.lang.invoke.SerializedLambda"); + } catch (ClassNotFoundException e) { + // Not running on Java 8. + return; + } + k.register(ClosureSerializer.Closure.class, new ClosureSerializer()); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/LocaleSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/LocaleSerializer.java new file mode 100644 index 0000000000..c5cab889f0 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/LocaleSerializer.java @@ -0,0 +1,32 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.util.Locale; + +/** + * The java serializer uses an cache of allocated instances so it is probably a bit hard to beat, so + * why bother + */ +public class LocaleSerializer extends JavaSerializer { + public static IKryoRegistrar registrar() { + return new SingleRegistrar(Locale.class, new LocaleSerializer()); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PackageRegistrar.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PackageRegistrar.java new file mode 100644 index 0000000000..633111b5cf --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PackageRegistrar.java @@ -0,0 +1,45 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.spotify.scio.vendor.chill.IKryoRegistrar; + +/** Creates a registrar for all the serializers in the chill.java package */ +public class PackageRegistrar { + + public static IKryoRegistrar all() { + return new IterableRegistrar( + ArraysAsListSerializer.registrar(), + BitSetSerializer.registrar(), + PriorityQueueSerializer.registrar(), + RegexSerializer.registrar(), + SqlDateSerializer.registrar(), + SqlTimeSerializer.registrar(), + TimestampSerializer.registrar(), + URISerializer.registrar(), + InetSocketAddressSerializer.registrar(), + UUIDSerializer.registrar(), + LocaleSerializer.registrar(), + SimpleDateFormatSerializer.registrar(), + UnmodifiableCollectionSerializer.registrar(), + UnmodifiableListSerializer.registrar(), + UnmodifiableMapSerializer.registrar(), + UnmodifiableSetSerializer.registrar(), + UnmodifiableSortedMapSerializer.registrar(), + UnmodifiableSortedSetSerializer.registrar()); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PriorityQueueSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PriorityQueueSerializer.java new file mode 100644 index 0000000000..178ec76b31 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/PriorityQueueSerializer.java @@ -0,0 +1,79 @@ +/* +Copyright 2012 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.lang.reflect.Field; +import java.util.Comparator; +import java.util.PriorityQueue; + +class PriorityQueueSerializer extends Serializer> { + private Field compField; + + public static IKryoRegistrar registrar() { + return new SingleRegistrar(PriorityQueue.class, new PriorityQueueSerializer()); + } + + public PriorityQueueSerializer() { + try { + compField = PriorityQueue.class.getDeclaredField("comparator"); + compField.setAccessible(true); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public Comparator getComparator(PriorityQueue q) { + try { + return (Comparator) compField.get(q); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void write(Kryo k, Output o, PriorityQueue q) { + k.writeClassAndObject(o, getComparator(q)); + o.writeInt(q.size(), true); + for (Object a : q) { + k.writeClassAndObject(o, a); + o.flush(); + } + } + + public PriorityQueue read(Kryo k, Input i, Class> c) { + Comparator comp = (Comparator) k.readClassAndObject(i); + int sz = i.readInt(true); + // can't create with size 0: + PriorityQueue result; + if (sz == 0) { + result = new PriorityQueue(1, comp); + } else { + result = new PriorityQueue(sz, comp); + } + int idx = 0; + while (idx < sz) { + result.add(k.readClassAndObject(i)); + idx += 1; + } + return result; + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/RegexSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/RegexSerializer.java new file mode 100644 index 0000000000..388779e2ab --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/RegexSerializer.java @@ -0,0 +1,42 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.util.regex.Pattern; + +public class RegexSerializer extends Serializer { + + public static IKryoRegistrar registrar() { + return new SingleRegistrar(Pattern.class, new RegexSerializer()); + } + + @Override + public void write(Kryo kryo, Output output, Pattern pattern) { + output.writeString(pattern.pattern()); + } + + @Override + public Pattern read(Kryo kryo, Input input, Class patternClass) { + return Pattern.compile(input.readString()); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SimpleDateFormatSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SimpleDateFormatSerializer.java new file mode 100644 index 0000000000..a800fb20f4 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SimpleDateFormatSerializer.java @@ -0,0 +1,32 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.text.SimpleDateFormat; + +/** + * This class fails with the Fields serializer. If it is a perf bottleneck, we could write a Kryo + * serializer + */ +public class SimpleDateFormatSerializer extends JavaSerializer { + public static IKryoRegistrar registrar() { + return new SingleRegistrar(SimpleDateFormat.class, new SimpleDateFormatSerializer()); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlDateSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlDateSerializer.java new file mode 100644 index 0000000000..c4824c4d96 --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlDateSerializer.java @@ -0,0 +1,42 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.sql.Date; + +public class SqlDateSerializer extends Serializer { + + public static IKryoRegistrar registrar() { + return new SingleRegistrar(Date.class, new SqlDateSerializer()); + } + + @Override + public void write(Kryo kryo, Output output, Date date) { + output.writeLong(date.getTime(), true); + } + + @Override + public Date read(Kryo kryo, Input input, Class dateClass) { + return new Date(input.readLong(true)); + } +} diff --git a/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlTimeSerializer.java b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlTimeSerializer.java new file mode 100644 index 0000000000..290c35d6db --- /dev/null +++ b/scio-core/src/main/java/com/spotify/scio/vendor/chill/java/SqlTimeSerializer.java @@ -0,0 +1,42 @@ +/* +Copyright 2013 Twitter, Inc. + +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.vendor.chill.java; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.spotify.scio.vendor.chill.IKryoRegistrar; +import com.spotify.scio.vendor.chill.SingleRegistrar; +import java.sql.Time; + +public class SqlTimeSerializer extends Serializer