You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/10/19 10:26:56 UTC

[GitHub] tzulitai commented on a change in pull request #6881: [FLINK-10605] [core] Upgrade AvroSerializer snapshot to implement new TypeSerializerSnapshot interface

tzulitai commented on a change in pull request #6881: [FLINK-10605] [core] Upgrade AvroSerializer snapshot to implement new TypeSerializerSnapshot interface
URL: https://github.com/apache/flink/pull/6881#discussion_r226600689
 
 

 ##########
 File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializerSnapshot.java
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * 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.flink.formats.avro.typeutils;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaCompatibility;
+import org.apache.avro.SchemaCompatibility.SchemaPairCompatibility;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An {@code Avro} specific implementation of a {@link TypeSerializerSnapshot}.
+ *
+ * @param <T> The data type that the originating serializer of this configuration serializes.
+ */
+public final class AvroSerializerSnapshot<T> implements TypeSerializerSnapshot<T> {
+	private Class<T> runtimeType;
+	private Schema schema;
+
+	@SuppressWarnings("WeakerAccess")
+	public AvroSerializerSnapshot() {
+		// this constructor is used when restoring from a checkpoint.
+	}
+
+	AvroSerializerSnapshot(Schema schema, Class<T> runtimeType) {
+		this.schema = schema;
+		this.runtimeType = runtimeType;
+	}
+
+	@Override
+	public int getCurrentVersion() {
+		return 1;
+	}
+
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		checkNotNull(runtimeType);
+		checkNotNull(schema);
+
+		out.writeUTF(runtimeType.getName());
+		out.writeUTF(schema.toString(false));
+	}
+
+	@Override
+	public void read(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException {
+		final String previousRuntimeClassName = in.readUTF();
+		final String previousSchemaDefinition = in.readUTF();
+
+		this.runtimeType = findClassOrThrow(userCodeClassLoader, previousRuntimeClassName);
+		this.schema = parseAvroSchema(previousSchemaDefinition);
+	}
+
+	@Override
+	public <NS extends TypeSerializer<T>> TypeSerializerSchemaCompatibility<T, NS>
+	resolveSchemaCompatibility(NS newSerializer) {
+		if (!(newSerializer instanceof AvroSerializer)) {
+			return TypeSerializerSchemaCompatibility.incompatible();
+		}
+		AvroSerializer<?> newAvroSerializer = (AvroSerializer<?>) newSerializer;
+		return resolveSchemaCompatibility(schema, newAvroSerializer.getAvroSchema());
+	}
+
+	@Override
+	public TypeSerializer<T> restoreSerializer() {
+		checkNotNull(runtimeType);
+		checkNotNull(schema);
+
+		if (AvroSerializer.isGenericRecord(runtimeType)) {
+			return new AvroSerializer<>(runtimeType, schema);
+		}
+		else {
+			return new AvroSerializer<>(runtimeType);
 
 Review comment:
   I think regardless of whether the type is Avro or specific, the `restoreSerializer` should always instantiate an AvroSerializer with the snapshotted schema.
   
   With `new AvroSerializer<>(runtimeType);`, the schema used, as far as I understand Avro, will be the new one attached to the generated Specific record.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services