You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2017/08/07 13:17:16 UTC

[01/11] flink git commit: [FLINK-7013] Introduce flink-shaded-netty-4

Repository: flink
Updated Branches:
  refs/heads/master 0936dd045 -> 1ceb89a97


http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java
index 3b844d8..3e6470b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java
@@ -23,10 +23,11 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestResult;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestType;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.util.ReferenceCountUtil;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java
index c6f46d1..2889e2e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java
@@ -24,17 +24,18 @@ import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.query.netty.message.KvStateRequest;
 import org.apache.flink.util.Preconditions;
 
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.stream.ChunkedWriteHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
index bced08b..1af55dc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
@@ -25,13 +25,14 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestType;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.ExceptionUtils;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.util.ReferenceCountUtil;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java
index a9d40a8..f0cc94a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java
@@ -27,10 +27,10 @@ import org.apache.flink.runtime.util.DataInputDeserializer;
 import org.apache.flink.runtime.util.DataOutputSerializer;
 import org.apache.flink.util.Preconditions;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.ByteBufInputStream;
-import io.netty.buffer.ByteBufOutputStream;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufInputStream;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufOutputStream;
 
 import java.io.IOException;
 import java.io.ObjectInputStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
index 8b4fc0e..12f5064 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.Channel;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
@@ -30,6 +29,9 @@ import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java
index b4fc46f..5754e36 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java
@@ -18,15 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPromise;
-import io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.netty.NettyTestUtil.NettyServerAndClient;
@@ -37,6 +28,17 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOutboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPromise;
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java
index d0e875b..3f2d363 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java
@@ -18,13 +18,15 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler;
-import io.netty.handler.codec.string.StringDecoder;
-import io.netty.handler.codec.string.StringEncoder;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.SecurityOptions;
 import org.apache.flink.util.NetUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.string.StringDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.string.StringEncoder;
+
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java
index 77de6bf..fa039ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java
@@ -18,13 +18,15 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.EventLoopGroup;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 import org.apache.flink.util.NetUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+
 import org.junit.Test;
 
 import java.lang.reflect.Field;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
index f514cbd..8200caa 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.event.task.IntegerTaskEvent;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -28,6 +26,10 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+
 import org.junit.Test;
 
 import java.nio.ByteBuffer;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyServerLowAndHighWatermarkTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyServerLowAndHighWatermarkTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyServerLowAndHighWatermarkTest.java
index e8b6550..e0128e7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyServerLowAndHighWatermarkTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyServerLowAndHighWatermarkTest.java
@@ -18,12 +18,13 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+
 import org.junit.Test;
 
 import java.util.concurrent.atomic.AtomicReference;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java
index 119a3d0..f664347 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java
@@ -18,12 +18,11 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.Channel;
-
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.NetUtils;
 
 import scala.Tuple2;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
 
 import java.net.InetAddress;
 import java.net.InetSocketAddress;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
index cabca23..91a052f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java
@@ -18,14 +18,16 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPromise;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.util.NetUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOutboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPromise;
+
 import org.junit.Ignore;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
index 26d791f..7093d32 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.UnpooledByteBufAllocator;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.flink.core.memory.HeapMemorySegment;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -36,6 +31,13 @@ import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel
 import org.apache.flink.runtime.io.network.util.TestBufferFactory;
 import org.apache.flink.runtime.testutils.DiscardingRecycler;
 import org.apache.flink.runtime.util.event.EventListener;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
index b969b1c..5c9b5c9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java
@@ -18,13 +18,15 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.flink.runtime.execution.CancelTaskException;
 import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
index 3c4ebb3..01a0b5f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
@@ -30,6 +26,12 @@ import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java
index 40af249..1e41236 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java
@@ -20,8 +20,9 @@ package org.apache.flink.runtime.query.netty;
 
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+
 import org.junit.Test;
 
 import java.nio.channels.ClosedChannelException;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
index 6b34305..6b21487 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
@@ -39,17 +39,18 @@ import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.util.NetUtils;
 
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
 import org.junit.AfterClass;
 import org.junit.Test;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
index d643ac7..4914ff7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
@@ -43,11 +43,12 @@ import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.util.TestLogger;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.embedded.EmbeddedChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
 import org.junit.AfterClass;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java
index 4f7e54d..f8213e1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java
@@ -36,18 +36,19 @@ import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
 import org.junit.AfterClass;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
index a0b5404..2567004 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
@@ -35,9 +35,10 @@ import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.UnpooledByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledByteBufAllocator;
+
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e57d555..bb157e9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -446,8 +446,8 @@ under the License.
 			</dependency>
 
 			<dependency>
-				<groupId>io.netty</groupId>
-				<artifactId>netty-all</artifactId>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-shaded-netty</artifactId>
 				<!-- Don't upgrade for now. Netty versions >= 4.0.28.Final
 				contain an improvement by Netty, which slices a Netty buffer
 				instead of doing a memory copy [1] in the
@@ -456,7 +456,7 @@ under the License.
 				errors.
 
 				[1] https://github.com/netty/netty/issues/3704 -->
-				<version>4.0.27.Final</version>
+				<version>4.0.27.Final-1.0</version>
 			</dependency>
 
 			<!-- We have to define the versions for httpcore and httpclient here such that a consistent
@@ -476,12 +476,6 @@ under the License.
 			</dependency>
 
 			<dependency>
-				<groupId>tv.cntt</groupId>
-				<artifactId>netty-router</artifactId>
-				<version>1.10</version>
-			</dependency>
-
-			<dependency>
 				<groupId>com.fasterxml.jackson.core</groupId>
 				<artifactId>jackson-core</artifactId>
 				<version>${jackson.version}</version>

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/tools/maven/checkstyle.xml
----------------------------------------------------------------------
diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml
index 6139a44..ef383f3 100644
--- a/tools/maven/checkstyle.xml
+++ b/tools/maven/checkstyle.xml
@@ -211,7 +211,7 @@ This file is based on the checkstyle file of Apache Beam.
     </module>
 
     <module name="IllegalImport">
-      <property name="illegalPkgs" value="autovalue.shaded, avro.shaded, com.google.api.client.repackaged, com.google.appengine.repackaged, io.netty.util.internal"/>
+      <property name="illegalPkgs" value="autovalue.shaded, avro.shaded, com.google.api.client.repackaged, com.google.appengine.repackaged, io.netty"/>
     </module>
 
     <module name="RedundantModifier">

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/tools/travis_mvn_watchdog.sh
----------------------------------------------------------------------
diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh
index a3790ea..557e1c9 100755
--- a/tools/travis_mvn_watchdog.sh
+++ b/tools/travis_mvn_watchdog.sh
@@ -292,6 +292,15 @@ check_shaded_artifacts() {
 		echo "=============================================================================="
 		return 1
 	fi
+
+    NETTY=`cat allClasses | grep '^io/netty' | wc -1`
+	if [ $NETTY != "0" ]; then
+		echo "=============================================================================="
+		echo "Detected $NETTY unshaded netty dependencies in fat jar"
+		echo "=============================================================================="
+		return 1
+	fi
+
 	return 0
 }
 


[03/11] flink git commit: [FLINK-4499] [build] Add spotbugs plugin

Posted by ch...@apache.org.
[FLINK-4499] [build] Add spotbugs plugin

This closes #4367.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/a1644076
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/a1644076
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/a1644076

Branch: refs/heads/master
Commit: a1644076ee0b1771777ffc9e5634e5b2ece89d00
Parents: 27429a7
Author: zentol <ch...@apache.org>
Authored: Mon Jul 17 16:59:22 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 pom.xml                          |  75 ++++++
 tools/maven/spotbugs-exclude.xml | 447 ++++++++++++++++++++++++++++++++++
 tools/travis_mvn_watchdog.sh     |   1 +
 3 files changed, 523 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a1644076/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 427e813..e57d555 100644
--- a/pom.xml
+++ b/pom.xml
@@ -525,6 +525,81 @@ under the License.
 		</profile>
 
 		<profile>
+			<id>spotbugs</id>
+			<activation>
+				<property>
+					<name>spotbugs</name>
+				</property>
+			</activation>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>com.github.hazendaz.spotbugs</groupId>
+						<artifactId>spotbugs-maven-plugin</artifactId>
+						<version>3.0.6</version>
+
+						<executions>
+							<execution>
+								<id>findbugs-run</id>
+								<phase>compile</phase>
+								<goals>
+									<goal>check</goal>
+								</goals>
+							</execution>
+						</executions>
+
+						<configuration>
+							<xmlOutput>true</xmlOutput>
+							<threshold>Low</threshold>
+							<effort>default</effort>
+							<findbugsXmlOutputDirectory>${project.build.directory}/spotbugs</findbugsXmlOutputDirectory>
+							<excludeFilterFile>tools/maven/spotbugs-exclude.xml</excludeFilterFile>
+							<failOnError>true</failOnError>
+						</configuration>
+					</plugin>
+
+					<plugin>
+						<groupId>org.codehaus.mojo</groupId>
+						<artifactId>xml-maven-plugin</artifactId>
+						<version>1.0.1</version>
+						<executions>
+							<execution>
+								<phase>verify</phase>
+								<goals>
+									<goal>transform</goal>
+								</goals>
+							</execution>
+						</executions>
+						<configuration>
+							<transformationSets>
+								<transformationSet>
+									<dir>${project.build.directory}/spotbugs</dir>
+									<outputDir>${project.build.directory}/spotbugs</outputDir>
+									<!-- A list of available stylesheets can be found here: https://github.com/findbugsproject/findbugs/tree/master/findbugs/src/xsl -->
+									<stylesheet>plain.xsl</stylesheet>
+									
+									<fileMappers>
+										<fileMapper
+											implementation="org.codehaus.plexus.components.io.filemappers.FileExtensionMapper">
+											<targetExtension>.html</targetExtension>
+										</fileMapper>
+									</fileMappers>
+								</transformationSet>
+							</transformationSets>
+						</configuration>
+						<dependencies>
+							<dependency>
+								<groupId>com.github.hazendaz.spotbugs</groupId>
+								<artifactId>spotbugs-maven-plugin</artifactId>
+								<version>3.0.6</version>
+							</dependency>
+						</dependencies>
+					</plugin>
+				</plugins>
+			</build>
+		</profile>
+
+		<profile>
 			<id>vendor-repos</id>
 			<!-- Add vendor maven repositories -->
 			<repositories>

http://git-wip-us.apache.org/repos/asf/flink/blob/a1644076/tools/maven/spotbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tools/maven/spotbugs-exclude.xml b/tools/maven/spotbugs-exclude.xml
new file mode 100644
index 0000000..c9e63cd
--- /dev/null
+++ b/tools/maven/spotbugs-exclude.xml
@@ -0,0 +1,447 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<FindBugsFilter>
+	<Match>
+		<Bug pattern="VA_FORMAT_STRING_USES_NEWLINE"/>
+	</Match>
+	<Match>
+		<Bug pattern="BC_BAD_CAST_TO_ABSTRACT_COLLECTION"/>
+	</Match>
+	<Match>
+		<Bug pattern="BC_EQUALS_METHOD_SHOULD_WORK_FOR_ALL_OBJECTS"/>
+	</Match>
+	<Match>
+		<Bug pattern="BC_IMPOSSIBLE_INSTANCEOF"/>
+	</Match>
+	<Match>
+		<Bug pattern="BC_UNCONFIRMED_CAST"/>
+	</Match>
+	<Match>
+		<Bug pattern="BC_UNCONFIRMED_CAST_OF_RETURN_VALUE"/>
+	</Match>
+	<Match>
+		<Bug pattern="BC_VACUOUS_INSTANCEOF"/>
+	</Match>
+	<Match>
+		<Bug pattern="BX_BOXING_IMMEDIATELY_UNBOXED"/>
+	</Match>
+	<Match>
+		<Bug pattern="BX_BOXING_IMMEDIATELY_UNBOXED_TO_PERFORM_COERCION"/>
+	</Match>
+	<Match>
+		<Bug pattern="BX_UNBOXING_IMMEDIATELY_REBOXED"/>
+	</Match>
+	<Match>
+		<Bug pattern="BIT_ADD_OF_SIGNED_BYTE"/>
+	</Match>
+	<Match>
+		<Bug pattern="BIT_SIGNED_CHECK"/>
+	</Match>
+	<Match>
+		<Bug pattern="CI_CONFUSED_INHERITANCE"/>
+	</Match>
+	<Match>
+		<Bug pattern="CN_IDIOM"/>
+	</Match>
+	<Match>
+		<Bug pattern="CN_IDIOM_NO_SUPER_CALL"/>
+	</Match>
+	<Match>
+		<Bug pattern="CN_IMPLEMENTS_CLONE_BUT_NOT_CLONEABLE"/>
+	</Match>
+	<Match>
+		<Bug pattern="CO_COMPARETO_INCORRECT_FLOATING"/>
+	</Match>
+	<Match>
+		<Bug pattern="DB_DUPLICATE_BRANCHES"/>
+	</Match>
+	<Match>
+		<Bug pattern="DB_DUPLICATE_SWITCH_CLAUSES"/>
+	</Match>
+	<Match>
+		<Bug pattern="DC_PARTIALLY_CONSTRUCTED"/>
+	</Match>
+	<Match>
+		<Bug pattern="DE_MIGHT_IGNORE"/>
+	</Match>
+	<Match>
+		<Bug pattern="DLS_DEAD_LOCAL_STORE"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_BOXED_PRIMITIVE_FOR_PARSING"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_BOXED_PRIMITIVE_TOSTRING"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_CONVERT_CASE"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_DEFAULT_ENCODING"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_EXIT"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_FP_NUMBER_CTOR"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_GC"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_NEW_FOR_GETCLASS"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_NEXTINT_VIA_NEXTDOUBLE"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_NUMBER_CTOR"/>
+	</Match>
+	<Match>
+		<Bug pattern="DM_STRING_TOSTRING"/>
+	</Match>
+	<Match>
+		<Bug pattern="DMI_ENTRY_SETS_MAY_REUSE_ENTRY_OBJECTS"/>
+	</Match>
+	<Match>
+		<Bug pattern="DMI_RANDOM_USED_ONLY_ONCE"/>
+	</Match>
+	<Match>
+		<Bug pattern="DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED"/>
+	</Match>
+	<Match>
+		<Bug pattern="DP_DO_INSIDE_DO_PRIVILEGED"/>
+	</Match>
+	<Match>
+		<Bug pattern="EI_EXPOSE_REP"/>D
+	</Match>
+	<Match>
+		<Bug pattern="EI_EXPOSE_REP2"/>
+	</Match>
+	<Match>
+		<Bug pattern="EQ_COMPARETO_USE_OBJECT_EQUALS"/>
+	</Match>
+	<Match>
+		<Bug pattern="EQ_DOESNT_OVERRIDE_EQUALS"/>
+	</Match>
+	<Match>
+		<Bug pattern="EQ_GETCLASS_AND_CLASS_CONSTANT"/>
+	</Match>
+	<Match>
+		<Bug pattern="EQ_OVERRIDING_EQUALS_NOT_SYMMETRIC"/>
+	</Match>
+	<Match>
+		<Bug pattern="EQ_UNUSUAL"/>
+	</Match>
+	<Match>
+		<Bug pattern="FE_TEST_IF_EQUAL_TO_NOT_A_NUMBER"/>
+	</Match>
+	<Match>
+		<Bug pattern="FE_FLOATING_POINT_EQUALITY"/>
+	</Match>
+	<Match>
+		<Bug pattern="HE_EQUALS_NO_HASHCODE"/>
+	</Match>
+	<Match>
+		<Bug pattern="HE_EQUALS_USE_HASHCODE"/>
+	</Match>
+	<Match>
+		<Bug pattern="HE_HASHCODE_NO_EQUALS"/>
+	</Match>
+	<Match>
+		<Bug pattern="HE_HASHCODE_USE_OBJECT_EQUALS"/>
+	</Match>
+	<Match>
+		<Bug pattern="ICAST_IDIV_CAST_TO_DOUBLE"/>
+	</Match>
+	<Match>
+		<Bug pattern="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG"/>
+	</Match>
+	<Match>
+		<Bug pattern="ICAST_QUESTIONABLE_UNSIGNED_RIGHT_SHIFT"/>
+	</Match>
+	<Match>
+		<Bug pattern="INT_BAD_COMPARISON_WITH_NONNEGATIVE_VALUE"/>
+	</Match>
+	<Match>
+		<Bug pattern="IP_PARAMETER_IS_DEAD_BUT_OVERWRITTEN"/>
+	</Match>
+	<Match>
+		<Bug pattern="IS2_INCONSISTENT_SYNC"/>
+	</Match>
+	<Match>
+		<Bug pattern="IT_NO_SUCH_ELEMENT"/>
+	</Match>
+	<Match>
+		<Bug pattern="JLM_JSR166_UTILCONCURRENT_MONITORENTER"/>
+	</Match>
+	<Match>
+		<Bug pattern="LI_LAZY_INIT_STATIC"/>
+	</Match>
+	<Match>
+		<Bug pattern="MS_MUTABLE_ARRAY"/>
+	</Match>
+	<Match>
+		<Bug pattern="MS_MUTABLE_COLLECTION_PKGPROTECT"/>
+	</Match>
+	<Match>
+		<Bug pattern="MS_PKGPROTECT"/>
+	</Match>
+	<Match>
+		<Bug pattern="MS_SHOULD_BE_FINAL"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_CLASS_NAMING_CONVENTION"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_CLASS_NOT_EXCEPTION"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_CONFUSING"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_FIELD_NAMING_CONVENTION"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_FUTURE_KEYWORD_USED_AS_IDENTIFIER"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_FUTURE_KEYWORD_USED_AS_MEMBER_IDENTIFIER"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS"/>
+	</Match>
+	<Match>
+		<Bug pattern="NO_NOTIFY_NOT_NOTIFYALL"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_ALWAYS_NULL"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_DEREFERENCE_OF_READLINE_VALUE"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_EQUALS_SHOULD_HANDLE_NULL_ARGUMENT"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_METHOD_NAMING_CONVENTION"/>
+	</Match>
+	<Match>
+		<Bug pattern="NM_SAME_SIMPLE_NAME_AS_INTERFACE"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_NONNULL_PARAM_VIOLATION"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_NULL_ON_SOME_PATH"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_NULL_ON_SOME_PATH_EXCEPTION"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_NULL_ON_SOME_PATH_MIGHT_BE_INFEASIBLE"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_NULL_PARAM_DEREF"/>
+	</Match>
+	<Match>
+		<Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+	</Match>
+	<Match>
+		<Bug pattern="NS_DANGEROUS_NON_SHORT_CIRCUIT"/>
+	</Match>
+	<Match>
+		<Bug pattern="NS_NON_SHORT_CIRCUIT"/>
+	</Match>
+	<Match>
+		<Bug pattern="OBL_UNSATISFIED_OBLIGATION"/>
+	</Match>
+	<Match>
+		<Bug pattern="OS_OPEN_STREAM"/>
+	</Match>
+	<Match>
+		<Bug pattern="OS_OPEN_STREAM_EXCEPTION_PATH"/>
+	</Match>
+	<Match>
+		<Bug pattern="PZLA_PREFER_ZERO_LENGTH_ARRAYS"/>
+	</Match>
+	<Match>
+		<Bug pattern="REC_CATCH_EXCEPTION"/>
+	</Match>
+	<Match>
+		<Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE"/>
+	</Match>
+	<Match>
+		<Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE"/>
+	</Match>
+	<Match>
+		<Bug pattern="RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"/>
+	</Match>
+	<Match>
+		<Bug pattern="RI_REDUNDANT_INTERFACES"/>
+	</Match>
+	<Match>
+		<Bug pattern="RpC_REPEATED_CONDITIONAL_TEST"/>
+	</Match>
+	<Match>
+		<Bug pattern="RR_NOT_CHECKED"/>
+	</Match>
+	<Match>
+		<Bug pattern="RV_EXCEPTION_NOT_THROWN"/>
+	</Match>
+	<Match>
+		<Bug pattern="RV_NEGATING_RESULT_OF_COMPARETO"/>
+	</Match>
+	<Match>
+		<Bug pattern="RV_RETURN_VALUE_IGNORED"/>
+	</Match>
+	<Match>
+		<Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+	</Match>
+	<Match>
+		<Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+	</Match>
+	<Match>
+		<Bug pattern="SA_LOCAL_SELF_ASSIGNMENT"/>
+	</Match>
+	<Match>
+		<Bug pattern="SBSC_USE_STRINGBUFFER_CONCATENATION"/>
+	</Match>
+	<Match>
+		<Bug pattern="SC_START_IN_CTOR"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_BAD_FIELD"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_BAD_FIELD_INNER_CLASS"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_BAD_FIELD_STORE"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_INNER_CLASS"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_NO_SERIALVERSIONID"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_NO_SUITABLE_CONSTRUCTOR"/>
+	</Match>
+	<Match>
+		<Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
+	</Match>
+	<Match>
+		<Bug pattern="SF_SWITCH_NO_DEFAULT"/>
+	</Match>
+	<Match>
+		<Bug pattern="SIC_INNER_SHOULD_BE_STATIC"/>
+	</Match>
+	<Match>
+		<Bug pattern="SIC_INNER_SHOULD_BE_STATIC_ANON"/>
+	</Match>
+	<Match>
+		<Bug pattern="SIC_INNER_SHOULD_BE_STATIC_NEEDS_THIS"/>
+	</Match>
+	<Match>
+		<Bug pattern="SIO_SUPERFLUOUS_INSTANCEOF"/>
+	</Match>
+	<Match>
+		<Bug pattern="SF_SWITCH_FALLTHROUGH"/>
+	</Match>
+	<Match>
+		<Bug pattern="SQL_NONCONSTANT_STRING_PASSED_TO_EXECUTE"/>
+	</Match>
+	<Match>
+		<Bug pattern="SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING"/>
+	</Match>
+	<Match>
+		<Bug pattern="SR_NOT_CHECKED"/>
+	</Match>
+	<Match>
+		<Bug pattern="SS_SHOULD_BE_STATIC"/>
+	</Match>
+	<Match>
+		<Bug pattern="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD"/>
+	</Match>
+	<Match>
+		<Bug pattern="UC_USELESS_CONDITION"/>
+	</Match>
+	<Match>
+		<Bug pattern="UC_USELESS_OBJECT"/>
+	</Match>
+	<Match>
+		<Bug pattern="UCF_USELESS_CONTROL_FLOW"/>
+	</Match>
+	<Match>
+		<Bug pattern="UL_UNRELEASED_LOCK"/>
+	</Match>
+	<Match>
+		<Bug pattern="UMAC_UNCALLABLE_METHOD_OF_ANONYMOUS_CLASS"/>
+	</Match>
+	<Match>
+		<Bug pattern="UPM_UNCALLED_PRIVATE_METHOD"/>
+	</Match>
+	<Match>
+		<Bug pattern="URF_UNREAD_FIELD"/>
+	</Match>
+	<Match>
+		<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD"/>
+	</Match>
+	<Match>
+		<Bug pattern="UUF_UNUSED_FIELD"/>
+	</Match>
+	<Match>
+		<Bug pattern="UUF_UNUSED_PUBLIC_OR_PROTECTED_FIELD"/>
+	</Match>
+	<Match>
+		<Bug pattern="UW_UNCOND_WAIT"/>
+	</Match>
+	<Match>
+		<Bug pattern="UWF_FIELD_NOT_INITIALIZED_IN_CONSTRUCTOR"/>
+	</Match>
+	<Match>
+		<Bug pattern="UWF_UNWRITTEN_FIELD"/>
+	</Match>
+	<Match>
+		<Bug pattern="UWF_UNWRITTEN_PUBLIC_OR_PROTECTED_FIELD"/>
+	</Match>
+	<Match>
+		<Bug pattern="VO_VOLATILE_INCREMENT"/>
+	</Match>
+	<Match>
+		<Bug pattern="VO_VOLATILE_REFERENCE_TO_ARRAY"/>
+	</Match>
+	<Match>
+		<Bug pattern="WA_NOT_IN_LOOP"/>
+	</Match>
+	<Match>
+		<Bug pattern="WMI_WRONG_MAP_ITERATOR"/>
+	</Match>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/flink/blob/a1644076/tools/travis_mvn_watchdog.sh
----------------------------------------------------------------------
diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh
index b837fbd..a3790ea 100755
--- a/tools/travis_mvn_watchdog.sh
+++ b/tools/travis_mvn_watchdog.sh
@@ -133,6 +133,7 @@ case $TEST in
 		# compile everything since dist needs it anyway
 		MVN_COMPILE_MODULES=""
 		MVN_TEST_MODULES="-pl $NEGATED_CORE,$NEGATED_LIBRARIES,$NEGATED_CONNECTORS,$NEGATED_TESTS"
+		MVN_COMPILE_OPTIONS="-Dspotbugs"
 	;;
 esac
 


[09/11] flink git commit: [hotfix] [gelly] Explicit type can be replaced with <>

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CirculantGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CirculantGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CirculantGraphTest.java
index b7197a4..79c3468 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CirculantGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CirculantGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -82,8 +80,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CompleteGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CompleteGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CompleteGraphTest.java
index 1791f2e..45446b3 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CompleteGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CompleteGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -80,8 +78,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CycleGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CycleGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CycleGraphTest.java
index e4e2960..dd0912a 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CycleGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/CycleGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -79,8 +77,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EchoGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EchoGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EchoGraphTest.java
index fc64d62..5d8bfc4 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EchoGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EchoGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -124,8 +122,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EmptyGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EmptyGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EmptyGraphTest.java
index 939d871..3590922 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EmptyGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/EmptyGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -74,8 +72,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/GridGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/GridGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/GridGraphTest.java
index c40d456..7281770 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/GridGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/GridGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -89,8 +87,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/HypercubeGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/HypercubeGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/HypercubeGraphTest.java
index 05f84cd..3ecd4a4 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/HypercubeGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/HypercubeGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -81,8 +79,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/PathGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/PathGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/PathGraphTest.java
index 19b0f25..6817e92 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/PathGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/PathGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -79,8 +77,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/RMatGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/RMatGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/RMatGraphTest.java
index 920fc4e..5f18a73 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/RMatGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/RMatGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory;
 import org.apache.flink.graph.generator.random.RandomGenerableFactory;
 import org.apache.flink.types.LongValue;
@@ -66,8 +64,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/SingletonEdgeGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/SingletonEdgeGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/SingletonEdgeGraphTest.java
index 045354d..f78b7ff 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/SingletonEdgeGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/SingletonEdgeGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -80,8 +78,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/StarGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/StarGraphTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/StarGraphTest.java
index ee9df74..9e9da5f 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/StarGraphTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/StarGraphTest.java
@@ -19,9 +19,7 @@
 package org.apache.flink.graph.generator;
 
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
@@ -81,8 +79,8 @@ extends GraphGeneratorTestBase {
 			.setParallelism(parallelism)
 			.generate();
 
-		graph.getVertices().output(new DiscardingOutputFormat<Vertex<LongValue, NullValue>>());
-		graph.getEdges().output(new DiscardingOutputFormat<Edge<LongValue, NullValue>>());
+		graph.getVertices().output(new DiscardingOutputFormat<>());
+		graph.getEdges().output(new DiscardingOutputFormat<>());
 
 		TestUtils.verifyParallelism(env, parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
index 70a2d15..c94e169 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
@@ -60,7 +60,7 @@ public class GSACompilerTest extends CompilerTestBase {
 		// compose test program
 
 		DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple3<>(
-			1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<Long, NullValue>());
+			1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<>());
 
 		Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new InitVertices(), env);
 
@@ -68,7 +68,7 @@ public class GSACompilerTest extends CompilerTestBase {
 			new GatherNeighborIds(), new SelectMinId(),
 			new UpdateComponentId(), 100).getVertices();
 
-		result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+		result.output(new DiscardingOutputFormat<>());
 
 		Plan p = env.createProgramPlan("GSA Connected Components");
 		OptimizedPlan op = compileNoStats(p);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
index 0dfbcb7..0501820 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
@@ -72,7 +72,7 @@ public class GSATranslationTest {
 		// ------------ construct the test program ------------------
 
 		DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple3<>(
-			1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<Long, NullValue>());
+			1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<>());
 
 		Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new InitVertices(), env);
 
@@ -89,7 +89,7 @@ public class GSATranslationTest {
 			new GatherNeighborIds(), new SelectMinId(),
 			new UpdateComponentId(), NUM_ITERATIONS, parameters).getVertices();
 
-		result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+		result.output(new DiscardingOutputFormat<>());
 
 		// ------------- validate the java program ----------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/ConnectedComponentsWithRandomisedEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/ConnectedComponentsWithRandomisedEdgesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/ConnectedComponentsWithRandomisedEdgesITCase.java
index 3e76005..8b8927e 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/ConnectedComponentsWithRandomisedEdgesITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/ConnectedComponentsWithRandomisedEdgesITCase.java
@@ -62,7 +62,7 @@ public class ConnectedComponentsWithRandomisedEdgesITCase extends JavaProgramTes
 
 		Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
 
-		DataSet<Vertex<Long, Long>> result = graph.run(new ConnectedComponents<Long, Long, NullValue>(100));
+		DataSet<Vertex<Long, Long>> result = graph.run(new ConnectedComponents<>(100));
 
 		result.writeAsCsv(resultPath, "\n", " ");
 		env.execute();

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
index a0aba65..626b754 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
@@ -25,7 +25,6 @@ import org.apache.flink.graph.library.clustering.directed.LocalClusteringCoeffic
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
 
 import org.apache.commons.math3.util.CombinatoricsUtils;
 import org.junit.Test;
@@ -52,7 +51,7 @@ extends AsmTestBase {
 			"(5,1,0)";
 
 		DataSet<Result<IntValue>> cc = directedSimpleGraph
-			.run(new LocalClusteringCoefficient<IntValue, NullValue, NullValue>());
+			.run(new LocalClusteringCoefficient<>());
 
 		TestBaseUtils.compareResultAsText(cc.collect(), expectedResult);
 	}
