You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2018/04/22 20:40:42 UTC
[06/17] flink git commit: [hotfix] [core] Fix OutputTag serialization
to not drop TypeInformation
[hotfix] [core] Fix OutputTag serialization to not drop TypeInformation
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/addc0c9d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/addc0c9d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/addc0c9d
Branch: refs/heads/master
Commit: addc0c9d7ab37e522fcf665a72583176060eb898
Parents: 32b583f
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Apr 17 20:04:16 2018 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Apr 22 16:28:34 2018 +0200
----------------------------------------------------------------------
.../src/main/java/org/apache/flink/util/OutputTag.java | 13 +++++--------
1 file changed, 5 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/addc0c9d/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/OutputTag.java b/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
index 4126924..05f15e0 100644
--- a/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
+++ b/flink-core/src/main/java/org/apache/flink/util/OutputTag.java
@@ -23,8 +23,6 @@ import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.typeutils.TypeExtractor;
-import java.io.IOException;
-import java.io.ObjectInputStream;
import java.io.Serializable;
/**
@@ -44,11 +42,11 @@ import java.io.Serializable;
@PublicEvolving
public class OutputTag<T> implements Serializable {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private final String id;
- private transient TypeInformation<T> typeInfo;
+ private final TypeInformation<T> typeInfo;
/**
* Creates a new named {@code OutputTag} with the given id.
@@ -83,10 +81,7 @@ public class OutputTag<T> implements Serializable {
this.typeInfo = Preconditions.checkNotNull(typeInfo, "TypeInformation cannot be null.");
}
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- typeInfo = null;
- }
+ // ------------------------------------------------------------------------
public String getId() {
return id;
@@ -96,6 +91,8 @@ public class OutputTag<T> implements Serializable {
return typeInfo;
}
+ // ------------------------------------------------------------------------
+
@Override
public boolean equals(Object obj) {
return obj instanceof OutputTag