/
SCollectionSyntax.scala
231 lines (213 loc) · 8.39 KB
/
SCollectionSyntax.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
/*
* Copyright 2019 Spotify AB.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package com.spotify.scio.avro.syntax
import com.google.protobuf.Message
import com.spotify.scio.avro._
import com.spotify.scio.avro.types.AvroType.HasAvroAnnotation
import com.spotify.scio.coders.Coder
import com.spotify.scio.io.ClosedTap
import com.spotify.scio.util.FilenamePolicySupplier
import com.spotify.scio.values._
import org.apache.avro.Schema
import org.apache.avro.file.CodecFactory
import org.apache.avro.specific.SpecificRecord
import org.apache.avro.generic.GenericRecord
import org.apache.beam.sdk.extensions.avro.io.AvroDatumFactory
import scala.reflect.ClassTag
import scala.reflect.runtime.universe._
final class GenericRecordSCollectionOps(private val self: SCollection[GenericRecord])
extends AnyVal {
/**
* Save this SCollection of type [[org.apache.avro.generic.GenericRecord GenericRecord]] as an
* Avro file.
*/
def saveAsAvroFile(
path: String,
schema: Schema,
numShards: Int = GenericRecordIO.WriteParam.DefaultNumShards,
suffix: String = GenericRecordIO.WriteParam.DefaultSuffix,
codec: CodecFactory = GenericRecordIO.WriteParam.DefaultCodec,
metadata: Map[String, AnyRef] = GenericRecordIO.WriteParam.DefaultMetadata,
shardNameTemplate: String = GenericRecordIO.WriteParam.DefaultShardNameTemplate,
tempDirectory: String = GenericRecordIO.WriteParam.DefaultTempDirectory,
filenamePolicySupplier: FilenamePolicySupplier =
GenericRecordIO.WriteParam.DefaultFilenamePolicySupplier,
prefix: String = GenericRecordIO.WriteParam.DefaultPrefix,
datumFactory: AvroDatumFactory[GenericRecord] = GenericRecordIO.WriteParam.DefaultDatumFactory
): ClosedTap[GenericRecord] = {
val param = GenericRecordIO.WriteParam(
numShards,
suffix,
codec,
metadata,
filenamePolicySupplier,
prefix,
shardNameTemplate,
tempDirectory,
datumFactory
)
self.write(GenericRecordIO(path, schema))(param)
}
}
final class ObjectFileSCollectionOps[T](private val self: SCollection[T]) extends AnyVal {
/**
* Save this SCollection as an object file using default serialization.
*
* Serialized objects are stored in Avro files to leverage Avro's block file format. Note that
* serialization is not guaranteed to be compatible across Scio releases.
*/
def saveAsObjectFile(
path: String,
numShards: Int = ObjectFileIO.WriteParam.DefaultNumShards,
suffix: String = ObjectFileIO.WriteParam.DefaultSuffixObjectFile,
codec: CodecFactory = ObjectFileIO.WriteParam.DefaultCodec,
metadata: Map[String, AnyRef] = ObjectFileIO.WriteParam.DefaultMetadata,
shardNameTemplate: String = ObjectFileIO.WriteParam.DefaultShardNameTemplate,
tempDirectory: String = ObjectFileIO.WriteParam.DefaultTempDirectory,
filenamePolicySupplier: FilenamePolicySupplier =
ObjectFileIO.WriteParam.DefaultFilenamePolicySupplier,
prefix: String = ObjectFileIO.WriteParam.DefaultPrefix
)(implicit coder: Coder[T]): ClosedTap[T] = {
val param = ObjectFileIO.WriteParam[GenericRecord](
numShards,
suffix,
codec,
metadata,
filenamePolicySupplier,
prefix,
shardNameTemplate,
tempDirectory
)
self.write(ObjectFileIO[T](path))(param)
}
}
final class SpecificRecordSCollectionOps[T <: SpecificRecord](private val self: SCollection[T])
extends AnyVal {
/**
* Save this SCollection of type [[org.apache.avro.specific.SpecificRecord SpecificRecord]] as an
* Avro file.
*/
def saveAsAvroFile(
path: String,
numShards: Int = SpecificRecordIO.WriteParam.DefaultNumShards,
suffix: String = SpecificRecordIO.WriteParam.DefaultSuffix,
codec: CodecFactory = SpecificRecordIO.WriteParam.DefaultCodec,
metadata: Map[String, AnyRef] = SpecificRecordIO.WriteParam.DefaultMetadata,
shardNameTemplate: String = SpecificRecordIO.WriteParam.DefaultShardNameTemplate,
tempDirectory: String = SpecificRecordIO.WriteParam.DefaultTempDirectory,
filenamePolicySupplier: FilenamePolicySupplier =
SpecificRecordIO.WriteParam.DefaultFilenamePolicySupplier,
prefix: String = SpecificRecordIO.WriteParam.DefaultPrefix,
datumFactory: AvroDatumFactory[T] = SpecificRecordIO.WriteParam.DefaultDatumFactory
)(implicit ct: ClassTag[T]): ClosedTap[T] = {
val param = SpecificRecordIO.WriteParam(
numShards,
suffix,
codec,
metadata,
filenamePolicySupplier,
prefix,
shardNameTemplate,
tempDirectory,
datumFactory
)
self.write(SpecificRecordIO[T](path))(param)
}
}
final class TypedAvroSCollectionOps[T <: HasAvroAnnotation](private val self: SCollection[T])
extends AnyVal {
/**
* Save this SCollection as an Avro file. Note that element type `T` must be a case class
* annotated with [[com.spotify.scio.avro.types.AvroType AvroType.toSchema]].
*/
def saveAsTypedAvroFile(
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 tt: TypeTag[T], coder: Coder[T]): ClosedTap[T] = {
val param = AvroTypedIO.WriteParam(
numShards,
suffix,
codec,
metadata,
filenamePolicySupplier,
prefix,
shardNameTemplate,
tempDirectory,
datumFactory
)
self.write(AvroTypedIO[T](path))(param)
}
}
final class ProtobufSCollectionOps[T <: Message](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(
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,
filenamePolicySupplier: FilenamePolicySupplier =
ProtobufIO.WriteParam.DefaultFilenamePolicySupplier,
prefix: String = ProtobufIO.WriteParam.DefaultPrefix
)(implicit ct: ClassTag[T]): ClosedTap[T] = {
val param = ProtobufIO.WriteParam[GenericRecord](
numShards,
suffix,
codec,
metadata,
filenamePolicySupplier,
prefix,
shardNameTemplate,
tempDirectory
)
self.write(ProtobufIO[T](path))(param)
}
}
/** Enhanced with Avro methods. */
trait SCollectionSyntax {
implicit def avroGenericRecordSCollectionOps(
c: SCollection[GenericRecord]
): GenericRecordSCollectionOps = new GenericRecordSCollectionOps(c)
implicit def avroObjectFileSCollectionOps[T](
c: SCollection[T]
): ObjectFileSCollectionOps[T] = new ObjectFileSCollectionOps[T](c)
implicit def avroSpecificRecordSCollectionOps[T <: SpecificRecord](
c: SCollection[T]
): SpecificRecordSCollectionOps[T] = new SpecificRecordSCollectionOps[T](c)
implicit def avroTypedAvroSCollectionOps[T <: HasAvroAnnotation](
c: SCollection[T]
): TypedAvroSCollectionOps[T] = new TypedAvroSCollectionOps[T](c)
implicit def avroProtobufSCollectionOps[T <: Message](
c: SCollection[T]
): ProtobufSCollectionOps[T] = new ProtobufSCollectionOps[T](c)
}