@@ -64,7 +63,7 @@ extends AsmTestBase {
 		long expectedTriangleCount = 2 * CombinatoricsUtils.binomialCoefficient((int) expectedDegree, 2);
 
 		DataSet<Result<LongValue>> cc = completeGraph
-			.run(new LocalClusteringCoefficient<LongValue, NullValue, NullValue>());
+			.run(new LocalClusteringCoefficient<>());
 
 		List<Result<LongValue>> results = cc.collect();
 
@@ -80,7 +79,7 @@ extends AsmTestBase {
 	public void testRMatGraph()
 			throws Exception {
 		DataSet<Result<LongValue>> cc = directedRMatGraph(10, 16)
-			.run(new LocalClusteringCoefficient<LongValue, NullValue, NullValue>());
+			.run(new LocalClusteringCoefficient<>());
 
 		Checksum checksum = new org.apache.flink.graph.asm.dataset.ChecksumHashCode<Result<LongValue>>()
 			.run(cc)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
index e8f8659..2d02907 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
@@ -95,7 +95,7 @@ extends AsmTestBase {
 		long expectedCount = completeGraphVertexCount * CombinatoricsUtils.binomialCoefficient((int) expectedDegree, 2) / 3;
 
 		DataSet<Result<LongValue>> tl = completeGraph
-			.run(new TriangleListing<LongValue, NullValue, NullValue>());
+			.run(new TriangleListing<>());
 
 		List<Result<LongValue>> results = tl.collect();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
index e00669b..d77707e 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoeff
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
 
 import org.apache.commons.math3.util.CombinatoricsUtils;
 import org.junit.Test;
@@ -53,7 +52,7 @@ extends AsmTestBase {
 			"(5,1,0)";
 
 		DataSet<Result<IntValue>> cc = undirectedSimpleGraph
-			.run(new LocalClusteringCoefficient<IntValue, NullValue, NullValue>());
+			.run(new LocalClusteringCoefficient<>());
 
 		TestBaseUtils.compareResultAsText(cc.collect(), expectedResult);
 	}
@@ -65,7 +64,7 @@ extends AsmTestBase {
 		long expectedTriangleCount = CombinatoricsUtils.binomialCoefficient((int) expectedDegree, 2);
 
 		DataSet<Result<LongValue>> cc = completeGraph
-			.run(new LocalClusteringCoefficient<LongValue, NullValue, NullValue>());
+			.run(new LocalClusteringCoefficient<>());
 
 		List<Result<LongValue>> results = cc.collect();
 
@@ -81,7 +80,7 @@ extends AsmTestBase {
 	public void testRMatGraph()
 			throws Exception {
 		DataSet<Result<LongValue>> cc = undirectedRMatGraph(10, 16)
-			.run(new LocalClusteringCoefficient<LongValue, NullValue, NullValue>());
+			.run(new LocalClusteringCoefficient<>());
 
 		Checksum checksum = new ChecksumHashCode<Result<LongValue>>()
 			.run(cc)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
index 6af1b01..2e34945 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
@@ -95,7 +95,7 @@ extends AsmTestBase {
 		long expectedCount = completeGraphVertexCount * CombinatoricsUtils.binomialCoefficient((int) expectedDegree, 2) / 3;
 
 		DataSet<Result<LongValue>> tl = completeGraph
-			.run(new TriangleListing<LongValue, NullValue, NullValue>());
+			.run(new TriangleListing<>());
 
 		Checksum checksum = new ChecksumHashCode<Result<LongValue>>()
 			.run(tl)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/linkanalysis/HITSTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/linkanalysis/HITSTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/linkanalysis/HITSTest.java
index 4471105..2c94260 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/linkanalysis/HITSTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/linkanalysis/HITSTest.java
@@ -115,7 +115,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Result<LongValue>> hits = directedRMatGraph(10, 16)
-			.run(new HITS<LongValue, NullValue, NullValue>(0.000001));
+			.run(new HITS<>(0.000001));
 
 		Map<Long, Result<LongValue>> results = new HashMap<>();
 		for (Result<LongValue> result :  new Collect<Result<LongValue>>().run(hits).execute()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/ChecksumHashCodeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/ChecksumHashCodeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/ChecksumHashCodeTest.java
index 9b1d18c..4deb491 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/ChecksumHashCodeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/ChecksumHashCodeTest.java
@@ -41,7 +41,7 @@ extends AsmTestBase {
 			env);
 
 		Checksum checksum = graph
-			.run(new ChecksumHashCode<Long, Long, Long>())
+			.run(new ChecksumHashCode<>())
 			.execute();
 
 		assertEquals(checksum.getCount(), 12L);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
index aa259a2..5afd0ee 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
@@ -49,7 +49,7 @@ extends AsmTestBase {
 	public void testSimpleGraph()
 			throws Exception {
 		DataSet<Result<IntValue>> aa = undirectedSimpleGraph
-			.run(new AdamicAdar<IntValue, NullValue, NullValue>());
+			.run(new AdamicAdar<>());
 
 		String expectedResult =
 			"(0,1," + ilog[2] + ")\n" +
@@ -105,7 +105,7 @@ extends AsmTestBase {
 		float expectedScore = (completeGraphVertexCount - 2) / (float) Math.log(completeGraphVertexCount - 1);
 
 		DataSet<Result<LongValue>> aa = completeGraph
-			.run(new AdamicAdar<LongValue, NullValue, NullValue>());
+			.run(new AdamicAdar<>());
 
 		for (Result<LongValue> result : aa.collect()) {
 			assertEquals(expectedScore, result.getAdamicAdarScore().getValue(), 0.00001);
@@ -116,7 +116,7 @@ extends AsmTestBase {
 	public void testRMatGraph()
 			throws Exception {
 		DataSet<Result<LongValue>> aa = undirectedRMatGraph(8, 8)
-			.run(new AdamicAdar<LongValue, NullValue, NullValue>());
+			.run(new AdamicAdar<>());
 
 		assertEquals(13954, aa.count());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
index d8cd298..2e59f93 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
@@ -42,7 +42,7 @@ extends AsmTestBase {
 	public void testSimpleGraph()
 			throws Exception {
 		DataSet<Result<IntValue>> ji = undirectedSimpleGraph
-			.run(new JaccardIndex<IntValue, NullValue, NullValue>());
+			.run(new JaccardIndex<>());
 
 		String expectedResult =
 			"(0,1,1,4)\n" +

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelCompilerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelCompilerTest.java
index 71937db..95dd96e 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelCompilerTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelCompilerTest.java
@@ -61,7 +61,7 @@ public class PregelCompilerTest extends CompilerTestBase {
 
 			DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
 				new Tuple2<>(1L, 1L), new Tuple2<>(2L, 2L))
-				.map(new Tuple2ToVertexMap<Long, Long>());
+				.map(new Tuple2ToVertexMap<>());
 
 			DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<>(1L, 2L))
 				.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
@@ -76,7 +76,7 @@ public class PregelCompilerTest extends CompilerTestBase {
 			DataSet<Vertex<Long, Long>> result = graph.runVertexCentricIteration(
 				new CCCompute(), null, 100).getVertices();
 
-			result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+			result.output(new DiscardingOutputFormat<>());
 		}
 
 		Plan p = env.createProgramPlan("Pregel Connected Components");
@@ -126,7 +126,7 @@ public class PregelCompilerTest extends CompilerTestBase {
 
 			DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
 				new Tuple2<>(1L, 1L), new Tuple2<>(2L, 2L))
-				.map(new Tuple2ToVertexMap<Long, Long>());
+				.map(new Tuple2ToVertexMap<>());
 
 			DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<>(1L, 2L))
 				.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
@@ -145,7 +145,7 @@ public class PregelCompilerTest extends CompilerTestBase {
 				new CCCompute(), null, 100, parameters)
 				.getVertices();
 
-			result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+			result.output(new DiscardingOutputFormat<>());
 		}
 
 		Plan p = env.createProgramPlan("Pregel Connected Components");
@@ -192,7 +192,7 @@ public class PregelCompilerTest extends CompilerTestBase {
 
 			DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
 				new Tuple2<>(1L, 1L), new Tuple2<>(2L, 2L))
-				.map(new Tuple2ToVertexMap<Long, Long>());
+				.map(new Tuple2ToVertexMap<>());
 
 			DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<>(1L, 2L))
 				.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
@@ -207,7 +207,7 @@ public class PregelCompilerTest extends CompilerTestBase {
 			DataSet<Vertex<Long, Long>> result = graph.runVertexCentricIteration(
 				new CCCompute(), new CCCombiner(), 100).getVertices();
 
-			result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+			result.output(new DiscardingOutputFormat<>());
 		}
 
 		Plan p = env.createProgramPlan("Pregel Connected Components");

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelTranslationTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelTranslationTest.java
index 8084e71..0e79f65 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelTranslationTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/pregel/PregelTranslationTest.java
@@ -88,7 +88,7 @@ public class PregelTranslationTest {
 		result = graph.runVertexCentricIteration(new MyCompute(), null,
 			NUM_ITERATIONS, parameters).getVertices();
 
-		result.output(new DiscardingOutputFormat<Vertex<String, Double>>());
+		result.output(new DiscardingOutputFormat<>());
 
 		// ------------- validate the java program ----------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
index 1c6d08e..901276c 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
@@ -62,7 +62,7 @@ public class SpargelCompilerTest extends CompilerTestBase {
 		// compose test program
 		DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
 			new Tuple2<>(1L, 1L), new Tuple2<>(2L, 2L))
-			.map(new Tuple2ToVertexMap<Long, Long>());
+			.map(new Tuple2ToVertexMap<>());
 
 		DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<>(1L, 2L))
 			.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
@@ -75,11 +75,11 @@ public class SpargelCompilerTest extends CompilerTestBase {
 		Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
 
 		DataSet<Vertex<Long, Long>> result = graph.runScatterGatherIteration(
-			new ConnectedComponents.CCMessenger<Long, Long>(BasicTypeInfo.LONG_TYPE_INFO),
-			new ConnectedComponents.CCUpdater<Long, Long>(), 100)
+			new ConnectedComponents.CCMessenger<>(BasicTypeInfo.LONG_TYPE_INFO),
+			new ConnectedComponents.CCUpdater<>(), 100)
 			.getVertices();
 
-		result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+		result.output(new DiscardingOutputFormat<>());
 
 		Plan p = env.createProgramPlan("Spargel Connected Components");
 		OptimizedPlan op = compileNoStats(p);
@@ -136,7 +136,7 @@ public class SpargelCompilerTest extends CompilerTestBase {
 
 		DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
 			new Tuple2<>(1L, 1L), new Tuple2<>(2L, 2L))
-			.map(new Tuple2ToVertexMap<Long, Long>());
+			.map(new Tuple2ToVertexMap<>());
 
 		DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<>(1L, 2L))
 			.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
@@ -153,11 +153,11 @@ public class SpargelCompilerTest extends CompilerTestBase {
 		parameters.addBroadcastSetForGatherFunction(broadcastVariableName, bcVar);
 
 		DataSet<Vertex<Long, Long>> result = graph.runScatterGatherIteration(
-			new ConnectedComponents.CCMessenger<Long, Long>(BasicTypeInfo.LONG_TYPE_INFO),
-			new ConnectedComponents.CCUpdater<Long, Long>(), 100)
+			new ConnectedComponents.CCMessenger<>(BasicTypeInfo.LONG_TYPE_INFO),
+			new ConnectedComponents.CCUpdater<>(), 100)
 			.getVertices();
 
-		result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+		result.output(new DiscardingOutputFormat<>());
 
 		Plan p = env.createProgramPlan("Spargel Connected Components");
 		OptimizedPlan op = compileNoStats(p);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
index d209a2d..cbbf0c4 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
@@ -91,7 +91,7 @@ public class SpargelTranslationTest {
 		result = graph.runScatterGatherIteration(new MessageFunctionNoEdgeValue(), new UpdateFunction(),
 			NUM_ITERATIONS, parameters).getVertices();
 
-		result.output(new DiscardingOutputFormat<Vertex<String, Double>>());
+		result.output(new DiscardingOutputFormat<>());
 
 		// ------------- validate the java program ----------------
 
@@ -154,7 +154,7 @@ public class SpargelTranslationTest {
 		result = graph.runScatterGatherIteration(new MessageFunctionNoEdgeValue(), new UpdateFunction(),
 			NUM_ITERATIONS, parameters).getVertices();
 
-		result.output(new DiscardingOutputFormat<Vertex<String, Double>>());
+		result.output(new DiscardingOutputFormat<>());
 
 		// ------------- validate the java program ----------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
index 2454b38..8b1ab91 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
@@ -21,7 +21,6 @@ package org.apache.flink.graph.test;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.spargel.GatherFunction;
@@ -53,8 +52,9 @@ public class CollectionModeSuperstepITCase extends TestLogger {
 				new MessageFunction(), new UpdateFunction(), 10);
 
 		result.getVertices().map(
-				new VertexToTuple2Map<Long, Long>()).output(
-						new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			new VertexToTuple2Map<>()).output(
+				new DiscardingOutputFormat<>());
+
 		env.execute();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/ScatterGatherConfigurationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/ScatterGatherConfigurationITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/ScatterGatherConfigurationITCase.java
index 139ff1e..94d981c 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/ScatterGatherConfigurationITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/ScatterGatherConfigurationITCase.java
@@ -137,7 +137,7 @@ public class ScatterGatherConfigurationITCase extends MultipleProgramsTestBase {
 		Graph<Long, Long, Long> res = graph.runScatterGatherIteration(
 			new MessageFunctionDefault(), new UpdateFunctionDefault(), 5);
 
-		DataSet<Tuple2<Long, Long>> data = res.getVertices().map(new VertexToTuple2Map<Long, Long>());
+		DataSet<Tuple2<Long, Long>> data = res.getVertices().map(new VertexToTuple2Map<>());
 		List<Tuple2<Long, Long>> result = data.collect();
 
 		expectedResult = "1,6\n" +

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
index 8b726f4..f01daec 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
@@ -20,14 +20,12 @@ package org.apache.flink.graph.test.operations;
 
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.test.TestGraphUtils;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.util.TestEnvironment;
-import org.apache.flink.types.LongValue;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.AfterClass;
@@ -77,7 +75,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
 
 		try {
-			graph.outDegrees().output(new DiscardingOutputFormat<Tuple2<Long, LongValue>>());
+			graph.outDegrees().output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("graph.outDegrees() did not fail.");
@@ -100,7 +98,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
 
 		try {
-			graph.inDegrees().output(new DiscardingOutputFormat<Tuple2<Long, LongValue>>());
+			graph.inDegrees().output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("graph.inDegrees() did not fail.");
@@ -123,7 +121,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
 
 		try {
-			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, LongValue>>());
+			graph.getDegrees().output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("graph.getDegrees() did not fail.");
@@ -146,7 +144,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
 
 		try {
-			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, LongValue>>());
+			graph.getDegrees().output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("graph.getDegrees() did not fail.");
@@ -169,7 +167,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 				TestGraphUtils.getLongLongEdgeInvalidSrcTrgData(env), env);
 
 		try {
-			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, LongValue>>());
+			graph.getDegrees().output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("graph.getDegrees() did not fail.");

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
index 991a420..eb2fe5b 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
@@ -122,7 +122,7 @@ public class GraphCreationITCase extends MultipleProgramsTestBase {
 		DataSet<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdgeData(env);
 
 		Graph<Long, Long, Long> graph = Graph.fromDataSet(vertices, edges, env);
-		Boolean valid = graph.validate(new InvalidVertexIdsValidator<Long, Long, Long>());
+		Boolean valid = graph.validate(new InvalidVertexIdsValidator<>());
 
 		//env.fromElements(result).writeAsText(resultPath);
 
@@ -144,7 +144,7 @@ public class GraphCreationITCase extends MultipleProgramsTestBase {
 		DataSet<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdgeData(env);
 
 		Graph<Long, Long, Long> graph = Graph.fromDataSet(vertices, edges, env);
-		Boolean valid = graph.validate(new InvalidVertexIdsValidator<Long, Long, Long>());
+		Boolean valid = graph.validate(new InvalidVertexIdsValidator<>());
 
 		String res = valid.toString(); //env.fromElements(valid);
 		List<String> result = new LinkedList<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
index 43ff124..e3909ca 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
@@ -62,7 +62,7 @@ public class JoinWithEdgesITCase extends MultipleProgramsTestBase {
 			TestGraphUtils.getLongLongEdgeData(env), env);
 
 		Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges()
-			.map(new EdgeToTuple3Map<Long, Long>()), new AddValuesMapper());
+			.map(new EdgeToTuple3Map<>()), new AddValuesMapper());
 
 		DataSet<Edge<Long, Long>> data = res.getEdges();
 		List<Edge<Long, Long>> result = data.collect();
@@ -90,7 +90,7 @@ public class JoinWithEdgesITCase extends MultipleProgramsTestBase {
 			TestGraphUtils.getLongLongEdgeData(env), env);
 
 		Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges().first(3)
-			.map(new EdgeToTuple3Map<Long, Long>()), new AddValuesMapper());
+			.map(new EdgeToTuple3Map<>()), new AddValuesMapper());
 
 		DataSet<Edge<Long, Long>> data = res.getEdges();
 		List<Edge<Long, Long>> result = data.collect();

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
index 181c1a7..fe5c52b 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
@@ -60,7 +60,7 @@ public class JoinWithVerticesITCase extends MultipleProgramsTestBase {
 			TestGraphUtils.getLongLongEdgeData(env), env);
 
 		Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices()
-			.map(new VertexToTuple2Map<Long, Long>()), new AddValuesMapper());
+			.map(new VertexToTuple2Map<>()), new AddValuesMapper());
 
 		DataSet<Vertex<Long, Long>> data = res.getVertices();
 		List<Vertex<Long, Long>> result = data.collect();
@@ -86,7 +86,7 @@ public class JoinWithVerticesITCase extends MultipleProgramsTestBase {
 			TestGraphUtils.getLongLongEdgeData(env), env);
 
 		Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices().first(3)
-			.map(new VertexToTuple2Map<Long, Long>()), new AddValuesMapper());
+			.map(new VertexToTuple2Map<>()), new AddValuesMapper());
 
 		DataSet<Vertex<Long, Long>> data = res.getVertices();
 		List<Vertex<Long, Long>> result = data.collect();

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
index 19e701d..6733ba1 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
@@ -84,7 +84,7 @@ public class ReduceOnEdgesWithExceptionITCase extends TestLogger {
 			DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
 					graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
 
-			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			verticesWithAllNeighbors.output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("Expected an exception.");
@@ -110,7 +110,7 @@ public class ReduceOnEdgesWithExceptionITCase extends TestLogger {
 			DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
 					graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
 
-			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			verticesWithAllNeighbors.output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("Expected an exception.");

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
index d3b97a1..1a60874 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
@@ -86,7 +86,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 			DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
 					graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
 
-			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("Expected an exception.");
@@ -113,7 +113,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 			DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
 					graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
 
-			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<>());
 			env.execute();
 
 			fail("Expected an exception.");
@@ -140,7 +140,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 			DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
 					graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
 
-			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<>());
 			env.execute();
 		} catch (Exception e) {
 			// We expect the job to fail with an exception
@@ -165,7 +165,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 			DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
 					graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
 
-			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<>());
 			env.execute();
 		} catch (Exception e) {
 			// We expect the job to fail with an exception

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/TypeExtractorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/TypeExtractorTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/TypeExtractorTest.java
index 4d9040c..cad07c0 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/TypeExtractorTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/TypeExtractorTest.java
@@ -58,7 +58,7 @@ public class TypeExtractorTest {
 	public void testMapVerticesType() throws Exception {
 
 		// test type extraction in mapVertices
-		DataSet<Vertex<Long, Tuple2<Long, Integer>>> outVertices = inputGraph.mapVertices(new VertexMapper<Long>()).getVertices();
+		DataSet<Vertex<Long, Tuple2<Long, Integer>>> outVertices = inputGraph.mapVertices(new VertexMapper<>()).getVertices();
 		Assert.assertTrue(new TupleTypeInfo(Vertex.class, BasicTypeInfo.LONG_TYPE_INFO,
 			new TupleTypeInfo<Tuple2<Long, Integer>>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO))
 			.equals(outVertices.getType()));
@@ -68,7 +68,7 @@ public class TypeExtractorTest {
 	public void testMapEdgesType() throws Exception {
 
 		// test type extraction in mapEdges
-		DataSet<Edge<Long, Tuple2<Long, Integer>>> outEdges = inputGraph.mapEdges(new EdgeMapper<Long>()).getEdges();
+		DataSet<Edge<Long, Tuple2<Long, Integer>>> outEdges = inputGraph.mapEdges(new EdgeMapper<>()).getEdges();
 		Assert.assertTrue(new TupleTypeInfo(Edge.class, BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO,
 			new TupleTypeInfo<Tuple2<Long, Integer>>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO))
 			.equals(outEdges.getType()));
@@ -76,7 +76,7 @@ public class TypeExtractorTest {
 
 	@Test
 	public void testFromDataSet() throws Exception {
-		DataSet<Vertex<Long, Tuple2<Long, Integer>>> outVertices = Graph.fromDataSet(edges, new VertexInitializer<Long>(), env)
+		DataSet<Vertex<Long, Tuple2<Long, Integer>>> outVertices = Graph.fromDataSet(edges, new VertexInitializer<>(), env)
 			.getVertices();
 		Assert.assertTrue(new TupleTypeInfo(Vertex.class, BasicTypeInfo.LONG_TYPE_INFO,
 			new TupleTypeInfo<Tuple2<Long, Integer>>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO))
@@ -85,7 +85,7 @@ public class TypeExtractorTest {
 
 	@Test
 	public void testGroupReduceOnEdges() throws Exception {
-		DataSet<Tuple2<Long, Long>> output = inputGraph.groupReduceOnEdges(new EdgesGroupFunction<Long, Long>(), EdgeDirection.OUT);
+		DataSet<Tuple2<Long, Long>> output = inputGraph.groupReduceOnEdges(new EdgesGroupFunction<>(), EdgeDirection.OUT);
 		Assert.assertTrue((new TupleTypeInfo<Tuple2<Long, Long>>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO)).equals(output.getType()));
 	}
 
@@ -113,7 +113,7 @@ public class TypeExtractorTest {
 
 		@Override
 		public void iterateEdges(Iterable<Tuple2<K, Edge<K, EV>>> edges, Collector<Tuple2<K, EV>> out) throws Exception {
-			out.collect(new Tuple2<K, EV>());
+			out.collect(new Tuple2<>());
 		}
 	}
 


[05/11] flink git commit: [FLINK-7343][kafka] Increase Xmx for tests

Posted by ch...@apache.org.
[FLINK-7343][kafka] Increase Xmx for tests

Sometimes 1000m was not enough memory to run at-least-once tests with broker failures on Travis

This closes #4456.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4406d486
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4406d486
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4406d486

Branch: refs/heads/master
Commit: 4406d4868320c72ce7c744748cbd7528ff4bc642
Parents: be8eb1a
Author: Piotr Nowojski <pi...@gmail.com>
Authored: Tue Aug 1 16:05:49 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 flink-connectors/flink-connector-kafka-0.10/pom.xml | 1 -
 flink-connectors/flink-connector-kafka-0.9/pom.xml  | 1 -
 2 files changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4406d486/flink-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml
index 0ecaebc..581640d 100644
--- a/flink-connectors/flink-connector-kafka-0.10/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml
@@ -204,7 +204,6 @@ under the License.
 				<configuration>
 					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
 					<forkCount>1</forkCount>
-					<argLine>-Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
 				</configuration>
 			</plugin>
 		</plugins>

http://git-wip-us.apache.org/repos/asf/flink/blob/4406d486/flink-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml
index 248f7e1..fef070d 100644
--- a/flink-connectors/flink-connector-kafka-0.9/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml
@@ -192,7 +192,6 @@ under the License.
 				<configuration>
 					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
 					<forkCount>1</forkCount>
-					<argLine>-Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
 				</configuration>
 			</plugin>
 			<!--


[02/11] flink git commit: [FLINK-7013] Introduce flink-shaded-netty-4

Posted by ch...@apache.org.
[FLINK-7013] Introduce flink-shaded-netty-4

This closes #4452.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/1ceb89a9
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/1ceb89a9
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/1ceb89a9

Branch: refs/heads/master
Commit: 1ceb89a979d560944e1099fa4700e46c09a79484
Parents: 614c18d
Author: zentol <ch...@apache.org>
Authored: Tue Jun 27 14:27:16 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 flink-mesos/pom.xml                             |  4 +-
 .../flink/mesos/util/MesosArtifactServer.java   | 79 ++++++++++----------
 flink-runtime-web/pom.xml                       |  4 +-
 .../runtime/webmonitor/HttpRequestHandler.java  | 44 +++++------
 .../webmonitor/PipelineErrorHandler.java        | 17 +++--
 .../webmonitor/RuntimeMonitorHandler.java       | 23 +++---
 .../webmonitor/RuntimeMonitorHandlerBase.java   | 12 +--
 .../runtime/webmonitor/WebRuntimeMonitor.java   |  3 +-
 .../files/StaticFileServerHandler.java          | 67 +++++++++--------
 .../handlers/AbstractJsonRequestHandler.java    | 12 +--
 .../handlers/ConstantTextHandler.java           | 22 +++---
 .../handlers/HandlerRedirectUtils.java          | 13 ++--
 .../JobCancellationWithSavepointHandlers.java   | 13 ++--
 .../webmonitor/handlers/RequestHandler.java     |  2 +-
 .../handlers/TaskManagerLogHandler.java         | 45 +++++------
 .../webmonitor/history/HistoryServer.java       |  3 +-
 .../HistoryServerStaticFileServerHandler.java   | 43 +++++------
 .../webmonitor/utils/WebFrontendBootstrap.java  | 25 ++++---
 .../runtime/webmonitor/WebFrontendITCase.java   |  3 +-
 .../webmonitor/WebRuntimeMonitorITCase.java     |  3 +-
 ...obCancellationWithSavepointHandlersTest.java |  7 +-
 .../handlers/TaskManagerLogHandlerTest.java     | 13 ++--
 ...istoryServerStaticFileServerHandlerTest.java |  3 +-
 .../webmonitor/testutils/HttpTestClient.java    | 51 ++++++-------
 flink-runtime/pom.xml                           | 11 ++-
 .../executiongraph/ExecutionAttemptID.java      |  3 +-
 .../io/network/netty/NettyBufferPool.java       | 12 +--
 .../runtime/io/network/netty/NettyClient.java   | 30 ++++----
 .../runtime/io/network/netty/NettyMessage.java  | 22 +++---
 .../runtime/io/network/netty/NettyProtocol.java |  2 +-
 .../runtime/io/network/netty/NettyServer.java   | 27 ++++---
 .../network/netty/PartitionRequestClient.java   |  8 +-
 .../netty/PartitionRequestClientFactory.java    |  7 +-
 .../netty/PartitionRequestClientHandler.java    |  8 +-
 .../network/netty/PartitionRequestProtocol.java |  3 +-
 .../io/network/netty/PartitionRequestQueue.java | 14 ++--
 .../netty/PartitionRequestServerHandler.java    |  6 +-
 .../partition/consumer/InputChannelID.java      |  3 +-
 .../jobgraph/IntermediateResultPartitionID.java |  3 +-
 .../runtime/query/netty/ChunkedByteBuf.java     |  8 +-
 .../runtime/query/netty/KvStateClient.java      | 27 +++----
 .../query/netty/KvStateClientHandler.java       |  9 ++-
 .../runtime/query/netty/KvStateServer.java      | 21 +++---
 .../query/netty/KvStateServerHandler.java       | 15 ++--
 .../netty/message/KvStateRequestSerializer.java |  8 +-
 .../netty/CancelPartitionRequestTest.java       |  4 +-
 .../netty/ClientTransportErrorHandlingTest.java | 20 ++---
 .../network/netty/NettyClientServerSslTest.java | 10 ++-
 .../netty/NettyConnectionManagerTest.java       |  8 +-
 .../netty/NettyMessageSerializationTest.java    |  6 +-
 .../NettyServerLowAndHighWatermarkTest.java     | 13 ++--
 .../runtime/io/network/netty/NettyTestUtil.java |  3 +-
 .../PartitionRequestClientFactoryTest.java      | 10 ++-
 .../PartitionRequestClientHandlerTest.java      | 12 +--
 .../netty/PartitionRequestQueueTest.java        |  4 +-
 .../netty/ServerTransportErrorHandlingTest.java | 10 ++-
 .../query/netty/KvStateClientHandlerTest.java   |  5 +-
 .../runtime/query/netty/KvStateClientTest.java  | 23 +++---
 .../query/netty/KvStateServerHandlerTest.java   | 11 +--
 .../runtime/query/netty/KvStateServerTest.java  | 25 ++++---
 .../message/KvStateRequestSerializerTest.java   |  7 +-
 pom.xml                                         | 12 +--
 tools/maven/checkstyle.xml                      |  2 +-
 tools/travis_mvn_watchdog.sh                    |  9 +++
 64 files changed, 509 insertions(+), 443 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-mesos/pom.xml
----------------------------------------------------------------------
diff --git a/flink-mesos/pom.xml b/flink-mesos/pom.xml
index 4dae731..ab3499f 100644
--- a/flink-mesos/pom.xml
+++ b/flink-mesos/pom.xml
@@ -108,8 +108,8 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>tv.cntt</groupId>
-			<artifactId>netty-router</artifactId>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-netty</artifactId>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
index 967d818..2627d25 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
@@ -26,34 +26,35 @@ import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.net.SSLUtils;
 
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.DefaultHttpResponse;
-import io.netty.handler.codec.http.FullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpRequest;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpServerCodec;
-import io.netty.handler.codec.http.LastHttpContent;
-import io.netty.handler.codec.http.router.Handler;
-import io.netty.handler.codec.http.router.Routed;
-import io.netty.handler.codec.http.router.Router;
-import io.netty.handler.ssl.SslHandler;
-import io.netty.handler.stream.ChunkedStream;
-import io.netty.handler.stream.ChunkedWriteHandler;
-import io.netty.util.CharsetUtil;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpServerCodec;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Handler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Router;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedStream;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil;
+
 import org.jets3t.service.utils.Mimetypes;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,17 +72,17 @@ import java.util.Map;
 
 import scala.Option;
 
-import static io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
-import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
-import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static io.netty.handler.codec.http.HttpMethod.GET;
-import static io.netty.handler.codec.http.HttpMethod.HEAD;
-import static io.netty.handler.codec.http.HttpResponseStatus.GONE;
-import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
-import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
-import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
-import static io.netty.handler.codec.http.HttpResponseStatus.OK;
-import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod.GET;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod.HEAD;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.GONE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 
 /**
  * A generic Mesos artifact server, designed specifically for use by the Mesos Fetcher.

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml
index 9a3c29f..cf8f307 100644
--- a/flink-runtime-web/pom.xml
+++ b/flink-runtime-web/pom.xml
@@ -56,8 +56,8 @@ under the License.
 			=================================================== -->
 
 		<dependency>
-			<groupId>tv.cntt</groupId>
-			<artifactId>netty-router</artifactId>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-netty</artifactId>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
index b481c78..f8b51d7 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
@@ -29,28 +29,28 @@ package org.apache.flink.runtime.webmonitor;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.util.ExceptionUtils;
 
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.HttpContent;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpMethod;
-import io.netty.handler.codec.http.HttpObject;
-import io.netty.handler.codec.http.HttpRequest;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
-import io.netty.handler.codec.http.LastHttpContent;
-import io.netty.handler.codec.http.QueryStringDecoder;
-import io.netty.handler.codec.http.QueryStringEncoder;
-import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
-import io.netty.handler.codec.http.multipart.DiskFileUpload;
-import io.netty.handler.codec.http.multipart.HttpDataFactory;
-import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
-import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
-import io.netty.handler.codec.http.multipart.InterfaceHttpData;
-import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObject;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.QueryStringDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.QueryStringEncoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DiskFileUpload;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpDataFactory;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.InterfaceHttpData;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
 
 import java.io.File;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/PipelineErrorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/PipelineErrorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/PipelineErrorHandler.java
index 52622f6..4de2911 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/PipelineErrorHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/PipelineErrorHandler.java
@@ -21,14 +21,15 @@ package org.apache.flink.runtime.webmonitor;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.util.ExceptionUtils;
 
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+
 import org.slf4j.Logger;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
index c5ee081..4777202 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
@@ -23,17 +23,18 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
 import org.apache.flink.util.ExceptionUtils;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.FullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
-import io.netty.handler.codec.http.router.KeepAliveWrite;
-import io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.KeepAliveWrite;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandlerBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandlerBase.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandlerBase.java
index 99d5df5..d524632 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandlerBase.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandlerBase.java
@@ -22,12 +22,12 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
 import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
 
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.router.KeepAliveWrite;
-import io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.KeepAliveWrite;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
 
 import scala.Option;
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index 4c4f6e0..cffd23b 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -74,8 +74,9 @@ import org.apache.flink.runtime.webmonitor.metrics.TaskManagerMetricsHandler;
 import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
 import org.apache.flink.util.FileUtils;
 
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Router;
+
 import akka.actor.ActorSystem;
-import io.netty.handler.codec.http.router.Router;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
index 4efa926..be6928e 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
@@ -30,27 +30,28 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
 import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
 
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.DefaultFileRegion;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.DefaultHttpResponse;
-import io.netty.handler.codec.http.FullHttpResponse;
-import io.netty.handler.codec.http.HttpChunkedInput;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpRequest;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.LastHttpContent;
-import io.netty.handler.codec.http.router.KeepAliveWrite;
-import io.netty.handler.codec.http.router.Routed;
-import io.netty.handler.ssl.SslHandler;
-import io.netty.handler.stream.ChunkedFile;
-import io.netty.util.CharsetUtil;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.KeepAliveWrite;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
+import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -77,18 +78,18 @@ import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
-import static io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
-import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
-import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static io.netty.handler.codec.http.HttpHeaders.Names.DATE;
-import static io.netty.handler.codec.http.HttpHeaders.Names.EXPIRES;
-import static io.netty.handler.codec.http.HttpHeaders.Names.IF_MODIFIED_SINCE;
-import static io.netty.handler.codec.http.HttpHeaders.Names.LAST_MODIFIED;
-import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
-import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
-import static io.netty.handler.codec.http.HttpResponseStatus.NOT_MODIFIED;
-import static io.netty.handler.codec.http.HttpResponseStatus.OK;
-import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.DATE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.EXPIRES;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.IF_MODIFIED_SINCE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.LAST_MODIFIED;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_MODIFIED;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
index 6831eaf..2b4a45f 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
@@ -20,12 +20,12 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.runtime.instance.ActorGateway;
 
-import io.netty.buffer.Unpooled;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.FullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
 
 import java.nio.charset.Charset;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java
index 61e2958..34898e7 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ConstantTextHandler.java
@@ -20,17 +20,17 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.configuration.ConfigConstants;
 
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
-import io.netty.handler.codec.http.router.KeepAliveWrite;
-import io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.KeepAliveWrite;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
 
 /**
  * Responder that returns a constant String.

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
index 510291a..9fbafb8 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
@@ -23,12 +23,13 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.webmonitor.files.MimeTypes;
 
-import io.netty.buffer.Unpooled;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
index 7dcac7f..7dd4a52 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
@@ -30,14 +30,15 @@ import org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess;
 import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
 
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+
 import akka.dispatch.OnComplete;
 import com.fasterxml.jackson.core.JsonGenerator;
-import io.netty.buffer.Unpooled;
-import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.FullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
 
 import javax.annotation.Nullable;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
index 66c30af..8646df9 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import org.apache.flink.runtime.instance.ActorGateway;
 
-import io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
index ce29721..3562874 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
@@ -43,25 +43,26 @@ import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandlerBase;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StringUtils;
 
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener;
+
 import akka.dispatch.Mapper;
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.DefaultFileRegion;
-import io.netty.handler.codec.http.DefaultHttpResponse;
-import io.netty.handler.codec.http.HttpChunkedInput;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpRequest;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.LastHttpContent;
-import io.netty.handler.codec.http.router.Routed;
-import io.netty.handler.ssl.SslHandler;
-import io.netty.handler.stream.ChunkedFile;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.GenericFutureListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -81,10 +82,10 @@ import scala.concurrent.ExecutionContextExecutor;
 import scala.concurrent.duration.FiniteDuration;
 import scala.reflect.ClassTag$;
 
-import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
-import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static io.netty.handler.codec.http.HttpResponseStatus.OK;
-import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
index d86bfb2..6da8115 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
@@ -35,7 +35,8 @@ import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 
-import io.netty.handler.codec.http.router.Router;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Router;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
index c14f3d8..c5943dc 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
@@ -28,21 +28,22 @@ package org.apache.flink.runtime.webmonitor.history;
 
 import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler;
 
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.DefaultFileRegion;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.handler.codec.http.DefaultHttpResponse;
-import io.netty.handler.codec.http.HttpChunkedInput;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpRequest;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.LastHttpContent;
-import io.netty.handler.codec.http.router.Routed;
-import io.netty.handler.ssl.SslHandler;
-import io.netty.handler.stream.ChunkedFile;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,12 +60,12 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Locale;
 
-import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
-import static io.netty.handler.codec.http.HttpHeaders.Names.IF_MODIFIED_SINCE;
-import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
-import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
-import static io.netty.handler.codec.http.HttpResponseStatus.OK;
-import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.IF_MODIFIED_SINCE;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
index 2638626..2c47478 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
@@ -24,18 +24,19 @@ import org.apache.flink.runtime.webmonitor.HttpRequestHandler;
 import org.apache.flink.runtime.webmonitor.PipelineErrorHandler;
 import org.apache.flink.util.Preconditions;
 
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.codec.http.HttpServerCodec;
-import io.netty.handler.codec.http.router.Handler;
-import io.netty.handler.codec.http.router.Router;
-import io.netty.handler.ssl.SslHandler;
-import io.netty.handler.stream.ChunkedWriteHandler;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpServerCodec;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Handler;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Router;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
 import org.slf4j.Logger;
 
 import javax.net.ssl.SSLContext;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
index db2b7eb..acf0e3b 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
@@ -31,10 +31,11 @@ import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.util.TestLogger;
 
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
-import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 23d8a51..8c4d8c3 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -36,9 +36,10 @@ import org.apache.flink.runtime.webmonitor.files.MimeTypes;
 import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
 import org.apache.flink.util.TestLogger;
 
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
-import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.curator.test.TestingServer;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
index 71b715a..64a07c8 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
@@ -28,13 +28,14 @@ import org.apache.flink.runtime.messages.JobManagerMessages.CancelJobWithSavepoi
 import org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess;
 import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
 
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
 import akka.dispatch.ExecutionContexts$;
 import akka.dispatch.Futures;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import io.netty.handler.codec.http.FullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpResponseStatus;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
index e1c3686..5846d75 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
@@ -33,12 +33,13 @@ import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.handler.codec.http.DefaultFullHttpRequest;
-import io.netty.handler.codec.http.HttpMethod;
-import io.netty.handler.codec.http.HttpVersion;
-import io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
+
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandlerTest.java
index cee9248..066de74 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandlerTest.java
@@ -21,7 +21,8 @@ package org.apache.flink.runtime.webmonitor.history;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
 
-import io.netty.handler.codec.http.router.Router;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Router;
+
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java
index af9a869..94dd5f8 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java
@@ -18,31 +18,32 @@
 
 package org.apache.flink.runtime.webmonitor.testutils;
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.codec.http.DefaultFullHttpRequest;
-import io.netty.handler.codec.http.HttpClientCodec;
-import io.netty.handler.codec.http.HttpContent;
-import io.netty.handler.codec.http.HttpContentDecompressor;
-import io.netty.handler.codec.http.HttpHeaders;
-import io.netty.handler.codec.http.HttpMethod;
-import io.netty.handler.codec.http.HttpObject;
-import io.netty.handler.codec.http.HttpRequest;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
-import io.netty.handler.codec.http.LastHttpContent;
-import io.netty.util.CharsetUtil;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPipeline;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpClientCodec;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpContent;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpContentDecompressor;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpMethod;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObject;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 5d236bc..e37e9c5 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -57,6 +57,11 @@ under the License.
 		</dependency>
 
 		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-netty</artifactId>
+		</dependency>
+
+		<dependency>
 			<groupId>org.apache.commons</groupId>
 			<artifactId>commons-lang3</artifactId>
 		</dependency>
@@ -66,12 +71,6 @@ under the License.
 			<artifactId>commons-cli</artifactId>
 		</dependency>
 
-		<dependency>
-			<groupId>io.netty</groupId>
-			<artifactId>netty-all</artifactId>
-			<!-- Version is set in root POM -->
-		</dependency>
-
 		<!-- See: https://groups.google.com/forum/#!msg/netty/-aAPDBNUnDg/SkGOXL2Ma2QJ -->
 		<dependency>
 			<groupId>org.javassist</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java
index daf89c1..f0ff3bc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java
@@ -18,9 +18,10 @@
 
 package org.apache.flink.runtime.executiongraph;
 
-import io.netty.buffer.ByteBuf;
 import org.apache.flink.util.AbstractID;
 
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
 /**
  * Unique identifier for the attempt to execute a tasks. Multiple attempts happen
  * in cases of failures and recovery.

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java
index 175726b..dfd3c32 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java
@@ -18,16 +18,18 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.CompositeByteBuf;
-import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.CompositeByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.PooledByteBufAllocator;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
 
 import java.lang.reflect.Field;
 
+import scala.Option;
+
 import static org.apache.flink.util.Preconditions.checkArgument;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
index 9b0bb00..485af2d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyClient.java
@@ -18,17 +18,19 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.epoll.Epoll;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollSocketChannel;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.ssl.SslHandler;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelException;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.epoll.Epoll;
+import org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -197,14 +199,14 @@ class NettyClient {
 		try {
 			return bootstrap.connect(serverSocketAddress);
 		}
-		catch (io.netty.channel.ChannelException e) {
+		catch (ChannelException e) {
 			if ( (e.getCause() instanceof java.net.SocketException &&
 					e.getCause().getMessage().equals("Too many open files")) ||
-				(e.getCause() instanceof io.netty.channel.ChannelException &&
+				(e.getCause() instanceof ChannelException &&
 						e.getCause().getCause() instanceof java.net.SocketException &&
 						e.getCause().getCause().getMessage().equals("Too many open files")))
 			{
-				throw new io.netty.channel.ChannelException(
+				throw new ChannelException(
 						"The operating system does not offer enough file handles to open the network connection. " +
 								"Please increase the number of of available file handles.", e.getCause());
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
index 93775df..de50f82 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java
@@ -18,17 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.ByteBufInputStream;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPromise;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.codec.MessageToMessageDecoder;
-
 import org.apache.flink.runtime.event.TaskEvent;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
@@ -38,6 +27,17 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufInputStream;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufOutputStream;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOutboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPromise;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder;
+
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java
index 6b33cc9..bcfe558 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
 
 public interface NettyProtocol {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java
index ee3e923..e8727d2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java
@@ -18,24 +18,27 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.epoll.Epoll;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollServerSocketChannel;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.ssl.SslHandler;
 import org.apache.flink.runtime.util.FatalExitExceptionHandler;
+
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.epoll.Epoll;
+import org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLEngine;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.concurrent.ThreadFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
index 7c06b55..7850974 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java
@@ -18,15 +18,17 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
 import org.apache.flink.runtime.event.TaskEvent;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 import org.apache.flink.runtime.util.AtomicDisposableReferenceCounter;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
index 040a8ef..c559ed8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
@@ -18,14 +18,15 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
 import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+
 import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
index 9f80abc..7db7ac4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import com.google.common.collect.Maps;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -33,6 +30,11 @@ import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 import org.apache.flink.runtime.util.event.EventListener;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+
+import com.google.common.collect.Maps;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
index d06a018..b6614b6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java
@@ -18,10 +18,11 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.ChannelHandler;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+
 import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder;
 import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder.createFrameLengthDecoder;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
index 5330dac..881eae8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
@@ -18,12 +18,6 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import com.google.common.collect.Sets;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -31,6 +25,14 @@ import org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse;
 import org.apache.flink.runtime.io.network.partition.ProducerFailedException;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+
+import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
index 1bd05f6..bd867da 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
@@ -18,14 +18,16 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.netty.NettyMessage.CancelPartitionRequest;
 import org.apache.flink.runtime.io.network.netty.NettyMessage.CloseRequest;
 import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelID.java
index dc6e918..ceeb83d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelID.java
@@ -18,9 +18,10 @@
 
 package org.apache.flink.runtime.io.network.partition.consumer;
 
-import io.netty.buffer.ByteBuf;
 import org.apache.flink.util.AbstractID;
 
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
 public class InputChannelID extends AbstractID {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java
index acd1899..24d76c4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java
@@ -18,9 +18,10 @@
 
 package org.apache.flink.runtime.jobgraph;
 
-import io.netty.buffer.ByteBuf;
 import org.apache.flink.util.AbstractID;
 
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
 public class IntermediateResultPartitionID extends AbstractID {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java
index f0f082f..e4fa809 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java
@@ -20,10 +20,10 @@ package org.apache.flink.runtime.query.netty;
 
 import org.apache.flink.util.Preconditions;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.handler.stream.ChunkedInput;
-import io.netty.handler.stream.ChunkedWriteHandler;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
 
 /**
  * A {@link ByteBuf} instance to be consumed in chunks by {@link ChunkedWriteHandler},

http://git-wip-us.apache.org/repos/asf/flink/blob/1ceb89a9/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java
index 769e816..60099d2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java
@@ -24,21 +24,22 @@ import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.util.Preconditions;
 
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
 import akka.dispatch.Futures;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.stream.ChunkedWriteHandler;
 
 import java.nio.channels.ClosedChannelException;
 import java.util.ArrayDeque;


[10/11] flink git commit: [hotfix] [gelly] Explicit type can be replaced with <>

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegrees.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegrees.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegrees.java
index 57045a1..db87eb9 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegrees.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegrees.java
@@ -52,7 +52,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Edge<K, Tuple2<EV, Degrees>>> {
 			.join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND)
 			.where(1)
 			.equalTo(0)
-			.with(new JoinEdgeWithVertexDegree<K, EV, Degrees>())
+			.with(new JoinEdgeWithVertexDegree<>())
 				.setParallelism(parallelism)
 				.name("Edge target degrees");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegrees.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegrees.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegrees.java
index 06a7fd2..a703789 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegrees.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegrees.java
@@ -93,18 +93,18 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, Degrees>> {
 			throws Exception {
 		// s, t, bitmask
 		DataSet<Tuple2<K, ByteValue>> vertexWithEdgeOrder = input.getEdges()
-			.flatMap(new EmitAndFlipEdge<K, EV>())
+			.flatMap(new EmitAndFlipEdge<>())
 				.setParallelism(parallelism)
 				.name("Emit and flip edge")
 			.groupBy(0, 1)
-			.reduceGroup(new ReduceBitmask<K>())
+			.reduceGroup(new ReduceBitmask<>())
 				.setParallelism(parallelism)
 				.name("Reduce bitmask");
 
 		// s, d(s)
 		DataSet<Vertex<K, Degrees>> vertexDegrees = vertexWithEdgeOrder
 			.groupBy(0)
-			.reduceGroup(new DegreeCount<K>())
+			.reduceGroup(new DegreeCount<>())
 				.setParallelism(parallelism)
 				.name("Degree count");
 
@@ -113,7 +113,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, Degrees>> {
 				.leftOuterJoin(vertexDegrees)
 				.where(0)
 				.equalTo(0)
-				.with(new JoinVertexWithVertexDegrees<K, VV>())
+				.with(new JoinVertexWithVertexDegrees<>())
 					.setParallelism(parallelism)
 					.name("Zero degree vertices");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java
index dc071cf..38c7995 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java
@@ -84,14 +84,14 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, LongValue>> {
 		// t
 		DataSet<Vertex<K, LongValue>> targetIds = input
 			.getEdges()
-			.map(new MapEdgeToTargetId<K, EV>())
+			.map(new MapEdgeToTargetId<>())
 				.setParallelism(parallelism)
 				.name("Edge to target ID");
 
 		// t, d(t)
 		DataSet<Vertex<K, LongValue>> targetDegree = targetIds
 			.groupBy(0)
-			.reduce(new DegreeCount<K>())
+			.reduce(new DegreeCount<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Degree count");
@@ -101,7 +101,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, LongValue>> {
 				.leftOuterJoin(targetDegree)
 				.where(0)
 				.equalTo(0)
-				.with(new JoinVertexWithVertexDegree<K, VV>())
+				.with(new JoinVertexWithVertexDegree<>())
 					.setParallelism(parallelism)
 					.name("Zero degree vertices");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java
index 4a4689b..ef9c781 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java
@@ -84,14 +84,14 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, LongValue>> {
 		// s
 		DataSet<Vertex<K, LongValue>> sourceIds = input
 			.getEdges()
-			.map(new MapEdgeToSourceId<K, EV>())
+			.map(new MapEdgeToSourceId<>())
 				.setParallelism(parallelism)
 				.name("Edge to source ID");
 
 		// s, d(s)
 		DataSet<Vertex<K, LongValue>> sourceDegree = sourceIds
 			.groupBy(0)
-			.reduce(new DegreeCount<K>())
+			.reduce(new DegreeCount<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Degree count");
@@ -101,7 +101,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, LongValue>> {
 				.leftOuterJoin(sourceDegree)
 				.where(0)
 				.equalTo(0)
-				.with(new JoinVertexWithVertexDegree<K, VV>())
+				.with(new JoinVertexWithVertexDegree<>())
 					.setParallelism(parallelism)
 					.name("Zero degree vertices");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java
index ff4285f..6825295 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java
@@ -89,7 +89,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Edge<K, Tuple3<EV, LongValue, L
 			.join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND)
 			.where(1)
 			.equalTo(0)
-			.with(new JoinEdgeDegreeWithVertexDegree<K, EV, LongValue>())
+			.with(new JoinEdgeDegreeWithVertexDegree<>())
 				.setParallelism(parallelism)
 				.name("Edge target degree");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java
index bd8ce3d..3fe05d9 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java
@@ -81,7 +81,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Edge<K, Tuple2<EV, LongValue>>>
 			.join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND)
 			.where(0)
 			.equalTo(0)
-			.with(new JoinEdgeWithVertexDegree<K, EV, LongValue>())
+			.with(new JoinEdgeWithVertexDegree<>())
 				.setParallelism(parallelism)
 				.name("Edge source degree");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java
index cb18d2c..6020ba3 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java
@@ -81,7 +81,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Edge<K, Tuple2<EV, LongValue>>>
 			.join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND)
 			.where(1)
 			.equalTo(0)
-			.with(new JoinEdgeWithVertexDegree<K, EV, LongValue>())
+			.with(new JoinEdgeWithVertexDegree<>())
 				.setParallelism(parallelism)
 				.name("Edge target degree");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java
index d2fad18..fee58a3 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java
@@ -103,7 +103,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, LongValue>> {
 	public DataSet<Vertex<K, LongValue>> runInternal(Graph<K, VV, EV> input)
 			throws Exception {
 		MapFunction<Edge<K, EV>, Vertex<K, LongValue>> mapEdgeToId = reduceOnTargetId.get() ?
-			new MapEdgeToTargetId<K, EV>() : new MapEdgeToSourceId<K, EV>();
+			new MapEdgeToTargetId<>() : new MapEdgeToSourceId<>();
 
 		// v
 		DataSet<Vertex<K, LongValue>> vertexIds = input
@@ -115,7 +115,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, LongValue>> {
 		// v, deg(v)
 		DataSet<Vertex<K, LongValue>> degree = vertexIds
 			.groupBy(0)
-			.reduce(new DegreeCount<K>())
+			.reduce(new DegreeCount<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Degree count");
@@ -126,7 +126,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Vertex<K, LongValue>> {
 				.leftOuterJoin(degree)
 				.where(0)
 				.equalTo(0)
-				.with(new JoinVertexWithVertexDegree<K, VV>())
+				.with(new JoinVertexWithVertexDegree<>())
 					.setParallelism(parallelism)
 					.name("Zero degree vertices");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegree.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegree.java
index 522d39c..41dc64b 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegree.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegree.java
@@ -138,7 +138,7 @@ extends GraphAlgorithmWrappingGraph<K, VV, EV, K, VV, EV> {
 
 		// u, d(u) if d(u) > maximumDegree
 		DataSet<Tuple1<K>> highDegreeVertices = vertexDegree
-			.flatMap(new DegreeFilter<K>(maximumDegree))
+			.flatMap(new DegreeFilter<>(maximumDegree))
 				.setParallelism(parallelism)
 				.name("Filter high-degree vertices");
 
@@ -150,7 +150,7 @@ extends GraphAlgorithmWrappingGraph<K, VV, EV, K, VV, EV> {
 			.leftOuterJoin(highDegreeVertices, joinHint)
 			.where(0)
 			.equalTo(0)
-			.with(new ProjectVertex<K, VV>())
+			.with(new ProjectVertex<>())
 				.setParallelism(parallelism)
 				.name("Project low-degree vertices");
 
@@ -160,13 +160,13 @@ extends GraphAlgorithmWrappingGraph<K, VV, EV, K, VV, EV> {
 			.leftOuterJoin(highDegreeVertices, joinHint)
 			.where(reduceOnTargetId.get() ? 1 : 0)
 			.equalTo(0)
-				.with(new ProjectEdge<K, EV>())
+				.with(new ProjectEdge<>())
 				.setParallelism(parallelism)
 				.name("Project low-degree edges by " + (reduceOnTargetId.get() ? "target" : "source"))
 			.leftOuterJoin(highDegreeVertices, joinHint)
 			.where(reduceOnTargetId.get() ? 0 : 1)
 			.equalTo(0)
-			.with(new ProjectEdge<K, EV>())
+			.with(new ProjectEdge<>())
 				.setParallelism(parallelism)
 				.name("Project low-degree edges by " + (reduceOnTargetId.get() ? "source" : "target"));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/directed/Simplify.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/directed/Simplify.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/directed/Simplify.java
index 511840a..0d4fa1e 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/directed/Simplify.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/directed/Simplify.java
@@ -41,7 +41,7 @@ extends GraphAlgorithmWrappingGraph<K, VV, EV, K, VV, EV> {
 		// Edges
 		DataSet<Edge<K, EV>> edges = input
 			.getEdges()
-			.filter(new RemoveSelfLoops<K, EV>())
+			.filter(new RemoveSelfLoops<>())
 				.setParallelism(parallelism)
 				.name("Remove self-loops")
 			.distinct(0, 1)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/undirected/Simplify.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/undirected/Simplify.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/undirected/Simplify.java
index 21db233..f00a162 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/undirected/Simplify.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/simple/undirected/Simplify.java
@@ -71,7 +71,7 @@ extends GraphAlgorithmWrappingGraph<K, VV, EV, K, VV, EV> {
 		// Edges
 		DataSet<Edge<K, EV>> edges = input
 			.getEdges()
-			.flatMap(new SymmetrizeAndRemoveSelfLoops<K, EV>(clipAndFlip))
+			.flatMap(new SymmetrizeAndRemoveSelfLoops<>(clipAndFlip))
 				.setParallelism(parallelism)
 				.name("Remove self-loops")
 			.distinct(0, 1)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java
index 4cb4e01..6dcf766 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java
@@ -88,7 +88,7 @@ public class Translate {
 		TupleTypeInfo<Vertex<NEW, VV>> returnType = new TupleTypeInfo<>(vertexClass, newType, vertexValueType);
 
 		return vertices
-			.map(new TranslateVertexId<OLD, NEW, VV>(translator))
+			.map(new TranslateVertexId<>(translator))
 			.returns(returnType)
 				.setParallelism(parallelism)
 				.name("Translate vertex IDs");
@@ -172,7 +172,7 @@ public class Translate {
 		TupleTypeInfo<Edge<NEW, EV>> returnType = new TupleTypeInfo<>(edgeClass, newType, newType, edgeValueType);
 
 		return edges
-			.map(new TranslateEdgeId<OLD, NEW, EV>(translator))
+			.map(new TranslateEdgeId<>(translator))
 			.returns(returnType)
 				.setParallelism(parallelism)
 				.name("Translate edge IDs");
@@ -257,7 +257,7 @@ public class Translate {
 		TupleTypeInfo<Vertex<K, NEW>> returnType = new TupleTypeInfo<>(vertexClass, idType, newType);
 
 		return vertices
-			.map(new TranslateVertexValue<K, OLD, NEW>(translator))
+			.map(new TranslateVertexValue<>(translator))
 			.returns(returnType)
 				.setParallelism(parallelism)
 				.name("Translate vertex values");
@@ -341,7 +341,7 @@ public class Translate {
 		TupleTypeInfo<Edge<K, NEW>> returnType = new TupleTypeInfo<>(edgeClass, idType, idType, newType);
 
 		return edges
-			.map(new TranslateEdgeValue<K, OLD, NEW>(translator))
+			.map(new TranslateEdgeValue<>(translator))
 			.returns(returnType)
 				.setParallelism(parallelism)
 				.name("Translate edge values");

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/bipartite/BipartiteGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/bipartite/BipartiteGraph.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/bipartite/BipartiteGraph.java
index 029e2c4..97c93e2 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/bipartite/BipartiteGraph.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/bipartite/BipartiteGraph.java
@@ -128,7 +128,7 @@ public class BipartiteGraph<KT, KB, VVT, VVB, EV> {
 		DataSet<Edge<KT, Tuple2<EV, EV>>> newEdges = edges.join(edges)
 			.where(1)
 			.equalTo(1)
-			.with(new ProjectionTopSimple<KT, KB, EV>())
+			.with(new ProjectionTopSimple<>())
 				.name("Simple top projection");
 
 		return Graph.fromDataSet(topVertices, newEdges, context);
@@ -172,7 +172,7 @@ public class BipartiteGraph<KT, KB, VVT, VVB, EV> {
 		DataSet<Edge<KB, Tuple2<EV, EV>>> newEdges =  edges.join(edges)
 			.where(0)
 			.equalTo(0)
-			.with(new ProjectionBottomSimple<KT, KB, EV>())
+			.with(new ProjectionBottomSimple<>())
 			.name("Simple bottom projection");
 
 		return Graph.fromDataSet(bottomVertices, newEdges, context);
@@ -218,7 +218,7 @@ public class BipartiteGraph<KT, KB, VVT, VVB, EV> {
 		DataSet<Edge<KT, Projection<KB, VVB, VVT, EV>>> newEdges = edgesWithVertices.join(edgesWithVertices)
 			.where(1)
 			.equalTo(1)
-			.with(new ProjectionTopFull<KT, KB, EV, VVT, VVB>())
+			.with(new ProjectionTopFull<>())
 				.name("Full top projection");
 
 		return Graph.fromDataSet(topVertices, newEdges, context);
@@ -284,7 +284,7 @@ public class BipartiteGraph<KT, KB, VVT, VVB, EV> {
 		DataSet<Edge<KB, Projection<KT, VVT, VVB, EV>>> newEdges = edgesWithVertices.join(edgesWithVertices)
 			.where(0)
 			.equalTo(0)
-			.with(new ProjectionBottomFull<KT, KB, EV, VVT, VVB>())
+			.with(new ProjectionBottomFull<>())
 				.name("Full bottom projection");
 
 		return Graph.fromDataSet(bottomVertices, newEdges, context);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/GraphGeneratorUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/GraphGeneratorUtils.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/GraphGeneratorUtils.java
index fca9d8b..d5a70f3 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/GraphGeneratorUtils.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/GraphGeneratorUtils.java
@@ -91,7 +91,7 @@ public class GraphGeneratorUtils {
 	 */
 	public static <K, EV> DataSet<Vertex<K, NullValue>> vertexSet(DataSet<Edge<K, EV>> edges, int parallelism) {
 		DataSet<Vertex<K, NullValue>> vertexSet = edges
-			.flatMap(new EmitSrcAndTarget<K, EV>())
+			.flatMap(new EmitSrcAndTarget<>())
 				.setParallelism(parallelism)
 				.name("Emit source and target labels");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/RMatGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/RMatGraph.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/RMatGraph.java
index d14d32c..1960aa3 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/RMatGraph.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/RMatGraph.java
@@ -156,7 +156,7 @@ extends GraphGeneratorBase<LongValue, NullValue, NullValue> {
 			.rebalance()
 				.setParallelism(parallelism)
 				.name("Rebalance")
-			.flatMap(new GenerateEdges<T>(vertexCount, scale, a, b, c, noiseEnabled, noise))
+			.flatMap(new GenerateEdges<>(vertexCount, scale, a, b, c, noiseEnabled, noise))
 				.setParallelism(parallelism)
 				.name("RMat graph edges");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
index 72e18ae..f09a890 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
@@ -58,7 +58,7 @@ public class GSAConfiguration extends IterationConfiguration {
 	 * @param data The data set to be broadcasted.
 	 */
 	public void addBroadcastSetForGatherFunction(String name, DataSet<?> data) {
-		this.bcVarsGather.add(new Tuple2<String, DataSet<?>>(name, data));
+		this.bcVarsGather.add(new Tuple2<>(name, data));
 	}
 
 	/**
@@ -68,7 +68,7 @@ public class GSAConfiguration extends IterationConfiguration {
 	 * @param data The data set to be broadcasted.
 	 */
 	public void addBroadcastSetForSumFunction(String name, DataSet<?> data) {
-		this.bcVarsSum.add(new Tuple2<String, DataSet<?>>(name, data));
+		this.bcVarsSum.add(new Tuple2<>(name, data));
 	}
 
 	/**
@@ -78,7 +78,7 @@ public class GSAConfiguration extends IterationConfiguration {
 	 * @param data The data set to be broadcasted.
 	 */
 	public void addBroadcastSetForApplyFunction(String name, DataSet<?> data) {
-		this.bcVarsApply.add(new Tuple2<String, DataSet<?>>(name, data));
+		this.bcVarsApply.add(new Tuple2<>(name, data));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
index 12d4977..5f04b70 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
@@ -169,24 +169,24 @@ public class GatherSumApplyIteration<K, VV, EV, M> implements CustomUnaryOperati
 			case OUT:
 				neighbors = iteration
 				.getWorkset().join(edgeDataSet)
-				.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>());
+				.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<>());
 				break;
 			case IN:
 				neighbors = iteration
 				.getWorkset().join(edgeDataSet)
-				.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<K, VV, EV>());
+				.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<>());
 				break;
 			case ALL:
 				neighbors =  iteration
 						.getWorkset().join(edgeDataSet)
-						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>()).union(iteration
+						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<>()).union(iteration
 								.getWorkset().join(edgeDataSet)
-								.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<K, VV, EV>()));
+								.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<>()));
 				break;
 			default:
 				neighbors = iteration
 						.getWorkset().join(edgeDataSet)
-						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>());
+						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<>());
 				break;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
index 9846286..ccf2bb1 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
@@ -73,14 +73,14 @@ public class CommunityDetection<K> implements GraphAlgorithm<K, Long, Double, Gr
 	public Graph<K, Long, Double> run(Graph<K, Long, Double> graph) {
 
 		DataSet<Vertex<K, Tuple2<Long, Double>>> initializedVertices = graph.getVertices()
-			.map(new AddScoreToVertexValuesMapper<K>());
+			.map(new AddScoreToVertexValuesMapper<>());
 
 		Graph<K, Tuple2<Long, Double>, Double> graphWithScoredVertices =
 			Graph.fromDataSet(initializedVertices, graph.getEdges(), graph.getContext()).getUndirected();
 
-		return graphWithScoredVertices.runScatterGatherIteration(new LabelMessenger<K>(),
-			new VertexLabelUpdater<K>(delta), maxIterations)
-				.mapVertices(new RemoveScoreFromVertexValuesMapper<K>());
+		return graphWithScoredVertices.runScatterGatherIteration(new LabelMessenger<>(),
+			new VertexLabelUpdater<>(delta), maxIterations)
+				.mapVertices(new RemoveScoreFromVertexValuesMapper<>());
 	}
 
 	@SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
index a3110ab..5cb8abe 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
@@ -73,12 +72,12 @@ public class ConnectedComponents<K, VV extends Comparable<VV>, EV>
 		TypeInformation<VV> valueTypeInfo = ((TupleTypeInfo<?>) graph.getVertices().getType()).getTypeAt(1);
 
 		Graph<K, VV, NullValue> undirectedGraph = graph
-			.mapEdges(new MapTo<Edge<K, EV>, NullValue>(NullValue.getInstance()))
+			.mapEdges(new MapTo<>(NullValue.getInstance()))
 			.getUndirected();
 
 		return undirectedGraph.runScatterGatherIteration(
-			new CCMessenger<K, VV>(valueTypeInfo),
-			new CCUpdater<K, VV>(),
+			new CCMessenger<>(valueTypeInfo),
+			new CCUpdater<>(),
 			maxIterations).getVertices();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
index 37e5cab..230f88e 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
@@ -74,13 +73,13 @@ public class GSAConnectedComponents<K, VV extends Comparable<VV>, EV>
 		TypeInformation<VV> valueTypeInfo = ((TupleTypeInfo<?>) graph.getVertices().getType()).getTypeAt(1);
 
 		Graph<K, VV, NullValue> undirectedGraph = graph
-			.mapEdges(new MapTo<Edge<K, EV>, NullValue>(NullValue.getInstance()))
+			.mapEdges(new MapTo<>(NullValue.getInstance()))
 			.getUndirected();
 
 		return undirectedGraph.runGatherSumApplyIteration(
 			new GatherNeighborIds<>(valueTypeInfo),
 			new SelectMinId<>(valueTypeInfo),
-			new UpdateComponentId<K, VV>(valueTypeInfo),
+			new UpdateComponentId<>(valueTypeInfo),
 			maxIterations).getVertices();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
index 2d0b8da..28e9168 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
@@ -51,9 +51,9 @@ public class GSASingleSourceShortestPaths<K, VV> implements
 	@Override
 	public DataSet<Vertex<K, Double>> run(Graph<K, VV, Double> input) {
 
-		return input.mapVertices(new InitVerticesMapper<K, VV>(srcVertexId))
+		return input.mapVertices(new InitVerticesMapper<>(srcVertexId))
 				.runGatherSumApplyIteration(new CalculateDistances(), new ChooseMinDistance(),
-						new UpdateDistance<K>(), maxIterations)
+					new UpdateDistance<>(), maxIterations)
 						.getVertices();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
index 1e700f4..880a67b 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
@@ -77,9 +76,9 @@ public class LabelPropagation<K, VV extends Comparable<VV>, EV>
 		TypeInformation<VV> valueType = ((TupleTypeInfo<?>) input.getVertices().getType()).getTypeAt(1);
 		// iteratively adopt the most frequent label among the neighbors of each vertex
 		return input
-			.mapEdges(new MapTo<Edge<K, EV>, NullValue>(NullValue.getInstance()))
+			.mapEdges(new MapTo<>(NullValue.getInstance()))
 			.runScatterGatherIteration(
-				new SendNewLabelToNeighbors<K, VV>(valueType), new UpdateVertexLabel<K, VV>(), maxIterations)
+				new SendNewLabelToNeighbors<>(valueType), new UpdateVertexLabel<>(), maxIterations)
 			.getVertices();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
index 15f0a84..8f41fa0 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
@@ -51,8 +51,8 @@ public class SingleSourceShortestPaths<K, VV> implements GraphAlgorithm<K, VV, D
 	@Override
 	public DataSet<Vertex<K, Double>> run(Graph<K, VV, Double> input) {
 
-		return input.mapVertices(new InitVerticesMapper<K, VV>(srcVertexId))
-				.runScatterGatherIteration(new MinDistanceMessenger<K>(), new VertexDistanceUpdater<K>(),
+		return input.mapVertices(new InitVerticesMapper<>(srcVertexId))
+				.runScatterGatherIteration(new MinDistanceMessenger<>(), new VertexDistanceUpdater<>(),
 				maxIterations).getVertices();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/Summarization.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/Summarization.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/Summarization.java
index 44ea988..a1498df 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/Summarization.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/Summarization.java
@@ -102,11 +102,11 @@ public class Summarization<K, VV, EV>
 		// group vertices by value and create vertex group items
 		DataSet<VertexGroupItem<K, VV>> vertexGroupItems = input.getVertices()
 				.groupBy(1)
-				.reduceGroup(new VertexGroupReducer<K, VV>());
+				.reduceGroup(new VertexGroupReducer<>());
 		// create super vertices
 		DataSet<Vertex<K, VertexValue<VV>>> summarizedVertices = vertexGroupItems
-				.filter(new VertexGroupItemToSummarizedVertexFilter<K, VV>())
-				.map(new VertexGroupItemToSummarizedVertexMapper<K, VV>());
+				.filter(new VertexGroupItemToSummarizedVertexFilter<>())
+				.map(new VertexGroupItemToSummarizedVertexMapper<>());
 
 		// -------------------------
 		// build super edges
@@ -114,22 +114,22 @@ public class Summarization<K, VV, EV>
 
 		// create mapping between vertices and their representative
 		DataSet<VertexWithRepresentative<K>> vertexToRepresentativeMap = vertexGroupItems
-			.filter(new VertexGroupItemToRepresentativeFilter<K, VV>())
-			.map(new VertexGroupItemToVertexWithRepresentativeMapper<K, VV>());
+			.filter(new VertexGroupItemToRepresentativeFilter<>())
+			.map(new VertexGroupItemToVertexWithRepresentativeMapper<>());
 		// join edges with vertex representatives and update source and target identifiers
 		DataSet<Edge<K, EV>> edgesForGrouping = input.getEdges()
 				.join(vertexToRepresentativeMap)
 				.where(0) 	// source vertex id
 				.equalTo(0) // vertex id
-				.with(new SourceVertexJoinFunction<K, EV>())
+				.with(new SourceVertexJoinFunction<>())
 				.join(vertexToRepresentativeMap)
 				.where(1) 	// target vertex id
 				.equalTo(0) // vertex id
-				.with(new TargetVertexJoinFunction<K, EV>());
+				.with(new TargetVertexJoinFunction<>());
 		// create super edges
 		DataSet<Edge<K, EdgeValue<EV>>> summarizedEdges = edgesForGrouping
 				.groupBy(0, 1, 2) // group by source id (0), target id (1) and edge value (2)
-				.reduceGroup(new EdgeGroupReducer<K, EV>());
+				.reduceGroup(new EdgeGroupReducer<>());
 
 		return Graph.fromDataSet(summarizedVertices, summarizedEdges, input.getContext());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/TriangleEnumerator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/TriangleEnumerator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/TriangleEnumerator.java
index 2ae6120..23f942c 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/TriangleEnumerator.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/TriangleEnumerator.java
@@ -65,21 +65,21 @@ public class TriangleEnumerator<K extends Comparable<K>, VV, EV> implements
 		DataSet<Edge<K, EV>> edges = input.getEdges();
 
 		// annotate edges with degrees
-		DataSet<EdgeWithDegrees<K>> edgesWithDegrees = edges.flatMap(new EdgeDuplicator<K, EV>())
-				.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new DegreeCounter<K, EV>())
-				.groupBy(EdgeWithDegrees.V1, EdgeWithDegrees.V2).reduce(new DegreeJoiner<K>());
+		DataSet<EdgeWithDegrees<K>> edgesWithDegrees = edges.flatMap(new EdgeDuplicator<>())
+				.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new DegreeCounter<>())
+				.groupBy(EdgeWithDegrees.V1, EdgeWithDegrees.V2).reduce(new DegreeJoiner<>());
 
 		// project edges by degrees
-		DataSet<Edge<K, NullValue>> edgesByDegree = edgesWithDegrees.map(new EdgeByDegreeProjector<K>());
+		DataSet<Edge<K, NullValue>> edgesByDegree = edgesWithDegrees.map(new EdgeByDegreeProjector<>());
 		// project edges by vertex id
-		DataSet<Edge<K, NullValue>> edgesById = edgesByDegree.map(new EdgeByIdProjector<K>());
+		DataSet<Edge<K, NullValue>> edgesById = edgesByDegree.map(new EdgeByIdProjector<>());
 
 		DataSet<Tuple3<K, K, K>> triangles = edgesByDegree
 				// build triads
 				.groupBy(EdgeWithDegrees.V1).sortGroup(EdgeWithDegrees.V2, Order.ASCENDING)
-				.reduceGroup(new TriadBuilder<K>())
+				.reduceGroup(new TriadBuilder<>())
 				// filter triads
-				.join(edgesById, JoinHint.REPARTITION_HASH_SECOND).where(Triad.V2, Triad.V3).equalTo(0, 1).with(new TriadFilter<K>());
+				.join(edgesById, JoinHint.REPARTITION_HASH_SECOND).where(Triad.V2, Triad.V3).equalTo(0, 1).with(new TriadFilter<>());
 
 		return triangles;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficient.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficient.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficient.java
index 981110f..55d3056 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficient.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficient.java
@@ -119,13 +119,13 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 
 		// u, edge count
 		DataSet<Tuple2<K, LongValue>> triangleVertices = triangles
-			.flatMap(new SplitTriangles<K>())
+			.flatMap(new SplitTriangles<>())
 				.name("Split triangle vertices");
 
 		// u, triangle count
 		DataSet<Tuple2<K, LongValue>> vertexTriangleCount = triangleVertices
 			.groupBy(0)
-			.reduce(new CountTriangles<K>())
+			.reduce(new CountTriangles<>())
 			.setCombineHint(CombineHint.HASH)
 				.name("Count triangles");
 
@@ -140,7 +140,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.leftOuterJoin(vertexTriangleCount)
 			.where(0)
 			.equalTo(0)
-			.with(new JoinVertexDegreeWithTriangleCount<K>())
+			.with(new JoinVertexDegreeWithTriangleCount<>())
 				.setParallelism(parallelism)
 				.name("Clustering coefficient");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
index 00b2210..52d3c10 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
@@ -84,11 +84,11 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 		// u, v, bitmask where u < v
 		DataSet<Tuple3<K, K, ByteValue>> filteredByID = input
 			.getEdges()
-			.map(new OrderByID<K, EV>())
+			.map(new OrderByID<>())
 				.setParallelism(parallelism)
 				.name("Order by ID")
 			.groupBy(0, 1)
-			.reduceGroup(new ReduceBitmask<K>())
+			.reduceGroup(new ReduceBitmask<>())
 				.setParallelism(parallelism)
 				.name("Flatten by ID");
 
@@ -99,11 +99,11 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 
 		// u, v, bitmask where deg(u) < deg(v) or (deg(u) == deg(v) and u < v)
 		DataSet<Tuple3<K, K, ByteValue>> filteredByDegree = pairDegrees
-			.map(new OrderByDegree<K, EV>())
+			.map(new OrderByDegree<>())
 				.setParallelism(parallelism)
 				.name("Order by degree")
 			.groupBy(0, 1)
-			.reduceGroup(new ReduceBitmask<K>())
+			.reduceGroup(new ReduceBitmask<>())
 				.setParallelism(parallelism)
 				.name("Flatten by degree");
 
@@ -111,7 +111,7 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 		DataSet<Tuple4<K, K, K, ByteValue>> triplets = filteredByDegree
 			.groupBy(0)
 			.sortGroup(1, Order.ASCENDING)
-			.reduceGroup(new GenerateTriplets<K>())
+			.reduceGroup(new GenerateTriplets<>())
 				.name("Generate triplets");
 
 		// u, v, w, bitmask where (u, v), (u, w), and (v, w) are edges in graph
@@ -119,16 +119,16 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 			.join(filteredByID, JoinOperatorBase.JoinHint.REPARTITION_HASH_SECOND)
 			.where(1, 2)
 			.equalTo(0, 1)
-			.with(new ProjectTriangles<K>())
+			.with(new ProjectTriangles<>())
 				.name("Triangle listing");
 
 		if (permuteResults) {
 			triangles = triangles
-				.flatMap(new PermuteResult<K>())
+				.flatMap(new PermuteResult<>())
 					.name("Permute triangle vertices");
 		} else if (sortTriangleVertices.get()) {
 			triangles = triangles
-				.map(new SortTriangleVertices<K>())
+				.map(new SortTriangleVertices<>())
 					.name("Sort triangle vertices");
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
index 0beb989..e1c7655 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
@@ -118,13 +118,13 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 
 		// u, 1
 		DataSet<Tuple2<K, LongValue>> triangleVertices = triangles
-			.flatMap(new SplitTriangles<K>())
+			.flatMap(new SplitTriangles<>())
 				.name("Split triangle vertices");
 
 		// u, triangle count
 		DataSet<Tuple2<K, LongValue>> vertexTriangleCount = triangleVertices
 			.groupBy(0)
-			.reduce(new CountTriangles<K>())
+			.reduce(new CountTriangles<>())
 			.setCombineHint(CombineHint.HASH)
 				.name("Count triangles");
 
@@ -139,7 +139,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.leftOuterJoin(vertexTriangleCount)
 			.where(0)
 			.equalTo(0)
-			.with(new JoinVertexDegreeWithTriangleCount<K>())
+			.with(new JoinVertexDegreeWithTriangleCount<>())
 				.setParallelism(parallelism)
 				.name("Clustering coefficient");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java
index 2472744..3f1b00a 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java
@@ -83,7 +83,7 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 		// u, v where u < v
 		DataSet<Tuple2<K, K>> filteredByID = input
 			.getEdges()
-			.flatMap(new FilterByID<K, EV>())
+			.flatMap(new FilterByID<>())
 				.setParallelism(parallelism)
 				.name("Filter by ID");
 
@@ -94,7 +94,7 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 
 		// u, v where deg(u) < deg(v) or (deg(u) == deg(v) and u < v)
 		DataSet<Tuple2<K, K>> filteredByDegree = pairDegree
-			.flatMap(new FilterByDegree<K, EV>())
+			.flatMap(new FilterByDegree<>())
 				.setParallelism(parallelism)
 				.name("Filter by degree");
 
@@ -102,7 +102,7 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 		DataSet<Tuple3<K, K, K>> triplets = filteredByDegree
 			.groupBy(0)
 			.sortGroup(1, Order.ASCENDING)
-			.reduceGroup(new GenerateTriplets<K>())
+			.reduceGroup(new GenerateTriplets<>())
 				.name("Generate triplets");
 
 		// u, v, w where (u, v), (u, w), and (v, w) are edges in graph, v < w
@@ -110,16 +110,16 @@ extends TriangleListingBase<K, VV, EV, Result<K>> {
 			.join(filteredByID, JoinOperatorBase.JoinHint.REPARTITION_HASH_SECOND)
 			.where(1, 2)
 			.equalTo(0, 1)
-			.with(new ProjectTriangles<K>())
+			.with(new ProjectTriangles<>())
 				.name("Triangle listing");
 
 		if (permuteResults) {
 			triangles = triangles
-				.flatMap(new PermuteResult<K>())
+				.flatMap(new PermuteResult<>())
 					.name("Permute triangle vertices");
 		} else if (sortTriangleVertices.get()) {
 			triangles = triangles
-				.map(new SortTriangleVertices<K>())
+				.map(new SortTriangleVertices<>())
 					.name("Sort triangle vertices");
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/HITS.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/HITS.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/HITS.java
index 6b41ee4..e59240b 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/HITS.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/HITS.java
@@ -129,17 +129,17 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			throws Exception {
 		DataSet<Tuple2<K, K>> edges = input
 			.getEdges()
-			.map(new ExtractEdgeIDs<K, EV>())
+			.map(new ExtractEdgeIDs<>())
 				.setParallelism(parallelism)
 				.name("Extract edge IDs");
 
 		// ID, hub, authority
 		DataSet<Tuple3<K, DoubleValue, DoubleValue>> initialScores = edges
-			.map(new InitializeScores<K>())
+			.map(new InitializeScores<>())
 				.setParallelism(parallelism)
 				.name("Initial scores")
 			.groupBy(0)
-			.reduce(new SumScores<K>())
+			.reduce(new SumScores<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Sum");
@@ -153,18 +153,18 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.coGroup(edges)
 			.where(0)
 			.equalTo(1)
-			.with(new Hubbiness<K>())
+			.with(new Hubbiness<>())
 				.setParallelism(parallelism)
 				.name("Hub")
 			.groupBy(0)
-			.reduce(new SumScore<K>())
+			.reduce(new SumScore<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Sum");
 
 		// sum-of-hubbiness-squared
 		DataSet<DoubleValue> hubbinessSumSquared = hubbiness
-			.map(new Square<K>())
+			.map(new Square<>())
 				.setParallelism(parallelism)
 				.name("Square")
 			.reduce(new Sum())
@@ -177,18 +177,18 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.coGroup(edges)
 			.where(0)
 			.equalTo(0)
-			.with(new Authority<K>())
+			.with(new Authority<>())
 				.setParallelism(parallelism)
 				.name("Authority")
 			.groupBy(0)
-			.reduce(new SumScore<K>())
+			.reduce(new SumScore<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Sum");
 
 		// sum-of-authority-squared
 		DataSet<DoubleValue> authoritySumSquared = authority
-			.map(new Square<K>())
+			.map(new Square<>())
 				.setParallelism(parallelism)
 				.name("Square")
 			.reduce(new Sum())
@@ -201,7 +201,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.fullOuterJoin(authority, JoinHint.REPARTITION_SORT_MERGE)
 			.where(0)
 			.equalTo(0)
-			.with(new JoinAndNormalizeHubAndAuthority<K>())
+			.with(new JoinAndNormalizeHubAndAuthority<>())
 			.withBroadcastSet(hubbinessSumSquared, HUBBINESS_SUM_SQUARED)
 			.withBroadcastSet(authoritySumSquared, AUTHORITY_SUM_SQUARED)
 				.setParallelism(parallelism)
@@ -214,7 +214,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 				.fullOuterJoin(scores, JoinHint.REPARTITION_SORT_MERGE)
 				.where(0)
 				.equalTo(0)
-				.with(new ChangeInScores<K>())
+				.with(new ChangeInScores<>())
 					.setParallelism(parallelism)
 					.name("Change in scores");
 
@@ -225,7 +225,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 
 		return iterative
 			.closeWith(passThrough)
-			.map(new TranslateResult<K>())
+			.map(new TranslateResult<>())
 				.setParallelism(parallelism)
 				.name("Map result");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/PageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/PageRank.java
index af56e50..71c37aa 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/PageRank.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/linkanalysis/PageRank.java
@@ -151,20 +151,20 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 		DataSet<Edge<K, LongValue>> edgeSourceDegree = input
 			.run(new EdgeSourceDegrees<K, VV, EV>()
 				.setParallelism(parallelism))
-			.map(new ExtractSourceDegree<K, EV>())
+			.map(new ExtractSourceDegree<>())
 				.setParallelism(parallelism)
 				.name("Extract source degree");
 
 		// vertices with zero in-edges
 		DataSet<Tuple2<K, DoubleValue>> sourceVertices = vertexDegree
-			.flatMap(new InitializeSourceVertices<K>())
+			.flatMap(new InitializeSourceVertices<>())
 			.withBroadcastSet(vertexCount, VERTEX_COUNT)
 				.setParallelism(parallelism)
 				.name("Initialize source vertex scores");
 
 		// s, initial pagerank(s)
 		DataSet<Tuple2<K, DoubleValue>> initialScores = vertexDegree
-			.map(new InitializeVertexScores<K>())
+			.map(new InitializeVertexScores<>())
 			.withBroadcastSet(vertexCount, VERTEX_COUNT)
 				.setParallelism(parallelism)
 				.name("Initialize scores");
@@ -178,18 +178,18 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.coGroup(edgeSourceDegree)
 			.where(0)
 			.equalTo(0)
-			.with(new SendScore<K>())
+			.with(new SendScore<>())
 				.setParallelism(parallelism)
 				.name("Send score")
 			.groupBy(0)
-			.reduce(new SumScore<K>())
+			.reduce(new SumScore<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Sum");
 
 		// ignored ID, total pagerank
 		DataSet<Tuple2<K, DoubleValue>> sumOfScores = vertexScores
-			.reduce(new SumVertexScores<K>())
+			.reduce(new SumVertexScores<>())
 				.setParallelism(parallelism)
 				.name("Sum");
 
@@ -198,7 +198,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.union(sourceVertices)
 				.setParallelism(parallelism)
 				.name("Union with source vertices")
-			.map(new AdjustScores<K>(dampingFactor))
+			.map(new AdjustScores<>(dampingFactor))
 				.withBroadcastSet(sumOfScores, SUM_OF_SCORES)
 				.withBroadcastSet(vertexCount, VERTEX_COUNT)
 					.setParallelism(parallelism)
@@ -211,7 +211,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 				.join(adjustedScores)
 				.where(0)
 				.equalTo(0)
-				.with(new ChangeInScores<K>())
+				.with(new ChangeInScores<>())
 					.setParallelism(parallelism)
 					.name("Change in scores");
 
@@ -222,7 +222,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 
 		return iterative
 			.closeWith(passThrough)
-			.map(new TranslateResult<K>())
+			.map(new TranslateResult<>())
 				.setParallelism(parallelism)
 				.name("Map result");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/directed/EdgeMetrics.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/directed/EdgeMetrics.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/directed/EdgeMetrics.java
index c88e401..7294fd1 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/directed/EdgeMetrics.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/directed/EdgeMetrics.java
@@ -92,15 +92,15 @@ extends GraphAnalyticBase<K, VV, EV, Result> {
 
 		// s, d(s), count of (u, v) where deg(u) < deg(v) or (deg(u) == deg(v) and u < v)
 		DataSet<Tuple3<K, Degrees, LongValue>> edgeStats = edgeDegreesPair
-			.flatMap(new EdgeStats<K, EV>())
+			.flatMap(new EdgeStats<>())
 				.setParallelism(parallelism)
 				.name("Edge stats")
 			.groupBy(0, 1)
-			.reduceGroup(new ReduceEdgeStats<K>())
+			.reduceGroup(new ReduceEdgeStats<>())
 				.setParallelism(parallelism)
 				.name("Reduce edge stats")
 			.groupBy(0)
-			.reduce(new SumEdgeStats<K>())
+			.reduce(new SumEdgeStats<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Sum edge stats");

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/undirected/EdgeMetrics.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/undirected/EdgeMetrics.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/undirected/EdgeMetrics.java
index 4c0d654..8c520e6 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/undirected/EdgeMetrics.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/undirected/EdgeMetrics.java
@@ -103,11 +103,11 @@ extends GraphAnalyticBase<K, VV, EV, Result> {
 
 		// s, d(s), count of (u, v) where deg(u) < deg(v) or (deg(u) == deg(v) and u < v)
 		DataSet<Tuple3<K, LongValue, LongValue>> edgeStats = edgeDegreePair
-			.map(new EdgeStats<K, EV>())
+			.map(new EdgeStats<>())
 				.setParallelism(parallelism)
 				.name("Edge stats")
 			.groupBy(0)
-			.reduce(new SumEdgeStats<K>())
+			.reduce(new SumEdgeStats<>())
 			.setCombineHint(CombineHint.HASH)
 				.setParallelism(parallelism)
 				.name("Sum edge stats");

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
index d761f60..752e206 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
@@ -153,7 +153,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 		DataSet<Tuple3<K, LongValue, FloatValue>> inverseLogDegree = input
 			.run(new VertexDegree<K, VV, EV>()
 				.setParallelism(parallelism))
-			.map(new VertexInverseLogDegree<K>())
+			.map(new VertexInverseLogDegree<>())
 				.setParallelism(parallelism)
 				.name("Vertex score");
 
@@ -172,7 +172,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 		DataSet<Tuple4<IntValue, K, K, FloatValue>> groupSpans = sourceInverseLogDegree
 			.groupBy(0)
 			.sortGroup(1, Order.ASCENDING)
-			.reduceGroup(new GenerateGroupSpans<K>())
+			.reduceGroup(new GenerateGroupSpans<>())
 				.setParallelism(parallelism)
 				.name("Generate group spans");
 
@@ -181,7 +181,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.rebalance()
 				.setParallelism(parallelism)
 				.name("Rebalance")
-			.flatMap(new GenerateGroups<K>())
+			.flatMap(new GenerateGroups<>())
 				.setParallelism(parallelism)
 				.name("Generate groups");
 
@@ -189,19 +189,19 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 		DataSet<Tuple3<K, K, FloatValue>> twoPaths = groups
 			.groupBy(0, 1)
 			.sortGroup(2, Order.ASCENDING)
-			.reduceGroup(new GenerateGroupPairs<K>())
+			.reduceGroup(new GenerateGroupPairs<>())
 				.name("Generate group pairs");
 
 		// t, u, adamic-adar score
 		GroupReduceOperator<Tuple3<K, K, FloatValue>, Result<K>> scores = twoPaths
 			.groupBy(0, 1)
-			.reduceGroup(new ComputeScores<K>(minimumScore, minimumRatio))
+			.reduceGroup(new ComputeScores<>(minimumScore, minimumRatio))
 				.name("Compute scores");
 
 		if (minimumRatio > 0.0f) {
 			// total score, number of pairs of neighbors
 			DataSet<Tuple2<FloatValue, LongValue>> sumOfScoresAndNumberOfNeighborPairs = inverseLogDegree
-				.map(new ComputeScoreFromVertex<K>())
+				.map(new ComputeScoreFromVertex<>())
 					.setParallelism(parallelism)
 					.name("Average score")
 				.sum(0)
@@ -213,7 +213,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 
 		if (mirrorResults) {
 			return scores
-				.flatMap(new MirrorResult<K, Result<K>>())
+				.flatMap(new MirrorResult<>())
 					.name("Mirror results");
 		} else {
 			return scores;

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
index 8e820ac..92bf9e3 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
@@ -199,7 +199,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 		DataSet<Tuple4<IntValue, K, K, IntValue>> groupSpans = neighborDegree
 			.groupBy(0)
 			.sortGroup(1, Order.ASCENDING)
-			.reduceGroup(new GenerateGroupSpans<K, EV>(groupSize))
+			.reduceGroup(new GenerateGroupSpans<>(groupSize))
 				.setParallelism(parallelism)
 				.name("Generate group spans");
 
@@ -208,7 +208,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			.rebalance()
 				.setParallelism(parallelism)
 				.name("Rebalance")
-			.flatMap(new GenerateGroups<K>())
+			.flatMap(new GenerateGroups<>())
 				.setParallelism(parallelism)
 				.name("Generate groups");
 
@@ -216,20 +216,20 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 		DataSet<Tuple3<K, K, IntValue>> twoPaths = groups
 			.groupBy(0, 1)
 			.sortGroup(2, Order.ASCENDING)
-			.reduceGroup(new GenerateGroupPairs<K>(groupSize))
+			.reduceGroup(new GenerateGroupPairs<>(groupSize))
 				.name("Generate group pairs");
 
 		// t, u, intersection, union
 		DataSet<Result<K>> scores = twoPaths
 			.groupBy(0, 1)
-			.reduceGroup(new ComputeScores<K>(unboundedScores,
+			.reduceGroup(new ComputeScores<>(unboundedScores,
 					minimumScoreNumerator, minimumScoreDenominator,
 					maximumScoreNumerator, maximumScoreDenominator))
 				.name("Compute scores");
 
 		if (mirrorResults) {
 			scores = scores
-				.flatMap(new MirrorResult<K, Result<K>>())
+				.flatMap(new MirrorResult<>())
 					.name("Mirror results");
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java
index 69fcc52..39b9bcf 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricConfiguration.java
@@ -48,7 +48,7 @@ public class VertexCentricConfiguration extends IterationConfiguration {
 	 * @param data The data set to be broadcasted.
 	 */
 	public void addBroadcastSet(String name, DataSet<?> data) {
-		this.bcVars.add(new Tuple2<String, DataSet<?>>(name, data));
+		this.bcVars.add(new Tuple2<>(name, data));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricIteration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricIteration.java
index c30b1a7..6c06a3a 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricIteration.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/VertexCentricIteration.java
@@ -176,9 +176,9 @@ public class VertexCentricIteration<K, VV, EV, Message>
 		DataSet<Tuple2<Vertex<K, VV>, Either<NullValue, Message>>> verticesWithMsgs =
 				iteration.getSolutionSet().join(iteration.getWorkset())
 				.where(0).equalTo(0)
-				.with(new AppendVertexState<K, VV, Message>())
-				.returns(new TupleTypeInfo<Tuple2<Vertex<K, VV>, Either<NullValue, Message>>>(
-						vertexType, nullableMsgTypeInfo));
+				.with(new AppendVertexState<>())
+				.returns(new TupleTypeInfo<>(
+					vertexType, nullableMsgTypeInfo));
 
 		VertexComputeUdf<K, VV, EV, Message> vertexUdf =
 			new VertexComputeUdf<>(computeFunction, intermediateTypeInfo);
@@ -190,11 +190,11 @@ public class VertexCentricIteration<K, VV, EV, Message>
 
 		// compute the solution set delta
 		DataSet<Vertex<K, VV>> solutionSetDelta = superstepComputation.flatMap(
-				new ProjectNewVertexValue<K, VV, Message>()).returns(vertexType);
+			new ProjectNewVertexValue<>()).returns(vertexType);
 
 		// compute the inbox of each vertex for the next superstep (new workset)
 		DataSet<Tuple2<K, Either<NullValue, Message>>> allMessages = superstepComputation.flatMap(
-				new ProjectMessages<K, VV, Message>()).returns(workSetTypeInfo);
+			new ProjectMessages<>()).returns(workSetTypeInfo);
 
 		DataSet<Tuple2<K, Either<NullValue, Message>>> newWorkSet = allMessages;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java
index 6a62847..0422f13 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/ScatterGatherConfiguration.java
@@ -59,7 +59,7 @@ public class ScatterGatherConfiguration extends IterationConfiguration {
 	 * @param data The data set to be broadcasted.
 	 */
 	public void addBroadcastSetForScatterFunction(String name, DataSet<?> data) {
-		this.bcVarsScatter.add(new Tuple2<String, DataSet<?>>(name, data));
+		this.bcVarsScatter.add(new Tuple2<>(name, data));
 	}
 
 	/**
@@ -69,7 +69,7 @@ public class ScatterGatherConfiguration extends IterationConfiguration {
 	 * @param data The data set to be broadcasted.
 	 */
 	public void addBroadcastSetForGatherFunction(String name, DataSet<?> data) {
-		this.bcVarsGather.add(new Tuple2<String, DataSet<?>>(name, data));
+		this.bcVarsGather.add(new Tuple2<>(name, data));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
index 3e2ac23..8082cd9 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
@@ -46,7 +46,7 @@ public class GraphUtils {
 	 */
 	public static <T> DataSet<LongValue> count(DataSet<T> input) {
 		return input
-			.map(new MapTo<T, LongValue>(new LongValue(1)))
+			.map(new MapTo<>(new LongValue(1)))
 				.returns(LONG_VALUE_TYPE_INFO)
 				.name("Emit 1")
 			.reduce(new AddLongValue())

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
index 57aa987..b85b6e6 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
@@ -48,11 +48,11 @@ public class InvalidVertexIdsValidator<K, VV, EV> extends GraphValidator<K, VV,
 	@Override
 	public boolean validate(Graph<K, VV, EV> graph) throws Exception {
 		DataSet<Tuple1<K>> edgeIds = graph.getEdges()
-				.flatMap(new MapEdgeIds<K, EV>()).distinct();
+				.flatMap(new MapEdgeIds<>()).distinct();
 		DataSet<K> invalidIds = graph.getVertices().coGroup(edgeIds).where(0)
-				.equalTo(0).with(new GroupInvalidIds<K, VV>()).first(1);
+				.equalTo(0).with(new GroupInvalidIds<>()).first(1);
 
-		return invalidIds.map(new KToTupleMap<K>()).count() == 0;
+		return invalidIds.map(new KToTupleMap<>()).count() == 0;
 	}
 
 	private static final class MapEdgeIds<K, EV> implements FlatMapFunction<Edge<K, EV>, Tuple1<K>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
index f89d4f5..1afb5da 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
@@ -120,7 +120,7 @@ public class AsmTestBase {
 
 		return new RMatGraph<>(env, new JDKRandomGeneratorFactory(), vertexCount, edgeCount)
 			.generate()
-			.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>());
+			.run(new org.apache.flink.graph.asm.simple.directed.Simplify<>());
 	}
 
 	/**
@@ -149,6 +149,6 @@ public class AsmTestBase {
 
 		return new RMatGraph<>(env, new JDKRandomGeneratorFactory(), vertexCount, edgeCount)
 			.generate()
-			.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(false));
+			.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<>(false));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
index 22b47fe..63bf133 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
@@ -53,7 +53,7 @@ extends AsmTestBase {
 			"(5,3,((null),(1,1,0),(4,2,2)))";
 
 		DataSet<Edge<IntValue, Tuple3<NullValue, Degrees, Degrees>>> degreesPair = directedSimpleGraph
-			.run(new EdgeDegreesPair<IntValue, NullValue, NullValue>());
+			.run(new EdgeDegreesPair<>());
 
 		TestBaseUtils.compareResultAsText(degreesPair.collect(), expectedResult);
 	}
@@ -62,7 +62,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Edge<LongValue, Tuple3<NullValue, Degrees, Degrees>>> degreesPair = directedRMatGraph(10, 16)
-			.run(new EdgeDegreesPair<LongValue, NullValue, NullValue>());
+			.run(new EdgeDegreesPair<>());
 
 		Checksum checksum = new ChecksumHashCode<Edge<LongValue, Tuple3<NullValue, Degrees, Degrees>>>()
 			.run(degreesPair)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
index f0d51d2..967cfb2 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
@@ -53,7 +53,7 @@ extends AsmTestBase {
 			"(5,3,((null),(1,1,0)))";
 
 		DataSet<Edge<IntValue, Tuple2<NullValue, Degrees>>> sourceDegrees = directedSimpleGraph
-				.run(new EdgeSourceDegrees<IntValue, NullValue, NullValue>());
+				.run(new EdgeSourceDegrees<>());
 
 		TestBaseUtils.compareResultAsText(sourceDegrees.collect(), expectedResult);
 	}
@@ -62,7 +62,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Edge<LongValue, Tuple2<NullValue, Degrees>>> sourceDegrees = directedRMatGraph(10, 16)
-			.run(new EdgeSourceDegrees<LongValue, NullValue, NullValue>());
+			.run(new EdgeSourceDegrees<>());
 
 		Checksum checksum = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue, Degrees>>>()
 			.run(sourceDegrees)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
index 6d58bb0..abb76c4 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
@@ -53,7 +53,7 @@ extends AsmTestBase {
 			"(5,3,((null),(4,2,2)))";
 
 		DataSet<Edge<IntValue, Tuple2<NullValue, Degrees>>> targetDegrees = directedSimpleGraph
-				.run(new EdgeTargetDegrees<IntValue, NullValue, NullValue>());
+				.run(new EdgeTargetDegrees<>());
 
 		TestBaseUtils.compareResultAsText(targetDegrees.collect(), expectedResult);
 	}
@@ -62,7 +62,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Edge<LongValue, Tuple2<NullValue, Degrees>>> targetDegrees = directedRMatGraph(10, 16)
-			.run(new EdgeTargetDegrees<LongValue, NullValue, NullValue>());
+			.run(new EdgeTargetDegrees<>());
 
 		Checksum checksum = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue, Degrees>>>()
 			.run(targetDegrees)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
index 5214282..91f354f 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
@@ -43,7 +43,7 @@ extends AsmTestBase {
 	public void testWithSimpleDirectedGraph()
 			throws Exception {
 		DataSet<Vertex<IntValue, Degrees>> degrees = directedSimpleGraph
-			.run(new VertexDegrees<IntValue, NullValue, NullValue>());
+			.run(new VertexDegrees<>());
 
 		String expectedResult =
 			"(0,(2,2,0))\n" +
@@ -60,7 +60,7 @@ extends AsmTestBase {
 	public void testWithSimpleUndirectedGraph()
 			throws Exception {
 		DataSet<Vertex<IntValue, Degrees>> degrees = undirectedSimpleGraph
-			.run(new VertexDegrees<IntValue, NullValue, NullValue>());
+			.run(new VertexDegrees<>());
 
 		String expectedResult =
 			"(0,(2,2,2))\n" +
@@ -100,7 +100,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 	throws Exception {
 		DataSet<Vertex<LongValue, Degrees>> degrees = directedRMatGraph(10, 16)
-			.run(new VertexDegrees<LongValue, NullValue, NullValue>());
+			.run(new VertexDegrees<>());
 
 		Checksum checksum = new ChecksumHashCode<Vertex<LongValue, Degrees>>()
 			.run(degrees)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
index 5f492e4..1cae2e7 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
@@ -59,7 +59,7 @@ extends AsmTestBase {
 			"(5,3,((null),1,4))";
 
 		DataSet<Edge<IntValue, Tuple3<NullValue, LongValue, LongValue>>> degreePairOnSourceId = undirectedSimpleGraph
-			.run(new EdgeDegreePair<IntValue, NullValue, NullValue>());
+			.run(new EdgeDegreePair<>());
 
 		TestBaseUtils.compareResultAsText(degreePairOnSourceId.collect(), expectedResult);
 
@@ -74,7 +74,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Edge<LongValue, Tuple3<NullValue, LongValue, LongValue>>> degreePairOnSourceId = undirectedRMatGraph(10, 16)
-			.run(new EdgeDegreePair<LongValue, NullValue, NullValue>());
+			.run(new EdgeDegreePair<>());
 
 		Checksum checksumOnSourceId = new ChecksumHashCode<Edge<LongValue, Tuple3<NullValue, LongValue, LongValue>>>()
 			.run(degreePairOnSourceId)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
index 393220d..2d8b2e3 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
@@ -59,7 +59,7 @@ extends AsmTestBase {
 			"(5,3,((null),1))";
 
 		DataSet<Edge<IntValue, Tuple2<NullValue, LongValue>>> sourceDegreeOnSourceId = undirectedSimpleGraph
-			.run(new EdgeSourceDegree<IntValue, NullValue, NullValue>());
+			.run(new EdgeSourceDegree<>());
 
 		TestBaseUtils.compareResultAsText(sourceDegreeOnSourceId.collect(), expectedResult);
 
@@ -74,7 +74,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> sourceDegreeOnSourceId = undirectedRMatGraph(10, 16)
-			.run(new EdgeSourceDegree<LongValue, NullValue, NullValue>());
+			.run(new EdgeSourceDegree<>());
 
 		Checksum checksumOnSourceId = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue, LongValue>>>()
 			.run(sourceDegreeOnSourceId)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
index 782296a..a7c88a1 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
@@ -59,7 +59,7 @@ extends AsmTestBase {
 			"(5,3,((null),4))";
 
 		DataSet<Edge<IntValue, Tuple2<NullValue, LongValue>>> targetDegreeOnTargetId = undirectedSimpleGraph
-				.run(new EdgeTargetDegree<IntValue, NullValue, NullValue>());
+				.run(new EdgeTargetDegree<>());
 
 		TestBaseUtils.compareResultAsText(targetDegreeOnTargetId.collect(), expectedResult);
 
@@ -74,7 +74,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> targetDegreeOnTargetId = undirectedRMatGraph(10, 16)
-			.run(new EdgeSourceDegree<LongValue, NullValue, NullValue>());
+			.run(new EdgeSourceDegree<>());
 
 		Checksum checksumOnTargetId = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue, LongValue>>>()
 			.run(targetDegreeOnTargetId)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
index 192782d..bc76bff 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
@@ -50,7 +50,7 @@ extends AsmTestBase {
 			"(5,1)";
 
 		DataSet<Vertex<IntValue, LongValue>> degreeOnSourceId = undirectedSimpleGraph
-			.run(new VertexDegree<IntValue, NullValue, NullValue>());
+			.run(new VertexDegree<>());
 
 		TestBaseUtils.compareResultAsText(degreeOnSourceId.collect(), expectedResult);
 
@@ -67,7 +67,7 @@ extends AsmTestBase {
 		long expectedDegree = completeGraphVertexCount - 1;
 
 		DataSet<Vertex<LongValue, LongValue>> degreeOnSourceId = completeGraph
-			.run(new VertexDegree<LongValue, NullValue, NullValue>());
+			.run(new VertexDegree<>());
 
 		for (Vertex<LongValue, LongValue> vertex : degreeOnSourceId.collect()) {
 			assertEquals(expectedDegree, vertex.getValue().getValue());
@@ -109,7 +109,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Vertex<LongValue, LongValue>> degreeOnSourceId = undirectedRMatGraph(10, 16)
-			.run(new VertexDegree<LongValue, NullValue, NullValue>());
+			.run(new VertexDegree<>());
 
 		Checksum checksumOnSourceId = new ChecksumHashCode<Vertex<LongValue, LongValue>>()
 			.run(degreeOnSourceId)

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
index f03d82c..51e7712 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
@@ -24,7 +24,6 @@ import org.apache.flink.graph.asm.dataset.ChecksumHashCode.Checksum;
 import org.apache.flink.graph.library.metric.ChecksumHashCode;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
 import org.apache.flink.types.NullValue;
 
 import org.junit.Test;
@@ -41,7 +40,7 @@ extends AsmTestBase {
 	public void testWithSimpleGraph()
 			throws Exception {
 		Graph<IntValue, NullValue, NullValue> graph = undirectedSimpleGraph
-			.run(new MaximumDegree<IntValue, NullValue, NullValue>(3));
+			.run(new MaximumDegree<>(3));
 
 		String expectedVerticesResult =
 			"(0,(null))\n" +
@@ -67,8 +66,8 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		Checksum checksum = undirectedRMatGraph(10, 16)
-			.run(new MaximumDegree<LongValue, NullValue, NullValue>(16))
-			.run(new ChecksumHashCode<LongValue, NullValue, NullValue>())
+			.run(new MaximumDegree<>(16))
+			.run(new ChecksumHashCode<>())
 			.execute();
 
 		assertEquals(805, checksum.getCount());

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/directed/SimplifyTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/directed/SimplifyTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/directed/SimplifyTest.java
index a3aad4b..751d030 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/directed/SimplifyTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/directed/SimplifyTest.java
@@ -70,7 +70,7 @@ public class SimplifyTest {
 			"(1,0,(null))";
 
 		Graph<IntValue, NullValue, NullValue> simpleGraph = graph
-			.run(new Simplify<IntValue, NullValue, NullValue>());
+			.run(new Simplify<>());
 
 		TestBaseUtils.compareResultAsText(simpleGraph.getEdges().collect(), expectedResult);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/undirected/SimplifyTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/undirected/SimplifyTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/undirected/SimplifyTest.java
index 6ff4292..68b4e0c 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/undirected/SimplifyTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/simple/undirected/SimplifyTest.java
@@ -71,7 +71,7 @@ public class SimplifyTest {
 			"(2,0,(null))";
 
 		Graph<IntValue, NullValue, NullValue> simpleGraph = graph
-			.run(new Simplify<IntValue, NullValue, NullValue>(false));
+			.run(new Simplify<>(false));
 
 		TestBaseUtils.compareResultAsText(simpleGraph.getEdges().collect(), expectedResult);
 	}
@@ -84,7 +84,7 @@ public class SimplifyTest {
 			"(1,0,(null))";
 
 		Graph<IntValue, NullValue, NullValue> simpleGraph = graph
-			.run(new Simplify<IntValue, NullValue, NullValue>(true));
+			.run(new Simplify<>(true));
 
 		TestBaseUtils.compareResultAsText(simpleGraph.getEdges().collect(), expectedResult);
 	}


[08/11] flink git commit: [FLINK-7348] [checkstyle] Allow redundant modifiers on methods / revert removal of final modifier

Posted by ch...@apache.org.
[FLINK-7348] [checkstyle] Allow redundant modifiers on methods / revert removal of final modifier

This closes #4458.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/614c18dc
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/614c18dc
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/614c18dc

Branch: refs/heads/master
Commit: 614c18dcd9b6424d87ea836e08ddf5a84cc53894
Parents: a164407
Author: zentol <ch...@apache.org>
Authored: Wed Aug 2 11:08:20 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/storm/wrappers/SpoutWrapper.java     | 2 +-
 .../api/java/operators/translation/PlanFilterOperator.java     | 2 +-
 .../java/org/apache/flink/graph/pregel/MessageIterator.java    | 6 +++---
 .../java/org/apache/flink/graph/spargel/MessageIterator.java   | 6 +++---
 tools/maven/checkstyle.xml                                     | 4 +++-
 5 files changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/614c18dc/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
index 458fffb..6d37b29 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
@@ -240,7 +240,7 @@ public final class SpoutWrapper<OUT> extends RichParallelSourceFunction<OUT> imp
 	}
 
 	@Override
-	public void run(final SourceContext<OUT> ctx) throws Exception {
+	public final void run(final SourceContext<OUT> ctx) throws Exception {
 		final GlobalJobParameters config = super.getRuntimeContext().getExecutionConfig()
 				.getGlobalJobParameters();
 		StormConfig stormConfig = new StormConfig();

http://git-wip-us.apache.org/repos/asf/flink/blob/614c18dc/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java
index ecf1aac..07287b8 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java
@@ -53,7 +53,7 @@ public class PlanFilterOperator<T> extends FilterOperatorBase<T, FlatMapFunction
 		}
 
 		@Override
-		public void flatMap(T value, Collector<T> out) throws Exception {
+		public final void flatMap(T value, Collector<T> out) throws Exception {
 			if (this.wrappedFunction.filter(value)) {
 				out.collect(value);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/614c18dc/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/MessageIterator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/MessageIterator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/MessageIterator.java
index f8dd926..8f9cf81 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/MessageIterator.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/pregel/MessageIterator.java
@@ -43,7 +43,7 @@ public final class MessageIterator<Message> implements Iterator<Message>, Iterab
 	}
 
 	@Override
-	public boolean hasNext() {
+	public final boolean hasNext() {
 		if (first != null) {
 			return true;
 		}
@@ -53,7 +53,7 @@ public final class MessageIterator<Message> implements Iterator<Message>, Iterab
 	}
 
 	@Override
-	public Message next() {
+	public final Message next() {
 		if (first != null) {
 			Message toReturn = first;
 			first = null;
@@ -63,7 +63,7 @@ public final class MessageIterator<Message> implements Iterator<Message>, Iterab
 	}
 
 	@Override
-	public void remove() {
+	public final void remove() {
 		throw new UnsupportedOperationException();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/614c18dc/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
index be36954..071a617 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
@@ -36,17 +36,17 @@ public final class MessageIterator<Message> implements Iterator<Message>, Iterab
 	}
 
 	@Override
-	public boolean hasNext() {
+	public final boolean hasNext() {
 		return this.source.hasNext();
 	}
 
 	@Override
-	public Message next() {
+	public final Message next() {
 		return this.source.next().f1;
 	}
 
 	@Override
-	public void remove() {
+	public final void remove() {
 		throw new UnsupportedOperationException();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/614c18dc/tools/maven/checkstyle.xml
----------------------------------------------------------------------
diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml
index 897802b..6139a44 100644
--- a/tools/maven/checkstyle.xml
+++ b/tools/maven/checkstyle.xml
@@ -217,8 +217,10 @@ This file is based on the checkstyle file of Apache Beam.
     <module name="RedundantModifier">
       <!-- Checks for redundant modifiers on various symbol definitions.
         See: http://checkstyle.sourceforge.net/config_modifier.html#RedundantModifier
+        
+        We exclude METHOD_DEF to allow final methods in final classes to make them more future-proof.
       -->
-      <property name="tokens" value="METHOD_DEF, VARIABLE_DEF, ANNOTATION_FIELD_DEF, INTERFACE_DEF, CLASS_DEF, ENUM_DEF"/>
+      <property name="tokens" value="VARIABLE_DEF, ANNOTATION_FIELD_DEF, INTERFACE_DEF, CLASS_DEF, ENUM_DEF"/>
     </module>
 
     <!--


[06/11] flink git commit: [FLINK-7349] [travis] Only execute checkstyle in misc profile

Posted by ch...@apache.org.
[FLINK-7349] [travis] Only execute checkstyle in misc profile

This closes #4460.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c4a5dd85
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c4a5dd85
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c4a5dd85

Branch: refs/heads/master
Commit: c4a5dd85a92fcc5d4561bada609e0385acde381f
Parents: 0936dd0
Author: zentol <ch...@apache.org>
Authored: Wed Aug 2 12:05:56 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 tools/travis_mvn_watchdog.sh | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c4a5dd85/tools/travis_mvn_watchdog.sh
----------------------------------------------------------------------
diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh
index a623798..b590b00 100755
--- a/tools/travis_mvn_watchdog.sh
+++ b/tools/travis_mvn_watchdog.sh
@@ -102,23 +102,28 @@ if [[ $PROFILE == *"include-kinesis"* ]]; then
 fi
 
 MVN_COMPILE_MODULES=""
+MVN_COMPILE_OPTIONS=""
 MVN_TEST_MODULES=""
 case $TEST in
 	(core)
 		MVN_COMPILE_MODULES="-pl $MODULES_CORE -am"
 		MVN_TEST_MODULES="-pl $MODULES_CORE"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
 	;;
 	(libraries)
 		MVN_COMPILE_MODULES="-pl $MODULES_LIBRARIES -am"
 		MVN_TEST_MODULES="-pl $MODULES_LIBRARIES"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
 	;;
 	(connectors)
 		MVN_COMPILE_MODULES="-pl $MODULES_CONNECTORS -am"
 		MVN_TEST_MODULES="-pl $MODULES_CONNECTORS"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
 	;;
 	(tests)
 		MVN_COMPILE_MODULES="-pl $MODULES_TESTS -am"
 		MVN_TEST_MODULES="-pl $MODULES_TESTS"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
 	;;
 	(misc)
 		NEGATED_CORE=\!${MODULES_CORE//,/,\!}
@@ -138,8 +143,11 @@ esac
 # -nsu option forbids downloading snapshot artifacts. The only snapshot artifacts we depend are from
 # Flink, which however should all be built locally. see FLINK-7230
 MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configuration=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn"
-MVN_COMPILE="mvn -nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -DskipTests -Dmaven.javadoc.skip=true -B $PROFILE $MVN_LOGGING_OPTIONS $MVN_COMPILE_MODULES clean install"
-MVN_TEST="mvn -nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.javadoc.skip=true -B $PROFILE $MVN_LOGGING_OPTIONS $MVN_TEST_MODULES verify"
+MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.javadoc.skip=true -B $MVN_LOGGING_OPTIONS"
+MVN_COMPILE_OPTIONS="$MVN_COMPILE_OPTIONS -DskipTests"
+
+MVN_COMPILE="mvn $MVN_COMMON_OPTIONS $MVN_COMPILE_OPTIONS $PROFILE $MVN_COMPILE_MODULES clean install"
+MVN_TEST="mvn $MVN_COMMON_OPTIONS $PROFILE $MVN_TEST_MODULES verify"
 
 MVN_PID="${ARTIFACTS_DIR}/watchdog.mvn.pid"
 MVN_EXIT="${ARTIFACTS_DIR}/watchdog.mvn.exit"


[07/11] flink git commit: [FLINK-6982] [guava] Reduce guava dependency usages

Posted by ch...@apache.org.
[FLINK-6982] [guava] Reduce guava dependency usages

This closes #4453.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0910bc53
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0910bc53
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0910bc53

Branch: refs/heads/master
Commit: 0910bc537207895414a2e853f62c15ce4d0d91ae
Parents: 4406d48
Author: zentol <ch...@apache.org>
Authored: Tue Jun 13 17:01:24 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 flink-java/pom.xml                              |  7 --
 .../api/java/sampling/RandomSamplerTest.java    | 22 +++---
 .../flink/api/java/tuple/TupleGenerator.java    |  7 +-
 .../flink/api/java/utils/ParameterToolTest.java | 79 +++++++++++---------
 flink-scala/pom.xml                             |  6 --
 flink-streaming-scala/pom.xml                   |  6 --
 flink-tests/pom.xml                             |  7 --
 .../test/accumulators/AccumulatorITCase.java    |  7 +-
 .../flink/test/example/java/PageRankITCase.java |  7 +-
 .../test/example/scala/PageRankITCase.java      |  7 +-
 .../apache/flink/test/io/CsvReaderITCase.java   |  5 +-
 .../scala/io/ScalaCsvReaderWithPOJOITCase.scala |  7 +-
 .../YARNSessionCapacitySchedulerITCase.java     |  7 +-
 flink-yarn/pom.xml                              |  6 --
 .../yarn/YarnApplicationMasterRunnerTest.java   | 20 ++---
 15 files changed, 85 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java/pom.xml b/flink-java/pom.xml
index 2dc243e..a996cbe 100644
--- a/flink-java/pom.xml
+++ b/flink-java/pom.xml
@@ -68,13 +68,6 @@ under the License.
 		<!-- test dependencies -->
 
 		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-test-utils-junit</artifactId>
 			<version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java
index db1d19f..40cc971 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java
@@ -22,8 +22,6 @@ import org.apache.flink.testutils.junit.RetryOnFailure;
 import org.apache.flink.testutils.junit.RetryRule;
 import org.apache.flink.util.Preconditions;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -31,7 +29,9 @@ import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 
@@ -274,14 +274,12 @@ public class RandomSamplerTest {
 	 * Test sampler without replacement, and verify that there should not exist any duplicate element in sampled result.
 	 */
 	private void verifyRandomSamplerDuplicateElements(final RandomSampler<Double> sampler) {
-		List<Double> list = Lists.newLinkedList(new Iterable<Double>() {
-			@Override
-			public Iterator<Double> iterator() {
-				return sampler.sample(source.iterator());
-			}
-		});
-		Set<Double> set = Sets.newHashSet(list);
-		assertTrue("There should not have duplicate element for sampler without replacement.", list.size() == set.size());
+		Iterator<Double> values = sampler.sample(source.iterator());
+		Set<Double> set = new HashSet<>();
+		while (values.hasNext()) {
+			double next = values.next();
+			assertTrue("Sampler returned duplicate element (" + next + "). Set=" + set, set.add(next));
+		}
 	}
 
 	private int getSize(Iterator<?> iterator) {
@@ -370,7 +368,7 @@ public class RandomSamplerTest {
 		Iterator<Double> sampled;
 		if (sampleOnPartitions) {
 			DistributedRandomSampler<Double> reservoirRandomSampler = (DistributedRandomSampler<Double>) sampler;
-			List<IntermediateSampleData<Double>> intermediateResult = Lists.newLinkedList();
+			List<IntermediateSampleData<Double>> intermediateResult = new LinkedList<>();
 			for (int i = 0; i < DEFAULT_PARTITION_NUMBER; i++) {
 				Iterator<IntermediateSampleData<Double>> partialIntermediateResult = reservoirRandomSampler.sampleInPartition(sourcePartitions[i].iterator());
 				while (partialIntermediateResult.hasNext()) {
@@ -381,7 +379,7 @@ public class RandomSamplerTest {
 		} else {
 			sampled = sampler.sample(source.iterator());
 		}
-		List<Double> list = Lists.newArrayList();
+		List<Double> list = new ArrayList<>();
 		while (sampled.hasNext()) {
 			list.add(sampled.next());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java b/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java
index dbc76c0..e063bc4 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java
@@ -18,11 +18,10 @@
 
 package org.apache.flink.api.java.tuple;
 
-import com.google.common.io.Files;
+import org.apache.flink.util.FileUtils;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.Scanner;
 
 /**
@@ -95,7 +94,7 @@ class TupleGenerator {
 	}
 
 	private static void insertCodeIntoFile(String code, File file) throws IOException {
-		String fileContent = Files.toString(file, StandardCharsets.UTF_8);
+		String fileContent = FileUtils.readFileUtf8(file);
 
 		try (Scanner s = new Scanner(fileContent)) {
 			StringBuilder sb = new StringBuilder();
@@ -137,7 +136,7 @@ class TupleGenerator {
 				sb.append(line).append("\n");
 			}
 			s.close();
-			Files.write(sb.toString(), file, StandardCharsets.UTF_8);
+			FileUtils.writeFileUtf8(file, sb.toString());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
index 1924ea3..f8549f8 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.api.java.utils;
 
-import com.google.common.collect.Sets;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -29,8 +28,10 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
 /**
  * Tests for {@link ParameterTool}.
@@ -163,7 +164,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedBoolean() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean", "true"});
-		Assert.assertEquals(Sets.newHashSet("boolean"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertTrue(parameter.getBoolean("boolean"));
@@ -177,7 +178,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedBooleanWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean", "true"});
-		Assert.assertEquals(Sets.newHashSet("boolean"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertTrue(parameter.getBoolean("boolean", false));
@@ -191,7 +192,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedBooleanWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean"});
-		Assert.assertEquals(Sets.newHashSet("boolean"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters());
 
 		parameter.getBoolean("boolean");
 		Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters());
@@ -202,7 +203,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedByte() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-byte", "1"});
-		Assert.assertEquals(Sets.newHashSet("byte"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(1, parameter.getByte("byte"));
@@ -216,7 +217,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedByteWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-byte", "1"});
-		Assert.assertEquals(Sets.newHashSet("byte"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(1, parameter.getByte("byte", (byte) 0));
@@ -230,7 +231,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedByteWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-byte"});
-		Assert.assertEquals(Sets.newHashSet("byte"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters());
 
 		exception.expect(RuntimeException.class);
 		exception.expectMessage("For input string: \"__NO_VALUE_KEY\"");
@@ -243,7 +244,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedShort() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-short", "2"});
-		Assert.assertEquals(Sets.newHashSet("short"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(2, parameter.getShort("short"));
@@ -257,7 +258,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedShortWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-short", "2"});
-		Assert.assertEquals(Sets.newHashSet("short"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(2, parameter.getShort("short", (short) 0));
@@ -271,7 +272,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedShortWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-short"});
-		Assert.assertEquals(Sets.newHashSet("short"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters());
 
 		exception.expect(RuntimeException.class);
 		exception.expectMessage("For input string: \"__NO_VALUE_KEY\"");
@@ -284,7 +285,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedInt() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-int", "4"});
-		Assert.assertEquals(Sets.newHashSet("int"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(4, parameter.getInt("int"));
@@ -298,7 +299,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedIntWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-int", "4"});
-		Assert.assertEquals(Sets.newHashSet("int"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(4, parameter.getInt("int", 0));
@@ -312,7 +313,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedIntWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-int"});
-		Assert.assertEquals(Sets.newHashSet("int"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters());
 
 		exception.expect(RuntimeException.class);
 		exception.expectMessage("For input string: \"__NO_VALUE_KEY\"");
@@ -325,7 +326,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedLong() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-long", "8"});
-		Assert.assertEquals(Sets.newHashSet("long"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(8, parameter.getLong("long"));
@@ -339,7 +340,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedLongWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-long", "8"});
-		Assert.assertEquals(Sets.newHashSet("long"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(8, parameter.getLong("long", 0));
@@ -353,7 +354,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedLongWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-long"});
-		Assert.assertEquals(Sets.newHashSet("long"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters());
 
 		exception.expect(RuntimeException.class);
 		exception.expectMessage("For input string: \"__NO_VALUE_KEY\"");
@@ -366,7 +367,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedFloat() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-float", "4"});
-		Assert.assertEquals(Sets.newHashSet("float"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001);
@@ -380,7 +381,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedFloatWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-float", "4"});
-		Assert.assertEquals(Sets.newHashSet("float"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(4.0, parameter.getFloat("float", 0.0f), 0.00001);
@@ -394,7 +395,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedFloatWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-float"});
-		Assert.assertEquals(Sets.newHashSet("float"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters());
 
 		exception.expect(RuntimeException.class);
 		exception.expectMessage("For input string: \"__NO_VALUE_KEY\"");
@@ -407,7 +408,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedDouble() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-double", "8"});
-		Assert.assertEquals(Sets.newHashSet("double"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001);
@@ -421,7 +422,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedDoubleWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-double", "8"});
-		Assert.assertEquals(Sets.newHashSet("double"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals(8.0, parameter.getDouble("double", 0.0), 0.00001);
@@ -435,7 +436,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedDoubleWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-double"});
-		Assert.assertEquals(Sets.newHashSet("double"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters());
 
 		exception.expect(RuntimeException.class);
 		exception.expectMessage("For input string: \"__NO_VALUE_KEY\"");
@@ -448,7 +449,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedString() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-string", "∞"});
-		Assert.assertEquals(Sets.newHashSet("string"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals("∞", parameter.get("string"));
@@ -462,7 +463,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedStringWithDefaultValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-string", "∞"});
-		Assert.assertEquals(Sets.newHashSet("string"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals("∞", parameter.get("string", "0.0"));
@@ -476,7 +477,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedStringWithMissingValue() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-string"});
-		Assert.assertEquals(Sets.newHashSet("string"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters());
 
 		parameter.get("string");
 		Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters());
@@ -487,7 +488,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedHas() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean"});
-		Assert.assertEquals(Sets.newHashSet("boolean"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertTrue(parameter.has("boolean"));
@@ -501,7 +502,7 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	@Test
 	public void testUnrequestedRequired() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-required", "∞"});
-		Assert.assertEquals(Sets.newHashSet("required"), parameter.getUnrequestedParameters());
+		Assert.assertEquals(createHashSet("required"), parameter.getUnrequestedParameters());
 
 		// test parameter access
 		Assert.assertEquals("∞", parameter.getRequired("required"));
@@ -516,35 +517,35 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 	public void testUnrequestedMultiple() {
 		ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean", "true", "-byte", "1",
 			"-short", "2", "-int", "4", "-long", "8", "-float", "4.0", "-double", "8.0", "-string", "∞"});
-		Assert.assertEquals(Sets.newHashSet("boolean", "byte", "short", "int", "long", "float", "double", "string"),
+		Assert.assertEquals(createHashSet("boolean", "byte", "short", "int", "long", "float", "double", "string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertTrue(parameter.getBoolean("boolean"));
-		Assert.assertEquals(Sets.newHashSet("byte", "short", "int", "long", "float", "double", "string"),
+		Assert.assertEquals(createHashSet("byte", "short", "int", "long", "float", "double", "string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertEquals(1, parameter.getByte("byte"));
-		Assert.assertEquals(Sets.newHashSet("short", "int", "long", "float", "double", "string"),
+		Assert.assertEquals(createHashSet("short", "int", "long", "float", "double", "string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertEquals(2, parameter.getShort("short"));
-		Assert.assertEquals(Sets.newHashSet("int", "long", "float", "double", "string"),
+		Assert.assertEquals(createHashSet("int", "long", "float", "double", "string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertEquals(4, parameter.getInt("int"));
-		Assert.assertEquals(Sets.newHashSet("long", "float", "double", "string"),
+		Assert.assertEquals(createHashSet("long", "float", "double", "string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertEquals(8, parameter.getLong("long"));
-		Assert.assertEquals(Sets.newHashSet("float", "double", "string"),
+		Assert.assertEquals(createHashSet("float", "double", "string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001);
-		Assert.assertEquals(Sets.newHashSet("double", "string"),
+		Assert.assertEquals(createHashSet("double", "string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001);
-		Assert.assertEquals(Sets.newHashSet("string"),
+		Assert.assertEquals(createHashSet("string"),
 			parameter.getUnrequestedParameters());
 
 		Assert.assertEquals("∞", parameter.get("string"));
@@ -567,4 +568,12 @@ public class ParameterToolTest extends AbstractParameterToolTest {
 
 		Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters());
 	}
+
+	private static <T> Set<T> createHashSet(T... elements) {
+		Set<T> set = new HashSet<>();
+		for (T element : elements) {
+			set.add(element);
+		}
+		return set;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index fd73253..654e15c 100644
--- a/flink-scala/pom.xml
+++ b/flink-scala/pom.xml
@@ -65,12 +65,6 @@ under the License.
 			<artifactId>asm</artifactId>
 			<version>${asm.version}</version>
 		</dependency>
-		
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
 
 		<!-- test dependencies -->
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-streaming-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml
index 586dca7..57ab1cb 100644
--- a/flink-streaming-scala/pom.xml
+++ b/flink-streaming-scala/pom.xml
@@ -68,12 +68,6 @@ under the License.
 			<artifactId>asm</artifactId>
 			<version>${asm.version}</version>
 		</dependency>
-		
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
 
 		<!-- test dependencies -->
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 215adc0..85d90b3 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -217,13 +217,6 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
index 3e35bd0..b7f54fa 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
@@ -35,10 +35,9 @@ import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.types.StringValue;
 import org.apache.flink.util.Collector;
 
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 import org.junit.Assert;
 
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
@@ -81,12 +80,12 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 		Assert.assertEquals(Double.valueOf(getParallelism()), res.getAccumulatorResult("open-close-counter"));
 
 		// Test histogram (words per line distribution)
-		Map<Integer, Integer> dist = Maps.newHashMap();
+		Map<Integer, Integer> dist = new HashMap<>();
 		dist.put(1, 1); dist.put(2, 1); dist.put(3, 1);
 		Assert.assertEquals(dist, res.getAccumulatorResult("words-per-line"));
 
 		// Test distinct words (custom accumulator)
-		Set<StringValue> distinctWords = Sets.newHashSet();
+		Set<StringValue> distinctWords = new HashSet<>();
 		distinctWords.add(new StringValue("one"));
 		distinctWords.add(new StringValue("two"));
 		distinctWords.add(new StringValue("three"));

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java
index daab163..ee7bf82 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java
@@ -22,9 +22,8 @@ package org.apache.flink.test.example.java;
 import org.apache.flink.examples.java.graph.PageRank;
 import org.apache.flink.test.testdata.PageRankData;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.FileUtils;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -57,10 +56,10 @@ public class PageRankITCase extends MultipleProgramsTestBase {
 	public void before() throws Exception{
 		resultPath = tempFolder.newFile().toURI().toString();
 		File verticesFile = tempFolder.newFile();
-		Files.write(PageRankData.VERTICES, verticesFile, Charsets.UTF_8);
+		FileUtils.writeFileUtf8(verticesFile, PageRankData.VERTICES);
 
 		File edgesFile = tempFolder.newFile();
-		Files.write(PageRankData.EDGES, edgesFile, Charsets.UTF_8);
+		FileUtils.writeFileUtf8(edgesFile, PageRankData.EDGES);
 
 		verticesPath = verticesFile.toURI().toString();
 		edgesPath = edgesFile.toURI().toString();

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java
index 93c4f56..ae18617 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java
@@ -22,9 +22,8 @@ package org.apache.flink.test.example.scala;
 import org.apache.flink.examples.scala.graph.PageRankBasic;
 import org.apache.flink.test.testdata.PageRankData;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.FileUtils;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -61,10 +60,10 @@ public class PageRankITCase extends MultipleProgramsTestBase {
 		resultPath = resultFile.toURI().toString();
 
 		File verticesFile = tempFolder.newFile();
-		Files.write(PageRankData.VERTICES, verticesFile, Charsets.UTF_8);
+		FileUtils.writeFileUtf8(verticesFile, PageRankData.VERTICES);
 
 		File edgesFile = tempFolder.newFile();
-		Files.write(PageRankData.EDGES, edgesFile, Charsets.UTF_8);
+		FileUtils.writeFileUtf8(edgesFile, PageRankData.EDGES);
 
 		verticesPath = verticesFile.toURI().toString();
 		edgesPath = edgesFile.toURI().toString();

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
index 82f699d..4fed163 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
@@ -30,9 +30,8 @@ import org.apache.flink.types.IntValue;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.ShortValue;
 import org.apache.flink.types.StringValue;
+import org.apache.flink.util.FileUtils;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -59,7 +58,7 @@ public class CsvReaderITCase extends MultipleProgramsTestBase {
 
 	private String createInputData(String data) throws Exception {
 		File file = tempFolder.newFile("input");
-		Files.write(data, file, Charsets.UTF_8);
+		FileUtils.writeFileUtf8(file, data);
 
 		return file.toURI().toString();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-tests/src/test/scala/org/apache/flink/api/scala/io/ScalaCsvReaderWithPOJOITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/io/ScalaCsvReaderWithPOJOITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/io/ScalaCsvReaderWithPOJOITCase.scala
index b2c5d0c..ffdd908 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/io/ScalaCsvReaderWithPOJOITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/io/ScalaCsvReaderWithPOJOITCase.scala
@@ -20,12 +20,11 @@ package org.apache.flink.api.scala.io
 
 import java.util.Locale
 
-import com.google.common.base.Charsets
-import com.google.common.io.Files
 import org.apache.flink.api.scala._
 import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.{TestBaseUtils, MultipleProgramsTestBase}
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
 import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.util.FileUtils
 import org.junit.Assert._
 import org.junit.rules.TemporaryFolder
 import org.junit.runner.RunWith
@@ -53,7 +52,7 @@ class ScalaCsvReaderWithPOJOITCase(mode: TestExecutionMode) extends MultipleProg
 
   def createInputData(data: String): String = {
     val dataFile = tempFolder.newFile("data")
-    Files.write(data, dataFile, Charsets.UTF_8)
+    FileUtils.writeFileUtf8(dataFile, data)
     dataFile.toURI.toString
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 5caea29..2e88836 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -30,9 +30,8 @@ import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.google.common.base.Joiner;
-import com.google.common.collect.Sets;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -234,7 +233,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 			NodeManager nm = yarnCluster.getNodeManager(nmId);
 			ConcurrentMap<ContainerId, Container> containers = nm.getNMContext().getContainers();
 			for (Map.Entry<ContainerId, Container> entry : containers.entrySet()) {
-				String command = Joiner.on(" ").join(entry.getValue().getLaunchContext().getCommands());
+				String command = StringUtils.join(entry.getValue().getLaunchContext().getCommands(), " ");
 				if (command.contains(YarnTaskManager.class.getSimpleName())) {
 					taskManagerContainer = entry.getKey();
 					nodeManager = nm;
@@ -568,7 +567,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 		@SuppressWarnings("unchecked")
 		Set<String> applicationTags = (Set<String>) applicationTagsMethod.invoke(report);
 
-		Assert.assertEquals(applicationTags, Sets.newHashSet("test-tag"));
+		Assert.assertEquals(applicationTags, Collections.singleton("test-tag"));
 	}
 
 	@After

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml
index ec0dd63..aa7603c 100644
--- a/flink-yarn/pom.xml
+++ b/flink-yarn/pom.xml
@@ -83,12 +83,6 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<dependency>
 			<groupId>com.data-artisans</groupId>
 			<artifactId>flakka-testkit_${scala.binary.version}</artifactId>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/0910bc53/flink-yarn/src/test/java/org/apache/flink/yarn/YarnApplicationMasterRunnerTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnApplicationMasterRunnerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnApplicationMasterRunnerTest.java
index 19a0352..b15374b 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnApplicationMasterRunnerTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnApplicationMasterRunnerTest.java
@@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
 import org.apache.flink.util.OperatingSystem;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.Assume;
@@ -37,6 +36,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 
 import static org.apache.flink.yarn.YarnConfigKeys.ENV_APP_ID;
@@ -90,14 +91,15 @@ public class YarnApplicationMasterRunnerTest {
 			}
 		}).when(yarnConf).getStrings(anyString(), Mockito.<String> anyVararg());
 
-		Map<String, String> env = ImmutableMap.<String, String> builder()
-			.put(ENV_APP_ID, "foo")
-			.put(ENV_CLIENT_HOME_DIR, home.getAbsolutePath())
-			.put(ENV_CLIENT_SHIP_FILES, "")
-			.put(ENV_FLINK_CLASSPATH, "")
-			.put(ENV_HADOOP_USER_NAME, "foo")
-			.put(FLINK_JAR_PATH, root.toURI().toString())
-			.build();
+		Map<String, String> env = new HashMap<>();
+		env.put(ENV_APP_ID, "foo");
+		env.put(ENV_CLIENT_HOME_DIR, home.getAbsolutePath());
+		env.put(ENV_CLIENT_SHIP_FILES, "");
+		env.put(ENV_FLINK_CLASSPATH, "");
+		env.put(ENV_HADOOP_USER_NAME, "foo");
+		env.put(FLINK_JAR_PATH, root.toURI().toString());
+		env = Collections.unmodifiableMap(env);
+
 		ContaineredTaskManagerParameters tmParams = mock(ContaineredTaskManagerParameters.class);
 		Configuration taskManagerConf = new Configuration();
 


[11/11] flink git commit: [hotfix] [gelly] Explicit type can be replaced with <>

Posted by ch...@apache.org.
[hotfix] [gelly] Explicit type can be replaced with <>

In Java 8 the diamond operator can be used in cases which would result
in an error in Java 7. In Gelly we have often desired to use the diamond
operator and only discovered an issue when running tests on TravisCI.

This closes #4457.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/27429a74
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/27429a74
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/27429a74

Branch: refs/heads/master
Commit: 27429a74f209d7c2bc0ef48f4c28847ce9d9ddf2
Parents: 0910bc5
Author: Greg Hogan <co...@greghogan.com>
Authored: Mon Jul 31 16:29:06 2017 -0400
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 .../graph/drivers/ConnectedComponents.java      |   4 +-
 .../apache/flink/graph/drivers/EdgeList.java    |   2 +-
 .../flink/graph/drivers/parameter/Simplify.java |   6 +-
 .../graph/examples/EuclideanGraphWeighing.java  |   8 +-
 .../flink/graph/examples/GSAPageRank.java       |   2 +-
 .../examples/GSASingleSourceShortestPaths.java  |   2 +-
 .../flink/graph/examples/IncrementalSSSP.java   |   2 +-
 .../flink/graph/examples/MusicProfiles.java     |  14 +-
 .../apache/flink/graph/examples/PageRank.java   |   4 +-
 .../apache/flink/graph/examples/PregelSSSP.java |   2 +-
 .../examples/SingleSourceShortestPaths.java     |   2 +-
 .../examples/data/CommunityDetectionData.java   |  66 ++++-----
 .../graph/examples/data/EuclideanGraphData.java |  40 +++---
 .../examples/data/IncrementalSSSPData.java      |  38 ++---
 .../examples/data/LabelPropagationData.java     |  68 ++++-----
 .../graph/examples/data/MusicProfilesData.java  |  84 +++++------
 .../flink/graph/examples/data/PageRankData.java |  20 +--
 .../graph/examples/data/TriangleCountData.java  |  20 +--
 .../graph/library/CommunityDetectionITCase.java |   4 +-
 .../graph/library/LabelPropagationITCase.java   |   4 +-
 .../graph/library/SummarizationITCase.java      |  12 +-
 .../graph/library/TriangleEnumeratorITCase.java |   2 +-
 .../flink/graph/test/GatherSumApplyITCase.java  |  10 +-
 .../graph/test/examples/PageRankITCase.java     |   8 +-
 .../main/java/org/apache/flink/graph/Graph.java | 140 +++++++++----------
 .../org/apache/flink/graph/GraphCsvReader.java  |   6 +-
 .../annotate/directed/EdgeDegreesPair.java      |   2 +-
 .../annotate/directed/EdgeSourceDegrees.java    |   2 +-
 .../annotate/directed/EdgeTargetDegrees.java    |   2 +-
 .../degree/annotate/directed/VertexDegrees.java |   8 +-
 .../annotate/directed/VertexInDegree.java       |   6 +-
 .../annotate/directed/VertexOutDegree.java      |   6 +-
 .../annotate/undirected/EdgeDegreePair.java     |   2 +-
 .../annotate/undirected/EdgeSourceDegree.java   |   2 +-
 .../annotate/undirected/EdgeTargetDegree.java   |   2 +-
 .../annotate/undirected/VertexDegree.java       |   6 +-
 .../degree/filter/undirected/MaximumDegree.java |   8 +-
 .../graph/asm/simple/directed/Simplify.java     |   2 +-
 .../graph/asm/simple/undirected/Simplify.java   |   2 +-
 .../flink/graph/asm/translate/Translate.java    |   8 +-
 .../flink/graph/bipartite/BipartiteGraph.java   |   8 +-
 .../graph/generator/GraphGeneratorUtils.java    |   2 +-
 .../apache/flink/graph/generator/RMatGraph.java |   2 +-
 .../flink/graph/gsa/GSAConfiguration.java       |   6 +-
 .../graph/gsa/GatherSumApplyIteration.java      |  10 +-
 .../flink/graph/library/CommunityDetection.java |   8 +-
 .../graph/library/ConnectedComponents.java      |   7 +-
 .../graph/library/GSAConnectedComponents.java   |   5 +-
 .../library/GSASingleSourceShortestPaths.java   |   4 +-
 .../flink/graph/library/LabelPropagation.java   |   5 +-
 .../library/SingleSourceShortestPaths.java      |   4 +-
 .../flink/graph/library/Summarization.java      |  16 +--
 .../flink/graph/library/TriangleEnumerator.java |  14 +-
 .../directed/LocalClusteringCoefficient.java    |   6 +-
 .../clustering/directed/TriangleListing.java    |  16 +--
 .../undirected/LocalClusteringCoefficient.java  |   6 +-
 .../clustering/undirected/TriangleListing.java  |  12 +-
 .../flink/graph/library/linkanalysis/HITS.java  |  24 ++--
 .../graph/library/linkanalysis/PageRank.java    |  18 +--
 .../library/metric/directed/EdgeMetrics.java    |   6 +-
 .../library/metric/undirected/EdgeMetrics.java  |   4 +-
 .../graph/library/similarity/AdamicAdar.java    |  14 +-
 .../graph/library/similarity/JaccardIndex.java  |  10 +-
 .../pregel/VertexCentricConfiguration.java      |   2 +-
 .../graph/pregel/VertexCentricIteration.java    |  10 +-
 .../spargel/ScatterGatherConfiguration.java     |   4 +-
 .../apache/flink/graph/utils/GraphUtils.java    |   2 +-
 .../validation/InvalidVertexIdsValidator.java   |   6 +-
 .../org/apache/flink/graph/asm/AsmTestBase.java |   4 +-
 .../annotate/directed/EdgeDegreesPairTest.java  |   4 +-
 .../directed/EdgeSourceDegreesTest.java         |   4 +-
 .../directed/EdgeTargetDegreesTest.java         |   4 +-
 .../annotate/directed/VertexDegreesTest.java    |   6 +-
 .../annotate/undirected/EdgeDegreePairTest.java |   4 +-
 .../undirected/EdgeSourceDegreeTest.java        |   4 +-
 .../undirected/EdgeTargetDegreeTest.java        |   4 +-
 .../annotate/undirected/VertexDegreeTest.java   |   6 +-
 .../filter/undirected/MaximumDegreeTest.java    |   7 +-
 .../graph/asm/simple/directed/SimplifyTest.java |   2 +-
 .../asm/simple/undirected/SimplifyTest.java     |   4 +-
 .../graph/generator/CirculantGraphTest.java     |   6 +-
 .../graph/generator/CompleteGraphTest.java      |   6 +-
 .../flink/graph/generator/CycleGraphTest.java   |   6 +-
 .../flink/graph/generator/EchoGraphTest.java    |   6 +-
 .../flink/graph/generator/EmptyGraphTest.java   |   6 +-
 .../flink/graph/generator/GridGraphTest.java    |   6 +-
 .../graph/generator/HypercubeGraphTest.java     |   6 +-
 .../flink/graph/generator/PathGraphTest.java    |   6 +-
 .../flink/graph/generator/RMatGraphTest.java    |   6 +-
 .../graph/generator/SingletonEdgeGraphTest.java |   6 +-
 .../flink/graph/generator/StarGraphTest.java    |   6 +-
 .../apache/flink/graph/gsa/GSACompilerTest.java |   4 +-
 .../flink/graph/gsa/GSATranslationTest.java     |   4 +-
 ...ctedComponentsWithRandomisedEdgesITCase.java |   2 +-
 .../LocalClusteringCoefficientTest.java         |   7 +-
 .../directed/TriangleListingTest.java           |   2 +-
 .../LocalClusteringCoefficientTest.java         |   7 +-
 .../undirected/TriangleListingTest.java         |   2 +-
 .../graph/library/linkanalysis/HITSTest.java    |   2 +-
 .../library/metric/ChecksumHashCodeTest.java    |   2 +-
 .../library/similarity/AdamicAdarTest.java      |   6 +-
 .../library/similarity/JaccardIndexTest.java    |   2 +-
 .../flink/graph/pregel/PregelCompilerTest.java  |  12 +-
 .../graph/pregel/PregelTranslationTest.java     |   2 +-
 .../graph/spargel/SpargelCompilerTest.java      |  16 +--
 .../graph/spargel/SpargelTranslationTest.java   |   4 +-
 .../test/CollectionModeSuperstepITCase.java     |   6 +-
 .../test/ScatterGatherConfigurationITCase.java  |   2 +-
 .../operations/DegreesWithExceptionITCase.java  |  12 +-
 .../test/operations/GraphCreationITCase.java    |   4 +-
 .../test/operations/JoinWithEdgesITCase.java    |   4 +-
 .../test/operations/JoinWithVerticesITCase.java |   4 +-
 .../ReduceOnEdgesWithExceptionITCase.java       |   4 +-
 .../ReduceOnNeighborsWithExceptionITCase.java   |   8 +-
 .../test/operations/TypeExtractorTest.java      |  10 +-
 115 files changed, 538 insertions(+), 568 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java
index 32f94c1..c40103b 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java
@@ -46,8 +46,8 @@ extends DriverBase<K, VV, EV> {
 	@Override
 	public DataSet plan(Graph<K, VV, EV> graph) throws Exception {
 		return graph
-			.mapVertices(new MapVertices<K, VV>())
-			.run(new GSAConnectedComponents<K, K, EV>(Integer.MAX_VALUE));
+			.mapVertices(new MapVertices<>())
+			.run(new GSAConnectedComponents<>(Integer.MAX_VALUE));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java
index 563908c..f0de90a 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java
@@ -54,7 +54,7 @@ extends DriverBase<K, VV, EV> {
 
 		if (hasNullValueEdges(edges)) {
 			return edges
-				.map(new EdgeToTuple2Map<K, EV>())
+				.map(new EdgeToTuple2Map<>())
 				.name("Edge to Tuple2")
 				.setParallelism(parallelism.getValue().intValue());
 		} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java
index 9fc937c..14ff9d0 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java
@@ -115,15 +115,15 @@ implements Parameter<Ordering> {
 		switch (value) {
 			case DIRECTED:
 				graph = graph
-					.run(new org.apache.flink.graph.asm.simple.directed.Simplify<T, NullValue, NullValue>());
+					.run(new org.apache.flink.graph.asm.simple.directed.Simplify<>());
 				break;
 			case UNDIRECTED:
 				graph = graph
-					.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<T, NullValue, NullValue>(false));
+					.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<>(false));
 				break;
 			case UNDIRECTED_CLIP_AND_FLIP:
 				graph = graph
-					.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<T, NullValue, NullValue>(true));
+					.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<>(true));
 				break;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/EuclideanGraphWeighing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/EuclideanGraphWeighing.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/EuclideanGraphWeighing.java
index 6380628..f426a9c 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/EuclideanGraphWeighing.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/EuclideanGraphWeighing.java
@@ -83,8 +83,8 @@ public class EuclideanGraphWeighing implements ProgramDescription {
 						Vertex<Long, Point> srcVertex = triplet.getSrcVertex();
 						Vertex<Long, Point> trgVertex = triplet.getTrgVertex();
 
-						return new Tuple3<Long, Long, Double>(srcVertex.getId(), trgVertex.getId(),
-								srcVertex.getValue().euclideanDistance(trgVertex.getValue()));
+						return new Tuple3<>(srcVertex.getId(), trgVertex.getId(),
+							srcVertex.getValue().euclideanDistance(trgVertex.getValue()));
 					}
 				});
 
@@ -186,7 +186,7 @@ public class EuclideanGraphWeighing implements ProgramDescription {
 
 						@Override
 						public Vertex<Long, Point> map(Tuple3<Long, Double, Double> value) throws Exception {
-							return new Vertex<Long, Point>(value.f0, new Point(value.f1, value.f2));
+							return new Vertex<>(value.f0, new Point(value.f1, value.f2));
 						}
 					});
 		} else {
@@ -203,7 +203,7 @@ public class EuclideanGraphWeighing implements ProgramDescription {
 
 						@Override
 						public Edge<Long, Double> map(Tuple2<Long, Long> tuple2) throws Exception {
-							return new Edge<Long, Double>(tuple2.f0, tuple2.f1, 0.0);
+							return new Edge<>(tuple2.f0, tuple2.f1, 0.0);
 						}
 					});
 		} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSAPageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSAPageRank.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSAPageRank.java
index 4508419..2cd9f4e 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSAPageRank.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSAPageRank.java
@@ -66,7 +66,7 @@ public class GSAPageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet
 		parameters.setOptNumVertices(true);
 
 		return networkWithWeights.runGatherSumApplyIteration(new GatherRanks(), new SumRanks(),
-				new UpdateRanks<K>(beta), maxIterations, parameters)
+			new UpdateRanks<>(beta), maxIterations, parameters)
 				.getVertices();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
index aa2b7e9..006ebd9 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
@@ -179,7 +179,7 @@ public class GSASingleSourceShortestPaths implements ProgramDescription {
 					.fieldDelimiter("\t")
 					.lineDelimiter("\n")
 					.types(Long.class, Long.class, Double.class)
-					.map(new Tuple3ToEdgeMap<Long, Double>());
+					.map(new Tuple3ToEdgeMap<>());
 		} else {
 			return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/IncrementalSSSP.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/IncrementalSSSP.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/IncrementalSSSP.java
index 197ad68..d5d770d 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/IncrementalSSSP.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/IncrementalSSSP.java
@@ -265,7 +265,7 @@ public class IncrementalSSSP implements ProgramDescription {
 
 	private static Edge<Long, Double> getEdgeToBeRemoved() {
 		if (fileOutput) {
-			return new Edge<Long, Double>(srcEdgeToBeRemoved, trgEdgeToBeRemoved, valEdgeToBeRemoved);
+			return new Edge<>(srcEdgeToBeRemoved, trgEdgeToBeRemoved, valEdgeToBeRemoved);
 		} else {
 			return IncrementalSSSPData.getDefaultEdgeToBeRemoved();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/MusicProfiles.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/MusicProfiles.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/MusicProfiles.java
index 43a0a9b..6afec71 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/MusicProfiles.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/MusicProfiles.java
@@ -144,7 +144,7 @@ public class MusicProfiles implements ProgramDescription {
 				.map(new MapFunction<Tuple2<Long, String>, Tuple2<String, Long>>() {
 					@Override
 					public Tuple2<String, Long> map(Tuple2<Long, String> tuple2) throws Exception {
-						return new Tuple2<String, Long>(tuple2.f1, tuple2.f0);
+						return new Tuple2<>(tuple2.f1, tuple2.f0);
 					}
 				});
 
@@ -154,7 +154,7 @@ public class MusicProfiles implements ProgramDescription {
 							public Long vertexJoin(Long vertexValue, Long inputValue) {
 								return inputValue;
 							}
-						}).run(new LabelPropagation<String, Long, NullValue>(maxIterations));
+						}).run(new LabelPropagation<>(maxIterations));
 
 		if (fileOutput) {
 			verticesWithCommunity.writeAsCsv(communitiesOutputPath, "\n", "\t");
@@ -172,7 +172,7 @@ public class MusicProfiles implements ProgramDescription {
 		public Tuple1<String> map(String value) {
 			String[] tokens = value.split("\\s+");
 			String songId = tokens[1].substring(1);
-			return new Tuple1<String>(songId);
+			return new Tuple1<>(songId);
 		}
 	}
 
@@ -211,7 +211,7 @@ public class MusicProfiles implements ProgramDescription {
 					topSong = edge.getTarget();
 				}
 			}
-			out.collect(new Tuple2<String, String>(vertex.getId(), topSong));
+			out.collect(new Tuple2<>(vertex.getId(), topSong));
 		}
 	}
 
@@ -219,14 +219,14 @@ public class MusicProfiles implements ProgramDescription {
 		Edge<String, NullValue>> {
 
 		public void reduce(Iterable<Edge<String, Integer>> edges, Collector<Edge<String, NullValue>> out) {
-			List<String> listeners = new ArrayList<String>();
+			List<String> listeners = new ArrayList<>();
 			for (Edge<String, Integer> edge : edges) {
 				listeners.add(edge.getSource());
 			}
 			for (int i = 0; i < listeners.size() - 1; i++) {
 				for (int j = i + 1; j < listeners.size(); j++) {
-					out.collect(new Edge<String, NullValue>(listeners.get(i),
-							listeners.get(j), NullValue.getInstance()));
+					out.collect(new Edge<>(listeners.get(i),
+						listeners.get(j), NullValue.getInstance()));
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PageRank.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PageRank.java
index a88f80e..19d80be 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PageRank.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PageRank.java
@@ -65,8 +65,8 @@ public class PageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Ve
 		ScatterGatherConfiguration parameters = new ScatterGatherConfiguration();
 		parameters.setOptNumVertices(true);
 
-		return networkWithWeights.runScatterGatherIteration(new RankMessenger<K>(),
-				new VertexRankUpdater<K>(beta), maxIterations, parameters)
+		return networkWithWeights.runScatterGatherIteration(new RankMessenger<>(),
+			new VertexRankUpdater<>(beta), maxIterations, parameters)
 				.getVertices();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PregelSSSP.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PregelSSSP.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PregelSSSP.java
index 97a7c5f..da227e0 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PregelSSSP.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/PregelSSSP.java
@@ -183,7 +183,7 @@ public class PregelSSSP implements ProgramDescription {
 					.fieldDelimiter("\t")
 					.ignoreComments("%")
 					.types(Long.class, Long.class, Double.class)
-					.map(new Tuple3ToEdgeMap<Long, Double>());
+					.map(new Tuple3ToEdgeMap<>());
 		} else {
 			return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/SingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/SingleSourceShortestPaths.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/SingleSourceShortestPaths.java
index 07c6b56..211ce39 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/SingleSourceShortestPaths.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/SingleSourceShortestPaths.java
@@ -188,7 +188,7 @@ public class SingleSourceShortestPaths implements ProgramDescription {
 					.lineDelimiter("\n")
 					.fieldDelimiter("\t")
 					.types(Long.class, Long.class, Double.class)
-					.map(new Tuple3ToEdgeMap<Long, Double>());
+					.map(new Tuple3ToEdgeMap<>());
 		} else {
 			return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/CommunityDetectionData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/CommunityDetectionData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/CommunityDetectionData.java
index 1d9b257..713cbbb 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/CommunityDetectionData.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/CommunityDetectionData.java
@@ -43,40 +43,40 @@ public class CommunityDetectionData {
 
 	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 2.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 3L, 4.0));
-		edges.add(new Edge<Long, Double>(2L, 4L, 5.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 6.0));
-		edges.add(new Edge<Long, Double>(5L, 6L, 7.0));
-		edges.add(new Edge<Long, Double>(5L, 7L, 8.0));
-		edges.add(new Edge<Long, Double>(6L, 7L, 9.0));
-		edges.add(new Edge<Long, Double>(7L, 12L, 10.0));
-		edges.add(new Edge<Long, Double>(8L, 9L, 11.0));
-		edges.add(new Edge<Long, Double>(8L, 10L, 12.0));
-		edges.add(new Edge<Long, Double>(8L, 11L, 13.0));
-		edges.add(new Edge<Long, Double>(9L, 10L, 14.0));
-		edges.add(new Edge<Long, Double>(9L, 11L, 15.0));
-		edges.add(new Edge<Long, Double>(10L, 11L, 16.0));
-		edges.add(new Edge<Long, Double>(10L, 12L, 17.0));
-		edges.add(new Edge<Long, Double>(11L, 12L, 18.0));
+		List<Edge<Long, Double>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 2L, 1.0));
+		edges.add(new Edge<>(1L, 3L, 2.0));
+		edges.add(new Edge<>(1L, 4L, 3.0));
+		edges.add(new Edge<>(2L, 3L, 4.0));
+		edges.add(new Edge<>(2L, 4L, 5.0));
+		edges.add(new Edge<>(3L, 5L, 6.0));
+		edges.add(new Edge<>(5L, 6L, 7.0));
+		edges.add(new Edge<>(5L, 7L, 8.0));
+		edges.add(new Edge<>(6L, 7L, 9.0));
+		edges.add(new Edge<>(7L, 12L, 10.0));
+		edges.add(new Edge<>(8L, 9L, 11.0));
+		edges.add(new Edge<>(8L, 10L, 12.0));
+		edges.add(new Edge<>(8L, 11L, 13.0));
+		edges.add(new Edge<>(9L, 10L, 14.0));
+		edges.add(new Edge<>(9L, 11L, 15.0));
+		edges.add(new Edge<>(10L, 11L, 16.0));
+		edges.add(new Edge<>(10L, 12L, 17.0));
+		edges.add(new Edge<>(11L, 12L, 18.0));
 
 		return env.fromCollection(edges);
 	}
 
 	public static DataSet<Edge<Long, Double>> getSimpleEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 2.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 3.0));
-		edges.add(new Edge<Long, Double>(1L, 5L, 4.0));
-		edges.add(new Edge<Long, Double>(2L, 6L, 5.0));
-		edges.add(new Edge<Long, Double>(6L, 7L, 6.0));
-		edges.add(new Edge<Long, Double>(6L, 8L, 7.0));
-		edges.add(new Edge<Long, Double>(7L, 8L, 8.0));
+		List<Edge<Long, Double>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 2L, 1.0));
+		edges.add(new Edge<>(1L, 3L, 2.0));
+		edges.add(new Edge<>(1L, 4L, 3.0));
+		edges.add(new Edge<>(1L, 5L, 4.0));
+		edges.add(new Edge<>(2L, 6L, 5.0));
+		edges.add(new Edge<>(6L, 7L, 6.0));
+		edges.add(new Edge<>(6L, 8L, 7.0));
+		edges.add(new Edge<>(7L, 8L, 8.0));
 
 		return env.fromCollection(edges);
 	}
@@ -84,11 +84,11 @@ public class CommunityDetectionData {
 	private CommunityDetectionData() {}
 
 	public static DataSet<Edge<Long, Double>> getTieEdgeDataSet(ExecutionEnvironment env) {
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 5L, 1.0));
+		List<Edge<Long, Double>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 2L, 1.0));
+		edges.add(new Edge<>(1L, 3L, 1.0));
+		edges.add(new Edge<>(1L, 4L, 1.0));
+		edges.add(new Edge<>(1L, 5L, 1.0));
 
 		return env.fromCollection(edges);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/EuclideanGraphData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/EuclideanGraphData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/EuclideanGraphData.java
index 1ac9272..5698241 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/EuclideanGraphData.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/EuclideanGraphData.java
@@ -40,10 +40,10 @@ public class EuclideanGraphData {
 
 	public static DataSet<Vertex<Long, EuclideanGraphWeighing.Point>> getDefaultVertexDataSet(ExecutionEnvironment env) {
 
-		List<Vertex<Long, EuclideanGraphWeighing.Point>> vertices = new ArrayList<Vertex<Long, EuclideanGraphWeighing.Point>>();
+		List<Vertex<Long, EuclideanGraphWeighing.Point>> vertices = new ArrayList<>();
 		for (int i = 1; i <= NUM_VERTICES; i++) {
-			vertices.add(new Vertex<Long, EuclideanGraphWeighing.Point>(new Long(i),
-					new EuclideanGraphWeighing.Point(new Double(i), new Double(i))));
+			vertices.add(new Vertex<>(new Long(i),
+				new EuclideanGraphWeighing.Point(new Double(i), new Double(i))));
 		}
 
 		return env.fromCollection(vertices);
@@ -55,23 +55,23 @@ public class EuclideanGraphData {
 
 	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 0.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 0.0));
-		edges.add(new Edge<Long, Double>(2L, 3L, 0.0));
-		edges.add(new Edge<Long, Double>(2L, 4L, 0.0));
-		edges.add(new Edge<Long, Double>(2L, 5L, 0.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 0.0));
-		edges.add(new Edge<Long, Double>(4L, 5L, 0.0));
-		edges.add(new Edge<Long, Double>(4L, 6L, 0.0));
-		edges.add(new Edge<Long, Double>(5L, 7L, 0.0));
-		edges.add(new Edge<Long, Double>(5L, 9L, 0.0));
-		edges.add(new Edge<Long, Double>(6L, 7L, 0.0));
-		edges.add(new Edge<Long, Double>(6L, 8L, 0.0));
-		edges.add(new Edge<Long, Double>(6L, 8L, 0.0));
-		edges.add(new Edge<Long, Double>(7L, 8L, 0.0));
-		edges.add(new Edge<Long, Double>(7L, 9L, 0.0));
-		edges.add(new Edge<Long, Double>(8L, 9L, 0.0));
+		List<Edge<Long, Double>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 2L, 0.0));
+		edges.add(new Edge<>(1L, 4L, 0.0));
+		edges.add(new Edge<>(2L, 3L, 0.0));
+		edges.add(new Edge<>(2L, 4L, 0.0));
+		edges.add(new Edge<>(2L, 5L, 0.0));
+		edges.add(new Edge<>(3L, 5L, 0.0));
+		edges.add(new Edge<>(4L, 5L, 0.0));
+		edges.add(new Edge<>(4L, 6L, 0.0));
+		edges.add(new Edge<>(5L, 7L, 0.0));
+		edges.add(new Edge<>(5L, 9L, 0.0));
+		edges.add(new Edge<>(6L, 7L, 0.0));
+		edges.add(new Edge<>(6L, 8L, 0.0));
+		edges.add(new Edge<>(6L, 8L, 0.0));
+		edges.add(new Edge<>(7L, 8L, 0.0));
+		edges.add(new Edge<>(7L, 9L, 0.0));
+		edges.add(new Edge<>(8L, 9L, 0.0));
 
 		return env.fromCollection(edges);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/IncrementalSSSPData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/IncrementalSSSPData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/IncrementalSSSPData.java
index eaa2cde..d86ba96 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/IncrementalSSSPData.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/IncrementalSSSPData.java
@@ -38,12 +38,12 @@ public class IncrementalSSSPData {
 
 	public static DataSet<Vertex<Long, Double>> getDefaultVertexDataSet(ExecutionEnvironment env) {
 
-		List<Vertex<Long, Double>> vertices = new ArrayList<Vertex<Long, Double>>();
-		vertices.add(new Vertex<Long, Double>(1L, 6.0));
-		vertices.add(new Vertex<Long, Double>(2L, 2.0));
-		vertices.add(new Vertex<Long, Double>(3L, 3.0));
-		vertices.add(new Vertex<Long, Double>(4L, 1.0));
-		vertices.add(new Vertex<Long, Double>(5L, 0.0));
+		List<Vertex<Long, Double>> vertices = new ArrayList<>();
+		vertices.add(new Vertex<>(1L, 6.0));
+		vertices.add(new Vertex<>(2L, 2.0));
+		vertices.add(new Vertex<>(3L, 3.0));
+		vertices.add(new Vertex<>(4L, 1.0));
+		vertices.add(new Vertex<>(5L, 0.0));
 
 		return env.fromCollection(vertices);
 	}
@@ -53,13 +53,13 @@ public class IncrementalSSSPData {
 
 	public static final DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 3L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 4L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 5L, 2.0));
-		edges.add(new Edge<Long, Double>(3L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 5.0));
-		edges.add(new Edge<Long, Double>(4L, 5L, 1.0));
+		List<Edge<Long, Double>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 3L, 3.0));
+		edges.add(new Edge<>(2L, 4L, 3.0));
+		edges.add(new Edge<>(2L, 5L, 2.0));
+		edges.add(new Edge<>(3L, 2L, 1.0));
+		edges.add(new Edge<>(3L, 5L, 5.0));
+		edges.add(new Edge<>(4L, 5L, 1.0));
 
 		return env.fromCollection(edges);
 	}
@@ -68,11 +68,11 @@ public class IncrementalSSSPData {
 
 	public static final DataSet<Edge<Long, Double>> getDefaultEdgesInSSSP(ExecutionEnvironment env) {
 
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 3L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 5L, 2.0));
-		edges.add(new Edge<Long, Double>(3L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(4L, 5L, 1.0));
+		List<Edge<Long, Double>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 3L, 3.0));
+		edges.add(new Edge<>(2L, 5L, 2.0));
+		edges.add(new Edge<>(3L, 2L, 1.0));
+		edges.add(new Edge<>(4L, 5L, 1.0));
 
 		return env.fromCollection(edges);
 	}
@@ -85,7 +85,7 @@ public class IncrementalSSSPData {
 
 	public static final Edge<Long, Double> getDefaultEdgeToBeRemoved() {
 
-		return new Edge<Long, Double>(2L, 5L, 2.0);
+		return new Edge<>(2L, 5L, 2.0);
 	}
 
 	public static final String RESULTED_VERTICES = "1," + Double.MAX_VALUE + "\n" + "2," + Double.MAX_VALUE + "\n"

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/LabelPropagationData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/LabelPropagationData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/LabelPropagationData.java
index 343ff70..21b3329 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/LabelPropagationData.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/LabelPropagationData.java
@@ -55,58 +55,58 @@ public class LabelPropagationData {
 
 	public static final DataSet<Vertex<Long, Long>> getDefaultVertexSet(ExecutionEnvironment env) {
 
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(1L, 10L));
-		vertices.add(new Vertex<Long, Long>(2L, 10L));
-		vertices.add(new Vertex<Long, Long>(3L, 30L));
-		vertices.add(new Vertex<Long, Long>(4L, 40L));
-		vertices.add(new Vertex<Long, Long>(5L, 40L));
-		vertices.add(new Vertex<Long, Long>(6L, 40L));
-		vertices.add(new Vertex<Long, Long>(7L, 40L));
+		List<Vertex<Long, Long>> vertices = new ArrayList<>();
+		vertices.add(new Vertex<>(1L, 10L));
+		vertices.add(new Vertex<>(2L, 10L));
+		vertices.add(new Vertex<>(3L, 30L));
+		vertices.add(new Vertex<>(4L, 40L));
+		vertices.add(new Vertex<>(5L, 40L));
+		vertices.add(new Vertex<>(6L, 40L));
+		vertices.add(new Vertex<>(7L, 40L));
 
 		return env.fromCollection(vertices);
 	}
 
 	public static final DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
-		edges.add(new Edge<Long, NullValue>(1L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(4L, 7L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(5L, 7L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(6L, 7L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(7L, 3L, NullValue.getInstance()));
+		List<Edge<Long, NullValue>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<>(2L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<>(4L, 7L, NullValue.getInstance()));
+		edges.add(new Edge<>(5L, 7L, NullValue.getInstance()));
+		edges.add(new Edge<>(6L, 7L, NullValue.getInstance()));
+		edges.add(new Edge<>(7L, 3L, NullValue.getInstance()));
 
 		return env.fromCollection(edges);
 	}
 
 	public static final DataSet<Vertex<Long, Long>> getTieVertexSet(ExecutionEnvironment env) {
 
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(1L, 10L));
-		vertices.add(new Vertex<Long, Long>(2L, 10L));
-		vertices.add(new Vertex<Long, Long>(3L, 10L));
-		vertices.add(new Vertex<Long, Long>(4L, 10L));
-		vertices.add(new Vertex<Long, Long>(5L, 0L));
-		vertices.add(new Vertex<Long, Long>(6L, 20L));
-		vertices.add(new Vertex<Long, Long>(7L, 20L));
-		vertices.add(new Vertex<Long, Long>(8L, 20L));
-		vertices.add(new Vertex<Long, Long>(9L, 20L));
+		List<Vertex<Long, Long>> vertices = new ArrayList<>();
+		vertices.add(new Vertex<>(1L, 10L));
+		vertices.add(new Vertex<>(2L, 10L));
+		vertices.add(new Vertex<>(3L, 10L));
+		vertices.add(new Vertex<>(4L, 10L));
+		vertices.add(new Vertex<>(5L, 0L));
+		vertices.add(new Vertex<>(6L, 20L));
+		vertices.add(new Vertex<>(7L, 20L));
+		vertices.add(new Vertex<>(8L, 20L));
+		vertices.add(new Vertex<>(9L, 20L));
 
 		return env.fromCollection(vertices);
 	}
 
 	public static final DataSet<Edge<Long, NullValue>> getTieEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
-		edges.add(new Edge<Long, NullValue>(1L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(4L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(5L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(6L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(7L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(8L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(9L, 5L, NullValue.getInstance()));
+		List<Edge<Long, NullValue>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(2L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(4L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(5L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(6L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(7L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(8L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(9L, 5L, NullValue.getInstance()));
 
 		return env.fromCollection(edges);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/MusicProfilesData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/MusicProfilesData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/MusicProfilesData.java
index df139f0..21e8730 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/MusicProfilesData.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/MusicProfilesData.java
@@ -32,53 +32,53 @@ import java.util.List;
 public class MusicProfilesData {
 
 	public static DataSet<Tuple3<String, String, Integer>> getUserSongTriplets(ExecutionEnvironment env) {
-		List<Tuple3<String, String, Integer>> triplets = new ArrayList<Tuple3<String, String, Integer>>();
-
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_1", 100));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_2", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_3", 20));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_4", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_5", 1));
-
-		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_6", 40));
-		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_7", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_8", 3));
-
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_1", 100));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_2", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_3", 20));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_8", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_9", 1));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_10", 8));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_11", 90));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_12", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_13", 34));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_14", 17));
-
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_1", 100));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_6", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_8", 20));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_12", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_13", 1));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_15", 1));
-
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_3", 300));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_4", 4));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_5", 5));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_8", 8));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_9", 9));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_10", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_12", 12));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_13", 13));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_15", 15));
-
-		triplets.add(new Tuple3<String, String, Integer>("user_6", "song_6", 30));
+		List<Tuple3<String, String, Integer>> triplets = new ArrayList<>();
+
+		triplets.add(new Tuple3<>("user_1", "song_1", 100));
+		triplets.add(new Tuple3<>("user_1", "song_2", 10));
+		triplets.add(new Tuple3<>("user_1", "song_3", 20));
+		triplets.add(new Tuple3<>("user_1", "song_4", 30));
+		triplets.add(new Tuple3<>("user_1", "song_5", 1));
+
+		triplets.add(new Tuple3<>("user_2", "song_6", 40));
+		triplets.add(new Tuple3<>("user_2", "song_7", 10));
+		triplets.add(new Tuple3<>("user_2", "song_8", 3));
+
+		triplets.add(new Tuple3<>("user_3", "song_1", 100));
+		triplets.add(new Tuple3<>("user_3", "song_2", 10));
+		triplets.add(new Tuple3<>("user_3", "song_3", 20));
+		triplets.add(new Tuple3<>("user_3", "song_8", 30));
+		triplets.add(new Tuple3<>("user_3", "song_9", 1));
+		triplets.add(new Tuple3<>("user_3", "song_10", 8));
+		triplets.add(new Tuple3<>("user_3", "song_11", 90));
+		triplets.add(new Tuple3<>("user_3", "song_12", 30));
+		triplets.add(new Tuple3<>("user_3", "song_13", 34));
+		triplets.add(new Tuple3<>("user_3", "song_14", 17));
+
+		triplets.add(new Tuple3<>("user_4", "song_1", 100));
+		triplets.add(new Tuple3<>("user_4", "song_6", 10));
+		triplets.add(new Tuple3<>("user_4", "song_8", 20));
+		triplets.add(new Tuple3<>("user_4", "song_12", 30));
+		triplets.add(new Tuple3<>("user_4", "song_13", 1));
+		triplets.add(new Tuple3<>("user_4", "song_15", 1));
+
+		triplets.add(new Tuple3<>("user_5", "song_3", 300));
+		triplets.add(new Tuple3<>("user_5", "song_4", 4));
+		triplets.add(new Tuple3<>("user_5", "song_5", 5));
+		triplets.add(new Tuple3<>("user_5", "song_8", 8));
+		triplets.add(new Tuple3<>("user_5", "song_9", 9));
+		triplets.add(new Tuple3<>("user_5", "song_10", 10));
+		triplets.add(new Tuple3<>("user_5", "song_12", 12));
+		triplets.add(new Tuple3<>("user_5", "song_13", 13));
+		triplets.add(new Tuple3<>("user_5", "song_15", 15));
+
+		triplets.add(new Tuple3<>("user_6", "song_6", 30));
 
 		return env.fromCollection(triplets);
 	}
 
 	public static DataSet<String> getMismatches(ExecutionEnvironment env) {
-		List<String> errors = new ArrayList<String>();
+		List<String> errors = new ArrayList<>();
 		errors.add("ERROR: <song_8 track_8> Sever");
 		errors.add("ERROR: <song_15 track_15> Black Trees");
 		return env.fromCollection(errors);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/PageRankData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/PageRankData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/PageRankData.java
index 1c3ebb0..0eb6c1d 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/PageRankData.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/PageRankData.java
@@ -52,16 +52,16 @@ public class PageRankData {
 
 	public static final DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(2L, 1L, 1.0));
-		edges.add(new Edge<Long, Double>(5L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(5L, 4L, 1.0));
-		edges.add(new Edge<Long, Double>(4L, 3L, 1.0));
-		edges.add(new Edge<Long, Double>(4L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 1.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 1.0));
+		List<Edge<Long, Double>> edges = new ArrayList<>();
+		edges.add(new Edge<>(2L, 1L, 1.0));
+		edges.add(new Edge<>(5L, 2L, 1.0));
+		edges.add(new Edge<>(5L, 4L, 1.0));
+		edges.add(new Edge<>(4L, 3L, 1.0));
+		edges.add(new Edge<>(4L, 2L, 1.0));
+		edges.add(new Edge<>(1L, 4L, 1.0));
+		edges.add(new Edge<>(1L, 2L, 1.0));
+		edges.add(new Edge<>(1L, 3L, 1.0));
+		edges.add(new Edge<>(3L, 5L, 1.0));
 
 		return env.fromCollection(edges);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/TriangleCountData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/TriangleCountData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/TriangleCountData.java
index 7f4f926..4346bd2 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/TriangleCountData.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/TriangleCountData.java
@@ -45,16 +45,16 @@ public class TriangleCountData {
 
 	public static DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 
-		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
-		edges.add(new Edge<Long, NullValue>(1L, 2L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(1L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 6L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(3L, 4L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(3L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(3L, 6L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(4L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(6L, 7L, NullValue.getInstance()));
+		List<Edge<Long, NullValue>> edges = new ArrayList<>();
+		edges.add(new Edge<>(1L, 2L, NullValue.getInstance()));
+		edges.add(new Edge<>(1L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<>(2L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<>(2L, 6L, NullValue.getInstance()));
+		edges.add(new Edge<>(3L, 4L, NullValue.getInstance()));
+		edges.add(new Edge<>(3L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(3L, 6L, NullValue.getInstance()));
+		edges.add(new Edge<>(4L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<>(6L, 7L, NullValue.getInstance()));
 
 		return env.fromCollection(edges);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/CommunityDetectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/CommunityDetectionITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/CommunityDetectionITCase.java
index 8b8c44e..2a80aea 100644
--- a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/CommunityDetectionITCase.java
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/CommunityDetectionITCase.java
@@ -53,7 +53,7 @@ public class CommunityDetectionITCase extends MultipleProgramsTestBase {
 		Graph<Long, Long, Double> inputGraph = Graph.fromDataSet(
 			CommunityDetectionData.getSimpleEdgeDataSet(env), new InitLabels(), env);
 
-		List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<Long>(1, CommunityDetectionData.DELTA))
+		List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<>(1, CommunityDetectionData.DELTA))
 			.getVertices().collect();
 
 		expected = CommunityDetectionData.COMMUNITIES_SINGLE_ITERATION;
@@ -69,7 +69,7 @@ public class CommunityDetectionITCase extends MultipleProgramsTestBase {
 		Graph<Long, Long, Double> inputGraph = Graph.fromDataSet(
 			CommunityDetectionData.getTieEdgeDataSet(env), new InitLabels(), env);
 
-		List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<Long>(1, CommunityDetectionData.DELTA))
+		List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<>(1, CommunityDetectionData.DELTA))
 			.getVertices().collect();
 		expected = CommunityDetectionData.COMMUNITIES_WITH_TIE;
 		compareResultAsTuples(result, expected);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/LabelPropagationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/LabelPropagationITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/LabelPropagationITCase.java
index d61c71d..d6dd3a0 100644
--- a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/LabelPropagationITCase.java
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/LabelPropagationITCase.java
@@ -55,7 +55,7 @@ public class LabelPropagationITCase extends MultipleProgramsTestBase {
 			LabelPropagationData.getDefaultEdgeDataSet(env), env);
 
 		List<Vertex<Long, Long>> result = inputGraph
-			.run(new LabelPropagation<Long, Long, NullValue>(1))
+			.run(new LabelPropagation<>(1))
 			.collect();
 
 		expectedResult = LabelPropagationData.LABELS_AFTER_1_ITERATION;
@@ -74,7 +74,7 @@ public class LabelPropagationITCase extends MultipleProgramsTestBase {
 			LabelPropagationData.getTieEdgeDataSet(env), env);
 
 		List<Vertex<Long, Long>> result = inputGraph
-			.run(new LabelPropagation<Long, Long, NullValue>(1))
+			.run(new LabelPropagation<>(1))
 			.collect();
 
 		expectedResult = LabelPropagationData.LABELS_WITH_TIE;

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/SummarizationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/SummarizationITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/SummarizationITCase.java
index baad6d0..f66caf4 100644
--- a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/SummarizationITCase.java
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/SummarizationITCase.java
@@ -72,7 +72,7 @@ public class SummarizationITCase extends MultipleProgramsTestBase {
 		List<Edge<Long, EdgeValue<String>>> summarizedEdges = new ArrayList<>();
 
 		Graph<Long, Summarization.VertexValue<String>, EdgeValue<String>> output =
-				input.run(new Summarization<Long, String, String>());
+				input.run(new Summarization<>());
 
 		output.getVertices().output(new LocalCollectionOutputFormat<>(summarizedVertices));
 		output.getEdges().output(new LocalCollectionOutputFormat<>(summarizedEdges));
@@ -90,13 +90,13 @@ public class SummarizationITCase extends MultipleProgramsTestBase {
 				SummarizationData.getVertices(env),
 				SummarizationData.getEdges(env),
 				env)
-			.run(new TranslateEdgeValues<Long, String, String, NullValue>(new ToNullValue<String>()));
+			.run(new TranslateEdgeValues<>(new ToNullValue<>()));
 
 		List<Vertex<Long, Summarization.VertexValue<String>>> summarizedVertices = new ArrayList<>();
 		List<Edge<Long, EdgeValue<NullValue>>> summarizedEdges = new ArrayList<>();
 
 		Graph<Long, Summarization.VertexValue<String>, EdgeValue<NullValue>> output =
-				input.run(new Summarization<Long, String, NullValue>());
+				input.run(new Summarization<>());
 
 		output.getVertices().output(new LocalCollectionOutputFormat<>(summarizedVertices));
 		output.getEdges().output(new LocalCollectionOutputFormat<>(summarizedEdges));
@@ -115,14 +115,14 @@ public class SummarizationITCase extends MultipleProgramsTestBase {
 				SummarizationData.getVertices(env),
 				SummarizationData.getEdges(env),
 				env)
-			.run(new TranslateVertexValues<Long, String, Long, String>(new StringToLong()))
-			.run(new TranslateEdgeValues<Long, Long, String, Long>(new StringToLong()));
+			.run(new TranslateVertexValues<>(new StringToLong()))
+			.run(new TranslateEdgeValues<>(new StringToLong()));
 
 		List<Vertex<Long, Summarization.VertexValue<Long>>> summarizedVertices = new ArrayList<>();
 		List<Edge<Long, EdgeValue<Long>>> summarizedEdges = new ArrayList<>();
 
 		Graph<Long, Summarization.VertexValue<Long>, EdgeValue<Long>> output =
-			input.run(new Summarization<Long, Long, Long>());
+			input.run(new Summarization<>());
 
 		output.getVertices().output(new LocalCollectionOutputFormat<>(summarizedVertices));
 		output.getEdges().output(new LocalCollectionOutputFormat<>(summarizedEdges));

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/TriangleEnumeratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/TriangleEnumeratorITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/TriangleEnumeratorITCase.java
index 2e1cc7c..d999bae 100644
--- a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/TriangleEnumeratorITCase.java
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/TriangleEnumeratorITCase.java
@@ -50,7 +50,7 @@ public class TriangleEnumeratorITCase extends MultipleProgramsTestBase {
 		Graph<Long, NullValue, NullValue> graph = Graph.fromDataSet(TriangleCountData.getDefaultEdgeDataSet(env),
 				env);
 
-		List<Tuple3<Long, Long, Long>> actualOutput = graph.run(new TriangleEnumerator<Long, NullValue, NullValue>()).collect();
+		List<Tuple3<Long, Long, Long>> actualOutput = graph.run(new TriangleEnumerator<>()).collect();
 		List<Tuple3<Long, Long, Long>> expectedResult = TriangleCountData.getListOfTriangles();
 
 		Assert.assertEquals(expectedResult.size(), actualOutput.size());

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
index 066f2c4..e44976e 100644
--- a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
@@ -67,10 +67,10 @@ public class GatherSumApplyITCase extends MultipleProgramsTestBase {
 
 		Graph<Long, Long, NullValue> inputGraph = Graph.fromDataSet(
 			ConnectedComponentsDefaultData.getDefaultEdgeDataSet(env),
-			new IdentityMapper<Long>(), env);
+			new IdentityMapper<>(), env);
 
 		List<Vertex<Long, Long>> result = inputGraph.run(
-			new GSAConnectedComponents<Long, Long, NullValue>(16)).collect();
+			new GSAConnectedComponents<>(16)).collect();
 
 		compareResultAsTuples(result, expectedResultCC);
 	}
@@ -85,10 +85,10 @@ public class GatherSumApplyITCase extends MultipleProgramsTestBase {
 			new LongToLongValue());
 
 		Graph<LongValue, LongValue, NullValue> inputGraph = Graph.fromDataSet(
-			edges, new IdentityMapper<LongValue>(), env);
+			edges, new IdentityMapper<>(), env);
 
 		List<Vertex<LongValue, LongValue>> result = inputGraph.run(
-			new GSAConnectedComponents<LongValue, LongValue, NullValue>(16)).collect();
+			new GSAConnectedComponents<>(16)).collect();
 
 		compareResultAsTuples(result, expectedResultCC);
 	}
@@ -106,7 +106,7 @@ public class GatherSumApplyITCase extends MultipleProgramsTestBase {
 			new InitMapperSSSP(), env);
 
 		List<Vertex<Long, Double>> result = inputGraph.run(
-			new GSASingleSourceShortestPaths<Long, NullValue>(1L, 16)).collect();
+			new GSASingleSourceShortestPaths<>(1L, 16)).collect();
 
 		String expectedResult = "1,0.0\n" +
 			"2,12.0\n" +

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/PageRankITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/PageRankITCase.java
index 62ed39f..bd2c50d 100644
--- a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/PageRankITCase.java
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/PageRankITCase.java
@@ -52,7 +52,7 @@ public class PageRankITCase extends MultipleProgramsTestBase {
 		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
 			PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
 
-		List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<Long>(0.85, 3))
+		List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<>(0.85, 3))
 			.collect();
 
 		compareWithDelta(result, 0.01);
@@ -65,7 +65,7 @@ public class PageRankITCase extends MultipleProgramsTestBase {
 		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
 			PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
 
-		List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<Long>(0.85, 3))
+		List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<>(0.85, 3))
 			.collect();
 
 		compareWithDelta(result, 0.01);
@@ -78,7 +78,7 @@ public class PageRankITCase extends MultipleProgramsTestBase {
 		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
 			PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
 
-		List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<Long>(0.85, 3))
+		List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<>(0.85, 3))
 			.collect();
 
 		compareWithDelta(result, 0.01);
@@ -91,7 +91,7 @@ public class PageRankITCase extends MultipleProgramsTestBase {
 		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
 			PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
 
-		List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<Long>(0.85, 3))
+		List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<>(0.85, 3))
 			.collect();
 
 		compareWithDelta(result, 0.01);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
index 670cefb..446a2ba 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
@@ -177,7 +177,7 @@ public class Graph<K, VV, EV> {
 			DataSet<Edge<K, EV>> edges, ExecutionEnvironment context) {
 
 		DataSet<Vertex<K, NullValue>> vertices = edges
-			.flatMap(new EmitSrcAndTarget<K, EV>())
+			.flatMap(new EmitSrcAndTarget<>())
 				.name("Source and target IDs")
 			.distinct()
 				.name("IDs");
@@ -221,7 +221,7 @@ public class Graph<K, VV, EV> {
 				Vertex.class, keyType, valueType);
 
 		DataSet<Vertex<K, VV>> vertices = edges
-			.flatMap(new EmitSrcAndTargetAsTuple1<K, EV>())
+			.flatMap(new EmitSrcAndTargetAsTuple1<>())
 				.name("Source and target IDs")
 			.distinct()
 				.name("IDs")
@@ -269,11 +269,11 @@ public class Graph<K, VV, EV> {
 			DataSet<Tuple3<K, K, EV>> edges, ExecutionEnvironment context) {
 
 		DataSet<Vertex<K, VV>> vertexDataSet = vertices
-			.map(new Tuple2ToVertexMap<K, VV>())
+			.map(new Tuple2ToVertexMap<>())
 				.name("Type conversion");
 
 		DataSet<Edge<K, EV>> edgeDataSet = edges
-			.map(new Tuple3ToEdgeMap<K, EV>())
+			.map(new Tuple3ToEdgeMap<>())
 				.name("Type conversion");
 
 		return fromDataSet(vertexDataSet, edgeDataSet, context);
@@ -296,7 +296,7 @@ public class Graph<K, VV, EV> {
 			ExecutionEnvironment context) {
 
 		DataSet<Edge<K, EV>> edgeDataSet = edges
-			.map(new Tuple3ToEdgeMap<K, EV>())
+			.map(new Tuple3ToEdgeMap<>())
 				.name("Type conversion");
 
 		return fromDataSet(edgeDataSet, context);
@@ -322,7 +322,7 @@ public class Graph<K, VV, EV> {
 			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
 
 		DataSet<Edge<K, EV>> edgeDataSet = edges
-			.map(new Tuple3ToEdgeMap<K, EV>())
+			.map(new Tuple3ToEdgeMap<>())
 				.name("Type conversion");
 
 		return fromDataSet(edgeDataSet, vertexValueInitializer, context);
@@ -343,7 +343,7 @@ public class Graph<K, VV, EV> {
 			ExecutionEnvironment context) {
 
 		DataSet<Edge<K, NullValue>> edgeDataSet = edges
-			.map(new Tuple2ToEdgeMap<K>())
+			.map(new Tuple2ToEdgeMap<>())
 				.name("To Edge");
 
 		return fromDataSet(edgeDataSet, context);
@@ -368,7 +368,7 @@ public class Graph<K, VV, EV> {
 			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
 
 		DataSet<Edge<K, NullValue>> edgeDataSet = edges
-			.map(new Tuple2ToEdgeMap<K>())
+			.map(new Tuple2ToEdgeMap<>())
 				.name("To Edge");
 
 		return fromDataSet(edgeDataSet, vertexValueInitializer, context);
@@ -465,14 +465,14 @@ public class Graph<K, VV, EV> {
 	 * @return the vertex DataSet as Tuple2.
 	 */
 	public DataSet<Tuple2<K, VV>> getVerticesAsTuple2() {
-		return vertices.map(new VertexToTuple2Map<K, VV>());
+		return vertices.map(new VertexToTuple2Map<>());
 	}
 
 	/**
 	 * @return the edge DataSet as Tuple3.
 	 */
 	public DataSet<Tuple3<K, K, EV>> getEdgesAsTuple3() {
-		return edges.map(new EdgeToTuple3Map<K, EV>());
+		return edges.map(new EdgeToTuple3Map<>());
 	}
 
 	/**
@@ -483,10 +483,10 @@ public class Graph<K, VV, EV> {
 	public DataSet<Triplet<K, VV, EV>> getTriplets() {
 		return this.getVertices()
 			.join(this.getEdges()).where(0).equalTo(0)
-			.with(new ProjectEdgeWithSrcValue<K, VV, EV>())
+			.with(new ProjectEdgeWithSrcValue<>())
 				.name("Project edge with source value")
 			.join(this.getVertices()).where(1).equalTo(0)
-			.with(new ProjectEdgeWithVertexValues<K, VV, EV>())
+			.with(new ProjectEdgeWithVertexValues<>())
 				.name("Project edge with vertex values");
 	}
 
@@ -628,7 +628,7 @@ public class Graph<K, VV, EV> {
 	 * @throws Exception
 	 */
 	public <NEW> Graph<NEW, VV, EV> translateGraphIds(TranslateFunction<K, NEW> translator) throws Exception {
-		return run(new TranslateGraphIds<K, NEW, VV, EV>(translator));
+		return run(new TranslateGraphIds<>(translator));
 	}
 
 	/**
@@ -640,7 +640,7 @@ public class Graph<K, VV, EV> {
 	 * @throws Exception
 	 */
 	public <NEW> Graph<K, NEW, EV> translateVertexValues(TranslateFunction<VV, NEW> translator) throws Exception {
-		return run(new TranslateVertexValues<K, VV, NEW, EV>(translator));
+		return run(new TranslateVertexValues<>(translator));
 	}
 
 	/**
@@ -652,7 +652,7 @@ public class Graph<K, VV, EV> {
 	 * @throws Exception
 	 */
 	public <NEW> Graph<K, VV, NEW> translateEdgeValues(TranslateFunction<EV, NEW> translator) throws Exception {
-		return run(new TranslateEdgeValues<K, VV, EV, NEW>(translator));
+		return run(new TranslateEdgeValues<>(translator));
 	}
 
 	/**
@@ -676,7 +676,7 @@ public class Graph<K, VV, EV> {
 
 		DataSet<Vertex<K, VV>> resultedVertices = this.getVertices()
 				.coGroup(inputDataSet).where(0).equalTo(0)
-				.with(new ApplyCoGroupToVertexValues<K, VV, T>(vertexJoinFunction))
+				.with(new ApplyCoGroupToVertexValues<>(vertexJoinFunction))
 					.name("Join with vertices");
 		return new Graph<>(resultedVertices, this.edges, this.context);
 	}
@@ -730,7 +730,7 @@ public class Graph<K, VV, EV> {
 
 		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
 				.coGroup(inputDataSet).where(0, 1).equalTo(0, 1)
-				.with(new ApplyCoGroupToEdgeValues<K, EV, T>(edgeJoinFunction))
+				.with(new ApplyCoGroupToEdgeValues<>(edgeJoinFunction))
 					.name("Join with edges");
 		return new Graph<>(this.vertices, resultedEdges, this.context);
 	}
@@ -787,7 +787,7 @@ public class Graph<K, VV, EV> {
 
 		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
 				.coGroup(inputDataSet).where(0).equalTo(0)
-				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(edgeJoinFunction))
+				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<>(edgeJoinFunction))
 					.name("Join with edges on source");
 
 		return new Graph<>(this.vertices, resultedEdges, this.context);
@@ -850,7 +850,7 @@ public class Graph<K, VV, EV> {
 
 		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
 				.coGroup(inputDataSet).where(1).equalTo(0)
-				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(edgeJoinFunction))
+				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<>(edgeJoinFunction))
 					.name("Join with edges on target");
 
 		return new Graph<>(this.vertices, resultedEdges, this.context);
@@ -869,9 +869,9 @@ public class Graph<K, VV, EV> {
 		DataSet<Vertex<K, VV>> filteredVertices = this.vertices.filter(vertexFilter);
 
 		DataSet<Edge<K, EV>> remainingEdges = this.edges.join(filteredVertices)
-				.where(0).equalTo(0).with(new ProjectEdge<K, VV, EV>())
+				.where(0).equalTo(0).with(new ProjectEdge<>())
 				.join(filteredVertices).where(1).equalTo(0)
-				.with(new ProjectEdge<K, VV, EV>()).name("Subgraph");
+				.with(new ProjectEdge<>()).name("Subgraph");
 
 		DataSet<Edge<K, EV>> filteredEdges = remainingEdges.filter(edgeFilter);
 
@@ -890,9 +890,9 @@ public class Graph<K, VV, EV> {
 		DataSet<Vertex<K, VV>> filteredVertices = this.vertices.filter(vertexFilter);
 
 		DataSet<Edge<K, EV>> remainingEdges = this.edges.join(filteredVertices)
-				.where(0).equalTo(0).with(new ProjectEdge<K, VV, EV>())
+				.where(0).equalTo(0).with(new ProjectEdge<>())
 				.join(filteredVertices).where(1).equalTo(0)
-				.with(new ProjectEdge<K, VV, EV>()).name("Filter on vertices");
+				.with(new ProjectEdge<>()).name("Filter on vertices");
 
 		return new Graph<>(filteredVertices, remainingEdges, this.context);
 	}
@@ -925,7 +925,7 @@ public class Graph<K, VV, EV> {
 	 */
 	public DataSet<Tuple2<K, LongValue>> outDegrees() {
 
-		return vertices.coGroup(edges).where(0).equalTo(0).with(new CountNeighborsCoGroup<K, VV, EV>())
+		return vertices.coGroup(edges).where(0).equalTo(0).with(new CountNeighborsCoGroup<>())
 			.name("Out-degree");
 	}
 
@@ -962,7 +962,7 @@ public class Graph<K, VV, EV> {
 	 */
 	public DataSet<Tuple2<K, LongValue>> inDegrees() {
 
-		return vertices.coGroup(edges).where(0).equalTo(1).with(new CountNeighborsCoGroup<K, VV, EV>())
+		return vertices.coGroup(edges).where(0).equalTo(1).with(new CountNeighborsCoGroup<>())
 			.name("In-degree");
 	}
 
@@ -985,7 +985,7 @@ public class Graph<K, VV, EV> {
 	public Graph<K, VV, EV> getUndirected() {
 
 		DataSet<Edge<K, EV>> undirectedEdges = edges.
-			flatMap(new RegularAndReversedEdgesMap<K, EV>()).name("To undirected graph");
+			flatMap(new RegularAndReversedEdgesMap<>()).name("To undirected graph");
 		return new Graph<>(vertices, undirectedEdges, this.context);
 	}
 
@@ -1014,7 +1014,7 @@ public class Graph<K, VV, EV> {
 			return vertices.coGroup(edges).where(0).equalTo(0)
 					.with(new ApplyCoGroupFunction<>(edgesFunction)).name("GroupReduce on out-edges");
 		case ALL:
-			return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<K, EV>())
+			return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<>())
 						.name("Emit edge"))
 					.where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges<>(edgesFunction))
 						.name("GroupReduce on in- and out-edges");
@@ -1051,7 +1051,7 @@ public class Graph<K, VV, EV> {
 						.with(new ApplyCoGroupFunction<>(edgesFunction))
 							.name("GroupReduce on out-edges").returns(typeInfo);
 			case ALL:
-				return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<K, EV>())
+				return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<>())
 							.name("Emit edge"))
 						.where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges<>(edgesFunction))
 							.name("GroupReduce on in- and out-edges").returns(typeInfo);
@@ -1105,17 +1105,17 @@ public class Graph<K, VV, EV> {
 
 		switch (direction) {
 			case IN:
-				return edges.map(new ProjectVertexIdMap<K, EV>(1)).name("Vertex ID")
+				return edges.map(new ProjectVertexIdMap<>(1)).name("Vertex ID")
 						.withForwardedFields("f1->f0")
 						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<>(edgesFunction))
 							.name("GroupReduce on in-edges").returns(typeInfo);
 			case OUT:
-				return edges.map(new ProjectVertexIdMap<K, EV>(0)).name("Vertex ID")
+				return edges.map(new ProjectVertexIdMap<>(0)).name("Vertex ID")
 						.withForwardedFields("f0")
 						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<>(edgesFunction))
 							.name("GroupReduce on out-edges").returns(typeInfo);
 			case ALL:
-				return edges.flatMap(new EmitOneEdgePerNode<K, EV>()).name("Emit edge")
+				return edges.flatMap(new EmitOneEdgePerNode<>()).name("Emit edge")
 						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<>(edgesFunction))
 							.name("GroupReduce on in- and out-edges").returns(typeInfo);
 			default:
@@ -1306,7 +1306,7 @@ public class Graph<K, VV, EV> {
 	 * @throws UnsupportedOperationException
 	 */
 	public Graph<K, VV, EV> reverse() throws UnsupportedOperationException {
-		DataSet<Edge<K, EV>> reversedEdges = edges.map(new ReverseEdgesMap<K, EV>()).name("Reverse edges");
+		DataSet<Edge<K, EV>> reversedEdges = edges.map(new ReverseEdgesMap<>()).name("Reverse edges");
 		return new Graph<>(vertices, reversedEdges, this.context);
 	}
 
@@ -1328,7 +1328,7 @@ public class Graph<K, VV, EV> {
 	 * @return The IDs of the vertices as DataSet
 	 */
 	public DataSet<K> getVertexIds() {
-		return vertices.map(new ExtractVertexIDMapper<K, VV>()).name("Vertex IDs");
+		return vertices.map(new ExtractVertexIDMapper<>()).name("Vertex IDs");
 	}
 
 	private static final class ExtractVertexIDMapper<K, VV>
@@ -1343,7 +1343,7 @@ public class Graph<K, VV, EV> {
 	 * @return The IDs of the edges as DataSet
 	 */
 	public DataSet<Tuple2<K, K>> getEdgeIds() {
-		return edges.map(new ExtractEdgeIDsMapper<K, EV>()).name("Edge IDs");
+		return edges.map(new ExtractEdgeIDsMapper<>()).name("Edge IDs");
 	}
 
 	@ForwardedFields("f0; f1")
@@ -1379,7 +1379,7 @@ public class Graph<K, VV, EV> {
 	public Graph<K, VV, EV> addVertices(List<Vertex<K, VV>> verticesToAdd) {
 		// Add the vertices
 		DataSet<Vertex<K, VV>> newVertices = this.vertices.coGroup(this.context.fromCollection(verticesToAdd))
-				.where(0).equalTo(0).with(new VerticesUnionCoGroup<K, VV>()).name("Add vertices");
+				.where(0).equalTo(0).with(new VerticesUnionCoGroup<>()).name("Add vertices");
 
 		return new Graph<>(newVertices, this.edges, this.context);
 	}
@@ -1433,9 +1433,9 @@ public class Graph<K, VV, EV> {
 
 		DataSet<Edge<K, EV>> validNewEdges = this.getVertices().join(newEdgesDataSet)
 				.where(0).equalTo(0)
-				.with(new JoinVerticesWithEdgesOnSrc<K, VV, EV>()).name("Join with source")
+				.with(new JoinVerticesWithEdgesOnSrc<>()).name("Join with source")
 				.join(this.getVertices()).where(1).equalTo(0)
-				.with(new JoinWithVerticesOnTrg<K, VV, EV>()).name("Join with target");
+				.with(new JoinWithVerticesOnTrg<>()).name("Join with target");
 
 		return Graph.fromDataSet(this.vertices, this.edges.union(validNewEdges), this.context);
 	}
@@ -1496,14 +1496,14 @@ public class Graph<K, VV, EV> {
 	private Graph<K, VV, EV> removeVertices(DataSet<Vertex<K, VV>> verticesToBeRemoved) {
 
 		DataSet<Vertex<K, VV>> newVertices = getVertices().coGroup(verticesToBeRemoved).where(0).equalTo(0)
-				.with(new VerticesRemovalCoGroup<K, VV>()).name("Remove vertices");
+				.with(new VerticesRemovalCoGroup<>()).name("Remove vertices");
 
 		DataSet <Edge< K, EV>> newEdges = newVertices.join(getEdges()).where(0).equalTo(0)
 				// if the edge source was removed, the edge will also be removed
-				.with(new ProjectEdgeToBeRemoved<K, VV, EV>()).name("Edges to be removed")
+				.with(new ProjectEdgeToBeRemoved<>()).name("Edges to be removed")
 				// if the edge target was removed, the edge will also be removed
 				.join(newVertices).where(1).equalTo(0)
-				.with(new ProjectEdge<K, VV, EV>()).name("Remove edges");
+				.with(new ProjectEdge<>()).name("Remove edges");
 
 		return new Graph<>(newVertices, newEdges, context);
 	}
@@ -1571,7 +1571,7 @@ public class Graph<K, VV, EV> {
 	public Graph<K, VV, EV> removeEdges(List<Edge<K, EV>> edgesToBeRemoved) {
 
 		DataSet<Edge<K, EV>> newEdges = getEdges().coGroup(this.context.fromCollection(edgesToBeRemoved))
-				.where(0, 1).equalTo(0, 1).with(new EdgeRemovalCoGroup<K, EV>()).name("Remove edges");
+				.where(0, 1).equalTo(0, 1).with(new EdgeRemovalCoGroup<>()).name("Remove edges");
 
 		return new Graph<>(this.vertices, newEdges, context);
 	}
@@ -1689,7 +1689,7 @@ public class Graph<K, VV, EV> {
 				.coGroup(edges)
 				.where(0, 1, 2)
 				.equalTo(0, 1, 2)
-				.with(new MatchingEdgeReducer<K, EV>())
+				.with(new MatchingEdgeReducer<>())
 					.name("Intersect edges");
 	}
 
@@ -1912,9 +1912,9 @@ public class Graph<K, VV, EV> {
 		case ALL:
 			// create <edge-sourceOrTargetVertex> pairs
 			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-					.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>()).name("Forward and reverse edges")
+					.flatMap(new EmitOneEdgeWithNeighborPerNode<>()).name("Forward and reverse edges")
 					.join(this.vertices).where(1).equalTo(0)
-					.with(new ProjectEdgeWithNeighbor<K, VV, EV>()).name("Edge with vertex");
+					.with(new ProjectEdgeWithNeighbor<>()).name("Edge with vertex");
 
 			return vertices.coGroup(edgesWithNeighbors)
 					.where(0).equalTo(0)
@@ -1962,9 +1962,9 @@ public class Graph<K, VV, EV> {
 			case ALL:
 				// create <edge-sourceOrTargetVertex> pairs
 				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>()).name("Forward and reverse edges")
+						.flatMap(new EmitOneEdgeWithNeighborPerNode<>()).name("Forward and reverse edges")
 						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectEdgeWithNeighbor<K, VV, EV>()).name("Edge with vertex");
+						.with(new ProjectEdgeWithNeighbor<>()).name("Edge with vertex");
 
 				return vertices.coGroup(edgesWithNeighbors)
 						.where(0).equalTo(0)
@@ -1997,7 +1997,7 @@ public class Graph<K, VV, EV> {
 			// create <edge-sourceVertex> pairs
 			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
 					.join(this.vertices).where(0).equalTo(0)
-					.with(new ProjectVertexIdJoin<K, VV, EV>(1))
+					.with(new ProjectVertexIdJoin<>(1))
 					.withForwardedFieldsFirst("f1->f0").name("Edge with source vertex ID");
 			return edgesWithSources.groupBy(0).reduceGroup(
 				new ApplyNeighborGroupReduceFunction<>(neighborsFunction)).name("Neighbors function");
@@ -2005,16 +2005,16 @@ public class Graph<K, VV, EV> {
 			// create <edge-targetVertex> pairs
 			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
 					.join(this.vertices).where(1).equalTo(0)
-					.with(new ProjectVertexIdJoin<K, VV, EV>(0))
+					.with(new ProjectVertexIdJoin<>(0))
 					.withForwardedFieldsFirst("f0").name("Edge with target vertex ID");
 			return edgesWithTargets.groupBy(0).reduceGroup(
 				new ApplyNeighborGroupReduceFunction<>(neighborsFunction)).name("Neighbors function");
 		case ALL:
 			// create <edge-sourceOrTargetVertex> pairs
 			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-					.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>()).name("Forward and reverse edges")
+					.flatMap(new EmitOneEdgeWithNeighborPerNode<>()).name("Forward and reverse edges")
 					.join(this.vertices).where(1).equalTo(0)
-					.with(new ProjectEdgeWithNeighbor<K, VV, EV>()).name("Edge with vertex ID");
+					.with(new ProjectEdgeWithNeighbor<>()).name("Edge with vertex ID");
 
 			return edgesWithNeighbors.groupBy(0).reduceGroup(
 				new ApplyNeighborGroupReduceFunction<>(neighborsFunction)).name("Neighbors function");
@@ -2046,7 +2046,7 @@ public class Graph<K, VV, EV> {
 				// create <edge-sourceVertex> pairs
 				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
 						.join(this.vertices).where(0).equalTo(0)
-						.with(new ProjectVertexIdJoin<K, VV, EV>(1))
+						.with(new ProjectVertexIdJoin<>(1))
 						.withForwardedFieldsFirst("f1->f0").name("Edge with source vertex ID");
 				return edgesWithSources.groupBy(0).reduceGroup(
 					new ApplyNeighborGroupReduceFunction<>(neighborsFunction))
@@ -2055,7 +2055,7 @@ public class Graph<K, VV, EV> {
 				// create <edge-targetVertex> pairs
 				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
 						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectVertexIdJoin<K, VV, EV>(0))
+						.with(new ProjectVertexIdJoin<>(0))
 						.withForwardedFieldsFirst("f0").name("Edge with target vertex ID");
 				return edgesWithTargets.groupBy(0).reduceGroup(
 					new ApplyNeighborGroupReduceFunction<>(neighborsFunction))
@@ -2063,9 +2063,9 @@ public class Graph<K, VV, EV> {
 			case ALL:
 				// create <edge-sourceOrTargetVertex> pairs
 				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
+						.flatMap(new EmitOneEdgeWithNeighborPerNode<>())
 						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectEdgeWithNeighbor<K, VV, EV>()).name("Edge with vertex ID");
+						.with(new ProjectEdgeWithNeighbor<>()).name("Edge with vertex ID");
 
 				return edgesWithNeighbors.groupBy(0).reduceGroup(
 					new ApplyNeighborGroupReduceFunction<>(neighborsFunction))
@@ -2246,27 +2246,27 @@ public class Graph<K, VV, EV> {
 				// create <vertex-source value> pairs
 				final DataSet<Tuple2<K, VV>> verticesWithSourceNeighborValues = edges
 						.join(this.vertices).where(0).equalTo(0)
-						.with(new ProjectVertexWithNeighborValueJoin<K, VV, EV>(1))
+						.with(new ProjectVertexWithNeighborValueJoin<>(1))
 						.withForwardedFieldsFirst("f1->f0").name("Vertex with in-neighbor value");
-				return verticesWithSourceNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
-						reduceNeighborsFunction)).name("Neighbors function");
+				return verticesWithSourceNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<>(
+					reduceNeighborsFunction)).name("Neighbors function");
 			case OUT:
 				// create <vertex-target value> pairs
 				DataSet<Tuple2<K, VV>> verticesWithTargetNeighborValues = edges
 						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectVertexWithNeighborValueJoin<K, VV, EV>(0))
+						.with(new ProjectVertexWithNeighborValueJoin<>(0))
 						.withForwardedFieldsFirst("f0").name("Vertex with out-neighbor value");
-				return verticesWithTargetNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
-						reduceNeighborsFunction)).name("Neighbors function");
+				return verticesWithTargetNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<>(
+					reduceNeighborsFunction)).name("Neighbors function");
 			case ALL:
 				// create <vertex-neighbor value> pairs
 				DataSet<Tuple2<K, VV>> verticesWithNeighborValues = edges
-						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
+						.flatMap(new EmitOneEdgeWithNeighborPerNode<>())
 						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectNeighborValue<K, VV, EV>()).name("Vertex with neighbor value");
+						.with(new ProjectNeighborValue<>()).name("Vertex with neighbor value");
 
-				return verticesWithNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
-						reduceNeighborsFunction)).name("Neighbors function");
+				return verticesWithNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<>(
+					reduceNeighborsFunction)).name("Neighbors function");
 			default:
 				throw new IllegalArgumentException("Illegal edge direction");
 		}
@@ -2306,22 +2306,22 @@ public class Graph<K, VV, EV> {
 
 		switch (direction) {
 			case IN:
-				return edges.map(new ProjectVertexWithEdgeValueMap<K, EV>(1))
+				return edges.map(new ProjectVertexWithEdgeValueMap<>(1))
 						.withForwardedFields("f1->f0")
 							.name("Vertex with in-edges")
-						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction))
+						.groupBy(0).reduce(new ApplyReduceFunction<>(reduceEdgesFunction))
 							.name("Reduce on edges");
 			case OUT:
-				return edges.map(new ProjectVertexWithEdgeValueMap<K, EV>(0))
+				return edges.map(new ProjectVertexWithEdgeValueMap<>(0))
 						.withForwardedFields("f0->f0")
 							.name("Vertex with out-edges")
-						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction))
+						.groupBy(0).reduce(new ApplyReduceFunction<>(reduceEdgesFunction))
 							.name("Reduce on edges");
 			case ALL:
-				return edges.flatMap(new EmitOneVertexWithEdgeValuePerNode<K, EV>())
+				return edges.flatMap(new EmitOneVertexWithEdgeValuePerNode<>())
 						.withForwardedFields("f2->f1")
 							.name("Vertex with all edges")
-						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction))
+						.groupBy(0).reduce(new ApplyReduceFunction<>(reduceEdgesFunction))
 							.name("Reduce on edges");
 			default:
 				throw new IllegalArgumentException("Illegal edge direction");

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
index 6f5570f..36d2001 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
@@ -158,7 +158,7 @@ public class GraphCsvReader {
 		DataSet<Edge<K, NullValue>> edges = edgeReader
 			.types(vertexKey, vertexKey)
 				.name(GraphCsvReader.class.getName())
-			.map(new Tuple2ToEdgeMap<K>())
+			.map(new Tuple2ToEdgeMap<>())
 				.name("Type conversion");
 
 		return Graph.fromDataSet(edges, executionContext);
@@ -183,7 +183,7 @@ public class GraphCsvReader {
 		DataSet<Edge<K, NullValue>> edges = edgeReader
 			.types(vertexKey, vertexKey)
 				.name(GraphCsvReader.class.getName())
-			.map(new Tuple2ToEdgeMap<K>())
+			.map(new Tuple2ToEdgeMap<>())
 				.name("To Edge");
 
 		// the vertex value can be provided by an input file or a user-defined mapper
@@ -191,7 +191,7 @@ public class GraphCsvReader {
 			DataSet<Vertex<K, VV>> vertices = vertexReader
 				.types(vertexKey, vertexValue)
 					.name(GraphCsvReader.class.getName())
-				.map(new Tuple2ToVertexMap<K, VV>())
+				.map(new Tuple2ToVertexMap<>())
 					.name("Type conversion");
 
 			return Graph.fromDataSet(vertices, edges, executionContext);

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPair.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPair.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPair.java
index 7191bc9..71264d3 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPair.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPair.java
@@ -58,7 +58,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Edge<K, Tuple3<EV, Degrees, Deg
 			.join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND)
 			.where(1)
 			.equalTo(0)
-			.with(new JoinEdgeDegreeWithVertexDegree<K, EV, Degrees>())
+			.with(new JoinEdgeDegreeWithVertexDegree<>())
 				.setParallelism(parallelism)
 				.name("Edge target degree");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/27429a74/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegrees.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegrees.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegrees.java
index 30d30fa..efae743 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegrees.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegrees.java
@@ -52,7 +52,7 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Edge<K, Tuple2<EV, Degrees>>> {
 			.join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND)
 			.where(0)
 			.equalTo(0)
-			.with(new JoinEdgeWithVertexDegree<K, EV, Degrees>())
+			.with(new JoinEdgeWithVertexDegree<>())
 				.setParallelism(parallelism)
 				.name("Edge source degrees");
 	}


[04/11] flink git commit: [FLINK-7350] [travis] Only execute japicmp in misc profile

Posted by ch...@apache.org.
[FLINK-7350] [travis] Only execute japicmp in misc profile

This closes #4461.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/be8eb1a6
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/be8eb1a6
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/be8eb1a6

Branch: refs/heads/master
Commit: be8eb1a6000ad7fb97c254a1bd46f5b058e04d45
Parents: c4a5dd8
Author: zentol <ch...@apache.org>
Authored: Wed Aug 2 12:06:56 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Aug 7 15:16:56 2017 +0200

----------------------------------------------------------------------
 pom.xml                      | 20 +++++++++++++++++++-
 tools/travis_mvn_watchdog.sh |  8 ++++----
 2 files changed, 23 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/be8eb1a6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 62f4e31..427e813 100644
--- a/pom.xml
+++ b/pom.xml
@@ -118,6 +118,7 @@ under the License.
 		<junit.version>4.12</junit.version>
 		<mockito.version>1.10.19</mockito.version>
 		<powermock.version>1.6.5</powermock.version>
+		<japicmp.skip>false</japicmp.skip>
 		<!--
 			Keeping the MiniKDC version fixed instead of taking hadoop version dependency
 			to support testing Kafka, ZK etc., modules that does not have Hadoop dependency
@@ -694,6 +695,23 @@ under the License.
 				</plugins>
 			</build>
 		</profile>
+		
+		<profile>
+			<!--japicmp 0.7 does not support deactivation from the command
+				line, so we have to use a workaround with profiles instead.
+				This can be removed when upgrading japicmp to 0.10+.
+				-->
+			<id>skip-japicmp</id>
+			<activation>
+				<property>
+					<name>japicmp.skip</name>
+					<value>true</value>
+				</property>
+			</activation>
+			<properties>
+				<japicmp.skip>true</japicmp.skip>
+			</properties>
+		</profile>
 
 		<profile>
 			<!-- used for SNAPSHOT and regular releases -->
@@ -1388,7 +1406,7 @@ under the License.
 							<!-- Don't break build on newly added maven modules -->
 							<ignoreNonResolvableArtifacts>true</ignoreNonResolvableArtifacts>
 						</parameter>
-						<skip>false</skip>
+						<skip>${japicmp.skip}</skip>
 						<dependencies>
 							<dependency>
 								<groupId>org.apache.flink</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/be8eb1a6/tools/travis_mvn_watchdog.sh
----------------------------------------------------------------------
diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh
index b590b00..b837fbd 100755
--- a/tools/travis_mvn_watchdog.sh
+++ b/tools/travis_mvn_watchdog.sh
@@ -108,22 +108,22 @@ case $TEST in
 	(core)
 		MVN_COMPILE_MODULES="-pl $MODULES_CORE -am"
 		MVN_TEST_MODULES="-pl $MODULES_CORE"
-		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true -Djapicmp.skip=true"
 	;;
 	(libraries)
 		MVN_COMPILE_MODULES="-pl $MODULES_LIBRARIES -am"
 		MVN_TEST_MODULES="-pl $MODULES_LIBRARIES"
-		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true -Djapicmp.skip=true"
 	;;
 	(connectors)
 		MVN_COMPILE_MODULES="-pl $MODULES_CONNECTORS -am"
 		MVN_TEST_MODULES="-pl $MODULES_CONNECTORS"
-		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true -Djapicmp.skip=true"
 	;;
 	(tests)
 		MVN_COMPILE_MODULES="-pl $MODULES_TESTS -am"
 		MVN_TEST_MODULES="-pl $MODULES_TESTS"
-		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true"
+		MVN_COMPILE_OPTIONS="-Dcheckstyle.skip=true -Djapicmp.skip=true"
 	;;
 	(misc)
 		NEGATED_CORE=\!${MODULES_CORE//,/,\!}