/
ParquetType.scala
210 lines (177 loc) · 7.69 KB
/
ParquetType.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
/*
* Copyright 2022 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 magnolify.parquet
import magnolify.shared.{Converter => _, _}
import org.apache.avro.{Schema => AvroSchema}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.Job
import org.apache.parquet.avro.AvroSchemaConverter
import org.apache.parquet.hadoop.{
api => hadoop,
ParquetInputFormat,
ParquetOutputFormat,
ParquetReader,
ParquetWriter
}
import org.apache.parquet.io.api._
import org.apache.parquet.io.{InputFile, OutputFile}
import org.apache.parquet.schema.MessageType
import org.slf4j.LoggerFactory
import org.typelevel.scalaccompat.annotation.nowarn
sealed trait ParquetArray
/**
* Add `import magnolify.parquet.ParquetArray.AvroCompat._` to generate AVRO schema on write
*/
object ParquetArray {
implicit case object default extends ParquetArray
object AvroCompat {
implicit case object avroCompat extends ParquetArray
}
}
sealed trait ParquetType[T] extends Serializable {
import ParquetType._
def schema: MessageType
def avroSchema: AvroSchema
val avroCompat: Boolean
def setupInput(job: Job): Unit = {
job.setInputFormatClass(classOf[ParquetInputFormat[T]])
ParquetInputFormat.setReadSupportClass(job, classOf[ReadSupport[T]])
job.getConfiguration.set(ReadTypeKey, SerializationUtils.toBase64(this))
}
def setupOutput(job: Job): Unit = {
job.setOutputFormatClass(classOf[ParquetOutputFormat[T]])
ParquetOutputFormat.setWriteSupportClass(job, classOf[WriteSupport[T]])
job.getConfiguration.set(WriteTypeKey, SerializationUtils.toBase64(this))
}
def readSupport: ReadSupport[T] = new ReadSupport[T](this)
def writeSupport: WriteSupport[T] = new WriteSupport[T](this)
def readBuilder(file: InputFile): ReadBuilder[T] = new ReadBuilder(file, readSupport)
def writeBuilder(file: OutputFile): WriteBuilder[T] = new WriteBuilder(file, writeSupport)
def write(c: RecordConsumer, v: T): Unit = ()
def newConverter: TypeConverter[T] = null
}
object ParquetType {
private val logger = LoggerFactory.getLogger(this.getClass)
implicit def apply[T](implicit f: ParquetField[T], pa: ParquetArray): ParquetType[T] =
ParquetType(CaseMapper.identity)
def apply[T](
cm: CaseMapper
)(implicit f: ParquetField[T], pa: ParquetArray): ParquetType[T] = f match {
case r: ParquetField.Record[_] =>
new ParquetType[T] {
@transient override lazy val schema: MessageType = Schema.message(r.schema(cm))
@transient override lazy val avroSchema: AvroSchema = {
val s = new AvroSchemaConverter().convert(schema)
// add doc to avro schema
val fieldDocs = f.fieldDocs(cm)
SchemaUtil.deepCopy(s, f.typeDoc, fieldDocs.get)
}
override val avroCompat: Boolean =
pa == ParquetArray.AvroCompat.avroCompat || f.hasAvroArray
override def write(c: RecordConsumer, v: T): Unit = r.write(c, v)(cm)
override def newConverter: TypeConverter[T] = r.newConverter
}
case _ =>
throw new IllegalArgumentException(s"ParquetType can only be created from Record. Got $f")
}
val ReadTypeKey = "parquet.type.read.type"
val WriteTypeKey = "parquet.type.write.type"
class ReadBuilder[T](file: InputFile, val readSupport: ReadSupport[T])
extends ParquetReader.Builder[T](file) {
override def getReadSupport: ReadSupport[T] = readSupport
}
class WriteBuilder[T](file: OutputFile, val writeSupport: WriteSupport[T])
extends ParquetWriter.Builder[T, WriteBuilder[T]](file) {
override def self(): WriteBuilder[T] = this
override def getWriteSupport(conf: Configuration): WriteSupport[T] = writeSupport
}
// From AvroReadSupport
private val AVRO_SCHEMA_METADATA_KEY = "parquet.avro.schema"
private val OLD_AVRO_SCHEMA_METADATA_KEY = "avro.schema"
class ReadSupport[T](private var parquetType: ParquetType[T]) extends hadoop.ReadSupport[T] {
def this() = this(null)
override def init(context: hadoop.InitContext): hadoop.ReadSupport.ReadContext = {
if (parquetType == null) {
// Use deprecated getConfiguration
// Recommended getParquetConfiguration is only available for parquet 1.14+
val readKeyType = context.getConfiguration.get(ReadTypeKey): @nowarn("cat=deprecation")
parquetType = SerializationUtils.fromBase64[ParquetType[T]](readKeyType)
}
val metadata = context.getKeyValueMetadata
val model = metadata.get(ParquetWriter.OBJECT_MODEL_NAME_PROP)
val isAvroFile = (model != null && model.contains("avro")) ||
metadata.containsKey(AVRO_SCHEMA_METADATA_KEY) ||
metadata.containsKey(OLD_AVRO_SCHEMA_METADATA_KEY)
if (isAvroFile && !parquetType.avroCompat) {
logger.warn(
"Parquet file was written from Avro records, " +
"`import magnolify.parquet.ParquetArray.AvroCompat._` to read correctly"
)
}
if (!isAvroFile && parquetType.avroCompat) {
logger.warn(
"Parquet file was not written from Avro records, " +
"remove `import magnolify.parquet.ParquetArray.AvroCompat._` to read correctly"
)
}
val requestedSchema = Schema.message(parquetType.schema)
Schema.checkCompatibility(context.getFileSchema, requestedSchema)
new hadoop.ReadSupport.ReadContext(requestedSchema, java.util.Collections.emptyMap())
}
override def prepareForRead(
configuration: Configuration,
keyValueMetaData: java.util.Map[String, String],
fileSchema: MessageType,
readContext: hadoop.ReadSupport.ReadContext
): RecordMaterializer[T] =
new RecordMaterializer[T] {
private val root = parquetType.newConverter
override def getCurrentRecord: T = root.get
override def getRootConverter: GroupConverter = root.asGroupConverter()
}
}
class WriteSupport[T](private var parquetType: ParquetType[T]) extends hadoop.WriteSupport[T] {
def this() = this(null)
override def getName: String = "magnolify"
private var recordConsumer: RecordConsumer = null
override def init(configuration: Configuration): hadoop.WriteSupport.WriteContext = {
if (parquetType == null) {
parquetType = SerializationUtils.fromBase64[ParquetType[T]](configuration.get(WriteTypeKey))
}
val schema = Schema.message(parquetType.schema)
val metadata = new java.util.HashMap[String, String]()
if (parquetType.avroCompat) {
// This overrides `WriteSupport#getName`
metadata.put(ParquetWriter.OBJECT_MODEL_NAME_PROP, "avro")
metadata.put(AVRO_SCHEMA_METADATA_KEY, parquetType.avroSchema.toString())
} else {
logger.warn(
"Parquet file is being written with no avro compatibility, this mode is not " +
"producing schema. Add `import magnolify.parquet.ParquetArray.AvroCompat._` to " +
"generate schema"
)
}
new hadoop.WriteSupport.WriteContext(schema, metadata)
}
override def prepareForWrite(recordConsumer: RecordConsumer): Unit =
this.recordConsumer = recordConsumer
override def write(record: T): Unit = {
recordConsumer.startMessage()
parquetType.write(recordConsumer, record)
recordConsumer.endMessage()
}
}
}