danny0405 commented on code in PR #8307: URL: https://github.com/apache/hudi/pull/8307#discussion_r1151359438
########## hudi-common/src/main/java/org/apache/hudi/avro/GenericAvroSerializer.java: ########## @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.avro; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.Schema; +import org.apache.avro.SchemaNormalization; +import org.apache.avro.generic.GenericContainer; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; + + +/** + * Custom serializer used for generic Avro containers. + * <p> + * Heavily adapted from: + * <p> + * <a href="https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala">GenericAvroSerializer.scala</a> + * <p> + * As {@link org.apache.hudi.common.util.SerializationUtils} is not shared between threads and does not concern any + * shuffling operations, compression and decompression cache is omitted as network IO is not a concern. + * <p> + * Unlike Spark's implementation, the class and constructor is not initialized with a predefined map of avro schemas. + * This is the case as schemas to read and write are not known beforehand. + * + * @param <D> the subtype of [[GenericContainer]] handled by this serializer + */ +public class GenericAvroSerializer<D extends GenericContainer> extends Serializer<D> { + + // reuses the same datum reader/writer since the same schema will be used many times + private final HashMap<Schema, DatumWriter<D>> writerCache = new HashMap<>(); + private final HashMap<Schema, DatumReader<D>> readerCache = new HashMap<>(); + + // fingerprinting is very expensive so this alleviates most of the work + private final HashMap<Schema, Long> fingerprintCache = new HashMap<>(); + private final HashMap<Long, Schema> schemaCache = new HashMap<>(); + + private Long getFingerprint(Schema schema) { + if (fingerprintCache.containsKey(schema)) { + return fingerprintCache.get(schema); + } else { + Long fingerprint = SchemaNormalization.parsingFingerprint64(schema); + fingerprintCache.put(schema, fingerprint); + return fingerprint; + } + } + + private Schema getSchema(Long fingerprint, byte[] schemaBytes) { + if (schemaCache.containsKey(fingerprint)) { + return schemaCache.get(fingerprint); + } else { + String schema = new String(schemaBytes, StandardCharsets.UTF_8); + Schema parsedSchema = new Schema.Parser().parse(schema); + schemaCache.put(fingerprint, parsedSchema); + return parsedSchema; + } + } + + private DatumWriter<D> getDatumWriter(Schema schema) { + DatumWriter<D> writer; + if (writerCache.containsKey(schema)) { + writer = writerCache.get(schema); + } else { + writer = new GenericDatumWriter<>(schema); + writerCache.put(schema, writer); + } + return writer; + } + + private DatumReader<D> getDatumReader(Schema schema) { + DatumReader<D> reader; + if (readerCache.containsKey(schema)) { + reader = readerCache.get(schema); + } else { + reader = new GenericDatumReader<>(schema); + readerCache.put(schema, reader); + } + return reader; + } + + private void serializeDatum(D datum, Output output) throws IOException { + Encoder encoder = EncoderFactory.get().directBinaryEncoder(output, null); + Schema schema = datum.getSchema(); + Long fingerprint = this.getFingerprint(schema); + byte[] schemaBytes = schema.toString().getBytes(StandardCharsets.UTF_8); Review Comment: This is Spark impl for serializing: ```scala def serializeDatum(datum: D, output: KryoOutput): Unit = { val encoder = EncoderFactory.get.binaryEncoder(output, null) val schema = datum.getSchema val fingerprint = fingerprintCache.getOrElseUpdate(schema, { SchemaNormalization.parsingFingerprint64(schema) }) schemas.get(fingerprint) match { case Some(_) => output.writeBoolean(true) output.writeLong(fingerprint) case None => output.writeBoolean(false) val compressedSchema = compress(schema) output.writeInt(compressedSchema.length) output.writeBytes(compressedSchema) } writerCache.getOrElseUpdate(schema, GenericData.get.createDatumWriter(schema)) .asInstanceOf[DatumWriter[D]] .write(datum, encoder) encoder.flush() } ``` So what's the point of the fingerprint here if we always serialize the schema bytes? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
