[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-11-01 Thread aljoscha
Github user aljoscha closed the pull request at:

https://github.com/apache/flink/pull/4919


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147711564
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
@@ -361,5 +376,247 @@ public void setPendingCommitTransactions(List 
pendingCommitTransactions) {
public void setContext(Optional context) {
this.context = context;
}
+
+   @Override
+   public boolean equals(Object o) {
+   if (this == o) {
+   return true;
+   }
+   if (o == null || getClass() != o.getClass()) {
+   return false;
+   }
+
+   State state = (State) o;
+
--- End diff --

I used IntelliJ to generate those. :sweat_smile:


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147704985
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
@@ -361,5 +376,247 @@ public void setPendingCommitTransactions(List 
pendingCommitTransactions) {
public void setContext(Optional context) {
this.context = context;
}
+
+   @Override
+   public boolean equals(Object o) {
+   if (this == o) {
+   return true;
+   }
+   if (o == null || getClass() != o.getClass()) {
+   return false;
+   }
+
+   State state = (State) o;
+
--- End diff --

Not a deal breaker but these if's are a bit too complicated.
Using `Objects.equals()` on each of the components and `&&` would simplify 
them.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147692203
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
+   @Override
+   public boolean isImmutableType() {
+   return true;
+   }
+
+   @Override
+   public KafkaTransactionState createInstance() {
+   return null;
+   }
+
+   @Override
+   public KafkaTransactionState copy(KafkaTransactionState from) {
+   return from;
+   }
+
+   @Override
+   public KafkaTransactionState copy(
+   KafkaTransactionState from,
+   KafkaTransactionState reuse) {
+   return from;
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   KafkaTransactionState record,
+   DataOutputView target) throws IOException {
+   if (record.transactionalId == null) {
+   target.writeBoolean(false);
+   } else {
+   target.writeBoolean(true);
+   target.writeUTF(record.transactionalId);
+   }
+   target.writeLong(record.producerId);
+   target.writeShort(record.epoch);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(DataInputView source) 
throws IOException {
+   String transactionalId = null;
+   if (source.readBoolean()) {
+   transactionalId = source.readUTF();
+   }
+   long producerId = source.readLong();
+   short epoch = source.readShort();
+   return new KafkaTransactionState(transactionalId, 
producerId, epoch, null);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(
+   KafkaTransactionState reuse,
+   DataInputView source) throws IOException {
+   return deserialize(source);
+   }
+
+   @Override
+   public void copy(
+   DataInputView source, DataOutputView target) 
throws IOException {
+   boolean hasTransactionalId = source.readBoolean();
--- End diff --

Yes, you are right.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147692117
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
@@ -362,4 +374,213 @@ public void setContext(Optional context) {
this.context = context;
}
}
+
+   /**
+* Custom {@link TypeSerializer} for the sink state.
+*/
+   static final class StateSerializer extends 
TypeSerializer> {
+
+   private final TypeSerializer transactionSerializer;
+   private final TypeSerializer contextSerializer;
+
+   public StateSerializer(
+   TypeSerializer transactionSerializer,
+   TypeSerializer contextSerializer) {
+   this.transactionSerializer = 
checkNotNull(transactionSerializer);
+   this.contextSerializer = 
checkNotNull(contextSerializer);
+   }
+
+   @Override
+   public boolean isImmutableType() {
+   return transactionSerializer.isImmutableType() && 
contextSerializer.isImmutableType();
+   }
+
+   @Override
+   public TypeSerializer> duplicate() {
+   return new StateSerializer<>(
+   transactionSerializer.duplicate(), 
contextSerializer.duplicate());
+   }
+
+   @Override
+   public State createInstance() {
+   return null;
+   }
+
+   @Override
+   public State copy(State from) {
+   TXN copiedPendingTransaction = 
transactionSerializer.copy(from.getPendingTransaction());
+   List copiedPendingCommitTransactions = new 
ArrayList<>();
+   for (TXN txn : from.getPendingCommitTransactions()) {
+   
copiedPendingCommitTransactions.add(transactionSerializer.copy(txn));
+   }
+   Optional copiedContext = 
from.getContext().map(contextSerializer::copy);
+   return new State<>(copiedPendingTransaction, 
copiedPendingCommitTransactions, copiedContext);
+   }
+
+   @Override
+   public State copy(
+   State from,
+   State reuse) {
+   return copy(from);
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   State record,
+   DataOutputView target) throws IOException {
+   
transactionSerializer.serialize(record.getPendingTransaction(), target);
+   List pendingCommitTransactions = 
record.getPendingCommitTransactions();
+   target.writeInt(pendingCommitTransactions.size());
+   for (TXN pendingTxn : pendingCommitTransactions) {
+   transactionSerializer.serialize(pendingTxn, 
target);
+   }
+   Optional context = record.getContext();
+   if (context.isPresent()) {
+   target.writeBoolean(true);
+   contextSerializer.serialize(context.get(), 
target);
+   } else {
+   target.writeBoolean(false);
+   }
+   }
+
+   @Override
+   public State deserialize(DataInputView source) 
throws IOException {
+   TXN pendingTxn = 
transactionSerializer.deserialize(source);
+   int numPendingCommitTxns = source.readInt();
+   List pendingCommitTxns = new 
ArrayList<>(numPendingCommitTxns);
+   for (int i = 0; i < numPendingCommitTxns; i++) {
+   
pendingCommitTxns.add(transactionSerializer.deserialize(source));
+   }
+   Optional context = Optional.empty();
+   boolean hasContext = source.readBoolean();
+   if (hasContext) {
+   context = 
Optional.of(contextSerializer.deserialize(source));
+   }
+   return new State<>(pendingTxn, pendingCommitTxns, 
context);
+   }
+
+   @Override
+   public State

[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147692002
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
@@ -362,4 +374,213 @@ public void setContext(Optional context) {
this.context = context;
}
}
+
+   /**
+* Custom {@link TypeSerializer} for the sink state.
+*/
+   static final class StateSerializer extends 
TypeSerializer> {
+
--- End diff --

fixing


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147691567
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
+   @Override
+   public boolean isImmutableType() {
+   return true;
+   }
+
+   @Override
+   public KafkaTransactionState createInstance() {
+   return null;
+   }
+
+   @Override
+   public KafkaTransactionState copy(KafkaTransactionState from) {
+   return from;
+   }
+
+   @Override
+   public KafkaTransactionState copy(
+   KafkaTransactionState from,
+   KafkaTransactionState reuse) {
+   return from;
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   KafkaTransactionState record,
+   DataOutputView target) throws IOException {
+   if (record.transactionalId == null) {
+   target.writeBoolean(false);
+   } else {
+   target.writeBoolean(true);
+   target.writeUTF(record.transactionalId);
+   }
+   target.writeLong(record.producerId);
+   target.writeShort(record.epoch);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(DataInputView source) 
throws IOException {
+   String transactionalId = null;
+   if (source.readBoolean()) {
+   transactionalId = source.readUTF();
+   }
+   long producerId = source.readLong();
+   short epoch = source.readShort();
+   return new KafkaTransactionState(transactionalId, 
producerId, epoch, null);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(
+   KafkaTransactionState reuse,
+   DataInputView source) throws IOException {
+   return deserialize(source);
+   }
+
+   @Override
+   public void copy(
+   DataInputView source, DataOutputView target) 
throws IOException {
+   boolean hasTransactionalId = source.readBoolean();
+   target.writeBoolean(hasTransactionalId);
+   target.writeUTF(source.readUTF());
+   target.writeLong(source.readLong());
+   target.writeShort(source.readShort());
+   }
+
+   @Override
+   public boolean canEqual(Object obj) {
+   return obj instanceof TransactionStateSerializer;
+   }
+   }
+
+   static class ContextStateSerializer extends 
TypeSerializerSingleton {
--- End diff --

fixing


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147691545
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
+   @Override
+   public boolean isImmutableType() {
+   return true;
+   }
+
+   @Override
+   public KafkaTransactionState createInstance() {
+   return null;
+   }
+
+   @Override
+   public KafkaTransactionState copy(KafkaTransactionState from) {
+   return from;
+   }
+
+   @Override
+   public KafkaTransactionState copy(
+   KafkaTransactionState from,
+   KafkaTransactionState reuse) {
+   return from;
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   KafkaTransactionState record,
+   DataOutputView target) throws IOException {
+   if (record.transactionalId == null) {
+   target.writeBoolean(false);
+   } else {
+   target.writeBoolean(true);
+   target.writeUTF(record.transactionalId);
+   }
+   target.writeLong(record.producerId);
+   target.writeShort(record.epoch);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(DataInputView source) 
throws IOException {
+   String transactionalId = null;
+   if (source.readBoolean()) {
+   transactionalId = source.readUTF();
+   }
+   long producerId = source.readLong();
+   short epoch = source.readShort();
+   return new KafkaTransactionState(transactionalId, 
producerId, epoch, null);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(
+   KafkaTransactionState reuse,
+   DataInputView source) throws IOException {
+   return deserialize(source);
+   }
+
+   @Override
+   public void copy(
+   DataInputView source, DataOutputView target) 
throws IOException {
+   boolean hasTransactionalId = source.readBoolean();
--- End diff --

No, I actually have to do this:
```
if (hasTransactionalId) {
target.writeUTF(source.readUTF());
}
```

But thanks for noticing. :sweat:


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147691305
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
--- End diff --

fixing


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147691153
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
--- End diff --

fixing


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147691059
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -958,29 +960,34 @@ public int compare(PartitionInfo o1, PartitionInfo 
o2) {
/**
 * State for handling transactions.
 */
-   public static class KafkaTransactionState {
+   static class KafkaTransactionState {
 
private final transient FlinkKafkaProducer 
producer;
 
@Nullable
-   public final String transactionalId;
+   final String transactionalId;
 
-   public final long producerId;
+   final long producerId;
 
-   public final short epoch;
+   final short epoch;
 
-   public KafkaTransactionState(String transactionalId, 
FlinkKafkaProducer producer) {
--- End diff --

fixing


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147675623
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
--- End diff --

Here we should check for `null` for the `transactionalIds`. If this is 
null, it will also break the `ContextStateSerializer.serialize()` method.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147678569
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
@@ -79,19 +92,18 @@
 * TwoPhaseCommitSinkFunction(TypeInformation.of(new 
TypeHint>() {}));
 * }
 * 
-* @param stateTypeInformation {@link TypeInformation} for POJO holding 
state of opened transactions.
-*/
-   public TwoPhaseCommitSinkFunction(TypeInformation> 
stateTypeInformation) {
-   this(new ListStateDescriptor>("state", 
stateTypeInformation));
-   }
-
-   /**
-* Instantiate {@link TwoPhaseCommitSinkFunction} with custom state 
descriptors.
 *
-* @param stateDescriptor descriptor for transactions POJO.
+* @param transactionSerializer {@link TypeSerializer} for the 
transaction type of this sink
+* @param contextSerializer {@link TypeSerializer} for the context type 
of this sink
 */
-   public TwoPhaseCommitSinkFunction(ListStateDescriptor> stateDescriptor) {
-   this.stateDescriptor = requireNonNull(stateDescriptor, 
"stateDescriptor is null");
+   public TwoPhaseCommitSinkFunction(
--- End diff --

I know this is not part of this PR, but lines 81, 82, 84 should be 
`transient`.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147673504
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
--- End diff --

Define a `serialVersionUID`.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147674429
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
+   @Override
+   public boolean isImmutableType() {
+   return true;
+   }
+
+   @Override
+   public KafkaTransactionState createInstance() {
+   return null;
+   }
+
+   @Override
+   public KafkaTransactionState copy(KafkaTransactionState from) {
+   return from;
+   }
+
+   @Override
+   public KafkaTransactionState copy(
+   KafkaTransactionState from,
+   KafkaTransactionState reuse) {
+   return from;
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   KafkaTransactionState record,
+   DataOutputView target) throws IOException {
+   if (record.transactionalId == null) {
+   target.writeBoolean(false);
+   } else {
+   target.writeBoolean(true);
+   target.writeUTF(record.transactionalId);
+   }
+   target.writeLong(record.producerId);
+   target.writeShort(record.epoch);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(DataInputView source) 
throws IOException {
+   String transactionalId = null;
+   if (source.readBoolean()) {
+   transactionalId = source.readUTF();
+   }
+   long producerId = source.readLong();
+   short epoch = source.readShort();
+   return new KafkaTransactionState(transactionalId, 
producerId, epoch, null);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(
+   KafkaTransactionState reuse,
+   DataInputView source) throws IOException {
+   return deserialize(source);
+   }
+
+   @Override
+   public void copy(
+   DataInputView source, DataOutputView target) 
throws IOException {
+   boolean hasTransactionalId = source.readBoolean();
--- End diff --

For uniformity, this can become: 
`target.writeBoolean(source.readBoolean())`.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147688912
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
@@ -362,4 +374,213 @@ public void setContext(Optional context) {
this.context = context;
}
}
+
+   /**
+* Custom {@link TypeSerializer} for the sink state.
+*/
+   static final class StateSerializer extends 
TypeSerializer> {
+
+   private final TypeSerializer transactionSerializer;
+   private final TypeSerializer contextSerializer;
+
+   public StateSerializer(
+   TypeSerializer transactionSerializer,
+   TypeSerializer contextSerializer) {
+   this.transactionSerializer = 
checkNotNull(transactionSerializer);
+   this.contextSerializer = 
checkNotNull(contextSerializer);
+   }
+
+   @Override
+   public boolean isImmutableType() {
+   return transactionSerializer.isImmutableType() && 
contextSerializer.isImmutableType();
+   }
+
+   @Override
+   public TypeSerializer> duplicate() {
+   return new StateSerializer<>(
+   transactionSerializer.duplicate(), 
contextSerializer.duplicate());
+   }
+
+   @Override
+   public State createInstance() {
+   return null;
+   }
+
+   @Override
+   public State copy(State from) {
+   TXN copiedPendingTransaction = 
transactionSerializer.copy(from.getPendingTransaction());
+   List copiedPendingCommitTransactions = new 
ArrayList<>();
+   for (TXN txn : from.getPendingCommitTransactions()) {
+   
copiedPendingCommitTransactions.add(transactionSerializer.copy(txn));
+   }
+   Optional copiedContext = 
from.getContext().map(contextSerializer::copy);
+   return new State<>(copiedPendingTransaction, 
copiedPendingCommitTransactions, copiedContext);
+   }
+
+   @Override
+   public State copy(
+   State from,
+   State reuse) {
+   return copy(from);
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   State record,
+   DataOutputView target) throws IOException {
+   
transactionSerializer.serialize(record.getPendingTransaction(), target);
+   List pendingCommitTransactions = 
record.getPendingCommitTransactions();
+   target.writeInt(pendingCommitTransactions.size());
+   for (TXN pendingTxn : pendingCommitTransactions) {
+   transactionSerializer.serialize(pendingTxn, 
target);
+   }
+   Optional context = record.getContext();
+   if (context.isPresent()) {
+   target.writeBoolean(true);
+   contextSerializer.serialize(context.get(), 
target);
+   } else {
+   target.writeBoolean(false);
+   }
+   }
+
+   @Override
+   public State deserialize(DataInputView source) 
throws IOException {
+   TXN pendingTxn = 
transactionSerializer.deserialize(source);
+   int numPendingCommitTxns = source.readInt();
+   List pendingCommitTxns = new 
ArrayList<>(numPendingCommitTxns);
+   for (int i = 0; i < numPendingCommitTxns; i++) {
+   
pendingCommitTxns.add(transactionSerializer.deserialize(source));
+   }
+   Optional context = Optional.empty();
+   boolean hasContext = source.readBoolean();
+   if (hasContext) {
+   context = 
Optional.of(contextSerializer.deserialize(source));
+   }
+   return new State<>(pendingTxn, pendingCommitTxns, 
context);
+   }
+
+   @Override
+   public State 

[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147676219
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
+   @Override
+   public boolean isImmutableType() {
+   return true;
+   }
+
+   @Override
+   public KafkaTransactionState createInstance() {
+   return null;
+   }
+
+   @Override
+   public KafkaTransactionState copy(KafkaTransactionState from) {
+   return from;
+   }
+
+   @Override
+   public KafkaTransactionState copy(
+   KafkaTransactionState from,
+   KafkaTransactionState reuse) {
+   return from;
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   KafkaTransactionState record,
+   DataOutputView target) throws IOException {
+   if (record.transactionalId == null) {
+   target.writeBoolean(false);
+   } else {
+   target.writeBoolean(true);
+   target.writeUTF(record.transactionalId);
+   }
+   target.writeLong(record.producerId);
+   target.writeShort(record.epoch);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(DataInputView source) 
throws IOException {
+   String transactionalId = null;
+   if (source.readBoolean()) {
+   transactionalId = source.readUTF();
+   }
+   long producerId = source.readLong();
+   short epoch = source.readShort();
+   return new KafkaTransactionState(transactionalId, 
producerId, epoch, null);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(
+   KafkaTransactionState reuse,
+   DataInputView source) throws IOException {
+   return deserialize(source);
+   }
+
+   @Override
+   public void copy(
+   DataInputView source, DataOutputView target) 
throws IOException {
+   boolean hasTransactionalId = source.readBoolean();
+   target.writeBoolean(hasTransactionalId);
+   target.writeUTF(source.readUTF());
+   target.writeLong(source.readLong());
+   target.writeShort(source.readShort());
+   }
+
+   @Override
+   public boolean canEqual(Object obj) {
+   return obj instanceof TransactionStateSerializer;
+   }
+   }
+
+   static class ContextStateSerializer extends 
TypeSerializerSingleton {
+   @Override
+   public boolean isImmutableType() {
+   return true;
+   }
+
+   @Override
+   public KafkaTransactionContext createInstance() {
+   return null;
+   }
+
+   @Override
+   public KafkaTransactionContext copy(KafkaTransactionContext 
from) {
+   return from;
+   }
+
+   @Override
+   public KafkaTransactionContext copy(
+   KafkaTransactionContext from,
+   KafkaTransactionContext reuse) {
+   return from;
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+ 

[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147678946
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
@@ -362,4 +374,213 @@ public void setContext(Optional context) {
this.context = context;
}
}
+
+   /**
+* Custom {@link TypeSerializer} for the sink state.
+*/
+   static final class StateSerializer extends 
TypeSerializer> {
+
--- End diff --

Define a `serialVersionUID`.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147676539
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -993,14 +1000,162 @@ public String toString() {
 * Context associated to this instance of the {@link 
FlinkKafkaProducer011}. User for keeping track of the
 * transactionalIds.
 */
-   public static class KafkaTransactionContext {
-   public final Set transactionalIds;
+   static class KafkaTransactionContext {
+   final Set transactionalIds;
 
-   public KafkaTransactionContext(Set transactionalIds) {
+   KafkaTransactionContext(Set transactionalIds) {
this.transactionalIds = transactionalIds;
}
}
 
+   static class TransactionStateSerializer extends 
TypeSerializerSingleton {
+   @Override
+   public boolean isImmutableType() {
+   return true;
+   }
+
+   @Override
+   public KafkaTransactionState createInstance() {
+   return null;
+   }
+
+   @Override
+   public KafkaTransactionState copy(KafkaTransactionState from) {
+   return from;
+   }
+
+   @Override
+   public KafkaTransactionState copy(
+   KafkaTransactionState from,
+   KafkaTransactionState reuse) {
+   return from;
+   }
+
+   @Override
+   public int getLength() {
+   return -1;
+   }
+
+   @Override
+   public void serialize(
+   KafkaTransactionState record,
+   DataOutputView target) throws IOException {
+   if (record.transactionalId == null) {
+   target.writeBoolean(false);
+   } else {
+   target.writeBoolean(true);
+   target.writeUTF(record.transactionalId);
+   }
+   target.writeLong(record.producerId);
+   target.writeShort(record.epoch);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(DataInputView source) 
throws IOException {
+   String transactionalId = null;
+   if (source.readBoolean()) {
+   transactionalId = source.readUTF();
+   }
+   long producerId = source.readLong();
+   short epoch = source.readShort();
+   return new KafkaTransactionState(transactionalId, 
producerId, epoch, null);
+   }
+
+   @Override
+   public KafkaTransactionState deserialize(
+   KafkaTransactionState reuse,
+   DataInputView source) throws IOException {
+   return deserialize(source);
+   }
+
+   @Override
+   public void copy(
+   DataInputView source, DataOutputView target) 
throws IOException {
+   boolean hasTransactionalId = source.readBoolean();
+   target.writeBoolean(hasTransactionalId);
+   target.writeUTF(source.readUTF());
+   target.writeLong(source.readLong());
+   target.writeShort(source.readShort());
+   }
+
+   @Override
+   public boolean canEqual(Object obj) {
+   return obj instanceof TransactionStateSerializer;
+   }
+   }
+
+   static class ContextStateSerializer extends 
TypeSerializerSingleton {
--- End diff --

Define a `serialVersionUID`.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread kl0u
Github user kl0u commented on a diff in the pull request:

https://github.com/apache/flink/pull/4919#discussion_r147672995
  
--- Diff: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ---
@@ -958,29 +960,34 @@ public int compare(PartitionInfo o1, PartitionInfo 
o2) {
/**
 * State for handling transactions.
 */
-   public static class KafkaTransactionState {
+   static class KafkaTransactionState {
 
private final transient FlinkKafkaProducer 
producer;
 
@Nullable
-   public final String transactionalId;
+   final String transactionalId;
 
-   public final long producerId;
+   final long producerId;
 
-   public final short epoch;
+   final short epoch;
 
-   public KafkaTransactionState(String transactionalId, 
FlinkKafkaProducer producer) {
--- End diff --

The `public` is not needed as the whole class is not public.


---


[GitHub] flink pull request #4919: [FLINK-7902] Use TypeSerializer in TwoPhaseCommitS...

2017-10-30 Thread aljoscha
GitHub user aljoscha opened a pull request:

https://github.com/apache/flink/pull/4919

 [FLINK-7902] Use TypeSerializer in TwoPhaseCommitSinkFunctions 

*Thank you very much for contributing to Apache Flink - we are happy that 
you want to help us improve Flink. To help the community review your 
contribution in the best possible way, please go through the checklist below, 
which will get the contribution into a shape in which it can be best reviewed.*

*Please understand that we do not do this to make contributions to Flink a 
hassle. In order to uphold a high standard of quality for code contributions, 
while at the same time managing a large number of contributions, we need 
contributors to prepare the contributions well, and give reviewers enough 
contextual information for the review. Please also understand that 
contributions that do not follow this guide will take longer to review and thus 
typically be picked up with lower priority by the community.*

## Contribution Checklist

  - Make sure that the pull request corresponds to a [JIRA 
issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are 
made for typos in JavaDoc or documentation files, which need no JIRA issue.
  
  - Name the pull request in the form "[FLINK-] [component] Title of 
the pull request", where *FLINK-* should be replaced by the actual issue 
number. Skip *component* if you are unsure about which is the best component.
  Typo fixes that have no associated JIRA issue should be named following 
this pattern: `[hotfix] [docs] Fix typo in event time introduction` or 
`[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.

  - Fill out the template below to describe the changes contributed by the 
pull request. That will give reviewers the context they need to do the review.
  
  - Make sure that the change passes the automated tests, i.e., `mvn clean 
verify` passes. You can set up Travis CI to do that following [this 
guide](http://flink.apache.org/contribute-code.html#best-practices).

  - Each pull request should address only one issue, not mix up code from 
multiple issues.
  
  - Each commit in the pull request has a meaningful commit message 
(including the JIRA id)

  - Once all items of the checklist are addressed, remove the above text 
and this checklist, leaving only the filled out template below.


**(The sections below can be removed for hotfixes of typos)**

## What is the purpose of the change

*(For example: This pull request makes task deployment go through the blob 
server, rather than through RPC. That way we avoid re-transferring them on each 
deployment (during recovery).)*


## Brief change log

*(for example:)*
  - *The TaskInfo is stored in the blob store on job creation time as a 
persistent artifact*
  - *Deployments RPC transmits only the blob storage reference*
  - *TaskManagers retrieve the TaskInfo from the blob cache*


## Verifying this change

*(Please pick either of the following options)*

This change is a trivial rework / code cleanup without any test coverage.

*(or)*

This change is already covered by existing tests, such as *(please describe 
tests)*.

*(or)*

This change added tests and can be verified as follows:

*(example:)*
  - *Added integration tests for end-to-end deployment with large payloads 
(100MB)*
  - *Extended integration test for recovery after master (JobManager) 
failure*
  - *Added test that validates that TaskInfo is transferred only once 
across recoveries*
  - *Manually verified the change by running a 4 node cluser with 2 
JobManagers and 4 TaskManagers, a stateful streaming program, and killing one 
JobManager and two TaskManagers during the execution, verifying that recovery 
happens correctly.*

## 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)

## Documentation

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



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/aljoscha/flink 
jira-7902-twophase-proper-serializer

Alternatively you can review and apply these changes as the patch at: