You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2017/05/17 13:46:08 UTC
[3/3] flink git commit: [FLINK-6589] [core] Deserialize ArrayList
with capacity of size+1 to prevent growth.
[FLINK-6589] [core] Deserialize ArrayList with capacity of size+1 to prevent growth.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/98f4fad9
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/98f4fad9
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/98f4fad9
Branch: refs/heads/release-1.3
Commit: 98f4fad93263b05f0e55562ddd81385430546225
Parents: 36cac0f
Author: Fabian Hueske <fh...@apache.org>
Authored: Mon May 15 21:41:51 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Wed May 17 15:29:46 2017 +0200
----------------------------------------------------------------------
.../apache/flink/api/common/typeutils/base/ListSerializer.java | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/98f4fad9/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
index aa9808e..1f271fe 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
@@ -82,7 +82,7 @@ public final class ListSerializer<T> extends TypeSerializer<List<T>> {
@Override
public TypeSerializer<List<T>> duplicate() {
TypeSerializer<T> duplicateElement = elementSerializer.duplicate();
- return duplicateElement == elementSerializer ? this : new ListSerializer<T>(duplicateElement);
+ return duplicateElement == elementSerializer ? this : new ListSerializer<>(duplicateElement);
}
@Override
@@ -129,7 +129,8 @@ public final class ListSerializer<T> extends TypeSerializer<List<T>> {
@Override
public List<T> deserialize(DataInputView source) throws IOException {
final int size = source.readInt();
- final List<T> list = new ArrayList<>(size);
+ // create new list with (size + 1) capacity to prevent expensive growth when a single element is added
+ final List<T> list = new ArrayList<>(size + 1);
for (int i = 0; i < size; i++) {
list.add(elementSerializer.deserialize(source));
}