Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-11329][core] Migrating CompositeSerializers #7590

Closed

Conversation

igalshilman
Copy link
Contributor

What is the purpose of the change

This PR migrates the following serializers to the new TypeSerializerSnapshot abstraction:

Brief change log

Each commit migrates a single serializer's TypeSerializerConfigSnapshot to the new TypeSerializerSnapsoht.

Verifying this change

Each migrated serializer has an associated StateMigrationTest that verifies snapshot backwards compatibility with previous versions.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (yes / no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (yes / no)
  • The serializers: (yes / no / don't know)
  • The runtime per-record code paths (performance sensitive): (yes / no / don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / no / don't know)
  • The S3 file system connector: (yes / no / don't know)

Documentation

  • Does this pull request introduce a new feature? (yes / no)
  • If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)

Copy link
Contributor

@tzulitai tzulitai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Review for commit 516d39a.

@@ -302,6 +299,7 @@ static PrecomputedParameters precompute(
}

/** Snapshot field serializers of composite type. */
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Add @deprecated message and direct to new snapshot class.

@@ -56,8 +57,8 @@ public String toString() {
/** Serializer for Serializer. */
public static class Serializer extends CompositeSerializer<ValueWithTs<?>> {

public Serializer(TypeSerializer<?> userValueSerializer) {
super(true, userValueSerializer, LongSerializer.INSTANCE);
public Serializer(TypeSerializer<?> valueSerializer, TypeSerializer<Long> timestampSerializer) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we need a public constructor that accepts the timestamp serializer.
This should be a private constructor used only by the snapshot class.

We should still have a public constructor that accepts the user value serializer, and by default just uses LongSerializer.INSTANCE as the new timestamp serializer.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As far as I can tell this is not a user facing serializer, rather used internally by the TtlSerializer,
and I think it is important to make it explicit that this is a composite serializer and these are the nested serializers that define it.

As a way to reduce verbosity, we can add a static factory method with an explicit name.
Feel free to decide for yourself :-)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, that's a fair argument that I can agree with. Since I don't have a strong opinion on this, I'll leave the constructors as is in this PR.

@Override
protected Serializer createOuterSerializerWithNestedSerializers(TypeSerializer<?>[] nestedSerializers) {
TypeSerializer<?> valueSerializer = nestedSerializers[0];
TypeSerializer<Long> timeSerializer = (TypeSerializer<Long>) nestedSerializers[1];
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: time --> timestamp for naming consistency

@@ -126,15 +128,15 @@ private IS createState() throws Exception {
@SuppressWarnings("unchecked")
private IS createValueState() throws Exception {
ValueStateDescriptor<TtlValue<SV>> ttlDescriptor = new ValueStateDescriptor<>(
stateDesc.getName(), new TtlSerializer<>(stateDesc.getSerializer()));
stateDesc.getName(), new TtlSerializer<>(LongSerializer.INSTANCE, stateDesc.getSerializer()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As mentioned above, having to pass in a LongSerializer.INSTANCE every time we're instantiating a TtlSerializer seems very redundant.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I disagree, same reasons as above.

@@ -56,8 +57,8 @@ public String toString() {
/** Serializer for Serializer. */
public static class Serializer extends CompositeSerializer<ValueWithTs<?>> {
Copy link
Contributor

@tzulitai tzulitai Jan 30, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This serializer class previously did not have a serialVersionUID defined.
Need to explicitly set it to what it was before, because I guess the serial version UID would have changed when adding the new constructors.

OTOH, there seems to be missing a migration test for this serializer, because that would have caught this problem.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah scratch that, just realized that this is only a serializer used in tests.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But still, I'd actually suggest adding that, to enforce good practices.

Copy link
Contributor

@tzulitai tzulitai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Review for 8bba092.
Changes here looks good 👍

}

return CompatibilityResult.requiresMigration();
}

/**
* Configuration snapshot for serializers of nullable types, containing the
* configuration snapshot of its original serializer.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Add @deprecated message.

Copy link
Contributor

@tzulitai tzulitai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

d876fd0 LGTM 👍

}
/**
* A snapshot for {@link RowSerializer}.
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Add @deprecated message.

Copy link
Contributor

@tzulitai tzulitai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

58a3692 LGTM 👍


return CompatibilityResult.requiresMigration();
public TypeSerializerSnapshot<TaggedUnion<T1, T2>> snapshotConfiguration() {
return new UnionSerializerSnapshot<>(this);
}
}

/**
* The {@link TypeSerializerConfigSnapshot} for the {@link UnionSerializer}.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Add @deprecated message

Copy link
Contributor

@tzulitai tzulitai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

c375ca0 LGTM 👍

Copy link
Contributor

@tzulitai tzulitai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

0fa9adb LGTM 👍

Copy link
Contributor

@tzulitai tzulitai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One problem with 307f6ee.
Please see comment.

@@ -31,6 +33,7 @@
* allow calling different base class constructors from subclasses, while we need that
* for the default empty constructor.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Add @deprecated message.

: ScalaOptionSerializerConfigSnapshot[A] =>
ensureCompatibilityInternal(optionSerializerConfigSnapshot)
case legacyOptionSerializerConfigSnapshot
: OptionSerializer.OptionSerializerConfigSnapshot[A] =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removing this path will lead to problems when restoring from Flink 1.3, because this snapshot class was used back in Flink 1.3.

OTOH, it should be possible to redirect OptionSerializerConfigSnapshot's compatibility check to the new ScalaOptionSerializerSnapshot.

tzulitai added a commit to tzulitai/flink that referenced this pull request Jan 30, 2019
…lizerConfigSnapshot to ScalaOptionSerializerSnapshot

This closes apache#7590.
tzulitai added a commit to tzulitai/flink that referenced this pull request Jan 30, 2019
…lizerConfigSnapshot to ScalaOptionSerializerSnapshot

This closes apache#7590.
@tzulitai
Copy link
Contributor

Thanks for the work @igalshilman!

For every serializer that is touched, I checked that:

  • The touched serializer has a serialVersionUID defined already so that we're not breaking backwards compatibility
  • The ensureCompatibility method is removed (except from CompositeSerializer)
  • Old config snapshot correctly delegates compatibility check to new serializer
  • Has migration tests that cover the change

LGTM. I'll merge https://github.com/tzulitai/flink/tree/igal-dawid-gordon-composite once Travis is green!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants